You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@subversion.apache.org by st...@apache.org on 2014/02/16 12:27:30 UTC

svn commit: r1568748 - in /subversion/trunk/subversion/libsvn_fs_x: cached_data.c index.c index.h verify.c

Author: stefan2
Date: Sun Feb 16 11:27:29 2014
New Revision: 1568748

URL: http://svn.apache.org/r1568748
Log:
In FSX, replace txn / rev dual interfaces in index and cached_data code
with the new unified change set based ID part structure.

This is part of a series of patches switching everything over to the
new change set number paradigm.

* subversion/libsvn_fs_x/index.h
  (svn_fs_x__item_offset): Replace the txn / rev+item dual interface
                           parameters with a single change set based
                           ID part struct reference.  Also remove FSFS
                           references from the docstring.

* subversion/libsvn_fs_x/index.c
  (svn_fs_x__item_offset): Extract rev num / txn id internally as needed. 

* subversion/libsvn_fs_x/verify.c
  (compare_l2p_to_p2l_index,
   compare_p2l_to_l2p_index): Update index API calls. 

* subversion/libsvn_fs_x/cached_data.c
  (dgb__log_access): Combine revision+item_index into a id part parameter.
                     Also, remove FSFS-related code and docstring sections.
  (open_and_seek_revision): Combine rev+item to a single id part parameter
                            and update index API call.
  (open_and_seek_transaction): Update index API call.
  (open_and_seek_representation): Update function call.
  (get_node_revision_body,
   svn_fs_x__get_node_revision,
   svn_fs_x__get_mergeinfo_count): Update local function and index API calls. 
  (rep_state_t): Combine revision + item index members into an id part.
  (create_rep_state_body): Update struct initialization and use the new
                           struct member wherever required by function and
                           API changes.
  (get_window_key): Update struct user.
  (auto_set_start_offset, 
   read_delta_window,
   read_container_window, 
   get_combined_window): Update struct access and use the new struct member
                         wherever required by function and API changes.
  (init_rep_state): Update struct initialization.
  (svn_fs_x__get_changes): Update function and index API calls.
  (block_read): Update local function as well as index API calls and
                slightly simplify internal code.

Modified:
    subversion/trunk/subversion/libsvn_fs_x/cached_data.c
    subversion/trunk/subversion/libsvn_fs_x/index.c
    subversion/trunk/subversion/libsvn_fs_x/index.h
    subversion/trunk/subversion/libsvn_fs_x/verify.c

Modified: subversion/trunk/subversion/libsvn_fs_x/cached_data.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_x/cached_data.c?rev=1568748&r1=1568747&r2=1568748&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_x/cached_data.c (original)
+++ subversion/trunk/subversion/libsvn_fs_x/cached_data.c Sun Feb 16 11:27:29 2014
@@ -46,8 +46,7 @@
 static svn_error_t *
 block_read(void **result,
            svn_fs_t *fs,
-           svn_revnum_t revision,
-           apr_uint64_t item_index,
+           const svn_fs_x__id_part_t *id,
            apr_file_t *revision_file,
            apr_pool_t *result_pool,
            apr_pool_t *scratch_pool);
@@ -58,17 +57,12 @@ block_read(void **result,
 */
 
 /* When SVN_FS_X__LOG_ACCESS has been defined, write a line to console
- * showing where REVISION, ITEM_INDEX is located in FS and use ITEM to
- * show details on it's contents if not NULL.  To support format 6 and
- * earlier repos, ITEM_TYPE (SVN_FS_X__ITEM_TYPE_*) must match ITEM.
- * Use SCRATCH_POOL for temporary allocations.
- *
- * For pre-format7 repos, the display will be restricted.
+ * showing where ID is located in FS and use ITEM to show details on it's
+ * contents if not NULL.  Use SCRATCH_POOL for temporary allocations.
  */
 static svn_error_t *
 dgb__log_access(svn_fs_t *fs,
-                svn_revnum_t revision,
-                apr_uint64_t item_index,
+                const svn_fs_x__id_part_t *id,
                 void *item,
                 int item_type,
                 apr_pool_t *scratch_pool)
@@ -79,18 +73,16 @@ dgb__log_access(svn_fs_t *fs,
   apr_off_t offset = -1;
   apr_off_t end_offset = 0;
   apr_uint32_t sub_item = 0;
-  apr_array_header_t *entries;
   svn_fs_x__p2l_entry_t *entry = NULL;
-  int i;
   static const char *types[] = {"<n/a>", "frep ", "drep ", "fprop", "dprop",
                                 "node ", "chgs ", "rep  ", "c:", "n:", "r:"};
   const char *description = "";
   const char *type = types[item_type];
   const char *pack = "";
+  svn_revnum_t revision = svn_fs_x__get_revnum(id->change_set);
 
   /* determine rev / pack file offset */
-  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, revision, NULL,
-                                item_index, scratch_pool));
+  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, id, scratch_pool));
 
   /* constructing the pack file description */
   if (revision < ffd->min_unpacked_rev)
@@ -146,44 +138,32 @@ dgb__log_access(svn_fs_t *fs,
         }
     }
 
-  /* some info is only available in format7 repos */
-  if (ffd->format >= SVN_FS_X__MIN_LOG_ADDRESSING_FORMAT)
-    {
-      /* reverse index lookup: get item description in ENTRY */
-      SVN_ERR(svn_fs_x__p2l_entry_lookup(&entry, fs, revision, offset,
-                                          scratch_pool));
-      if (entry)
-        {
-          /* more details */
-          end_offset = offset + entry->size;
-          type = types[entry->type];
-
-          /* merge the sub-item number with the container type */
-          if (   entry->type == SVN_FS_X__ITEM_TYPE_CHANGES_CONT
-              || entry->type == SVN_FS_X__ITEM_TYPE_NODEREVS_CONT
-              || entry->type == SVN_FS_X__ITEM_TYPE_REPS_CONT)
-            type = apr_psprintf(scratch_pool, "%s%-3d", type, sub_item);
-        }
-
-      /* line output */
-      printf("%5s%4lx:%04lx -%4lx:%04lx %s %7ld %5"APR_UINT64_T_FMT"   %s\n",
-             pack, (long)(offset / ffd->block_size),
-             (long)(offset % ffd->block_size),
-             (long)(end_offset / ffd->block_size),
-             (long)(end_offset % ffd->block_size),
-             type, revision, item_index, description);
-    }
-  else
-    {
-      /* reduced logging for format 6 and earlier */
-      printf("%5s%10" APR_UINT64_T_HEX_FMT " %s %7ld %7" APR_UINT64_T_FMT \
-             "   %s\n",
-             pack, (apr_uint64_t)(offset), type, revision, item_index,
-             description);
-    }
+  /* reverse index lookup: get item description in ENTRY */
+  SVN_ERR(svn_fs_x__p2l_entry_lookup(&entry, fs, revision, offset,
+                                      scratch_pool));
+  if (entry)
+    {
+      /* more details */
+      end_offset = offset + entry->size;
+      type = types[entry->type];
+
+      /* merge the sub-item number with the container type */
+      if (   entry->type == SVN_FS_X__ITEM_TYPE_CHANGES_CONT
+          || entry->type == SVN_FS_X__ITEM_TYPE_NODEREVS_CONT
+          || entry->type == SVN_FS_X__ITEM_TYPE_REPS_CONT)
+        type = apr_psprintf(scratch_pool, "%s%-3d", type, sub_item);
+    }
+
+  /* line output */
+  printf("%5s%4lx:%04lx -%4lx:%04lx %s %7ld %5"APR_UINT64_T_FMT"   %s\n",
+          pack, (long)(offset / ffd->block_size),
+          (long)(offset % ffd->block_size),
+          (long)(end_offset / ffd->block_size),
+          (long)(end_offset % ffd->block_size),
+          type, revision, id->number, description);
 
 #endif
-  
+
   return SVN_NO_ERROR;
 }
 
@@ -202,26 +182,24 @@ aligned_seek(svn_fs_t *fs,
                                                   pool));
 }
 
-/* Open the revision file for revision REV in filesystem FS and store
-   the newly opened file in FILE.  Seek to location OFFSET before
+/* Open the revision file for the item given by ID in filesystem FS and
+   store the newly opened file in FILE.  Seek to the item's location before
    returning.  Perform temporary allocations in POOL. */
 static svn_error_t *
 open_and_seek_revision(apr_file_t **file,
                        svn_fs_t *fs,
-                       svn_revnum_t rev,
-                       apr_uint64_t item,
+                       const svn_fs_x__id_part_t *id,
                        apr_pool_t *pool)
 {
   apr_file_t *rev_file;
   apr_off_t offset = -1;
   apr_uint32_t sub_item = 0;
+  svn_revnum_t rev = svn_fs_x__get_revnum(id->change_set);
 
   SVN_ERR(svn_fs_x__ensure_revision_exists(rev, fs, pool));
 
   SVN_ERR(svn_fs_x__open_pack_or_rev_file(&rev_file, fs, rev, pool));
-  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, rev,
-                                SVN_FS_X__INVALID_TXN_ID, item,
-                                pool));
+  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, id, pool));
   SVN_ERR(aligned_seek(fs, rev_file, NULL, offset, pool));
 
   *file = rev_file;
@@ -242,13 +220,16 @@ open_and_seek_transaction(apr_file_t **f
   apr_off_t offset;
   apr_uint32_t sub_item = 0;
 
+  svn_fs_x__id_part_t id;
+  id.change_set = svn_fs_x__change_set_by_txn(rep->txn_id);
+  id.number = rep->item_index;
+
   SVN_ERR(svn_io_file_open(&rev_file,
                            svn_fs_x__path_txn_proto_rev(fs, rep->txn_id,
                                                         pool),
                            APR_READ | APR_BUFFERED, APR_OS_DEFAULT, pool));
 
-  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, SVN_INVALID_REVNUM,
-                                rep->txn_id, rep->item_index, pool));
+  SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, &id, pool));
   SVN_ERR(aligned_seek(fs, rev_file, NULL, offset, pool));
 
   *file = rev_file;
@@ -266,8 +247,12 @@ open_and_seek_representation(apr_file_t 
                              apr_pool_t *pool)
 {
   if (! svn_fs_x__id_txn_used(rep->txn_id))
-    return open_and_seek_revision(file_p, fs, rep->revision, rep->item_index,
-                                  pool);
+    {
+      svn_fs_x__id_part_t id;
+      id.change_set = svn_fs_x__change_set_by_rev(rep->revision);
+      id.number = rep->item_index;
+      return open_and_seek_revision(file_p, fs, &id, pool);
+    }
   else
     return open_and_seek_transaction(file_p, fs, rep, pool);
 }
@@ -336,10 +321,8 @@ get_node_revision_body(node_revision_t *
         {
           apr_off_t offset;
           apr_uint32_t sub_item;
-          SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs,
-                                        revision,
-                                        SVN_FS_X__INVALID_TXN_ID,
-                                        noderev_id->number, pool));
+          SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, noderev_id,
+                                        pool));
           key.revision = svn_fs_x__packed_base_rev(fs, revision);
           key.second = offset;
 
@@ -368,16 +351,12 @@ get_node_revision_body(node_revision_t *
         }
 
       /* someone needs to read the data from this file: */
-      err = open_and_seek_revision(&revision_file, fs,
-                                   revision,
-                                   noderev_id->number,
-                                   pool);
+      err = open_and_seek_revision(&revision_file, fs, noderev_id, pool);
 
       /* block-read will parse the whole block and will also return
           the one noderev that we need right now. */
       SVN_ERR(block_read((void **)noderev_p, fs,
-                         revision,
-                         noderev_id->number,
+                         noderev_id,
                          revision_file,
                          pool,
                          pool));
@@ -404,8 +383,7 @@ svn_fs_x__get_node_revision(node_revisio
                                id_string->data);
     }
 
-  SVN_ERR(dgb__log_access(fs, svn_fs_x__get_revnum(noderev_id->change_set),
-                          noderev_id->number, *noderev_p,
+  SVN_ERR(dgb__log_access(fs, noderev_id, *noderev_p,
                           SVN_FS_X__ITEM_TYPE_NODEREV, pool));
 
   return svn_error_trace(err);
@@ -441,9 +419,7 @@ svn_fs_x__get_mergeinfo_count(apr_int64_
           svn_boolean_t is_cached;
 
           SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs,
-                                        revision,
-                                        SVN_FS_X__INVALID_TXN_ID,
-                                        noderev_id->number, pool));
+                                        noderev_id, pool));
           key.revision = svn_fs_x__packed_base_rev(fs, revision);
           key.second = offset;
 
@@ -509,10 +485,8 @@ typedef struct rep_state_t
   svn_cache__t *window_cache;
                     /* Caches un-deltified windows. May be NULL. */
   svn_cache__t *combined_cache;
-                    /* revision containing the representation */
-  svn_revnum_t revision;
-                    /* representation's item index in REVISION */
-  apr_uint64_t item_index;
+                    /* ID addressing the representation */
+  svn_fs_x__id_part_t rep_id;
                     /* length of the header at the start of the rep.
                        0 iff this is rep is stored in a container
                        (i.e. does not have a header) */
@@ -566,8 +540,8 @@ create_rep_state_body(rep_state_t **rep_
 
   /* continue constructing RS and RA */
   rs->size = rep->size;
-  rs->revision = rep->revision;
-  rs->item_index = rep->item_index;
+  rs->rep_id.change_set = svn_fs_x__change_set_by_rev(rep->revision);
+  rs->rep_id.number = rep->item_index;
   rs->window_cache = ffd->txdelta_window_cache;
   rs->combined_cache = ffd->combined_window_cache;
   rs->ver = -1;
@@ -601,11 +575,10 @@ create_rep_state_body(rep_state_t **rep_
       /* we will need the on-disk location for non-txn reps */
       apr_off_t offset;
       apr_uint32_t sub_item;
+
       if (! svn_fs_x__id_txn_used(rep->txn_id))
-        SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item,
-                                      fs, rep->revision,
-                                      SVN_FS_X__INVALID_TXN_ID,
-                                      rep->item_index, pool));
+        SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, &rs->rep_id,
+                                      pool));
 
       /* is rep stored in some star-deltified container? */
       if (! svn_fs_x__id_txn_used(rep->txn_id))
@@ -670,15 +643,14 @@ create_rep_state_body(rep_state_t **rep_
 
       if (! svn_fs_x__id_txn_used(rep->txn_id))
         {
-          SVN_ERR(block_read(NULL, fs, rep->revision, rep->item_index,
-                              rs->file->file, pool, pool));
+          SVN_ERR(block_read(NULL, fs, &rs->rep_id, rs->file->file, pool, pool));
           if (ffd->rep_header_cache)
             SVN_ERR(svn_cache__set(ffd->rep_header_cache, &key, rh, pool));
         }
     }
 
-  SVN_ERR(dgb__log_access(fs, rep->revision, rep->item_index, rh,
-                          SVN_FS_X__ITEM_TYPE_ANY_REP, pool));
+  SVN_ERR(dgb__log_access(fs, &rs->rep_id, rh, SVN_FS_X__ITEM_TYPE_ANY_REP,
+                          pool));
 
   rs->header_size = rh->header_size;
   *rep_state = rs;
@@ -863,9 +835,11 @@ struct rep_read_baton
 static window_cache_key_t *
 get_window_key(window_cache_key_t *key, rep_state_t *rs)
 {
-  assert(rs->revision <= APR_UINT32_MAX);
-  key->revision = (apr_uint32_t)rs->revision;
-  key->item_index = rs->item_index;
+  svn_revnum_t revision = svn_fs_x__get_revnum(rs->rep_id.change_set);
+  assert(revision <= APR_UINT32_MAX);
+
+  key->revision = (apr_uint32_t)revision;
+  key->item_index = rs->rep_id.number;
   key->chunk_index = rs->chunk_index;
 
   return key;
@@ -1214,9 +1188,7 @@ auto_set_start_offset(rep_state_t *rs, a
   if (rs->start == -1)
     {
       SVN_ERR(svn_fs_x__item_offset(&rs->start, &rs->sub_item,
-                                    rs->file->fs, rs->revision,
-                                    SVN_FS_X__INVALID_TXN_ID,
-                                    rs->item_index, pool));
+                                    rs->file->fs, &rs->rep_id, pool));
       rs->start += rs->header_size;
     }
 
@@ -1262,8 +1234,8 @@ read_delta_window(svn_txdelta_window_t *
   apr_off_t end_offset;
   SVN_ERR_ASSERT(rs->chunk_index <= this_chunk);
 
-  SVN_ERR(dgb__log_access(rs->file->fs, rs->revision, rs->item_index,
-                          NULL, SVN_FS_X__ITEM_TYPE_ANY_REP, pool));
+  SVN_ERR(dgb__log_access(rs->file->fs, &rs->rep_id, NULL,
+                          SVN_FS_X__ITEM_TYPE_ANY_REP, pool));
 
   /* Read the next window.  But first, try to find it in the cache. */
   SVN_ERR(get_cached_window(nwin, rs, this_chunk, &is_cached, pool));
@@ -1276,10 +1248,10 @@ read_delta_window(svn_txdelta_window_t *
   /* invoke the 'block-read' feature for non-txn data.
      However, don't do that if we are in the middle of some representation,
      because the block is unlikely to contain other data. */
-  if (rs->chunk_index == 0 && SVN_IS_VALID_REVNUM(rs->revision))
+  if (rs->chunk_index == 0 && svn_fs_x__is_revision(rs->rep_id.change_set))
     {
-      SVN_ERR(block_read(NULL, rs->file->fs, rs->revision, rs->item_index,
-                         rs->file->file, pool, pool));
+      SVN_ERR(block_read(NULL, rs->file->fs, &rs->rep_id, rs->file->file,
+                         pool, pool));
 
       /* reading the whole block probably also provided us with the
          desired txdelta window */
@@ -1326,7 +1298,7 @@ read_delta_window(svn_txdelta_window_t *
 
   /* the window has not been cached before, thus cache it now
    * (if caching is used for them at all) */
-  if (SVN_IS_VALID_REVNUM(rs->revision))
+  if (svn_fs_x__is_revision(rs->rep_id.change_set))
     SVN_ERR(set_cached_window(*nwin, rs, start_offset, pool));
 
   return SVN_NO_ERROR;
@@ -1343,9 +1315,10 @@ read_container_window(svn_stringbuf_t **
   svn_fs_t *fs = rs->file->fs;
   fs_x_data_t *ffd = fs->fsap_data;
   pair_cache_key_t key;
+  svn_revnum_t revision = svn_fs_x__get_revnum(rs->rep_id.change_set);
 
   SVN_ERR(auto_set_start_offset(rs, pool));
-  key.revision = svn_fs_x__packed_base_rev(fs, rs->revision);
+  key.revision = svn_fs_x__packed_base_rev(fs, revision);
   key.second = rs->start;
 
   /* already in cache? */
@@ -1366,8 +1339,8 @@ read_container_window(svn_stringbuf_t **
   if (extractor == NULL)
     {
       SVN_ERR(auto_open_shared_file(rs->file));
-      SVN_ERR(block_read((void **)&extractor, fs, rs->revision,
-                         rs->item_index, rs->file->file, pool, pool));
+      SVN_ERR(block_read((void **)&extractor, fs, &rs->rep_id,
+                         rs->file->file, pool, pool));
     }
 
   SVN_ERR(svn_fs_x__extractor_drive(nwin, extractor, rs->current, size,
@@ -1447,7 +1420,7 @@ get_combined_window(svn_stringbuf_t **re
          single chunk.  Only then will no other chunk need a deeper RS
          list than the cached chunk. */
       if (   (rb->chunk_index == 0) && (rs->current == rs->size)
-          && SVN_IS_VALID_REVNUM(rs->revision))
+          && svn_fs_x__is_revision(rs->rep_id.change_set))
         SVN_ERR(set_cached_combined_window(buf, rs, new_pool));
 
       rs->chunk_index++;
@@ -1514,8 +1487,7 @@ init_rep_state(rep_state_t *rs,
   shared_file->pool = pool;
 
   rs->file = shared_file;
-  rs->revision = svn_fs_x__get_revnum(entry->items[0].change_set);
-  rs->item_index = entry->items[0].number;
+  rs->rep_id = entry->items[0];
   rs->header_size = rep_header->header_size;
   rs->start = entry->offset + rs->header_size;
   rs->current = 4;
@@ -2261,6 +2233,10 @@ svn_fs_x__get_changes(apr_array_header_t
   svn_boolean_t found;
   fs_x_data_t *ffd = fs->fsap_data;
 
+  svn_fs_x__id_part_t id;
+  id.change_set = svn_fs_x__change_set_by_rev(rev);
+  id.number = SVN_FS_X__ITEM_INDEX_CHANGES;
+
   /* try cache lookup first */
 
   if (ffd->changes_container_cache && svn_fs_x__is_packed_rev(fs, rev))
@@ -2269,9 +2245,7 @@ svn_fs_x__get_changes(apr_array_header_t
       apr_uint32_t sub_item;
       pair_cache_key_t key;
 
-      SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, rev,
-                                    SVN_FS_X__INVALID_TXN_ID,
-                                    SVN_FS_X__ITEM_INDEX_CHANGES, pool));
+      SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, &id, pool));
       key.revision = svn_fs_x__packed_base_rev(fs, rev);
       key.second = offset;
 
@@ -2299,15 +2273,14 @@ svn_fs_x__get_changes(apr_array_header_t
                                               pool));
 
       /* 'block-read' will also provide us with the desired data */
-      SVN_ERR(block_read((void **)changes, fs,
-                         rev, SVN_FS_X__ITEM_INDEX_CHANGES,
-                         revision_file, pool, pool));
+      SVN_ERR(block_read((void **)changes, fs, &id, revision_file,
+                         pool, pool));
 
       SVN_ERR(svn_io_file_close(revision_file, pool));
     }
 
-  SVN_ERR(dgb__log_access(fs, rev, SVN_FS_X__ITEM_INDEX_CHANGES, *changes,
-                          SVN_FS_X__ITEM_TYPE_CHANGES, pool));
+  SVN_ERR(dgb__log_access(fs, &id, *changes, SVN_FS_X__ITEM_TYPE_CHANGES,
+                          pool));
 
   return SVN_NO_ERROR;
 }
@@ -2611,8 +2584,7 @@ block_read_reps_container(svn_fs_x__rep_
 static svn_error_t *
 block_read(void **result,
            svn_fs_t *fs,
-           svn_revnum_t revision,
-           apr_uint64_t item_index,
+           const svn_fs_x__id_part_t *id,
            apr_file_t *revision_file,
            apr_pool_t *result_pool,
            apr_pool_t *scratch_pool)
@@ -2621,6 +2593,7 @@ block_read(void **result,
   apr_off_t offset, wanted_offset = 0;
   apr_off_t block_start = 0;
   apr_uint32_t wanted_sub_item = 0;
+  svn_revnum_t revision = svn_fs_x__get_revnum(id->change_set);
   apr_array_header_t *entries;
   int run_count = 0;
   int i;
@@ -2633,9 +2606,8 @@ block_read(void **result,
 
   /* index lookup: find the OFFSET of the item we *must* read plus (in the
    * "do-while" block) the list of items in the same block. */
-  SVN_ERR(svn_fs_x__item_offset(&wanted_offset, &wanted_sub_item, fs,
-                                revision, SVN_FS_X__INVALID_TXN_ID,
-                                item_index, iterpool));
+  SVN_ERR(svn_fs_x__item_offset(&wanted_offset, &wanted_sub_item, fs, id,
+                                iterpool));
 
   offset = wanted_offset;
   do
@@ -2661,9 +2633,8 @@ block_read(void **result,
           is_result =    result
                       && entry->offset == wanted_offset
                       && entry->item_count >= wanted_sub_item
-                      && entry->items[wanted_sub_item].change_set
-                         == svn_fs_x__change_set_by_rev(revision)
-                      && entry->items[wanted_sub_item].number == item_index;
+                      && svn_fs_x__id_part_eq(entry->items + wanted_sub_item,
+                                              id);
 
           /* select the pool that we want the item to be allocated in */
           pool = is_result ? result_pool : iterpool;

Modified: subversion/trunk/subversion/libsvn_fs_x/index.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_x/index.c?rev=1568748&r1=1568747&r2=1568748&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_x/index.c (original)
+++ subversion/trunk/subversion/libsvn_fs_x/index.c Sun Feb 16 11:27:29 2014
@@ -2675,17 +2675,17 @@ svn_error_t *
 svn_fs_x__item_offset(apr_off_t *offset,
                       apr_uint32_t *sub_item,
                       svn_fs_t *fs,
-                      svn_revnum_t revision,
-                      svn_fs_x__txn_id_t txn_id,
-                      apr_uint64_t item_index,
+                      const svn_fs_x__id_part_t *item_id,
                       apr_pool_t *pool)
 {
-  if (txn_id != SVN_FS_X__INVALID_TXN_ID)
-    SVN_ERR(l2p_proto_index_lookup(offset, sub_item,
-                                   fs, txn_id, item_index, pool));
+  if (svn_fs_x__is_txn(item_id->change_set))
+    SVN_ERR(l2p_proto_index_lookup(offset, sub_item, fs,
+                                   svn_fs_x__get_txn_id(item_id->change_set),
+                                   item_id->number, pool));
   else
-    SVN_ERR(l2p_index_lookup(offset, sub_item,
-                             fs, revision, item_index, pool));
+    SVN_ERR(l2p_index_lookup(offset, sub_item, fs,
+                             svn_fs_x__get_revnum(item_id->change_set),
+                             item_id->number, pool));
 
   return SVN_NO_ERROR;
 }

Modified: subversion/trunk/subversion/libsvn_fs_x/index.h
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_x/index.h?rev=1568748&r1=1568747&r2=1568748&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_x/index.h (original)
+++ subversion/trunk/subversion/libsvn_fs_x/index.h Sun Feb 16 11:27:29 2014
@@ -201,20 +201,14 @@ svn_fs_x__p2l_item_lookup(svn_fs_x__id_p
                           apr_pool_t *pool);
 
 /* Use the log-to-phys mapping files in FS to find the packed / non-packed /
- * proto-rev file offset and container sub-item of either (REVISION,
- * ITEM_INDEX) or (TXN_ID, ITEM_INDEX).  *SUB_ITEM will be 0 for non-
- * container items.  For committed revision, TXN_ID must be NULL.  For
- * format 6 and older repositories, we simply map the revision local offset
- * given as ITEM_INDEX to the actual file offset (when packed).
- * Use POOL for allocations.
+ * proto-rev file offset and container sub-item of ITEM_ID.  *SUB_ITEM will
+ * be 0 for non-container items.  Use POOL for allocations.
  */
 svn_error_t *
 svn_fs_x__item_offset(apr_off_t *offset,
                       apr_uint32_t *sub_item,
                       svn_fs_t *fs,
-                      svn_revnum_t revision,
-                      svn_fs_x__txn_id_t txn_id,
-                      apr_uint64_t item_index,
+                      const svn_fs_x__id_part_t *item_id,
                       apr_pool_t *pool);
 
 /* Use the log-to-phys indexes in FS to determine the maximum item indexes

Modified: subversion/trunk/subversion/libsvn_fs_x/verify.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_x/verify.c?rev=1568748&r1=1568747&r2=1568748&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_x/verify.c (original)
+++ subversion/trunk/subversion/libsvn_fs_x/verify.c Sun Feb 16 11:27:29 2014
@@ -193,9 +193,8 @@ compare_l2p_to_p2l_index(svn_fs_t *fs,
           l2p_item.number = k;
 
           /* get L2P entry.  Ignore unused entries. */
-          SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs,
-                                        revision, SVN_FS_X__INVALID_TXN_ID,
-                                        k, iterpool));
+          SVN_ERR(svn_fs_x__item_offset(&offset, &sub_item, fs, &l2p_item,
+                                        iterpool));
           if (offset == -1)
             continue;
 
@@ -296,9 +295,7 @@ compare_p2l_to_l2p_index(svn_fs_t *fs,
                 = svn_fs_x__get_revnum(p2l_item->change_set);
 
               SVN_ERR(svn_fs_x__item_offset(&l2p_offset, &sub_item, fs,
-                                            revision,
-                                            SVN_FS_X__INVALID_TXN_ID,
-                                            p2l_item->number, iterpool));
+                                            p2l_item, iterpool));
 
               if (sub_item != k || l2p_offset != entry->offset)
                 return svn_error_createf(SVN_ERR_FS_ITEM_INDEX_INCONSISTENT,