You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@subversion.apache.org by rh...@apache.org on 2012/06/30 17:34:03 UTC

svn commit: r1355747 - in /subversion/trunk/subversion/libsvn_wc: update_editor.c wc_db.c

Author: rhuijben
Date: Sat Jun 30 15:34:02 2012
New Revision: 1355747

URL: http://svn.apache.org/viewvc?rev=1355747&view=rev
Log:
Make the update editor install tree conflicts via the new conflict skels.

* subversion/libsvn_wc/update_editor.c
  (dir_baton): Use svn_skel_t* for tree_conflicts.
  (mark_directory_edited): Install as conflict.
  (create_tree_conflict,
   check_tree_conflict): Create a tree conflict skel.

  (delete_entry): Collect tree conflict as skel. Install with operation.
    Delay notification until we installed the operation and/or tree conflict.

  (add_directory): Collect as skel. Rebuild skel if necessary for replacement.
  (open_directory): Collect as skel.
  (close_directory): Use existing tree conflict skel if one exists instead of
    potentially overwriting the tree conflict once we store the skel.
    Avoid setting operation data twice, until we fixed all callers to delay
    until here.

  (add_file): Collect as skel. Rebuild skel if necessary for replacement.
  (open_file): Collect as skel.
  (close_file): Use existing tree conflict skel if one exists instead of
    potentially overwriting the tree conflict once we store the skel.
    Avoid setting operation data twice, until we fixed all callers to delay
    until here.

* subversion/libsvn_wc/wc_db.c
  (insert_base_node,
   insert_working_node
   set_actual_props): Install wq items and conflicts last, to allow the current
     fallback code to read the final db state on installing the conflict.
     (And to avoid doing useless work when a db transaction might fail).

Modified:
    subversion/trunk/subversion/libsvn_wc/update_editor.c
    subversion/trunk/subversion/libsvn_wc/wc_db.c

Modified: subversion/trunk/subversion/libsvn_wc/update_editor.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_wc/update_editor.c?rev=1355747&r1=1355746&r2=1355747&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_wc/update_editor.c (original)
+++ subversion/trunk/subversion/libsvn_wc/update_editor.c Sat Jun 30 15:34:02 2012
@@ -329,7 +329,7 @@ struct dir_baton
   const char *changed_author;
 
   /* If not NULL, contains a mapping of const char* basenames of children that
-     have been deleted to their svn_wc_conflict_description2_t* tree conflicts.
+     have been deleted to their svn_skel_t* tree conflicts.
      We store this hash to allow replacements to continue under a just
      installed tree conflict.
 
@@ -359,7 +359,7 @@ struct dir_baton
   svn_boolean_t edited;
 
   /* The tree conflict to install once the node is really edited */
-  svn_wc_conflict_description2_t *edit_conflict;
+  svn_skel_t *edit_conflict;
 
   /* The bump information for this directory. */
   struct bump_dir_info *bump_info;
@@ -781,7 +781,7 @@ struct file_baton
   svn_boolean_t edited;
 
   /* The tree conflict to install once the node is really edited */
-  svn_wc_conflict_description2_t *edit_conflict;
+  svn_skel_t *edit_conflict;
 };
 
 
@@ -873,10 +873,10 @@ mark_directory_edited(struct dir_baton *
   if (db->edit_conflict)
     {
       /* We have a (delayed) tree conflict to install */
-      SVN_ERR(svn_wc__db_op_set_tree_conflict(db->edit_baton->db,
-                                              db->local_abspath,
-                                              db->edit_conflict,
-                                              scratch_pool));
+      SVN_ERR(svn_wc__db_op_mark_conflict(db->edit_baton->db,
+                                          db->local_abspath,
+                                          db->edit_conflict, NULL,
+                                          scratch_pool));
 
       do_notification(db->edit_baton, db->local_abspath, svn_node_dir,
                       svn_wc_notify_tree_conflict, scratch_pool);
@@ -901,10 +901,10 @@ mark_file_edited(struct file_baton *fb, 
   if (fb->edit_conflict)
     {
       /* We have a (delayed) tree conflict to install */
-      SVN_ERR(svn_wc__db_op_set_tree_conflict(fb->edit_baton->db,
-                                              fb->local_abspath,
-                                              fb->edit_conflict,
-                                              scratch_pool));
+      SVN_ERR(svn_wc__db_op_mark_conflict(fb->edit_baton->db,
+                                          fb->local_abspath,
+                                          fb->edit_conflict, NULL,
+                                          scratch_pool));
 
       do_notification(fb->edit_baton, fb->local_abspath, svn_node_file,
                       svn_wc_notify_tree_conflict, scratch_pool);
@@ -1300,7 +1300,7 @@ node_has_local_mods(svn_boolean_t *modif
  * check_tree_conflict(), with the slight modification that this function
  * relies on the reason passed in REASON instead of actively looking for one. */
 static svn_error_t *
-create_tree_conflict(svn_wc_conflict_description2_t **pconflict,
+create_tree_conflict(svn_skel_t **pconflict,
                      struct edit_baton *eb,
                      const char *local_abspath,
                      svn_wc_conflict_reason_t reason,
@@ -1465,13 +1465,22 @@ create_tree_conflict(svn_wc_conflict_des
                                                       their_node_kind,
                                                       result_pool);
 
-  *pconflict = svn_wc_conflict_description_create_tree2(
-                   local_abspath, conflict_node_kind,
-                   eb->switch_relpath ?
-                     svn_wc_operation_switch : svn_wc_operation_update,
-                   src_left_version, src_right_version, result_pool);
-  (*pconflict)->action = action;
-  (*pconflict)->reason = reason;
+  *pconflict = svn_wc__conflict_skel_create(result_pool);
+
+  SVN_ERR(svn_wc__conflict_skel_add_tree_conflict(*pconflict,
+                                                  eb->db, local_abspath,
+                                                  reason,
+                                                  action,
+                                                  result_pool, scratch_pool));
+
+  if (eb->switch_relpath)
+    SVN_ERR(svn_wc__conflict_skel_set_op_switch(*pconflict,
+                                                src_left_version,
+                                                result_pool, scratch_pool));
+  else
+    SVN_ERR(svn_wc__conflict_skel_set_op_update(*pconflict,
+                                                src_left_version,
+                                                result_pool, scratch_pool));
 
   return SVN_NO_ERROR;
 }
@@ -1505,7 +1514,7 @@ create_tree_conflict(svn_wc_conflict_des
  * SCRACTH_POOl.
  */
 static svn_error_t *
-check_tree_conflict(svn_wc_conflict_description2_t **pconflict,
+check_tree_conflict(svn_skel_t **pconflict,
                     struct edit_baton *eb,
                     const char *local_abspath,
                     svn_wc__db_status_t working_status,
@@ -1789,7 +1798,7 @@ delete_entry(const char *path,
   svn_boolean_t conflicted;
   svn_boolean_t have_base;
   svn_boolean_t have_work;
-  svn_wc_conflict_description2_t *tree_conflict = NULL;
+  svn_skel_t *tree_conflict = NULL;
   svn_skel_t *work_item = NULL;
   svn_wc__db_status_t status;
   svn_wc__db_status_t base_status;
@@ -1909,6 +1918,7 @@ delete_entry(const char *path,
 
   if (tree_conflict != NULL)
     {
+      svn_wc_conflict_reason_t reason;
       /* When we raise a tree conflict on a node, we don't want to mark the
        * node as skipped, to allow a replacement to continue doing at least
        * a bit of its work (possibly adding a not present node, for the
@@ -1919,15 +1929,12 @@ delete_entry(const char *path,
       apr_hash_set(pb->deletion_conflicts, apr_pstrdup(pb->pool, base),
                    APR_HASH_KEY_STRING, tree_conflict);
 
-      SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db,
-                                              local_abspath,
-                                              tree_conflict,
-                                              scratch_pool));
-
-      do_notification(eb, local_abspath, svn_node_unknown,
-                      svn_wc_notify_tree_conflict, scratch_pool);
+      SVN_ERR(svn_wc__conflict_read_tree_conflict(&reason, NULL,
+                                                  eb->db, local_abspath,
+                                                  tree_conflict,
+                                                  scratch_pool, scratch_pool));
 
-      if (tree_conflict->reason == svn_wc_conflict_reason_edited)
+      if (reason == svn_wc_conflict_reason_edited)
         {
           /* The item exists locally and has some sort of local mod.
            * It no longer exists in the repository at its target URL@REV.
@@ -1941,11 +1948,10 @@ delete_entry(const char *path,
           /* Fall through to remove the BASE_NODEs properly, with potentially
              keeping a not-present marker */
         }
-      else if (tree_conflict->reason == svn_wc_conflict_reason_deleted
-               || tree_conflict->reason == svn_wc_conflict_reason_moved_away
-               || tree_conflict->reason ==
-                     svn_wc_conflict_reason_moved_away_and_edited
-               || tree_conflict->reason == svn_wc_conflict_reason_replaced)
+      else if (reason == svn_wc_conflict_reason_deleted
+               || reason == svn_wc_conflict_reason_moved_away
+               || reason == svn_wc_conflict_reason_moved_away_and_edited
+               || reason == svn_wc_conflict_reason_replaced)
         {
           /* The item does not exist locally because it was already shadowed.
            * We must complete the deletion, leaving the tree conflict info
@@ -2002,15 +2008,25 @@ delete_entry(const char *path,
       eb->target_deleted = TRUE;
     }
 
-  SVN_ERR(svn_wc__db_wq_add(eb->db, pb->local_abspath, work_item,
-                                scratch_pool));
+  if (tree_conflict)
+  {
+    SVN_ERR(svn_wc__db_op_mark_conflict(eb->db, local_abspath,
+                                        tree_conflict, work_item,
+                                        scratch_pool));
+  }
+  else
+    SVN_ERR(svn_wc__db_wq_add(eb->db, pb->local_abspath, work_item,
+                              scratch_pool));
 
   SVN_ERR(svn_wc__wq_run(eb->db, pb->local_abspath,
                          eb->cancel_func, eb->cancel_baton,
                          scratch_pool));
 
-  /* Notify. (If tree_conflict, we've already notified.) */
-  if (tree_conflict == NULL)
+  /* Notify. */
+  if (tree_conflict)
+    do_notification(eb, local_abspath, svn_node_unknown,
+                    svn_wc_notify_tree_conflict, scratch_pool);
+  else
     {
       svn_wc_notify_action_t action = svn_wc_notify_update_delete;
       svn_node_kind_t node_kind;
@@ -2049,7 +2065,7 @@ add_directory(const char *path,
   svn_kind_t wc_kind;
   svn_boolean_t conflicted;
   svn_boolean_t versioned_locally_and_present;
-  svn_wc_conflict_description2_t *tree_conflict = NULL;
+  svn_skel_t *tree_conflict = NULL;
   svn_error_t *err;
 
   SVN_ERR_ASSERT(! (copyfrom_path || SVN_IS_VALID_REVNUM(copyfrom_rev)));
@@ -2177,15 +2193,37 @@ add_directory(const char *path,
 
       if (tree_conflict)
         {
+          svn_wc_conflict_reason_t reason;
           /* So this deletion wasn't just a deletion, it is actually a
-             replacement. Luckily we still have the conflict so we can
-             just update it. */
+             replacement. Let's install a better tree conflict. */
+
+          /* ### Should store the conflict in DB to allow reinstalling
+             ### with theoretically more data in close_directory() */
+
+          SVN_ERR(svn_wc__conflict_read_tree_conflict(&reason, NULL,
+                                                      eb->db,
+                                                      db->local_abspath,
+                                                      tree_conflict,
+                                                      db->pool, db->pool));
+
+          tree_conflict = svn_wc__conflict_skel_create(db->pool);
 
-          /* ### What else should we update? */
-          tree_conflict->action = svn_wc_conflict_action_replace;
+          SVN_ERR(svn_wc__conflict_skel_add_tree_conflict(
+                                        tree_conflict,
+                                        eb->db, db->local_abspath,
+                                        reason, svn_wc_conflict_action_replace,
+                                        db->pool, db->pool));
+
+          /* Node does not pre-exist so no original location */
+          if (eb->switch_relpath)
+            SVN_ERR(svn_wc__conflict_skel_set_op_switch(tree_conflict, NULL,
+                                                        db->pool, db->pool));
+          else
+            SVN_ERR(svn_wc__conflict_skel_set_op_update(tree_conflict, NULL,
+                                                        db->pool, db->pool));
 
-          SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db, db->local_abspath,
-                                                  tree_conflict, pool));
+          SVN_ERR(svn_wc__db_op_mark_conflict(eb->db, db->local_abspath,
+                                              tree_conflict, NULL, db->pool));
 
           /* And now stop checking for conflicts here and just perform
              a shadowed update */
@@ -2344,8 +2382,9 @@ add_directory(const char *path,
 
   if (tree_conflict != NULL)
     {
-      SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db, db->local_abspath,
-                                              tree_conflict, pool));
+      SVN_ERR(svn_wc__db_op_mark_conflict(eb->db, db->local_abspath,
+                                          tree_conflict, NULL,
+                                          db->pool));
 
       db->already_notified = TRUE;
 
@@ -2403,7 +2442,7 @@ open_directory(const char *path,
   struct edit_baton *eb = pb->edit_baton;
   svn_boolean_t have_work;
   svn_boolean_t conflicted;
-  svn_wc_conflict_description2_t *tree_conflict = NULL;
+  svn_skel_t *tree_conflict = NULL;
   svn_wc__db_status_t status, base_status;
   svn_kind_t wc_kind;
   svn_error_t *err;
@@ -2514,14 +2553,18 @@ open_directory(const char *path,
   /* Remember the roots of any locally deleted trees. */
   if (tree_conflict != NULL)
     {
+      svn_wc_conflict_reason_t reason;
       db->edit_conflict = tree_conflict;
       /* Other modifications wouldn't be a tree conflict */
-      SVN_ERR_ASSERT(
-                tree_conflict->reason == svn_wc_conflict_reason_deleted ||
-                tree_conflict->reason == svn_wc_conflict_reason_moved_away ||
-                tree_conflict->reason ==
-                  svn_wc_conflict_reason_moved_away_and_edited ||
-                tree_conflict->reason == svn_wc_conflict_reason_replaced);
+
+      SVN_ERR(svn_wc__conflict_read_tree_conflict(&reason, NULL,
+                                                  eb->db, db->local_abspath,
+                                                  tree_conflict,
+                                                  db->pool, db->pool));
+      SVN_ERR_ASSERT(reason == svn_wc_conflict_reason_deleted
+                     || reason == svn_wc_conflict_reason_moved_away
+                     || reason == svn_wc_conflict_reason_moved_away_and_edited
+                     || reason == svn_wc_conflict_reason_replaced);
 
       /* Continue updating BASE */
       db->shadowed = TRUE;
@@ -2612,6 +2655,9 @@ close_directory(void *dir_baton,
       return SVN_NO_ERROR;
     }
 
+  if (db->edited)
+    conflict_skel = db->edit_conflict;
+
   SVN_ERR(svn_categorize_props(db->propchanges, &entry_prop_changes,
                                &dav_prop_changes, &regular_prop_changes, pool));
 
@@ -2924,7 +2970,16 @@ close_directory(void *dir_baton,
       if (conflict_skel)
         {
           svn_skel_t *work_item;
-          if (eb->switch_relpath)
+          svn_boolean_t completed;
+
+          SVN_ERR(svn_wc__conflict_skel_is_complete(&completed,
+                                                    conflict_skel));
+
+          if (completed)
+            {
+              /* Avoid assertion */
+            }
+          else if (eb->switch_relpath)
             SVN_ERR(svn_wc__conflict_skel_set_op_switch(
                         conflict_skel,
                         db->adding_dir
@@ -3205,7 +3260,7 @@ add_file(const char *path,
   apr_pool_t *scratch_pool;
   svn_boolean_t conflicted = FALSE;
   svn_boolean_t versioned_locally_and_present = FALSE;
-  svn_wc_conflict_description2_t *tree_conflict = NULL;
+  svn_skel_t *tree_conflict = NULL;
   svn_error_t *err = SVN_NO_ERROR;
 
   SVN_ERR_ASSERT(! (copyfrom_path || SVN_IS_VALID_REVNUM(copyfrom_rev)));
@@ -3319,15 +3374,37 @@ add_file(const char *path,
 
       if (tree_conflict)
         {
+          svn_wc_conflict_reason_t reason;
           /* So this deletion wasn't just a deletion, it is actually a
-             replacement. Luckily we still have the conflict so we can
-             just update it. */
+             replacement. Let's install a better tree conflict. */
 
-          /* ### What else should we update? */
-          tree_conflict->action = svn_wc_conflict_action_replace;
+          /* ### Should store the conflict in DB to allow reinstalling
+             ### with theoretically more data in close_directory() */
 
-          SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db, fb->local_abspath,
-                                                  tree_conflict, pool));
+          SVN_ERR(svn_wc__conflict_read_tree_conflict(&reason, NULL,
+                                                      eb->db,
+                                                      fb->local_abspath,
+                                                      tree_conflict,
+                                                      fb->pool, fb->pool));
+
+          tree_conflict = svn_wc__conflict_skel_create(fb->pool);
+
+          SVN_ERR(svn_wc__conflict_skel_add_tree_conflict(
+                                        tree_conflict,
+                                        eb->db, fb->local_abspath,
+                                        reason, svn_wc_conflict_action_replace,
+                                        fb->pool, fb->pool));
+
+          /* Node does not pre-exist so no original location */
+          if (eb->switch_relpath)
+            SVN_ERR(svn_wc__conflict_skel_set_op_switch(tree_conflict, NULL,
+                                                        fb->pool, fb->pool));
+          else
+            SVN_ERR(svn_wc__conflict_skel_set_op_update(tree_conflict, NULL,
+                                                        fb->pool, fb->pool));
+
+          SVN_ERR(svn_wc__db_op_mark_conflict(eb->db, fb->local_abspath,
+                                              tree_conflict, NULL, fb->pool));
 
           /* And now stop checking for conflicts here and just perform
              a shadowed update */
@@ -3466,10 +3543,10 @@ add_file(const char *path,
 
   if (tree_conflict != NULL)
     {
-      SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db,
-                                              fb->local_abspath,
-                                              tree_conflict,
-                                              scratch_pool));
+      SVN_ERR(svn_wc__db_op_mark_conflict(eb->db,
+                                          fb->local_abspath,
+                                          tree_conflict, NULL,
+                                          scratch_pool));
 
       fb->already_notified = TRUE;
       do_notification(eb, fb->local_abspath, svn_node_file,
@@ -3506,7 +3583,7 @@ open_file(const char *path,
   svn_boolean_t have_work;
   svn_wc__db_status_t status;
   svn_kind_t wc_kind;
-  svn_wc_conflict_description2_t *tree_conflict = NULL;
+  svn_skel_t *tree_conflict = NULL;
 
   /* the file_pool can stick around for a *long* time, so we want to use
      a subpool for any temporary allocations. */
@@ -3602,15 +3679,18 @@ open_file(const char *path,
   /* Is this path the victim of a newly-discovered tree conflict? */
   if (tree_conflict != NULL)
     {
+      svn_wc_conflict_reason_t reason;
       fb->edit_conflict = tree_conflict;
-
       /* Other modifications wouldn't be a tree conflict */
-      SVN_ERR_ASSERT(
-                tree_conflict->reason == svn_wc_conflict_reason_deleted ||
-                tree_conflict->reason == svn_wc_conflict_reason_moved_away ||
-                tree_conflict->reason ==
-                  svn_wc_conflict_reason_moved_away_and_edited ||
-                tree_conflict->reason == svn_wc_conflict_reason_replaced);
+
+      SVN_ERR(svn_wc__conflict_read_tree_conflict(&reason, NULL,
+                                                  eb->db, fb->local_abspath,
+                                                  tree_conflict,
+                                                  fb->pool, fb->pool));
+      SVN_ERR_ASSERT(reason == svn_wc_conflict_reason_deleted
+                     || reason == svn_wc_conflict_reason_moved_away
+                     || reason == svn_wc_conflict_reason_moved_away_and_edited
+                     || reason == svn_wc_conflict_reason_replaced);
 
       /* Continue updating BASE */
       fb->shadowed = TRUE;
@@ -4174,6 +4254,9 @@ close_file(void *file_baton,
       return SVN_NO_ERROR;
     }
 
+  if (fb->edited)
+    conflict_skel = fb->edit_conflict;
+
   if (expected_md5_digest)
     SVN_ERR(svn_checksum_parse_hex(&expected_md5_checksum, svn_checksum_md5,
                                    expected_md5_digest, scratch_pool));
@@ -4322,7 +4405,7 @@ close_file(void *file_baton,
 
       if (local_is_link != incoming_is_link)
         {
-          svn_wc_conflict_description2_t *tree_conflict = NULL;
+          svn_skel_t *tree_conflict = NULL;
 
           fb->shadowed = TRUE;
           fb->obstruction_found = TRUE;
@@ -4334,10 +4417,10 @@ close_file(void *file_baton,
                                        svn_wc_conflict_action_add,
                                        svn_node_file, fb->new_relpath,
                                        scratch_pool, scratch_pool));
-          SVN_ERR(svn_wc__db_op_set_tree_conflict(eb->db,
-                                                  fb->local_abspath,
-                                                  tree_conflict,
-                                                  scratch_pool));
+          SVN_ERR(svn_wc__db_op_mark_conflict(eb->db,
+                                              fb->local_abspath,
+                                              tree_conflict, NULL,
+                                              scratch_pool));
 
           fb->already_notified = TRUE;
           do_notification(eb, fb->local_abspath, svn_node_unknown,
@@ -4529,8 +4612,16 @@ close_file(void *file_baton,
   if (conflict_skel)
     {
       svn_skel_t *work_item;
+      svn_boolean_t completed;
 
-      if (eb->switch_relpath)
+      SVN_ERR(svn_wc__conflict_skel_is_complete(&completed,
+                                                    conflict_skel));
+
+      if (completed)
+        {
+          /* Avoid assertion */
+        }
+      else if (eb->switch_relpath)
         SVN_ERR(svn_wc__conflict_skel_set_op_switch(
                     conflict_skel,
                     fb->adding_file

Modified: subversion/trunk/subversion/libsvn_wc/wc_db.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_wc/wc_db.c?rev=1355747&r1=1355746&r2=1355747&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_wc/wc_db.c (original)
+++ subversion/trunk/subversion/libsvn_wc/wc_db.c Sat Jun 30 15:34:02 2012
@@ -780,8 +780,6 @@ insert_base_node(void *baton,
         }
       SVN_ERR(svn_sqlite__reset(stmt));
     }
-  if (pibb->conflict)
-    SVN_ERR(mark_conflict(wcroot, local_relpath, pibb->conflict, scratch_pool));
 
   SVN_ERR(svn_sqlite__get_statement(&stmt, wcroot->sdb, STMT_INSERT_NODE));
   SVN_ERR(svn_sqlite__bindf(stmt, "isdsisr"
@@ -897,6 +895,8 @@ insert_base_node(void *baton,
     }
 
   SVN_ERR(add_work_items(wcroot->sdb, pibb->work_items, scratch_pool));
+  if (pibb->conflict)
+    SVN_ERR(mark_conflict(wcroot, local_relpath, pibb->conflict, scratch_pool));
 
   return SVN_NO_ERROR;
 }
@@ -1084,11 +1084,6 @@ insert_working_node(void *baton,
       SVN_ERR(svn_sqlite__step_done(stmt));
     }
 
-  if (piwb->conflict)
-    SVN_ERR(mark_conflict(wcroot, local_relpath, piwb->conflict,
-                          scratch_pool));
-  SVN_ERR(add_work_items(wcroot->sdb, piwb->work_items, scratch_pool));
-
   if (piwb->not_present_op_depth > 0
       && piwb->not_present_op_depth < piwb->op_depth)
     {
@@ -1110,6 +1105,11 @@ insert_working_node(void *baton,
       SVN_ERR(svn_sqlite__step_done(stmt));
     }
 
+  SVN_ERR(add_work_items(wcroot->sdb, piwb->work_items, scratch_pool));
+  if (piwb->conflict)
+    SVN_ERR(mark_conflict(wcroot, local_relpath, piwb->conflict,
+                          scratch_pool));
+
   return SVN_NO_ERROR;
 }
 
@@ -4749,12 +4749,6 @@ set_props_txn(void *baton,
   struct set_props_baton_t *spb = baton;
   apr_hash_t *pristine_props;
 
-  /* First order of business: insert all the work items.  */
-  SVN_ERR(add_work_items(wcroot->sdb, spb->work_items, scratch_pool));
-
-  if (spb->conflict)
-    SVN_ERR(mark_conflict(wcroot, local_relpath, spb->conflict, scratch_pool));
-
   /* Check if the props are modified. If no changes, then wipe out the
      ACTUAL props.  PRISTINE_PROPS==NULL means that any
      ACTUAL props are okay as provided, so go ahead and set them.  */
@@ -4781,6 +4775,11 @@ set_props_txn(void *baton,
       SVN_ERR(db_record_fileinfo(&rb, wcroot, local_relpath, scratch_pool));
     }
 
+  /* And finally.  */
+  SVN_ERR(add_work_items(wcroot->sdb, spb->work_items, scratch_pool));
+  if (spb->conflict)
+    SVN_ERR(mark_conflict(wcroot, local_relpath, spb->conflict, scratch_pool));
+
   return SVN_NO_ERROR;
 }
 
@@ -5284,11 +5283,12 @@ mark_conflict(svn_wc__db_wcroot_t *wcroo
                                                         scratch_pool));
         }
 
-      /*if (tree_conflict)
+      if (tree_conflict)
         {
           svn_wc_conflict_description2_t *desc;
           svn_wc_conflict_version_t *v1;
           svn_wc_conflict_version_t *v2;
+          svn_node_kind_t tc_kind;
 
           SVN_ERR(svn_wc__conflict_read_tree_conflict(&local_change,
                                                       &incoming_change,
@@ -5305,16 +5305,83 @@ mark_conflict(svn_wc__db_wcroot_t *wcroo
                     ? APR_ARRAY_IDX(locations, 1, svn_wc_conflict_version_t *)
                     : NULL;
 
-          desc = svn_wc_conflict_description_create_tree2(
-                                local_abspath,
-                                svn_node_unknown,
-                                operation,
-                                v1, v2,
-                                scratch_pool);
+          if (incoming_change != svn_wc_conflict_action_delete
+              && (operation == svn_wc_operation_update
+                  || operation == svn_wc_operation_switch))
+            {
+              svn_wc__db_status_t status;
+              svn_revnum_t revision;
+              const char *repos_relpath;
+              apr_int64_t repos_id;
+              svn_kind_t kind;
+              svn_error_t *err;
+
+              /* ### Theoretically we should just fetch the BASE information
+                     here. This code might need tweaks until all tree conflicts
+                     are installed in the proper state */
+
+              SVN_ERR_ASSERT(v2 == NULL); /* Not set for update and switch */
+
+              /* With an update or switch we have to fetch the second location
+                 for a tree conflict from WORKING. (For text or prop from BASE)
+               */
+              err = base_get_info(&status, &kind, &revision,
+                                  &repos_relpath, &repos_id, NULL, NULL, NULL,
+                                  NULL, NULL, NULL, NULL, NULL, NULL,
+                                  wcroot, local_relpath,
+                                  scratch_pool, scratch_pool);
+
+              if (err)
+                {
+                  if (err && err->apr_err != SVN_ERR_WC_PATH_NOT_FOUND)
+                    return svn_error_trace(err);
+
+                  svn_error_clear(err);
+                  /* Ignore BASE */
+
+                  tc_kind = svn_node_file; /* Avoid assertion */
+                }
+              else if (repos_relpath)
+                {
+                  const char *repos_root_url;
+                  const char *repos_uuid;
+
+                  SVN_ERR(fetch_repos_info(&repos_root_url, &repos_uuid,
+                                           wcroot->sdb, repos_id,
+                                           scratch_pool));
+
+                  v2 = svn_wc_conflict_version_create2(repos_root_url,
+                                                       repos_uuid,
+                                                       repos_relpath,
+                                                       revision,
+                                                svn__node_kind_from_kind(kind),
+                                                       scratch_pool);
+                  tc_kind = svn__node_kind_from_kind(kind);
+                }
+              else
+                tc_kind = svn_node_file; /* Avoid assertion */
+            }
+          else
+            {
+              if (v2)
+                tc_kind = v2->node_kind;
+              else if (v1)
+                tc_kind = v1->node_kind;
+              else
+                tc_kind = svn_node_file; /* Avoid assertion */
+            }
+
+          desc = svn_wc_conflict_description_create_tree2(local_abspath,
+                                                          tc_kind,
+                                                          operation,
+                                                          v1, v2,
+                                                          scratch_pool);
+          desc->reason = local_change;
+          desc->action = incoming_change;
 
           SVN_ERR(temp_op_set_tree_conflict(desc, wcroot, local_relpath,
                                             scratch_pool));
-        }*/
+        }
       SVN_ERR(svn_wc__db_close(db));
     }
 #else