You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@subversion.apache.org by ph...@apache.org on 2009/12/09 17:33:33 UTC

svn commit: r888863 - in /subversion/branches/obliterate-like-deltify: ./ subversion/include/ subversion/libsvn_fs/ subversion/libsvn_fs_base/ subversion/libsvn_fs_fs/ subversion/tests/libsvn_fs/

Author: philip
Date: Wed Dec  9 16:33:28 2009
New Revision: 888863

URL: http://svn.apache.org/viewvc?rev=888863&view=rev
Log:
An experimental BDB obliterate based on deltify.  It passes one simple
libsvn_fs regression test.  No FSFS support.

* BRANCH-README: Update.

* subversion/include/svn_fs.h
  (svn_fs_obliterate): New.

* subversion/libsvn_fs/fs-loader.c
  (svn_fs_obliterate): New.

* subversion/libsvn_fs/fs-loader.h
  (fs_vtable_t): Add obliterate.

* subversion/libsvn_fs_base/fs.c
  (fs_vtable): Add svn_fs_base__obliterate.

* subversion/libsvn_fs_fs/fs.c
  (fs_vtable): Add NULL for svn_fs_base__obliterate.

* subversion/tests/libsvn_fs/fs-test.c
  (obliterate_2): New.
  (test_funcs): Add obliterate_2.

* subversion/libsvn_fs_base/tree.h
  (svn_fs_base__obliterate): New.

* subversion/libsvn_fs_base/tree.c
  (struct txn_obliterate_args, txn_body_obliterate,
   svn_fs_base__obliterate): New.

* subversion/libsvn_fs_base/dag.h
  (svn_fs_base__dag_obliterate): New.

* subversion/libsvn_fs_base/dag.c
  (svn_fs_base__dag_obliterate): New.

* subversion/libsvn_fs_base/reps-strings.h
  (svn_fs_base__rep_obliterate): New.

* subversion/libsvn_fs_base/reps-strings.c
  (svn_fs_base__rep_obliterate): New.

Modified:
    subversion/branches/obliterate-like-deltify/BRANCH-README
    subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.c
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.h
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.c
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.h
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/fs.c
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.c
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.h
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.c
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.h
    subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_fs/fs.c
    subversion/branches/obliterate-like-deltify/subversion/tests/libsvn_fs/fs-test.c

Modified: subversion/branches/obliterate-like-deltify/BRANCH-README
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/BRANCH-README?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/BRANCH-README (original)
+++ subversion/branches/obliterate-like-deltify/BRANCH-README Wed Dec  9 16:33:28 2009
@@ -8,3 +8,16 @@
 to file and directory properties, but anything that changes the tree
 structure or anything that needs new node-rev-ids is not going to
 work.
+
+At present the svn_fs.h interface exists to allow testing; I haven't
+really thought much about what it should look like.
+
+- Does it work?  It passes the one regression test in fs-test.
+
+- Is it safe to run on a live repository?  I don't know.
+
+- Does it break younger revisions?  I don't know.  Probably not,
+  because the old rep remains it the database.
+
+- The old rep is not deleted, can it leak?  I don't know.  Could diff
+  leak it?  Does diff even work properly?

Modified: subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h (original)
+++ subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h Wed Dec  9 16:33:28 2009
@@ -1840,6 +1840,16 @@
                         svn_revnum_t revision,
                         apr_pool_t *pool);
 
+/** Obliterate the text associated with @revision of @a path in @a fs,
+ * where @a path must be a file.  The text is replaced with the text
+ * of the previous version of the file.
+ */
+svn_error_t *
+svn_fs_obliterate(svn_fs_t *fs,
+                  const char *path,
+                  svn_revnum_t revision,
+                  apr_pool_t *pool);
+
 
 /** Set @a *value_p to the value of the property named @a propname on
  * revision @a rev in the filesystem @a fs.  If @a rev has no property by

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.c Wed Dec  9 16:33:28 2009
@@ -1152,6 +1152,13 @@
 }
 
 svn_error_t *
+svn_fs_obliterate(svn_fs_t *fs, const char *path, svn_revnum_t revision,
+                  apr_pool_t *pool)
+{
+  return fs->vtable->obliterate(fs, path, revision, pool);
+}
+
+svn_error_t *
 svn_fs_revision_prop(svn_string_t **value_p, svn_fs_t *fs, svn_revnum_t rev,
                      const char *propname, apr_pool_t *pool)
 {

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.h
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.h?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.h (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs/fs-loader.h Wed Dec  9 16:33:28 2009
@@ -176,6 +176,9 @@
   svn_error_t *(*list_transactions)(apr_array_header_t **names_p,
                                     svn_fs_t *fs, apr_pool_t *pool);
   svn_error_t *(*deltify)(svn_fs_t *fs, svn_revnum_t rev, apr_pool_t *pool);
+  /* ### Do we want obliterate in the FS interface? */
+  svn_error_t *(*obliterate)(svn_fs_t *fs, const char *path, svn_revnum_t rev,
+                             apr_pool_t *pool);
   svn_error_t *(*lock)(svn_lock_t **lock, svn_fs_t *fs,
                        const char *path, const char *token,
                        const char *comment, svn_boolean_t is_dav_comment,

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.c Wed Dec  9 16:33:28 2009
@@ -1534,6 +1534,32 @@
   return SVN_NO_ERROR;
 }
 
+svn_error_t *
+svn_fs_base__dag_obliterate(dag_node_t *node,
+                            dag_node_t *pred_node,
+                            trail_t *trail,
+                            apr_pool_t *pool)
+{
+  node_revision_t *node_rev, *pred_node_rev;
+  svn_fs_t *fs = svn_fs_base__dag_get_fs(node);
+  const char *pred_key;
+
+  SVN_ERR(svn_fs_bdb__get_node_revision(&node_rev, fs, node->id, trail, pool));
+  if (pred_node)
+    {
+      SVN_ERR(svn_fs_bdb__get_node_revision(&pred_node_rev, fs, pred_node->id,
+                                            trail, pool));
+      pred_key = pred_node_rev->data_key;
+    }
+  else
+    {
+      pred_key = NULL;
+    }
+
+  return svn_fs_base__rep_obliterate(trail->fs, node_rev->data_key, pred_key,
+                                     trail, pool);
+}
+
 
 /* Maybe store a `checksum-reps' index record for the representation whose
    key is REP.  (If there's already a rep for this checksum, we don't

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.h
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.h?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.h (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/dag.h Wed Dec  9 16:33:28 2009
@@ -539,6 +539,12 @@
                                       trail_t *trail,
                                       apr_pool_t *pool);
 
+/* Obliterate NODE's representation by constructing an empty delta
+   to PRED_NODE. */
+svn_error_t *svn_fs_base__dag_obliterate(dag_node_t *node,
+                                         dag_node_t *pred_node,
+                                         trail_t *trail,
+                                         apr_pool_t *pool);
 
 /* Index NODE's backing data representations by their checksum.  Do
    this as part of TRAIL.  Use POOL for allocations. */

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/fs.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/fs.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/fs.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/fs.c Wed Dec  9 16:33:28 2009
@@ -491,6 +491,7 @@
   svn_fs_base__purge_txn,
   svn_fs_base__list_transactions,
   svn_fs_base__deltify,
+  svn_fs_base__obliterate,
   svn_fs_base__lock,
   svn_fs_base__generate_lock_token,
   svn_fs_base__unlock,

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.c Wed Dec  9 16:33:28 2009
@@ -1621,3 +1621,143 @@
 
   return SVN_NO_ERROR;
 }
+
+svn_error_t *svn_fs_base__rep_obliterate(svn_fs_t *fs,
+                                         const char *key,
+                                         const char *pred_key,
+                                         trail_t *trail,
+                                         apr_pool_t *pool)
+{
+  svn_stream_t *new_stream;
+  struct write_svndiff_strings_baton new_baton;
+  svn_stream_t *pred_stream1, *pred_stream2;
+  svn_txdelta_stream_t *txdelta_stream;
+  base_fs_data_t *bfd = fs->fsap_data;
+  svn_txdelta_window_handler_t new_handler;
+  void *new_handler_baton;
+  apr_pool_t *wpool;
+  apr_array_header_t *windows;
+  window_write_t *ww;
+  svn_txdelta_window_t *window;
+  svn_filesize_t tview_off = 0;
+  svn_filesize_t diffsize = 0;
+  const unsigned char *digest;
+  representation_t *pred_rep;
+  representation_t new_rep;
+  rep_delta_chunk_t *chunk;
+  apr_array_header_t *chunks;
+  int i;
+
+  /* ### Temporary error, really need an empty rep here to support
+      obliteration of the first version. */
+  if (!pred_key)
+    return svn_error_createf
+      (SVN_ERR_UNSUPPORTED_FEATURE, NULL,
+       _("Attempting to obliterate '%s' without a predecessor "),
+       key);
+
+  if (!strcmp(key, pred_key))
+    return svn_error_createf(SVN_ERR_FS_CORRUPT, NULL,
+                             _("Attempt to obliterate '%s' using itself "),
+                             key);
+
+  new_baton.fs = fs;
+  new_baton.trail = trail;
+  new_baton.header_read = FALSE;
+  new_stream = svn_stream_create(&new_baton, pool);
+  svn_stream_set_write(new_stream, write_svndiff_strings);
+
+  /* ### Is there a simpler way to write an empty delta? */
+  SVN_ERR(svn_fs_base__rep_contents_read_stream(&pred_stream1, fs, pred_key,
+                                                TRUE, trail, pool));
+  SVN_ERR(svn_fs_base__rep_contents_read_stream(&pred_stream2, fs, pred_key,
+                                                TRUE, trail, pool));
+  svn_txdelta(&txdelta_stream, pred_stream1, pred_stream2, pool);
+
+  if (bfd->format >= SVN_FS_BASE__MIN_SVNDIFF1_FORMAT)
+    svn_txdelta_to_svndiff2(&new_handler, &new_handler_baton,
+                            new_stream, 1, pool);
+  else
+    svn_txdelta_to_svndiff2(&new_handler, &new_handler_baton,
+                            new_stream, 0, pool);
+
+  wpool = svn_pool_create(pool);
+  windows = apr_array_make(pool, 1, sizeof(ww));
+  do
+    {
+      new_baton.size = 0;
+      new_baton.key = NULL;
+      svn_pool_clear(wpool);
+
+      SVN_ERR(svn_txdelta_next_window(&window, txdelta_stream, wpool));
+      SVN_ERR(new_handler(window, new_handler_baton));
+      if (window)
+        {
+          ww = apr_pcalloc(pool, sizeof(*ww));
+          ww->key = new_baton.key;
+          ww->svndiff_len = new_baton.size;
+          ww->text_off = tview_off;
+          ww->text_len = window->tview_len;
+          APR_ARRAY_PUSH(windows, window_write_t *) = ww;
+          tview_off += window->tview_len;
+          diffsize += ww->svndiff_len;
+        }
+    } while (window);
+
+  svn_pool_destroy(wpool);
+
+  digest = svn_txdelta_md5_digest(txdelta_stream);
+  if (!digest)
+    return svn_error_createf(SVN_ERR_DELTA_MD5_CHECKSUM_ABSENT, NULL,
+                             _("Failed to calculate MD5 digest for '%s'"),
+                             pred_key);
+  /* ### Check the digest against something?  pred_rep->md5_checksum? */
+
+  SVN_ERR(svn_fs_bdb__read_rep(&pred_rep, fs, pred_key, trail, pool));
+  new_rep.md5_checksum = svn_checksum_dup(pred_rep->md5_checksum, pool);
+  new_rep.sha1_checksum = svn_checksum_dup(pred_rep->sha1_checksum, pool);
+  new_rep.kind = rep_kind_delta;
+  new_rep.txn_id = NULL;
+
+  chunks = apr_array_make(pool, windows->nelts, sizeof(chunk));
+
+  for (i = 0; i < windows->nelts; i++)
+    {
+      ww = APR_ARRAY_IDX(windows, i, window_write_t *);
+
+      chunk = apr_palloc(pool, sizeof(*chunk));
+      chunk->offset = ww->text_off;
+
+      chunk->version = new_baton.version;
+      chunk->string_key = ww->key;
+      chunk->size = ww->text_len;
+      chunk->rep_key = pred_key;
+
+      APR_ARRAY_PUSH(chunks, rep_delta_chunk_t *) = chunk;
+    }
+
+  new_rep.contents.delta.chunks = chunks;
+  SVN_ERR(svn_fs_bdb__write_rep(fs, key, &new_rep, trail, pool));
+
+#if 0
+  /* ### Might have younger revs referring to the old rep? */
+  {
+    representation_t *old_rep;
+    apr_array_header_t *orig_str_keys;
+    SVN_ERR(svn_fs_bdb__read_rep(&old_rep, fs, key, trail, pool));
+    if (old_rep->kind == rep_kind_fulltext)
+      {
+        const char *str_key = old_rep->contents.fulltext.string_key;
+        orig_str_keys = apr_array_make(pool, 1, sizeof(str_key));
+        APR_ARRAY_PUSH(orig_str_keys, const char *) = str_key;
+      }
+    else if (old_rep->kind == rep_kind_delta)
+      SVN_ERR(delta_string_keys(&orig_str_keys, old_rep, pool));
+    else /* unknown kind */
+      return UNKNOWN_NODE_KIND(key);
+    SVN_ERR(delete_strings(orig_str_keys, fs, trail, pool));
+  }
+#endif
+
+  return SVN_NO_ERROR;
+}

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.h
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.h?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.h (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/reps-strings.h Wed Dec  9 16:33:28 2009
@@ -168,6 +168,15 @@
                                       trail_t *trail,
                                       apr_pool_t *pool);
 
+/* Obliterate KEY's data by creating a new rep by constructing an
+   empty delta from PRED_KEY's data.  KEY's data remains in the
+   database in case some other key's data is derived from it. */
+svn_error_t *svn_fs_base__rep_obliterate(svn_fs_t *fs,
+                                         const char *key,
+                                         const char *pred_key,
+                                         trail_t *trail,
+                                         apr_pool_t *pool);
+
 
 
 #ifdef __cplusplus

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.c Wed Dec  9 16:33:28 2009
@@ -2910,6 +2910,93 @@
   return deltify_mutable(fs, root, "/", NULL, svn_node_dir, txn_id, pool);
 }
 
+struct txn_obliterate_args
+{
+  const svn_fs_id_t *id;
+  svn_boolean_t has_pred;
+  const svn_fs_id_t *pred_id;
+};
+
+static svn_error_t *
+txn_body_obliterate(void *baton, trail_t *trail)
+{
+  struct txn_obliterate_args *args = baton;
+  dag_node_t *node, *pred_node;
+
+  SVN_ERR(svn_fs_base__dag_get_node(&node, trail->fs, args->id,
+                                    trail, trail->pool));
+  if (args->has_pred)
+    {
+      SVN_ERR(svn_fs_base__dag_get_node(&pred_node, trail->fs, args->pred_id,
+                                        trail, trail->pool));
+    }
+  else
+    {
+      pred_node = NULL;
+    }
+
+  SVN_ERR(svn_fs_base__dag_obliterate(node, pred_node, trail, trail->pool));
+
+  return SVN_NO_ERROR;
+}
+
+svn_error_t *
+svn_fs_base__obliterate(svn_fs_t *fs,
+                        const char *path,
+                        svn_revnum_t revision,
+                        apr_pool_t *pool)
+{
+  svn_fs_root_t *root;
+  const char *txn_id;
+  struct rev_get_txn_id_args get_txn_args;
+  const svn_fs_id_t *id;
+  svn_node_kind_t kind;
+  struct txn_pred_count_args pred_count_args;
+  struct txn_obliterate_args oblit_args;
+
+  SVN_ERR(svn_fs_base__revision_root(&root, fs, revision, pool));
+  get_txn_args.txn_id = &txn_id;
+  get_txn_args.revision = revision;
+  SVN_ERR(svn_fs_base__retry_txn(fs, txn_body_rev_get_txn_id, &get_txn_args,
+                                 FALSE, pool));
+
+  SVN_ERR(base_node_id(&id, root, path, pool));
+  if (strcmp(svn_fs_base__id_txn_id(id), txn_id))
+    return svn_error_createf(SVN_ERR_FS_NOT_MUTABLE, NULL,
+                             _("Unexpected immutable node at '%s'"), path);
+
+  SVN_ERR(base_check_path(&kind, root, path, pool));
+  if (kind != svn_node_file)
+    return svn_error_createf(SVN_ERR_FS_NOT_FILE, NULL,
+                             _("Cannot obliterate '%s' as it is not a file"),
+                             path);
+
+  pred_count_args.id = id;
+  SVN_ERR(svn_fs_base__retry_txn(fs, txn_body_pred_count, &pred_count_args,
+                                 FALSE, pool));
+
+  if (pred_count_args.pred_count > 0)
+    {
+      struct txn_pred_id_args pred_id_args;
+
+      pred_id_args.id = id;
+      pred_id_args.pool = pool;
+      SVN_ERR(svn_fs_base__retry_txn(fs, txn_body_pred_id, &pred_id_args,
+                                     FALSE, pool));
+
+      oblit_args.has_pred = TRUE;
+      oblit_args.pred_id = pred_id_args.pred_id;
+    }
+  else
+    {
+      oblit_args.has_pred = FALSE;
+    }
+  oblit_args.id = id;
+
+  return svn_fs_base__retry_txn(fs, txn_body_obliterate, &oblit_args,
+                                TRUE, pool);
+}
+
 
 
 /* Modifying directories */

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.h
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.h?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.h (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_base/tree.h Wed Dec  9 16:33:28 2009
@@ -40,6 +40,9 @@
 svn_error_t *svn_fs_base__deltify(svn_fs_t *fs, svn_revnum_t rev,
                                   apr_pool_t *pool);
 
+svn_error_t *svn_fs_base__obliterate(svn_fs_t *fs, const char *path,
+                                     svn_revnum_t rev, apr_pool_t *pool);
+
 svn_error_t *svn_fs_base__commit_txn(const char **conflict_p,
                                      svn_revnum_t *new_rev, svn_fs_txn_t *txn,
                                      apr_pool_t *pool);

Modified: subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_fs/fs.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_fs/fs.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_fs/fs.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/libsvn_fs_fs/fs.c Wed Dec  9 16:33:28 2009
@@ -155,6 +155,7 @@
   svn_fs_fs__purge_txn,
   svn_fs_fs__list_transactions,
   svn_fs_fs__deltify,
+  NULL, /* ### FIXME */
   svn_fs_fs__lock,
   svn_fs_fs__generate_lock_token,
   svn_fs_fs__unlock,

Modified: subversion/branches/obliterate-like-deltify/subversion/tests/libsvn_fs/fs-test.c
URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/tests/libsvn_fs/fs-test.c?rev=888863&r1=888862&r2=888863&view=diff
==============================================================================
--- subversion/branches/obliterate-like-deltify/subversion/tests/libsvn_fs/fs-test.c (original)
+++ subversion/branches/obliterate-like-deltify/subversion/tests/libsvn_fs/fs-test.c Wed Dec  9 16:33:28 2009
@@ -4749,6 +4749,126 @@
                           "Feature and test are still under development");
 }
 
+
+static svn_error_t *
+obliterate_2(const svn_test_opts_t *opts, apr_pool_t *pool)
+{
+  apr_pool_t *subpool = svn_pool_create(pool);
+  svn_fs_t *fs;
+  svn_fs_txn_t *txn;
+  svn_fs_root_t *txn_root, *rev_root;
+  svn_revnum_t youngest_rev = 0;
+
+  SVN_ERR(svn_test__create_fs(&fs, "test-repo-obliterate-2", opts, pool));
+
+  /* Revision 1 */
+  SVN_ERR(svn_fs_begin_txn(&txn, fs, youngest_rev, subpool));
+  SVN_ERR(svn_fs_txn_root(&txn_root, txn, subpool));
+  SVN_ERR(svn_fs_make_dir(txn_root, "A", subpool));
+  SVN_ERR(svn_fs_make_file(txn_root, "A/foo", subpool));
+  SVN_ERR(svn_test__set_file_contents(txn_root, "A/foo", "1\n", subpool));
+  SVN_ERR(svn_fs_commit_txn(NULL, &youngest_rev, txn, subpool));
+  svn_pool_clear(subpool);
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "1\n" }
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, youngest_rev, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+
+  /* Revision 2 */
+  SVN_ERR(svn_fs_begin_txn(&txn, fs, youngest_rev, subpool));
+  SVN_ERR(svn_fs_txn_root(&txn_root, txn, subpool));
+  SVN_ERR(svn_test__set_file_contents(txn_root, "A/foo", "2\n", subpool));
+  SVN_ERR(svn_fs_commit_txn(NULL, &youngest_rev, txn, subpool));
+  svn_pool_clear(subpool);
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "2\n" }
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, youngest_rev, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+
+  /* Revision 3 */
+  SVN_ERR(svn_fs_begin_txn(&txn, fs, youngest_rev, subpool));
+  SVN_ERR(svn_fs_txn_root(&txn_root, txn, subpool));
+  SVN_ERR(svn_fs_revision_root(&rev_root, fs, youngest_rev, subpool));
+  SVN_ERR(svn_fs_copy(rev_root, "A", txn_root, "B", pool));
+  SVN_ERR(svn_fs_commit_txn(NULL, &youngest_rev, txn, subpool));
+  svn_pool_clear(subpool);
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "2\n" },
+      { "B", 0 },
+      { "B/foo", "2\n" },
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, youngest_rev, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+
+  /* Revision 4 */
+  SVN_ERR(svn_fs_begin_txn(&txn, fs, youngest_rev, subpool));
+  SVN_ERR(svn_fs_txn_root(&txn_root, txn, subpool));
+  SVN_ERR(svn_test__set_file_contents(txn_root, "A/foo", "3\n", subpool));
+  SVN_ERR(svn_fs_commit_txn(NULL, &youngest_rev, txn, subpool));
+  svn_pool_clear(subpool);
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "3\n" },
+      { "B", 0 },
+      { "B/foo", "2\n" },
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, youngest_rev, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+
+  /* Obliterate A/foo@2 affects both A/foo and B/foo */
+  SVN_ERR(svn_fs_obliterate(fs, "A/foo", 2, subpool));
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "1\n" },
+      { "B", 0 },
+      { "B/foo", "1\n" },
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, 2, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected, 2,
+                                    subpool));
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, 3, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+  {
+    svn_test__tree_entry_t expected[] = {
+      { "A", 0 },
+      { "A/foo", "3\n" },
+      { "B", 0 },
+      { "B/foo", "1\n" },
+    };
+    SVN_ERR(svn_fs_revision_root(&rev_root, fs, 4, subpool));
+    SVN_ERR(svn_test__validate_tree(rev_root, expected,
+                                    sizeof(expected)/sizeof(expected[0]),
+                                    subpool));
+  }
+
+  return SVN_NO_ERROR;
+}
+
 /* ------------------------------------------------------------------------ */
 
 /* The test table.  */
@@ -4830,5 +4950,7 @@
                        "create and modify small file"),
     SVN_TEST_OPTS_WIMP(obliterate_1,
                        "obliterate 1", "obliterate is in development"),
+    SVN_TEST_OPTS_PASS(obliterate_2,
+                       "obliterate experiment"),
     SVN_TEST_NULL
   };



Re: svn commit: r888863 - in /subversion/branches/obliterate-like-deltify: ./ subversion/include/ subversion/libsvn_fs/ subversion/libsvn_fs_base/ subversion/libsvn_fs_fs/ subversion/tests/libsvn_fs/

Posted by Philip Martin <ph...@wandisco.com>.
Blair Zajac <bl...@orcaware.com> writes:

>> +/** Obliterate the text associated with @revision of @a path in @a fs,
>> + * where @a path must be a file.  The text is replaced with the text
>> + * of the previous version of the file.
>
> I haven't been following the obliterate discussion closely, but how
> does this work if you just committed the file so there isn't a
> previous version of the file?  The documentation should state that
> there, me thinks.

The code has comments about that in svn_fs_base__rep_obliterate.  I
think I can construct a delta from an empty file, or perhaps just an
empty fulltext.

-- 
Philip

Re: svn commit: r888863 - in /subversion/branches/obliterate-like-deltify: ./ subversion/include/ subversion/libsvn_fs/ subversion/libsvn_fs_base/ subversion/libsvn_fs_fs/ subversion/tests/libsvn_fs/

Posted by Blair Zajac <bl...@orcaware.com>.
On Dec 9, 2009, at 8:33 AM, philip@apache.org wrote:

> Author: philip
> Date: Wed Dec  9 16:33:28 2009
> New Revision: 888863
> 

> 
> Modified: subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h
> URL: http://svn.apache.org/viewvc/subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h?rev=888863&r1=888862&r2=888863&view=diff
> ==============================================================================
> --- subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h (original)
> +++ subversion/branches/obliterate-like-deltify/subversion/include/svn_fs.h Wed Dec  9 16:33:28 2009
> @@ -1840,6 +1840,16 @@
>                         svn_revnum_t revision,
>                         apr_pool_t *pool);
> 
> +/** Obliterate the text associated with @revision of @a path in @a fs,
> + * where @a path must be a file.  The text is replaced with the text
> + * of the previous version of the file.

I haven't been following the obliterate discussion closely, but how does this work if you just committed the file so there isn't a previous version of the file?  The documentation should state that there, me thinks.

Blair