places/bookmark_sync/
engine.rs

1/* This Source Code Form is subject to the terms of the Mozilla Public
2 * License, v. 2.0. If a copy of the MPL was not distributed with this
3 * file, You can obtain one at http://mozilla.org/MPL/2.0/. */
4
5use super::incoming::IncomingApplicator;
6use super::record::{
7    BookmarkItemRecord, BookmarkRecord, BookmarkRecordId, FolderRecord, QueryRecord,
8    SeparatorRecord,
9};
10use super::{SyncedBookmarkKind, SyncedBookmarkValidity};
11use crate::db::{GlobalChangeCounterTracker, PlacesDb, SharedPlacesDb};
12use crate::error::*;
13use crate::frecency::{calculate_frecency, DEFAULT_FRECENCY_SETTINGS};
14use crate::storage::{
15    bookmarks::{
16        bookmark_sync::{create_synced_bookmark_roots, reset},
17        BookmarkRootGuid,
18    },
19    delete_pending_temp_tables, get_meta, put_meta,
20};
21use crate::types::{BookmarkType, SyncStatus, UnknownFields};
22use dogear::{
23    self, AbortSignal, CompletionOps, Content, Item, MergedRoot, TelemetryEvent, Tree, UploadItem,
24    UploadTombstone,
25};
26use interrupt_support::SqlInterruptScope;
27use rusqlite::ErrorCode;
28use rusqlite::Row;
29use sql_support::ConnExt;
30use std::cell::RefCell;
31use std::collections::HashMap;
32use std::fmt;
33use std::sync::Arc;
34use sync15::bso::{IncomingBso, OutgoingBso};
35use sync15::engine::{CollSyncIds, CollectionRequest, EngineSyncAssociation, SyncEngine};
36use sync15::{telemetry, CollectionName, ServerTimestamp};
37use sync_guid::Guid as SyncGuid;
38use types::Timestamp;
39pub const LAST_SYNC_META_KEY: &str = "bookmarks_last_sync_time";
40// Note that all engines in this crate should use a *different* meta key
41// for the global sync ID, because engines are reset individually.
42pub const GLOBAL_SYNCID_META_KEY: &str = "bookmarks_global_sync_id";
43pub const COLLECTION_SYNCID_META_KEY: &str = "bookmarks_sync_id";
44pub const COLLECTION_NAME: &str = "bookmarks";
45
46/// The maximum number of URLs for which to recalculate frecencies at once.
47/// This is a trade-off between write efficiency and transaction time: higher
48/// maximums mean fewer write statements, but longer transactions, possibly
49/// blocking writes from other connections.
50const MAX_FRECENCIES_TO_RECALCULATE_PER_CHUNK: usize = 400;
51
52/// Adapts an interruptee to a Dogear abort signal.
53struct MergeInterruptee<'a>(&'a SqlInterruptScope);
54
55impl AbortSignal for MergeInterruptee<'_> {
56    #[inline]
57    fn aborted(&self) -> bool {
58        self.0.was_interrupted()
59    }
60}
61
62fn stage_incoming(
63    db: &PlacesDb,
64    scope: &SqlInterruptScope,
65    inbound: Vec<IncomingBso>,
66    incoming_telemetry: &mut telemetry::EngineIncoming,
67) -> Result<()> {
68    let mut tx = db.begin_transaction()?;
69
70    let applicator = IncomingApplicator::new(db);
71
72    for incoming in inbound {
73        applicator.apply_bso(incoming)?;
74        incoming_telemetry.applied(1);
75        if tx.should_commit() {
76            // Trigger frecency updates for all new origins.
77            debug!("Updating origins for new synced URLs since last commit");
78            delete_pending_temp_tables(db)?;
79        }
80        tx.maybe_commit()?;
81        scope.err_if_interrupted()?;
82    }
83
84    debug!("Updating origins for new synced URLs in last chunk");
85    delete_pending_temp_tables(db)?;
86
87    tx.commit()?;
88    Ok(())
89}
90
91fn db_has_changes(db: &PlacesDb) -> Result<bool> {
92    // In the first subquery, we check incoming items with needsMerge = true
93    // except the tombstones who don't correspond to any local bookmark because
94    // we don't store them yet, hence never "merged" (see bug 1343103).
95    let sql = format!(
96        "SELECT
97            EXISTS (
98                SELECT 1
99                FROM moz_bookmarks_synced v
100                LEFT JOIN moz_bookmarks b ON v.guid = b.guid
101                WHERE v.needsMerge AND
102                (NOT v.isDeleted OR b.guid NOT NULL)
103            ) OR EXISTS (
104                WITH RECURSIVE
105                {}
106                SELECT 1
107                FROM localItems
108                WHERE syncChangeCounter > 0
109            ) OR EXISTS (
110                SELECT 1
111                FROM moz_bookmarks_deleted
112            )
113         AS hasChanges",
114        LocalItemsFragment("localItems")
115    );
116    Ok(db
117        .try_query_row(
118            &sql,
119            [],
120            |row| -> rusqlite::Result<_> { row.get::<_, bool>(0) },
121            false,
122        )?
123        .unwrap_or(false))
124}
125
126/// Builds a temporary table with the merge states of all nodes in the merged
127/// tree, then updates the local tree to match the merged tree.
128///
129/// Conceptually, we examine the merge state of each item, and either leave the
130/// item unchanged, upload the local side, apply the remote side, or apply and
131/// then reupload the remote side with a new structure.
132fn update_local_items_in_places(
133    db: &PlacesDb,
134    scope: &SqlInterruptScope,
135    now: Timestamp,
136    ops: &CompletionOps<'_>,
137) -> Result<()> {
138    // Build a table of new and updated items.
139    debug!("Staging apply remote item ops");
140    sql_support::each_sized_chunk(
141        &ops.apply_remote_items,
142        sql_support::default_max_variable_number() / 3,
143        |chunk, _| -> Result<()> {
144            // CTEs in `WITH` clauses aren't indexed, so this query needs a
145            // full table scan on `ops`. But that's okay; a separate temp
146            // table for ops would also need a full scan. Note that we need
147            // both the local _and_ remote GUIDs here, because we haven't
148            // changed the local GUIDs yet.
149            let sql = format!(
150                "WITH ops(mergedGuid, localGuid, remoteGuid, remoteType,
151                          level) AS (
152                     VALUES {ops}
153                 )
154                 INSERT INTO itemsToApply(mergedGuid, localId, remoteId,
155                                          remoteGuid, newLevel, newKind,
156                                          localDateAdded, remoteDateAdded,
157                                          lastModified, oldTitle, newTitle,
158                                          oldPlaceId, newPlaceId,
159                                          newKeyword)
160                 SELECT n.mergedGuid, b.id, v.id,
161                        v.guid, n.level, n.remoteType,
162                        b.dateAdded, v.dateAdded,
163                        MAX(v.dateAdded, {now}), b.title, v.title,
164                        b.fk, v.placeId,
165                        v.keyword
166                 FROM ops n
167                 JOIN moz_bookmarks_synced v ON v.guid = n.remoteGuid
168                 LEFT JOIN moz_bookmarks b ON b.guid = n.localGuid",
169                ops = sql_support::repeat_display(chunk.len(), ",", |index, f| {
170                    let op = &chunk[index];
171                    write!(
172                        f,
173                        "(?, ?, ?, {}, {})",
174                        SyncedBookmarkKind::from(op.remote_node().kind) as u8,
175                        op.level
176                    )
177                }),
178                now = now,
179            );
180
181            // We can't avoid allocating here, since we're binding four
182            // parameters per descendant. Rust's `SliceConcatExt::concat`
183            // is semantically equivalent, but requires a second allocation,
184            // which we _can_ avoid by writing this out.
185            let mut params = Vec::with_capacity(chunk.len() * 3);
186            for op in chunk.iter() {
187                scope.err_if_interrupted()?;
188
189                let merged_guid = op.merged_node.guid.as_str();
190                params.push(Some(merged_guid));
191
192                let local_guid = op
193                    .merged_node
194                    .merge_state
195                    .local_node()
196                    .map(|node| node.guid.as_str());
197                params.push(local_guid);
198
199                let remote_guid = op.remote_node().guid.as_str();
200                params.push(Some(remote_guid));
201            }
202
203            db.execute(&sql, rusqlite::params_from_iter(params))?;
204            Ok(())
205        },
206    )?;
207
208    debug!("Staging change GUID ops");
209    sql_support::each_sized_chunk(
210        &ops.change_guids,
211        sql_support::default_max_variable_number() / 2,
212        |chunk, _| -> Result<()> {
213            let sql = format!(
214                "INSERT INTO changeGuidOps(localGuid, mergedGuid,
215                                           syncStatus, level, lastModified)
216                 VALUES {}",
217                sql_support::repeat_display(chunk.len(), ",", |index, f| {
218                    let op = &chunk[index];
219                    // If only the local GUID changed, the item was deduped, so we
220                    // can mark it as syncing. Otherwise, we changed an invalid
221                    // GUID locally or remotely, so we leave its original sync
222                    // status in place until we've uploaded it.
223                    let sync_status = if op.merged_node.remote_guid_changed() {
224                        None
225                    } else {
226                        Some(SyncStatus::Normal as u8)
227                    };
228                    write!(
229                        f,
230                        "(?, ?, {}, {}, {})",
231                        NullableFragment(sync_status),
232                        op.level,
233                        now
234                    )
235                }),
236            );
237
238            let mut params = Vec::with_capacity(chunk.len() * 2);
239            for op in chunk.iter() {
240                scope.err_if_interrupted()?;
241
242                let local_guid = op.local_node().guid.as_str();
243                params.push(local_guid);
244
245                let merged_guid = op.merged_node.guid.as_str();
246                params.push(merged_guid);
247            }
248
249            db.execute(&sql, rusqlite::params_from_iter(params))?;
250            Ok(())
251        },
252    )?;
253
254    debug!("Staging apply new local structure ops");
255    sql_support::each_sized_chunk(
256        &ops.apply_new_local_structure,
257        sql_support::default_max_variable_number() / 2,
258        |chunk, _| -> Result<()> {
259            let sql = format!(
260                "INSERT INTO applyNewLocalStructureOps(
261                     mergedGuid, mergedParentGuid, position, level,
262                     lastModified
263                 )
264                 VALUES {}",
265                sql_support::repeat_display(chunk.len(), ",", |index, f| {
266                    let op = &chunk[index];
267                    write!(f, "(?, ?, {}, {}, {})", op.position, op.level, now)
268                }),
269            );
270
271            let mut params = Vec::with_capacity(chunk.len() * 2);
272            for op in chunk.iter() {
273                scope.err_if_interrupted()?;
274
275                let merged_guid = op.merged_node.guid.as_str();
276                params.push(merged_guid);
277
278                let merged_parent_guid = op.merged_parent_node.guid.as_str();
279                params.push(merged_parent_guid);
280            }
281
282            db.execute(&sql, rusqlite::params_from_iter(params))?;
283            Ok(())
284        },
285    )?;
286
287    debug!("Removing tombstones for revived items");
288    sql_support::each_chunk_mapped(
289        &ops.delete_local_tombstones,
290        |op| op.guid().as_str(),
291        |chunk, _| -> Result<()> {
292            scope.err_if_interrupted()?;
293            db.execute(
294                &format!(
295                    "DELETE FROM moz_bookmarks_deleted
296                     WHERE guid IN ({})",
297                    sql_support::repeat_sql_vars(chunk.len())
298                ),
299                rusqlite::params_from_iter(chunk),
300            )?;
301            Ok(())
302        },
303    )?;
304
305    debug!("Inserting new tombstones for non-syncable and invalid items");
306    sql_support::each_chunk_mapped(
307        &ops.insert_local_tombstones,
308        |op| op.remote_node().guid.as_str().to_owned(),
309        |chunk, _| -> Result<()> {
310            scope.err_if_interrupted()?;
311            db.execute(
312                &format!(
313                    "INSERT INTO moz_bookmarks_deleted(guid, dateRemoved)
314                     VALUES {}",
315                    sql_support::repeat_display(chunk.len(), ",", |_, f| write!(f, "(?, {})", now)),
316                ),
317                rusqlite::params_from_iter(chunk),
318            )?;
319            Ok(())
320        },
321    )?;
322
323    debug!("Flag frecencies for removed bookmark URLs as stale");
324    sql_support::each_chunk_mapped(
325        &ops.delete_local_items,
326        |op| op.local_node().guid.as_str().to_owned(),
327        |chunk, _| -> Result<()> {
328            scope.err_if_interrupted()?;
329            db.execute(
330                &format!(
331                    "REPLACE INTO moz_places_stale_frecencies(
332                         place_id, stale_at
333                     )
334                     SELECT b.fk, {now}
335                     FROM moz_bookmarks b
336                     WHERE b.guid IN ({vars})
337                     AND b.fk NOT NULL",
338                    now = now,
339                    vars = sql_support::repeat_sql_vars(chunk.len())
340                ),
341                rusqlite::params_from_iter(chunk),
342            )?;
343            Ok(())
344        },
345    )?;
346
347    debug!("Removing deleted items from Places");
348    sql_support::each_chunk_mapped(
349        &ops.delete_local_items,
350        |op| op.local_node().guid.as_str().to_owned(),
351        |chunk, _| -> Result<()> {
352            scope.err_if_interrupted()?;
353            db.execute(
354                &format!(
355                    "DELETE FROM moz_bookmarks
356                     WHERE guid IN ({})",
357                    sql_support::repeat_sql_vars(chunk.len())
358                ),
359                rusqlite::params_from_iter(chunk),
360            )?;
361            Ok(())
362        },
363    )?;
364
365    debug!("Changing GUIDs");
366    scope.err_if_interrupted()?;
367    db.execute_batch("DELETE FROM changeGuidOps")?;
368
369    debug!("Applying remote items");
370    apply_remote_items(db, scope, now)?;
371
372    // Fires the `applyNewLocalStructure` trigger.
373    debug!("Applying new local structure");
374    scope.err_if_interrupted()?;
375    db.execute_batch("DELETE FROM applyNewLocalStructureOps")?;
376
377    // Similar to the check in apply_remote_items, however we do a post check
378    // to see if dogear was unable to fix up the issue
379    let orphaned_count: i64 = db.query_row(
380        "WITH RECURSIVE orphans(id) AS (
381           SELECT b.id
382           FROM moz_bookmarks b
383           WHERE b.parent IS NOT NULL
384             AND NOT EXISTS (
385               SELECT 1 FROM moz_bookmarks p WHERE p.id = b.parent
386             )
387           UNION
388           SELECT c.id
389           FROM moz_bookmarks c
390           JOIN orphans o ON c.parent = o.id
391         )
392         SELECT COUNT(*) FROM orphans;",
393        [],
394        |row| row.get(0),
395    )?;
396
397    if orphaned_count > 0 {
398        warn!("Found {} orphaned bookmarks after sync", orphaned_count);
399        error_support::report_error!(
400            "places-sync-bookmarks-orphaned",
401            "found local orphaned bookmarks after we applied new local structure ops: {}",
402            orphaned_count,
403        );
404    }
405
406    debug!("Resetting change counters for items that shouldn't be uploaded");
407    sql_support::each_chunk_mapped(
408        &ops.set_local_merged,
409        |op| op.merged_node.guid.as_str(),
410        |chunk, _| -> Result<()> {
411            scope.err_if_interrupted()?;
412            db.execute(
413                &format!(
414                    "UPDATE moz_bookmarks SET
415                         syncChangeCounter = 0
416                     WHERE guid IN ({})",
417                    sql_support::repeat_sql_vars(chunk.len()),
418                ),
419                rusqlite::params_from_iter(chunk),
420            )?;
421            Ok(())
422        },
423    )?;
424
425    debug!("Bumping change counters for items that should be uploaded");
426    sql_support::each_chunk_mapped(
427        &ops.set_local_unmerged,
428        |op| op.merged_node.guid.as_str(),
429        |chunk, _| -> Result<()> {
430            scope.err_if_interrupted()?;
431            db.execute(
432                &format!(
433                    "UPDATE moz_bookmarks SET
434                         syncChangeCounter = 1
435                     WHERE guid IN ({})",
436                    sql_support::repeat_sql_vars(chunk.len()),
437                ),
438                rusqlite::params_from_iter(chunk),
439            )?;
440            Ok(())
441        },
442    )?;
443
444    debug!("Flagging applied remote items as merged");
445    sql_support::each_chunk_mapped(
446        &ops.set_remote_merged,
447        |op| op.guid().as_str(),
448        |chunk, _| -> Result<()> {
449            scope.err_if_interrupted()?;
450            db.execute(
451                &format!(
452                    "UPDATE moz_bookmarks_synced SET
453                         needsMerge = 0
454                     WHERE guid IN ({})",
455                    sql_support::repeat_sql_vars(chunk.len()),
456                ),
457                rusqlite::params_from_iter(chunk),
458            )?;
459            Ok(())
460        },
461    )?;
462
463    Ok(())
464}
465
466fn apply_remote_items(db: &PlacesDb, scope: &SqlInterruptScope, now: Timestamp) -> Result<()> {
467    // Remove all keywords from old and new URLs, and remove new keywords
468    // from all existing URLs. The `NOT NULL` conditions are important; they
469    // ensure that SQLite uses our partial indexes on `itemsToApply`,
470    // instead of a table scan.
471    debug!("Removing old keywords");
472    scope.err_if_interrupted()?;
473    db.execute_batch(
474        "DELETE FROM moz_keywords
475         WHERE place_id IN (SELECT oldPlaceId FROM itemsToApply
476                            WHERE oldPlaceId NOT NULL) OR
477               place_id IN (SELECT newPlaceId FROM itemsToApply
478                            WHERE newPlaceId NOT NULL) OR
479               keyword IN (SELECT newKeyword FROM itemsToApply
480                           WHERE newKeyword NOT NULL)",
481    )?;
482
483    debug!("Removing old tags");
484    scope.err_if_interrupted()?;
485    db.execute_batch(
486        "DELETE FROM moz_tags_relation
487         WHERE place_id IN (SELECT oldPlaceId FROM itemsToApply
488                            WHERE oldPlaceId NOT NULL) OR
489               place_id IN (SELECT newPlaceId FROM itemsToApply
490                            WHERE newPlaceId NOT NULL)",
491    )?;
492
493    // Due to bug 1935797, we try to add additional logging on what exact
494    // guids are colliding as it could shed light on what's going on
495    debug!("Checking for potential GUID collisions before upserting items");
496    let collision_check_sql = "
497        SELECT ia.localId, ia.mergedGuid, ia.remoteGuid, b.id, b.guid
498        FROM itemsToApply ia
499        JOIN moz_bookmarks b ON ia.mergedGuid = b.guid
500        WHERE (ia.localId IS NULL OR ia.localId != b.id)
501    ";
502
503    let potential_collisions: Vec<(Option<i64>, String, String, i64, String)> = db
504        .prepare(collision_check_sql)?
505        .query_map([], |row| {
506            let ia_local_id: Option<i64> = row.get(0)?;
507            let ia_merged_guid: String = row.get(1)?;
508            let ia_remote_guid: String = row.get(2)?;
509            let bmk_id: i64 = row.get(3)?;
510            let bmk_guid: String = row.get(4)?;
511            Ok((
512                ia_local_id,
513                ia_merged_guid,
514                ia_remote_guid,
515                bmk_id,
516                bmk_guid,
517            ))
518        })?
519        .filter_map(|entry| entry.ok())
520        .collect();
521
522    if !potential_collisions.is_empty() {
523        // Log details about the collisions
524        for (ia_local_id, ia_merged_guid, ia_remote_guid, bmk_id, bmk_guid) in &potential_collisions
525        {
526            error_support::breadcrumb!(
527                "Found GUID collision: ia_localId={:?}, ia_mergedGuid={}, ia_remoteGuid={}, mb_id={}, mb_guid={}",
528                ia_local_id,
529                ia_merged_guid,
530                ia_remote_guid,
531                bmk_id,
532                bmk_guid
533            );
534        }
535    }
536
537    // Due to bug 1935797, we need to check if any users have any
538    // undetected orphaned bookmarks and report them
539    let orphaned_count: i64 = db.query_row(
540        "WITH RECURSIVE orphans(id) AS (
541           SELECT b.id
542           FROM moz_bookmarks b
543           WHERE b.parent IS NOT NULL
544             AND NOT EXISTS (
545               SELECT 1 FROM moz_bookmarks p WHERE p.id = b.parent
546             )
547           UNION
548           SELECT c.id
549           FROM moz_bookmarks c
550           JOIN orphans o ON c.parent = o.id
551         )
552         SELECT COUNT(*) FROM orphans;",
553        [],
554        |row| row.get(0),
555    )?;
556
557    if orphaned_count > 0 {
558        warn!("Found {} orphaned bookmarks during sync", orphaned_count);
559        error_support::breadcrumb!(
560            "places-sync-bookmarks-orphaned: found local orphans before upsert {}",
561            orphaned_count
562        );
563    }
564
565    // Insert and update items, temporarily using the Places root for new
566    // items' parent IDs, and -1 for positions. We'll fix these up later,
567    // when we apply the new local structure. This `INSERT` is a full table
568    // scan on `itemsToApply`. The no-op `WHERE` clause is necessary to
569    // avoid a parsing ambiguity.
570    debug!("Upserting new items");
571    let upsert_sql = format!(
572        "INSERT INTO moz_bookmarks(id, guid, parent,
573                                   position, type, fk, title,
574                                   dateAdded,
575                                   lastModified,
576                                   syncStatus, syncChangeCounter)
577         SELECT localId, mergedGuid, (SELECT id FROM moz_bookmarks
578                                      WHERE guid = '{root_guid}'),
579                -1, {type_fragment}, newPlaceId, newTitle,
580                /* Pick the older of the local and remote date added. We'll
581                   weakly reupload any items with an older local date. */
582                MIN(IFNULL(localDateAdded, remoteDateAdded), remoteDateAdded),
583                /* The last modified date should always be newer than the date
584                   added, so we pick the newer of the two here. */
585                MAX(lastModified, remoteDateAdded),
586                {sync_status}, 0
587         FROM itemsToApply
588         WHERE 1
589         ON CONFLICT(id) DO UPDATE SET
590           title = excluded.title,
591           dateAdded = excluded.dateAdded,
592           lastModified = excluded.lastModified,
593           fk = excluded.fk,
594           syncStatus = {sync_status}
595       /* Due to bug 1935797, we found scenarios where users had bookmarks with GUIDs that matched
596        * incoming records BUT for one reason or another dogear doesn't believe it exists locally
597        * This handles the case where we try to insert a new bookmark with a GUID that already exists,
598        * updating the existing record instead of failing with a constraint violation.
599        * Usually the above conflict will catch most of these scenarios and there's no issue of
600        * any dupes being added here since users that hit this before would've just failed the bookmark sync
601        */
602        ON CONFLICT(guid) DO UPDATE SET
603           title = excluded.title,
604           dateAdded = excluded.dateAdded,
605           lastModified = excluded.lastModified,
606           fk = excluded.fk,
607           syncStatus = {sync_status}",
608        root_guid = BookmarkRootGuid::Root.as_guid().as_str(),
609        type_fragment = ItemTypeFragment("newKind"),
610        sync_status = SyncStatus::Normal as u8,
611    );
612
613    scope.err_if_interrupted()?;
614    let result = db.execute_batch(&upsert_sql);
615
616    // In trying to debug bug 1935797 - relaxing the trigger caused a spike on
617    // guid collisions, we want to report on this during the upsert to see
618    // if we can discern any obvious signs
619    if let Err(rusqlite::Error::SqliteFailure(e, _)) = &result {
620        if e.code == ErrorCode::ConstraintViolation {
621            error_support::report_error!(
622                "places-sync-bookmarks-constraint-violation",
623                "Hit a constraint violation {:?}",
624                result
625            );
626        }
627    }
628    // Return the original result
629    result?;
630
631    debug!("Flagging frecencies for recalculation");
632    scope.err_if_interrupted()?;
633    db.execute_batch(&format!(
634        "REPLACE INTO moz_places_stale_frecencies(place_id, stale_at)
635         SELECT oldPlaceId, {now} FROM itemsToApply
636         WHERE newKind = {bookmark_kind} AND (
637                   oldPlaceId IS NULL <> newPlaceId IS NULL OR
638                   oldPlaceId <> newPlaceId
639               )
640         UNION ALL
641         SELECT newPlaceId, {now} FROM itemsToApply
642         WHERE newKind = {bookmark_kind} AND (
643                   newPlaceId IS NULL <> oldPlaceId IS NULL OR
644                   newPlaceId <> oldPlaceId
645               )",
646        now = now,
647        bookmark_kind = SyncedBookmarkKind::Bookmark as u8,
648    ))?;
649
650    debug!("Inserting new keywords for new URLs");
651    scope.err_if_interrupted()?;
652    db.execute_batch(
653        "INSERT OR IGNORE INTO moz_keywords(keyword, place_id)
654         SELECT newKeyword, newPlaceId
655         FROM itemsToApply
656         WHERE newKeyword NOT NULL",
657    )?;
658
659    debug!("Inserting new tags for new URLs");
660    scope.err_if_interrupted()?;
661    db.execute_batch(
662        "INSERT OR IGNORE INTO moz_tags_relation(tag_id, place_id)
663         SELECT r.tagId, n.newPlaceId
664         FROM itemsToApply n
665         JOIN moz_bookmarks_synced_tag_relation r ON r.itemId = n.remoteId",
666    )?;
667
668    Ok(())
669}
670
671/// Stores a snapshot of all locally changed items in a temporary table for
672/// upload. This is called from within the merge transaction, to ensure that
673/// changes made during the sync don't cause us to upload inconsistent
674/// records.
675///
676/// Conceptually, `itemsToUpload` is a transient "view" of locally changed
677/// items. The local change counter is the persistent record of items that
678/// we need to upload, so, if upload is interrupted or fails, we'll stage
679/// the items again on the next sync.
680fn stage_items_to_upload(
681    db: &PlacesDb,
682    scope: &SqlInterruptScope,
683    upload_items: &[UploadItem<'_>],
684    upload_tombstones: &[UploadTombstone<'_>],
685) -> Result<()> {
686    debug!("Cleaning up staged items left from last sync");
687    scope.err_if_interrupted()?;
688    db.execute_batch("DELETE FROM itemsToUpload")?;
689
690    // Stage remotely changed items with older local creation dates. These are
691    // tracked "weakly": if the upload is interrupted or fails, we won't
692    // reupload the record on the next sync.
693    debug!("Staging items with older local dates added");
694    scope.err_if_interrupted()?;
695    db.execute_batch(&format!(
696        "INSERT OR IGNORE INTO itemsToUpload(id, guid, syncChangeCounter,
697                                             parentGuid, parentTitle, dateAdded,
698                                             kind, title, placeId, url,
699                                             keyword, position)
700         {}
701         JOIN itemsToApply n ON n.mergedGuid = b.guid
702         WHERE n.localDateAdded < n.remoteDateAdded",
703        UploadItemsFragment("b")
704    ))?;
705
706    debug!("Staging remaining locally changed items for upload");
707    sql_support::each_chunk_mapped(
708        upload_items,
709        |op| op.merged_node.guid.as_str(),
710        |chunk, _| -> Result<()> {
711            let sql = format!(
712                "INSERT OR IGNORE INTO itemsToUpload(id, guid, syncChangeCounter,
713                                                  parentGuid, parentTitle,
714                                                  dateAdded, kind, title,
715                                                  placeId, url, keyword,
716                                                  position)
717                 {upload_items_fragment}
718                 WHERE b.guid IN ({vars})",
719                vars = sql_support::repeat_sql_vars(chunk.len()),
720                upload_items_fragment = UploadItemsFragment("b")
721            );
722
723            db.execute(&sql, rusqlite::params_from_iter(chunk))?;
724            Ok(())
725        },
726    )?;
727
728    // Record the child GUIDs of locally changed folders, which we use to
729    // populate the `children` array in the record.
730    debug!("Staging structure to upload");
731    scope.err_if_interrupted()?;
732    db.execute_batch(
733        "INSERT INTO structureToUpload(guid, parentId, position)
734         SELECT b.guid, b.parent, b.position
735         FROM moz_bookmarks b
736         JOIN itemsToUpload o ON o.id = b.parent",
737    )?;
738
739    // Stage tags for outgoing bookmarks.
740    debug!("Staging tags to upload");
741    scope.err_if_interrupted()?;
742    db.execute_batch(
743        "INSERT INTO tagsToUpload(id, tag)
744         SELECT o.id, t.tag
745         FROM itemsToUpload o
746         JOIN moz_tags_relation r ON r.place_id = o.placeId
747         JOIN moz_tags t ON t.id = r.tag_id",
748    )?;
749
750    // Finally, stage tombstones for deleted items.
751    debug!("Staging tombstones to upload");
752    sql_support::each_chunk_mapped(
753        upload_tombstones,
754        |op| op.guid().as_str(),
755        |chunk, _| -> Result<()> {
756            scope.err_if_interrupted()?;
757            db.execute(
758                &format!(
759                    "INSERT OR IGNORE INTO itemsToUpload(
760                     guid, syncChangeCounter, isDeleted
761                 )
762                 VALUES {}",
763                    sql_support::repeat_display(chunk.len(), ",", |_, f| write!(f, "(?, 1, 1)")),
764                ),
765                rusqlite::params_from_iter(chunk),
766            )?;
767            Ok(())
768        },
769    )?;
770
771    Ok(())
772}
773
774/// Inflates Sync records for all staged outgoing items.
775fn fetch_outgoing_records(db: &PlacesDb, scope: &SqlInterruptScope) -> Result<Vec<OutgoingBso>> {
776    let mut changes = Vec::new();
777    let mut child_record_ids_by_local_parent_id: HashMap<i64, Vec<BookmarkRecordId>> =
778        HashMap::new();
779    let mut tags_by_local_id: HashMap<i64, Vec<String>> = HashMap::new();
780
781    let mut stmt = db.prepare(
782        "SELECT parentId, guid FROM structureToUpload
783         ORDER BY parentId, position",
784    )?;
785    let mut results = stmt.query([])?;
786    while let Some(row) = results.next()? {
787        scope.err_if_interrupted()?;
788        let local_parent_id = row.get::<_, i64>("parentId")?;
789        let child_guid = row.get::<_, SyncGuid>("guid")?;
790        let child_record_ids = child_record_ids_by_local_parent_id
791            .entry(local_parent_id)
792            .or_default();
793        child_record_ids.push(child_guid.into());
794    }
795
796    let mut stmt = db.prepare("SELECT id, tag FROM tagsToUpload")?;
797    let mut results = stmt.query([])?;
798    while let Some(row) = results.next()? {
799        scope.err_if_interrupted()?;
800        let local_id = row.get::<_, i64>("id")?;
801        let tag = row.get::<_, String>("tag")?;
802        let tags = tags_by_local_id.entry(local_id).or_default();
803        tags.push(tag);
804    }
805
806    let mut stmt = db.prepare(
807        "SELECT i.id, i.syncChangeCounter, i.guid, i.isDeleted, i.kind, i.keyword,
808                i.url, IFNULL(i.title, '') AS title, i.position, i.parentGuid,
809                IFNULL(i.parentTitle, '') AS parentTitle, i.dateAdded, m.unknownFields
810         FROM itemsToUpload i
811         LEFT JOIN moz_bookmarks_synced m ON i.guid == m.guid
812         ",
813    )?;
814    let mut results = stmt.query([])?;
815    while let Some(row) = results.next()? {
816        scope.err_if_interrupted()?;
817        let guid = row.get::<_, SyncGuid>("guid")?;
818        let is_deleted = row.get::<_, bool>("isDeleted")?;
819        if is_deleted {
820            changes.push(OutgoingBso::new_tombstone(
821                BookmarkRecordId::from(guid).as_guid().clone().into(),
822            ));
823            continue;
824        }
825        let parent_guid = row.get::<_, SyncGuid>("parentGuid")?;
826        let parent_title = row.get::<_, String>("parentTitle")?;
827        let date_added = row.get::<_, i64>("dateAdded")?;
828        let unknown_fields = match row.get::<_, Option<String>>("unknownFields")? {
829            None => UnknownFields::new(),
830            Some(s) => serde_json::from_str(&s)?,
831        };
832        let record: BookmarkItemRecord = match SyncedBookmarkKind::from_u8(row.get("kind")?)? {
833            SyncedBookmarkKind::Bookmark => {
834                let local_id = row.get::<_, i64>("id")?;
835                let title = row.get::<_, String>("title")?;
836                let url = row.get::<_, String>("url")?;
837                BookmarkRecord {
838                    record_id: guid.into(),
839                    parent_record_id: Some(parent_guid.into()),
840                    parent_title: Some(parent_title),
841                    date_added: Some(date_added),
842                    has_dupe: true,
843                    title: Some(title),
844                    url: Some(url),
845                    keyword: row.get::<_, Option<String>>("keyword")?,
846                    tags: tags_by_local_id.remove(&local_id).unwrap_or_default(),
847                    unknown_fields,
848                }
849                .into()
850            }
851            SyncedBookmarkKind::Query => {
852                let title = row.get::<_, String>("title")?;
853                let url = row.get::<_, String>("url")?;
854                QueryRecord {
855                    record_id: guid.into(),
856                    parent_record_id: Some(parent_guid.into()),
857                    parent_title: Some(parent_title),
858                    date_added: Some(date_added),
859                    has_dupe: true,
860                    title: Some(title),
861                    url: Some(url),
862                    tag_folder_name: None,
863                    unknown_fields,
864                }
865                .into()
866            }
867            SyncedBookmarkKind::Folder => {
868                let title = row.get::<_, String>("title")?;
869                let local_id = row.get::<_, i64>("id")?;
870                let children = child_record_ids_by_local_parent_id
871                    .remove(&local_id)
872                    .unwrap_or_default();
873                FolderRecord {
874                    record_id: guid.into(),
875                    parent_record_id: Some(parent_guid.into()),
876                    parent_title: Some(parent_title),
877                    date_added: Some(date_added),
878                    has_dupe: true,
879                    title: Some(title),
880                    children,
881                    unknown_fields,
882                }
883                .into()
884            }
885            SyncedBookmarkKind::Livemark => continue,
886            SyncedBookmarkKind::Separator => {
887                let position = row.get::<_, i64>("position")?;
888                SeparatorRecord {
889                    record_id: guid.into(),
890                    parent_record_id: Some(parent_guid.into()),
891                    parent_title: Some(parent_title),
892                    date_added: Some(date_added),
893                    has_dupe: true,
894                    position: Some(position),
895                    unknown_fields,
896                }
897                .into()
898            }
899        };
900        changes.push(OutgoingBso::from_content_with_id(record)?);
901    }
902
903    Ok(changes)
904}
905
906/// Decrements the change counter, updates the sync status, and cleans up
907/// tombstones for successfully synced items. Sync calls this method at the
908/// end of each bookmark sync.
909fn push_synced_items(
910    db: &PlacesDb,
911    scope: &SqlInterruptScope,
912    uploaded_at: ServerTimestamp,
913    records_synced: Vec<SyncGuid>,
914) -> Result<()> {
915    // Flag all successfully synced records as uploaded. This `UPDATE` fires
916    // the `pushUploadedChanges` trigger, which updates local change
917    // counters and writes the items back to the synced bookmarks table.
918    let mut tx = db.begin_transaction()?;
919
920    let guids = records_synced
921        .into_iter()
922        .map(|id| BookmarkRecordId::from_payload_id(id).into())
923        .collect::<Vec<SyncGuid>>();
924    sql_support::each_chunk(&guids, |chunk, _| -> Result<()> {
925        db.execute(
926            &format!(
927                "UPDATE itemsToUpload SET
928                     uploadedAt = {uploaded_at}
929                     WHERE guid IN ({values})",
930                uploaded_at = uploaded_at.as_millis(),
931                values = sql_support::repeat_sql_values(chunk.len())
932            ),
933            rusqlite::params_from_iter(chunk),
934        )?;
935        tx.maybe_commit()?;
936        scope.err_if_interrupted()?;
937        Ok(())
938    })?;
939
940    // Fast-forward the last sync time, so that we don't download the
941    // records we just uploaded on the next sync.
942    put_meta(db, LAST_SYNC_META_KEY, &uploaded_at.as_millis())?;
943
944    // Clean up.
945    db.execute_batch("DELETE FROM itemsToUpload")?;
946    tx.commit()?;
947
948    Ok(())
949}
950
951pub(crate) fn update_frecencies(db: &PlacesDb, scope: &SqlInterruptScope) -> Result<()> {
952    let mut tx = db.begin_transaction()?;
953
954    let mut frecencies = Vec::with_capacity(MAX_FRECENCIES_TO_RECALCULATE_PER_CHUNK);
955    loop {
956        let sql = format!(
957            "SELECT place_id FROM moz_places_stale_frecencies
958             ORDER BY stale_at DESC
959             LIMIT {}",
960            MAX_FRECENCIES_TO_RECALCULATE_PER_CHUNK
961        );
962        let mut stmt = db.prepare_maybe_cached(&sql, true)?;
963        let mut results = stmt.query([])?;
964        while let Some(row) = results.next()? {
965            let place_id = row.get("place_id")?;
966            // Frecency recalculation runs several statements, so check to
967            // make sure we aren't interrupted before each calculation.
968            scope.err_if_interrupted()?;
969            let frecency =
970                calculate_frecency(db, &DEFAULT_FRECENCY_SETTINGS, place_id, Some(false))?;
971            frecencies.push((place_id, frecency));
972        }
973        if frecencies.is_empty() {
974            break;
975        }
976
977        // Update all frecencies in one fell swoop...
978        db.execute_batch(&format!(
979            "WITH frecencies(id, frecency) AS (
980               VALUES {}
981             )
982             UPDATE moz_places SET
983               frecency = (SELECT frecency FROM frecencies f
984                           WHERE f.id = id)
985             WHERE id IN (SELECT f.id FROM frecencies f)",
986            sql_support::repeat_display(frecencies.len(), ",", |index, f| {
987                let (id, frecency) = frecencies[index];
988                write!(f, "({}, {})", id, frecency)
989            })
990        ))?;
991        tx.maybe_commit()?;
992        scope.err_if_interrupted()?;
993
994        // ...And remove them from the stale table.
995        db.execute_batch(&format!(
996            "DELETE FROM moz_places_stale_frecencies
997             WHERE place_id IN ({})",
998            sql_support::repeat_display(frecencies.len(), ",", |index, f| {
999                let (id, _) = frecencies[index];
1000                write!(f, "{}", id)
1001            })
1002        ))?;
1003        tx.maybe_commit()?;
1004        scope.err_if_interrupted()?;
1005
1006        // If the query returned fewer URLs than the maximum, we're done.
1007        // Otherwise, we might have more, so clear the ones we just
1008        // recalculated and fetch the next chunk.
1009        if frecencies.len() < MAX_FRECENCIES_TO_RECALCULATE_PER_CHUNK {
1010            break;
1011        }
1012        frecencies.clear();
1013    }
1014
1015    tx.commit()?;
1016
1017    Ok(())
1018}
1019
1020// Short-lived struct that's constructed each sync
1021pub struct BookmarksSyncEngine {
1022    db: Arc<SharedPlacesDb>,
1023    // Pub so that it can be used by the PlacesApi methods.  Once all syncing goes through the
1024    // `SyncManager` we should be able to make this private.
1025    pub(crate) scope: SqlInterruptScope,
1026}
1027
1028impl BookmarksSyncEngine {
1029    pub fn new(db: Arc<SharedPlacesDb>) -> Result<Self> {
1030        Ok(Self {
1031            scope: db.begin_interrupt_scope()?,
1032            db,
1033        })
1034    }
1035}
1036
1037impl SyncEngine for BookmarksSyncEngine {
1038    #[inline]
1039    fn collection_name(&self) -> CollectionName {
1040        COLLECTION_NAME.into()
1041    }
1042
1043    fn stage_incoming(
1044        &self,
1045        inbound: Vec<IncomingBso>,
1046        telem: &mut telemetry::Engine,
1047    ) -> anyhow::Result<()> {
1048        let conn = self.db.lock();
1049        // Stage all incoming items.
1050        let mut incoming_telemetry = telemetry::EngineIncoming::new();
1051        stage_incoming(&conn, &self.scope, inbound, &mut incoming_telemetry)?;
1052        telem.incoming(incoming_telemetry);
1053        Ok(())
1054    }
1055
1056    fn apply(
1057        &self,
1058        timestamp: ServerTimestamp,
1059        telem: &mut telemetry::Engine,
1060    ) -> anyhow::Result<Vec<OutgoingBso>> {
1061        let conn = self.db.lock();
1062        // write the timestamp now, so if we are interrupted merging or
1063        // creating outgoing changesets we don't need to re-apply the same
1064        // records.
1065        put_meta(&conn, LAST_SYNC_META_KEY, &timestamp.as_millis())?;
1066
1067        // Merge.
1068        let mut merger = Merger::with_telemetry(&conn, &self.scope, timestamp, telem);
1069        merger.merge()?;
1070        Ok(fetch_outgoing_records(&conn, &self.scope)?)
1071    }
1072
1073    fn set_uploaded(
1074        &self,
1075        new_timestamp: ServerTimestamp,
1076        ids: Vec<SyncGuid>,
1077    ) -> anyhow::Result<()> {
1078        let conn = self.db.lock();
1079        push_synced_items(&conn, &self.scope, new_timestamp, ids)?;
1080        Ok(update_frecencies(&conn, &self.scope)?)
1081    }
1082
1083    fn sync_finished(&self) -> anyhow::Result<()> {
1084        let conn = self.db.lock();
1085        conn.pragma_update(None, "wal_checkpoint", "PASSIVE")?;
1086        Ok(())
1087    }
1088
1089    fn get_collection_request(
1090        &self,
1091        server_timestamp: ServerTimestamp,
1092    ) -> anyhow::Result<Option<CollectionRequest>> {
1093        let conn = self.db.lock();
1094        let since =
1095            ServerTimestamp(get_meta::<i64>(&conn, LAST_SYNC_META_KEY)?.unwrap_or_default());
1096        Ok(if since == server_timestamp {
1097            None
1098        } else {
1099            Some(
1100                CollectionRequest::new(self.collection_name())
1101                    .full()
1102                    .newer_than(since),
1103            )
1104        })
1105    }
1106
1107    fn get_sync_assoc(&self) -> anyhow::Result<EngineSyncAssociation> {
1108        let conn = self.db.lock();
1109        let global = get_meta(&conn, GLOBAL_SYNCID_META_KEY)?;
1110        let coll = get_meta(&conn, COLLECTION_SYNCID_META_KEY)?;
1111        Ok(if let (Some(global), Some(coll)) = (global, coll) {
1112            EngineSyncAssociation::Connected(CollSyncIds { global, coll })
1113        } else {
1114            EngineSyncAssociation::Disconnected
1115        })
1116    }
1117
1118    fn reset(&self, assoc: &EngineSyncAssociation) -> anyhow::Result<()> {
1119        let conn = self.db.lock();
1120        reset(&conn, assoc)?;
1121        Ok(())
1122    }
1123
1124    /// Erases all local items. Unlike `reset`, this keeps all synced items
1125    /// until the next sync, when they will be replaced with tombstones. This
1126    /// also preserves the sync ID and last sync time.
1127    ///
1128    /// Conceptually, the next sync will merge an empty local tree, and a full
1129    /// remote tree.
1130    fn wipe(&self) -> anyhow::Result<()> {
1131        let conn = self.db.lock();
1132        let tx = conn.begin_transaction()?;
1133        let sql = format!(
1134            "INSERT INTO moz_bookmarks_deleted(guid, dateRemoved)
1135             SELECT guid, now()
1136             FROM moz_bookmarks
1137             WHERE guid NOT IN {roots} AND
1138                   syncStatus = {sync_status};
1139
1140             UPDATE moz_bookmarks SET
1141               syncChangeCounter = syncChangeCounter + 1
1142             WHERE guid IN {roots};
1143
1144             DELETE FROM moz_bookmarks
1145             WHERE guid NOT IN {roots};",
1146            roots = RootsFragment(&[
1147                BookmarkRootGuid::Root,
1148                BookmarkRootGuid::Menu,
1149                BookmarkRootGuid::Mobile,
1150                BookmarkRootGuid::Toolbar,
1151                BookmarkRootGuid::Unfiled
1152            ]),
1153            sync_status = SyncStatus::Normal as u8
1154        );
1155        conn.execute_batch(&sql)?;
1156        create_synced_bookmark_roots(&conn)?;
1157        tx.commit()?;
1158        Ok(())
1159    }
1160}
1161
1162#[derive(Default)]
1163struct Driver {
1164    validation: RefCell<telemetry::Validation>,
1165}
1166
1167impl dogear::Driver for Driver {
1168    fn generate_new_guid(&self, _invalid_guid: &dogear::Guid) -> dogear::Result<dogear::Guid> {
1169        Ok(SyncGuid::random().as_str().into())
1170    }
1171
1172    fn record_telemetry_event(&self, event: TelemetryEvent) {
1173        // Record validation telemetry for remote trees.
1174        if let TelemetryEvent::FetchRemoteTree(stats) = event {
1175            self.validation
1176                .borrow_mut()
1177                .problem("orphans", stats.problems.orphans)
1178                .problem("misparentedRoots", stats.problems.misparented_roots)
1179                .problem(
1180                    "multipleParents",
1181                    stats.problems.multiple_parents_by_children,
1182                )
1183                .problem("missingParents", stats.problems.missing_parent_guids)
1184                .problem("nonFolderParents", stats.problems.non_folder_parent_guids)
1185                .problem(
1186                    "parentChildDisagreements",
1187                    stats.problems.parent_child_disagreements,
1188                )
1189                .problem("missingChildren", stats.problems.missing_children);
1190        }
1191    }
1192}
1193
1194// The "merger", which is just a thin wrapper for dogear.
1195pub(crate) struct Merger<'a> {
1196    db: &'a PlacesDb,
1197    scope: &'a SqlInterruptScope,
1198    remote_time: ServerTimestamp,
1199    local_time: Timestamp,
1200    // Used for where the merger is not the one which should be managing the
1201    // transaction, e.g. in the case of bookmarks import. The only impact this has
1202    // is on the `apply()` function. Always false unless the caller explicitly
1203    // turns it on, to avoid accidentally enabling unintentionally.
1204    external_transaction: bool,
1205    telem: Option<&'a mut telemetry::Engine>,
1206    // Allows us to abort applying the result of the merge if the local tree
1207    // changed since we fetched it.
1208    global_change_tracker: GlobalChangeCounterTracker,
1209}
1210
1211impl<'a> Merger<'a> {
1212    #[cfg(test)]
1213    pub(crate) fn new(
1214        db: &'a PlacesDb,
1215        scope: &'a SqlInterruptScope,
1216        remote_time: ServerTimestamp,
1217    ) -> Self {
1218        Self {
1219            db,
1220            scope,
1221            remote_time,
1222            local_time: Timestamp::now(),
1223            external_transaction: false,
1224            telem: None,
1225            global_change_tracker: db.global_bookmark_change_tracker(),
1226        }
1227    }
1228
1229    pub(crate) fn with_telemetry(
1230        db: &'a PlacesDb,
1231        scope: &'a SqlInterruptScope,
1232        remote_time: ServerTimestamp,
1233        telem: &'a mut telemetry::Engine,
1234    ) -> Self {
1235        Self {
1236            db,
1237            scope,
1238            remote_time,
1239            local_time: Timestamp::now(),
1240            external_transaction: false,
1241            telem: Some(telem),
1242            global_change_tracker: db.global_bookmark_change_tracker(),
1243        }
1244    }
1245
1246    #[cfg(test)]
1247    fn with_localtime(
1248        db: &'a PlacesDb,
1249        scope: &'a SqlInterruptScope,
1250        remote_time: ServerTimestamp,
1251        local_time: Timestamp,
1252    ) -> Self {
1253        Self {
1254            db,
1255            scope,
1256            remote_time,
1257            local_time,
1258            external_transaction: false,
1259            telem: None,
1260            global_change_tracker: db.global_bookmark_change_tracker(),
1261        }
1262    }
1263
1264    pub(crate) fn merge(&mut self) -> Result<()> {
1265        use dogear::Store;
1266        if !db_has_changes(self.db)? {
1267            return Ok(());
1268        }
1269        // Merge and stage outgoing items via dogear.
1270        let driver = Driver::default();
1271        self.prepare()?;
1272        let result = self.merge_with_driver(&driver, &MergeInterruptee(self.scope));
1273        debug!("merge completed: {:?}", result);
1274
1275        // Record telemetry in all cases, even if the merge fails.
1276        if let Some(ref mut telem) = self.telem {
1277            telem.validation(driver.validation.into_inner());
1278        }
1279        result
1280    }
1281
1282    /// Prepares synced bookmarks for merging.
1283    fn prepare(&self) -> Result<()> {
1284        // Sync and Fennec associate keywords with bookmarks, and don't sync
1285        // POST data; Rust Places associates them with URLs, and also doesn't
1286        // support POST data; Desktop associates keywords with (URL, POST data)
1287        // pairs, and multiple bookmarks may have the same URL.
1288        //
1289        // When a keyword changes, clients should reupload all bookmarks with
1290        // the affected URL (bug 1328737). Just in case, we flag any synced
1291        // bookmarks that have different keywords for the same URL, or the same
1292        // keyword for different URLs, for reupload.
1293        self.scope.err_if_interrupted()?;
1294        debug!("Flagging bookmarks with mismatched keywords for reupload");
1295        let sql = format!(
1296            "UPDATE moz_bookmarks_synced SET
1297               validity = {reupload}
1298             WHERE validity = {valid} AND (
1299               placeId IN (
1300                 /* Same URL, different keywords. `COUNT` ignores NULLs, so
1301                    we need to count them separately. This handles cases where
1302                    a keyword was removed from one, but not all bookmarks with
1303                    the same URL. */
1304                 SELECT placeId FROM moz_bookmarks_synced
1305                 GROUP BY placeId
1306                 HAVING COUNT(DISTINCT keyword) +
1307                        COUNT(DISTINCT CASE WHEN keyword IS NULL
1308                                       THEN 1 END) > 1
1309               ) OR keyword IN (
1310                 /* Different URLs, same keyword. Bookmarks with keywords but
1311                    without URLs are already invalid, so we don't need to handle
1312                    NULLs here. */
1313                 SELECT keyword FROM moz_bookmarks_synced
1314                 WHERE keyword NOT NULL
1315                 GROUP BY keyword
1316                 HAVING COUNT(DISTINCT placeId) > 1
1317               )
1318             )",
1319            reupload = SyncedBookmarkValidity::Reupload as u8,
1320            valid = SyncedBookmarkValidity::Valid as u8,
1321        );
1322        self.db.execute_batch(&sql)?;
1323
1324        // Like keywords, Sync associates tags with bookmarks, but Places
1325        // associates them with URLs. This means multiple bookmarks with the
1326        // same URL should have the same tags. In practice, different tags for
1327        // bookmarks with the same URL are some of the most common validation
1328        // errors we see.
1329        //
1330        // Unlike keywords, the relationship between URLs and tags in many-many:
1331        // multiple URLs can have the same tag, and a URL can have multiple
1332        // tags. So, to find mismatches, we need to compare the tags for each
1333        // URL with the tags for each item.
1334        //
1335        // We could fetch both lists of tags, sort them, and then compare them.
1336        // But there's a trick here: we're only interested in whether the tags
1337        // _match_, not the tags themselves. So we sum the tag IDs!
1338        //
1339        // This has two advantages: we don't have to sort IDs, since addition is
1340        // commutative, and we can compare two integers much more efficiently
1341        // than two string lists! If a bookmark has mismatched tags, the sum of
1342        // its tag IDs in `tagsByItemId` won't match the sum in `tagsByPlaceId`,
1343        // and we'll flag the item for reupload.
1344        self.scope.err_if_interrupted()?;
1345        debug!("Flagging bookmarks with mismatched tags for reupload");
1346        let sql = format!(
1347            "WITH
1348             tagsByPlaceId(placeId, tagIds) AS (
1349                 /* For multiple bookmarks with the same URL, each group will
1350                    have one tag per bookmark. So, if bookmarks A1, A2, and A3
1351                    have the same URL A with tag T, T will be in the group three
1352                    times. But we only want to count each tag once per URL, so
1353                    we use `SUM(DISTINCT)`. */
1354                 SELECT v.placeId, SUM(DISTINCT t.tagId)
1355                 FROM moz_bookmarks_synced v
1356                 JOIN moz_bookmarks_synced_tag_relation t ON t.itemId = v.id
1357                 WHERE v.placeId NOT NULL
1358                 GROUP BY v.placeId
1359             ),
1360             tagsByItemId(itemId, tagIds) AS (
1361                 /* But here, we can use a plain `SUM`, since we're grouping by
1362                    item ID, and an item can't have duplicate tags thanks to the
1363                    primary key on the relation table. */
1364                 SELECT t.itemId, SUM(t.tagId)
1365                 FROM moz_bookmarks_synced_tag_relation t
1366                 GROUP BY t.itemId
1367             )
1368             UPDATE moz_bookmarks_synced SET
1369                 validity = {reupload}
1370             WHERE validity = {valid} AND id IN (
1371                 SELECT v.id FROM moz_bookmarks_synced v
1372                 JOIN tagsByPlaceId u ON v.placeId = u.placeId
1373                 /* This left join is important: if A1 has tags and A2 doesn't,
1374                    we want to flag A2 for reupload. */
1375                 LEFT JOIN tagsByItemId t ON t.itemId = v.id
1376                 /* Unlike `<>`, `IS NOT` compares NULLs. */
1377                 WHERE t.tagIds IS NOT u.tagIds
1378             )",
1379            reupload = SyncedBookmarkValidity::Reupload as u8,
1380            valid = SyncedBookmarkValidity::Valid as u8,
1381        );
1382        self.db.execute_batch(&sql)?;
1383
1384        Ok(())
1385    }
1386
1387    /// Creates a local tree item from a row in the `localItems` CTE.
1388    fn local_row_to_item(&self, row: &Row<'_>) -> Result<(Item, Option<Content>)> {
1389        let guid = row.get::<_, SyncGuid>("guid")?;
1390        let url_href = row.get::<_, Option<String>>("url")?;
1391        let kind = match row.get::<_, BookmarkType>("type")? {
1392            BookmarkType::Bookmark => match url_href.as_ref() {
1393                Some(u) if u.starts_with("place:") => SyncedBookmarkKind::Query,
1394                _ => SyncedBookmarkKind::Bookmark,
1395            },
1396            BookmarkType::Folder => SyncedBookmarkKind::Folder,
1397            BookmarkType::Separator => SyncedBookmarkKind::Separator,
1398        };
1399        let mut item = Item::new(guid.as_str().into(), kind.into());
1400        // Note that this doesn't account for local clock skew.
1401        let age = self
1402            .local_time
1403            .duration_since(row.get::<_, Timestamp>("localModified")?)
1404            .unwrap_or_default();
1405        item.age = age.as_secs() as i64 * 1000 + i64::from(age.subsec_millis());
1406        item.needs_merge = row.get::<_, u32>("syncChangeCounter")? > 0;
1407
1408        let content = if item.guid == dogear::ROOT_GUID {
1409            None
1410        } else {
1411            match row.get::<_, SyncStatus>("syncStatus")? {
1412                SyncStatus::Normal => None,
1413                _ => match kind {
1414                    SyncedBookmarkKind::Bookmark | SyncedBookmarkKind::Query => {
1415                        let title = row.get::<_, String>("title")?;
1416                        url_href.map(|url_href| Content::Bookmark { title, url_href })
1417                    }
1418                    SyncedBookmarkKind::Folder | SyncedBookmarkKind::Livemark => {
1419                        let title = row.get::<_, String>("title")?;
1420                        Some(Content::Folder { title })
1421                    }
1422                    SyncedBookmarkKind::Separator => Some(Content::Separator),
1423                },
1424            }
1425        };
1426
1427        Ok((item, content))
1428    }
1429
1430    /// Creates a remote tree item from a row in `moz_bookmarks_synced`.
1431    fn remote_row_to_item(&self, row: &Row<'_>) -> Result<(Item, Option<Content>)> {
1432        let guid = row.get::<_, SyncGuid>("guid")?;
1433        let kind = SyncedBookmarkKind::from_u8(row.get("kind")?)?;
1434        let mut item = Item::new(guid.as_str().into(), kind.into());
1435        // note that serverModified in this table is an int with ms, which isn't
1436        // the format of a ServerTimestamp - so we convert it into a number
1437        // of seconds before creating a ServerTimestamp and doing duration_since.
1438        let age = self
1439            .remote_time
1440            .duration_since(ServerTimestamp(row.get::<_, i64>("serverModified")?))
1441            .unwrap_or_default();
1442        item.age = age.as_secs() as i64 * 1000 + i64::from(age.subsec_millis());
1443        item.needs_merge = row.get("needsMerge")?;
1444        item.validity = SyncedBookmarkValidity::from_u8(row.get("validity")?)?.into();
1445
1446        let content = if item.guid == dogear::ROOT_GUID || !item.needs_merge {
1447            None
1448        } else {
1449            match kind {
1450                SyncedBookmarkKind::Bookmark | SyncedBookmarkKind::Query => {
1451                    let title = row.get::<_, String>("title")?;
1452                    let url_href = row.get::<_, Option<String>>("url")?;
1453                    url_href.map(|url_href| Content::Bookmark { title, url_href })
1454                }
1455                SyncedBookmarkKind::Folder | SyncedBookmarkKind::Livemark => {
1456                    let title = row.get::<_, String>("title")?;
1457                    Some(Content::Folder { title })
1458                }
1459                SyncedBookmarkKind::Separator => Some(Content::Separator),
1460            }
1461        };
1462
1463        Ok((item, content))
1464    }
1465}
1466
1467impl dogear::Store for Merger<'_> {
1468    type Ok = ();
1469    type Error = Error;
1470
1471    /// Builds a fully rooted, consistent tree from all local items and
1472    /// tombstones.
1473    fn fetch_local_tree(&self) -> Result<Tree> {
1474        let mut stmt = self.db.prepare(&format!(
1475            "SELECT guid, type, syncChangeCounter, syncStatus,
1476                    lastModified AS localModified,
1477                    NULL AS url
1478             FROM moz_bookmarks
1479             WHERE guid = '{root_guid}'",
1480            root_guid = BookmarkRootGuid::Root.as_guid().as_str(),
1481        ))?;
1482        let mut results = stmt.query([])?;
1483        let mut builder = match results.next()? {
1484            Some(row) => {
1485                let (item, _) = self.local_row_to_item(row)?;
1486                Tree::with_root(item)
1487            }
1488            None => return Err(Error::Corruption(Corruption::InvalidLocalRoots)),
1489        };
1490
1491        // Add items and contents to the builder, keeping track of their
1492        // structure in a separate map. We can't call `p.by_structure(...)`
1493        // after adding the item, because this query might return rows for
1494        // children before their parents. This approach also lets us scan
1495        // `moz_bookmarks` once, using the index on `(b.parent, b.position)`
1496        // to avoid a temp B-tree for the `ORDER BY`.
1497        let mut child_guids_by_parent_guid: HashMap<SyncGuid, Vec<dogear::Guid>> = HashMap::new();
1498        let mut stmt = self.db.prepare(&format!(
1499            "SELECT b.guid, p.guid AS parentGuid, b.type, b.syncChangeCounter,
1500                    b.syncStatus, b.lastModified AS localModified,
1501                    IFNULL(b.title, '') AS title,
1502                    {url_fragment} AS url
1503             FROM moz_bookmarks b
1504             JOIN moz_bookmarks p ON p.id = b.parent
1505             WHERE b.guid <> '{root_guid}'
1506             ORDER BY b.parent, b.position",
1507            url_fragment = UrlOrPlaceIdFragment::PlaceId("b.fk"),
1508            root_guid = BookmarkRootGuid::Root.as_guid().as_str(),
1509        ))?;
1510        let mut results = stmt.query([])?;
1511
1512        while let Some(row) = results.next()? {
1513            self.scope.err_if_interrupted()?;
1514
1515            let (item, content) = self.local_row_to_item(row)?;
1516
1517            let parent_guid = row.get::<_, SyncGuid>("parentGuid")?;
1518            child_guids_by_parent_guid
1519                .entry(parent_guid)
1520                .or_default()
1521                .push(item.guid.clone());
1522
1523            let mut p = builder.item(item)?;
1524            if let Some(content) = content {
1525                p.content(content);
1526            }
1527        }
1528
1529        // At this point, we've added entries for all items to the tree, so
1530        // we can add their structure info.
1531        for (parent_guid, child_guids) in &child_guids_by_parent_guid {
1532            for child_guid in child_guids {
1533                self.scope.err_if_interrupted()?;
1534                builder
1535                    .parent_for(child_guid)
1536                    .by_structure(&parent_guid.as_str().into())?;
1537            }
1538        }
1539
1540        // Note tombstones for locally deleted items.
1541        let mut stmt = self.db.prepare("SELECT guid FROM moz_bookmarks_deleted")?;
1542        let mut results = stmt.query([])?;
1543        while let Some(row) = results.next()? {
1544            self.scope.err_if_interrupted()?;
1545            let guid = row.get::<_, SyncGuid>("guid")?;
1546            builder.deletion(guid.as_str().into());
1547        }
1548
1549        let tree = Tree::try_from(builder)?;
1550        Ok(tree)
1551    }
1552
1553    /// Builds a fully rooted tree from all synced items and tombstones.
1554    fn fetch_remote_tree(&self) -> Result<Tree> {
1555        // Unlike the local tree, items and structure are stored separately, so
1556        // we use three separate statements to fetch the root, its descendants,
1557        // and their structure.
1558        let sql = format!(
1559            "SELECT guid, serverModified, kind, needsMerge, validity
1560             FROM moz_bookmarks_synced
1561             WHERE NOT isDeleted AND
1562                   guid = '{root_guid}'",
1563            root_guid = BookmarkRootGuid::Root.as_guid().as_str()
1564        );
1565        let mut builder = self
1566            .db
1567            .try_query_row(
1568                &sql,
1569                [],
1570                |row| -> Result<_> {
1571                    let (root, _) = self.remote_row_to_item(row)?;
1572                    Ok(Tree::with_root(root))
1573                },
1574                false,
1575            )?
1576            .ok_or(Error::Corruption(Corruption::InvalidSyncedRoots))?;
1577        builder.reparent_orphans_to(&dogear::UNFILED_GUID);
1578
1579        let sql = format!(
1580            "SELECT v.guid, v.parentGuid, v.serverModified, v.kind,
1581                    IFNULL(v.title, '') AS title, v.needsMerge, v.validity,
1582                    v.isDeleted, {url_fragment} AS url
1583             FROM moz_bookmarks_synced v
1584             WHERE v.guid <> '{root_guid}'
1585             ORDER BY v.guid",
1586            url_fragment = UrlOrPlaceIdFragment::PlaceId("v.placeId"),
1587            root_guid = BookmarkRootGuid::Root.as_guid().as_str()
1588        );
1589        let mut stmt = self.db.prepare(&sql)?;
1590        let mut results = stmt.query([])?;
1591        while let Some(row) = results.next()? {
1592            self.scope.err_if_interrupted()?;
1593
1594            let is_deleted = row.get::<_, bool>("isDeleted")?;
1595            if is_deleted {
1596                let needs_merge = row.get::<_, bool>("needsMerge")?;
1597                if !needs_merge {
1598                    // Ignore already-merged tombstones. These aren't persisted
1599                    // locally, so merging them is a no-op.
1600                    continue;
1601                }
1602                let guid = row.get::<_, SyncGuid>("guid")?;
1603                builder.deletion(guid.as_str().into());
1604            } else {
1605                let (item, content) = self.remote_row_to_item(row)?;
1606                let mut p = builder.item(item)?;
1607                if let Some(content) = content {
1608                    p.content(content);
1609                }
1610                if let Some(parent_guid) = row.get::<_, Option<SyncGuid>>("parentGuid")? {
1611                    p.by_parent_guid(parent_guid.as_str().into())?;
1612                }
1613            }
1614        }
1615
1616        let sql = format!(
1617            "SELECT guid, parentGuid FROM moz_bookmarks_synced_structure
1618             WHERE guid <> '{root_guid}'
1619             ORDER BY parentGuid, position",
1620            root_guid = BookmarkRootGuid::Root.as_guid().as_str()
1621        );
1622        let mut stmt = self.db.prepare(&sql)?;
1623        let mut results = stmt.query([])?;
1624        while let Some(row) = results.next()? {
1625            self.scope.err_if_interrupted()?;
1626            let guid = row.get::<_, SyncGuid>("guid")?;
1627            let parent_guid = row.get::<_, SyncGuid>("parentGuid")?;
1628            builder
1629                .parent_for(&guid.as_str().into())
1630                .by_children(&parent_guid.as_str().into())?;
1631        }
1632
1633        let tree = Tree::try_from(builder)?;
1634        Ok(tree)
1635    }
1636
1637    fn apply(&mut self, root: MergedRoot<'_>) -> Result<()> {
1638        let ops = root.completion_ops_with_signal(&MergeInterruptee(self.scope))?;
1639
1640        if ops.is_empty() {
1641            // If we don't have any items to apply, upload, or delete,
1642            // no need to open a transaction at all.
1643            return Ok(());
1644        }
1645
1646        let tx = if !self.external_transaction {
1647            Some(self.db.begin_transaction()?)
1648        } else {
1649            None
1650        };
1651
1652        // If the local tree has changed since we started the merge, we abort
1653        // in the expectation it will succeed next time.
1654        if self.global_change_tracker.changed() {
1655            info!("Aborting update of local items as local tree changed while merging");
1656            if let Some(tx) = tx {
1657                tx.rollback()?;
1658            }
1659            return Ok(());
1660        }
1661
1662        debug!("Updating local items in Places");
1663        update_local_items_in_places(self.db, self.scope, self.local_time, &ops)?;
1664
1665        debug!("Staging items to upload");
1666        stage_items_to_upload(
1667            self.db,
1668            self.scope,
1669            &ops.upload_items,
1670            &ops.upload_tombstones,
1671        )?;
1672
1673        self.db.execute_batch("DELETE FROM itemsToApply;")?;
1674        if let Some(tx) = tx {
1675            tx.commit()?;
1676        }
1677        Ok(())
1678    }
1679}
1680
1681/// A helper that formats an optional value so that it can be included in a SQL
1682/// statement. `None` values become SQL `NULL`s.
1683struct NullableFragment<T>(Option<T>);
1684
1685impl<T> fmt::Display for NullableFragment<T>
1686where
1687    T: fmt::Display,
1688{
1689    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1690        match &self.0 {
1691            Some(v) => v.fmt(f),
1692            None => write!(f, "NULL"),
1693        }
1694    }
1695}
1696
1697/// A helper that interpolates a SQL `CASE` expression for converting a synced
1698/// item kind to a local item type. The expression evaluates to `NULL` if the
1699/// kind is unknown.
1700struct ItemTypeFragment(&'static str);
1701
1702impl fmt::Display for ItemTypeFragment {
1703    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1704        write!(
1705            f,
1706            "(CASE WHEN {col} IN ({bookmark_kind}, {query_kind})
1707                        THEN {bookmark_type}
1708                   WHEN {col} IN ({folder_kind}, {livemark_kind})
1709                        THEN {folder_type}
1710                   WHEN {col} = {separator_kind}
1711                        THEN {separator_type}
1712              END)",
1713            col = self.0,
1714            bookmark_kind = SyncedBookmarkKind::Bookmark as u8,
1715            query_kind = SyncedBookmarkKind::Query as u8,
1716            bookmark_type = BookmarkType::Bookmark as u8,
1717            folder_kind = SyncedBookmarkKind::Folder as u8,
1718            livemark_kind = SyncedBookmarkKind::Livemark as u8,
1719            folder_type = BookmarkType::Folder as u8,
1720            separator_kind = SyncedBookmarkKind::Separator as u8,
1721            separator_type = BookmarkType::Separator as u8,
1722        )
1723    }
1724}
1725
1726/// Formats a `SELECT` statement for staging local items in the `itemsToUpload`
1727/// table.
1728struct UploadItemsFragment(&'static str);
1729
1730impl fmt::Display for UploadItemsFragment {
1731    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1732        write!(
1733            f,
1734            "SELECT {alias}.id, {alias}.guid, {alias}.syncChangeCounter,
1735                    p.guid AS parentGuid, p.title AS parentTitle,
1736                    {alias}.dateAdded, {kind_fragment} AS kind,
1737                    {alias}.title, h.id AS placeId, h.url,
1738                    (SELECT k.keyword FROM moz_keywords k
1739                     WHERE k.place_id = h.id) AS keyword,
1740                    {alias}.position
1741                FROM moz_bookmarks {alias}
1742                JOIN moz_bookmarks p ON p.id = {alias}.parent
1743                LEFT JOIN moz_places h ON h.id = {alias}.fk",
1744            alias = self.0,
1745            kind_fragment = item_kind_fragment(self.0, "type", UrlOrPlaceIdFragment::Url("h.url")),
1746        )
1747    }
1748}
1749
1750/// A helper that interpolates a named SQL common table expression (CTE) for
1751/// local items. The CTE may be included in a `WITH RECURSIVE` clause.
1752struct LocalItemsFragment<'a>(&'a str);
1753
1754impl fmt::Display for LocalItemsFragment<'_> {
1755    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1756        write!(
1757            f,
1758            "{name}(id, guid, parentId, parentGuid, position, type, title, parentTitle,
1759                    placeId, dateAdded, lastModified, syncChangeCounter, level) AS (
1760             SELECT b.id, b.guid, 0, NULL, b.position, b.type, b.title, NULL,
1761                    b.fk, b.dateAdded, b.lastModified, b.syncChangeCounter, 0
1762             FROM moz_bookmarks b
1763             WHERE b.guid = '{root_guid}'
1764             UNION ALL
1765             SELECT b.id, b.guid, s.id, s.guid, b.position, b.type, b.title, s.title,
1766                    b.fk, b.dateAdded, b.lastModified, b.syncChangeCounter, s.level + 1
1767             FROM moz_bookmarks b
1768             JOIN {name} s ON s.id = b.parent)",
1769            name = self.0,
1770            root_guid = BookmarkRootGuid::Root.as_guid().as_str()
1771        )
1772    }
1773}
1774
1775fn item_kind_fragment(
1776    table_name: &'static str,
1777    type_column_name: &'static str,
1778    url_or_place_id_fragment: UrlOrPlaceIdFragment,
1779) -> ItemKindFragment {
1780    ItemKindFragment {
1781        table_name,
1782        type_column_name,
1783        url_or_place_id_fragment,
1784    }
1785}
1786
1787/// A helper that interpolates a SQL `CASE` expression for converting a local
1788/// item type to a synced item kind. The expression evaluates to `NULL` if the
1789/// type is unknown.
1790struct ItemKindFragment {
1791    /// The name of the Places bookmarks table.
1792    table_name: &'static str,
1793    /// The name of the column containing the Places item type.
1794    type_column_name: &'static str,
1795    /// The column containing the item's URL or Place ID.
1796    url_or_place_id_fragment: UrlOrPlaceIdFragment,
1797}
1798
1799impl fmt::Display for ItemKindFragment {
1800    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1801        write!(
1802            f,
1803            "(CASE {table_name}.{type_column_name}
1804              WHEN {bookmark_type} THEN (
1805                  CASE substr({url}, 1, 6)
1806                  /* Queries are bookmarks with a 'place:' URL scheme. */
1807                  WHEN 'place:' THEN {query_kind}
1808                  ELSE {bookmark_kind}
1809                  END
1810              )
1811              WHEN {folder_type} THEN {folder_kind}
1812              WHEN {separator_type} THEN {separator_kind}
1813              END)",
1814            table_name = self.table_name,
1815            type_column_name = self.type_column_name,
1816            bookmark_type = BookmarkType::Bookmark as u8,
1817            url = self.url_or_place_id_fragment,
1818            query_kind = SyncedBookmarkKind::Query as u8,
1819            bookmark_kind = SyncedBookmarkKind::Bookmark as u8,
1820            folder_type = BookmarkType::Folder as u8,
1821            folder_kind = SyncedBookmarkKind::Folder as u8,
1822            separator_type = BookmarkType::Separator as u8,
1823            separator_kind = SyncedBookmarkKind::Separator as u8,
1824        )
1825    }
1826}
1827
1828/// A helper that interpolates a SQL expression for querying a local item's
1829/// URL. Note that the `&'static str` for each variant specifies the _name of
1830/// the column_ containing the URL or ID, not the URL or ID itself.
1831enum UrlOrPlaceIdFragment {
1832    /// The name of the column containing the URL. This avoids a subquery if
1833    /// a column for the URL already exists in the query.
1834    Url(&'static str),
1835    /// The name of the column containing the Place ID. This writes out a
1836    /// subquery to look up the URL.
1837    PlaceId(&'static str),
1838}
1839
1840impl fmt::Display for UrlOrPlaceIdFragment {
1841    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1842        match self {
1843            UrlOrPlaceIdFragment::Url(s) => write!(f, "{}", s),
1844            UrlOrPlaceIdFragment::PlaceId(s) => {
1845                write!(f, "(SELECT h.url FROM moz_places h WHERE h.id = {})", s)
1846            }
1847        }
1848    }
1849}
1850
1851/// A helper that interpolates a SQL list containing the given bookmark
1852/// root GUIDs.
1853struct RootsFragment<'a>(&'a [BookmarkRootGuid]);
1854
1855impl fmt::Display for RootsFragment<'_> {
1856    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1857        f.write_str("(")?;
1858        for (i, guid) in self.0.iter().enumerate() {
1859            if i != 0 {
1860                f.write_str(",")?;
1861            }
1862            write!(f, "'{}'", guid.as_str())?;
1863        }
1864        f.write_str(")")
1865    }
1866}
1867
1868#[cfg(test)]
1869mod tests {
1870    use super::*;
1871    use crate::api::places_api::{test::new_mem_api, ConnectionType, PlacesApi};
1872    use crate::bookmark_sync::tests::SyncedBookmarkItem;
1873    use crate::db::PlacesDb;
1874    use crate::storage::{
1875        bookmarks::{
1876            get_raw_bookmark, insert_bookmark, update_bookmark, BookmarkPosition,
1877            InsertableBookmark, UpdatableBookmark, USER_CONTENT_ROOTS,
1878        },
1879        history::frecency_stale_at,
1880        tags,
1881    };
1882    use crate::tests::{
1883        assert_json_tree as assert_local_json_tree, insert_json_tree as insert_local_json_tree,
1884    };
1885    use dogear::{Store as DogearStore, Validity};
1886    use rusqlite::{Error as RusqlError, ErrorCode};
1887    use serde_json::{json, Value};
1888    use std::{
1889        borrow::Cow,
1890        time::{Duration, SystemTime},
1891    };
1892    use sync15::bso::{IncomingBso, IncomingKind};
1893    use sync15::engine::CollSyncIds;
1894    use sync_guid::Guid;
1895    use url::Url;
1896
1897    // A helper type to simplify writing table-driven tests with synced items.
1898    struct ExpectedSyncedItem<'a>(SyncGuid, Cow<'a, SyncedBookmarkItem>);
1899
1900    impl<'a> ExpectedSyncedItem<'a> {
1901        fn new(
1902            guid: impl Into<SyncGuid>,
1903            expected: &'a SyncedBookmarkItem,
1904        ) -> ExpectedSyncedItem<'a> {
1905            ExpectedSyncedItem(guid.into(), Cow::Borrowed(expected))
1906        }
1907
1908        fn with_properties(
1909            guid: impl Into<SyncGuid>,
1910            expected: &'a SyncedBookmarkItem,
1911            f: impl FnOnce(&mut SyncedBookmarkItem) -> &mut SyncedBookmarkItem + 'static,
1912        ) -> ExpectedSyncedItem<'a> {
1913            let mut expected = expected.clone();
1914            f(&mut expected);
1915            ExpectedSyncedItem(guid.into(), Cow::Owned(expected))
1916        }
1917
1918        fn check(&self, conn: &PlacesDb) -> Result<()> {
1919            let actual =
1920                SyncedBookmarkItem::get(conn, &self.0)?.expect("Expected synced item should exist");
1921            assert_eq!(&actual, &*self.1);
1922            Ok(())
1923        }
1924    }
1925
1926    fn create_sync_engine(api: &PlacesApi) -> BookmarksSyncEngine {
1927        BookmarksSyncEngine::new(api.get_sync_connection().unwrap()).unwrap()
1928    }
1929
1930    fn engine_apply_incoming(
1931        engine: &BookmarksSyncEngine,
1932        incoming: Vec<IncomingBso>,
1933    ) -> Vec<OutgoingBso> {
1934        let mut telem = telemetry::Engine::new(engine.collection_name());
1935        engine
1936            .stage_incoming(incoming, &mut telem)
1937            .expect("Should stage incoming");
1938        engine
1939            .apply(ServerTimestamp(0), &mut telem)
1940            .expect("Should apply")
1941    }
1942
1943    // Applies the incoming records, and also "finishes" the sync by pretending
1944    // we uploaded the outgoing items and marks them as uploaded.
1945    // Returns the GUIDs of the outgoing items.
1946    fn apply_incoming(
1947        api: &PlacesApi,
1948        remote_time: ServerTimestamp,
1949        records_json: Value,
1950    ) -> Vec<Guid> {
1951        // suck records into the engine.
1952        let engine = create_sync_engine(api);
1953
1954        let incoming = match records_json {
1955            Value::Array(records) => records
1956                .into_iter()
1957                .map(|record| {
1958                    let timestamp = record
1959                        .as_object()
1960                        .and_then(|r| r.get("modified"))
1961                        .map(|v| {
1962                            serde_json::from_value(v.clone())
1963                                .expect("Should deserialize server modified")
1964                        })
1965                        .unwrap_or(remote_time);
1966                    IncomingBso::from_test_content_ts(record, timestamp)
1967                })
1968                .collect(),
1969            Value::Object(ref r) => {
1970                let timestamp = r
1971                    .get("modified")
1972                    .map(|v| {
1973                        serde_json::from_value(v.clone())
1974                            .expect("Should deserialize server modified")
1975                    })
1976                    .unwrap_or(remote_time);
1977                vec![IncomingBso::from_test_content_ts(records_json, timestamp)]
1978            }
1979            _ => panic!("unexpected json value"),
1980        };
1981
1982        engine_apply_incoming(&engine, incoming);
1983
1984        let sync_db = api.get_sync_connection().unwrap();
1985        let syncer = sync_db.lock();
1986        let mut stmt = syncer
1987            .prepare("SELECT guid FROM itemsToUpload")
1988            .expect("Should prepare statement to fetch uploaded GUIDs");
1989        let uploaded_guids: Vec<Guid> = stmt
1990            .query_and_then([], |row| -> rusqlite::Result<_> { row.get::<_, Guid>(0) })
1991            .expect("Should fetch uploaded GUIDs")
1992            .map(std::result::Result::unwrap)
1993            .collect();
1994
1995        push_synced_items(&syncer, &engine.scope, remote_time, uploaded_guids.clone())
1996            .expect("Should push synced changes back to the engine");
1997        uploaded_guids
1998    }
1999
2000    fn assert_incoming_creates_local_tree(
2001        api: &PlacesApi,
2002        records_json: Value,
2003        local_folder: &SyncGuid,
2004        local_tree: Value,
2005    ) {
2006        apply_incoming(api, ServerTimestamp(0), records_json);
2007        assert_local_json_tree(
2008            &api.get_sync_connection().unwrap().lock(),
2009            local_folder,
2010            local_tree,
2011        );
2012    }
2013
2014    #[test]
2015    fn test_fetch_remote_tree() -> Result<()> {
2016        let records = vec![
2017            json!({
2018                "id": "qqVTRWhLBOu3",
2019                "type": "bookmark",
2020                "parentid": "unfiled",
2021                "parentName": "Unfiled Bookmarks",
2022                "dateAdded": 1_381_542_355_843u64,
2023                "title": "The title",
2024                "bmkUri": "https://example.com",
2025                "tags": [],
2026            }),
2027            json!({
2028                "id": "unfiled",
2029                "type": "folder",
2030                "parentid": "places",
2031                "parentName": "",
2032                "dateAdded": 0,
2033                "title": "Unfiled Bookmarks",
2034                "children": ["qqVTRWhLBOu3"],
2035                "tags": [],
2036            }),
2037        ];
2038
2039        let api = new_mem_api();
2040        let db = api.get_sync_connection().unwrap();
2041        let conn = db.lock();
2042
2043        // suck records into the database.
2044        let interrupt_scope = conn.begin_interrupt_scope()?;
2045
2046        let incoming = records
2047            .into_iter()
2048            .map(IncomingBso::from_test_content)
2049            .collect();
2050
2051        stage_incoming(
2052            &conn,
2053            &interrupt_scope,
2054            incoming,
2055            &mut telemetry::EngineIncoming::new(),
2056        )
2057        .expect("Should apply incoming and stage outgoing records");
2058
2059        let merger = Merger::new(&conn, &interrupt_scope, ServerTimestamp(0));
2060
2061        let tree = merger.fetch_remote_tree()?;
2062
2063        // should be each user root, plus the real root, plus the bookmark we added.
2064        assert_eq!(tree.guids().count(), USER_CONTENT_ROOTS.len() + 2);
2065
2066        let node = tree
2067            .node_for_guid(&"qqVTRWhLBOu3".into())
2068            .expect("should exist");
2069        assert!(node.needs_merge);
2070        assert_eq!(node.validity, Validity::Valid);
2071        assert_eq!(node.level(), 2);
2072        assert!(node.is_syncable());
2073
2074        let node = tree
2075            .node_for_guid(&BookmarkRootGuid::Unfiled.as_guid().as_str().into())
2076            .expect("should exist");
2077        assert!(node.needs_merge);
2078        assert_eq!(node.validity, Validity::Valid);
2079        assert_eq!(node.level(), 1);
2080        assert!(node.is_syncable());
2081
2082        let node = tree
2083            .node_for_guid(&BookmarkRootGuid::Menu.as_guid().as_str().into())
2084            .expect("should exist");
2085        assert!(!node.needs_merge);
2086        assert_eq!(node.validity, Validity::Valid);
2087        assert_eq!(node.level(), 1);
2088        assert!(node.is_syncable());
2089
2090        let node = tree
2091            .node_for_guid(&BookmarkRootGuid::Root.as_guid().as_str().into())
2092            .expect("should exist");
2093        assert_eq!(node.validity, Validity::Valid);
2094        assert_eq!(node.level(), 0);
2095        assert!(!node.is_syncable());
2096
2097        // We should have changes.
2098        assert!(db_has_changes(&conn).unwrap());
2099        Ok(())
2100    }
2101
2102    #[test]
2103    fn test_fetch_local_tree() -> Result<()> {
2104        let now = SystemTime::now();
2105        let previously_ts: Timestamp = (now - Duration::new(10, 0)).into();
2106        let api = new_mem_api();
2107        let writer = api.open_connection(ConnectionType::ReadWrite)?;
2108        let sync_db = api.get_sync_connection().unwrap();
2109        let syncer = sync_db.lock();
2110
2111        writer
2112            .execute("UPDATE moz_bookmarks SET syncChangeCounter = 0", [])
2113            .expect("should work");
2114
2115        insert_local_json_tree(
2116            &writer,
2117            json!({
2118                "guid": &BookmarkRootGuid::Unfiled.as_guid(),
2119                "children": [
2120                    {
2121                        "guid": "bookmark1___",
2122                        "title": "the bookmark",
2123                        "url": "https://www.example.com/",
2124                        "last_modified": previously_ts,
2125                        "date_added": previously_ts,
2126                    },
2127                ]
2128            }),
2129        );
2130
2131        let interrupt_scope = syncer.begin_interrupt_scope()?;
2132        let merger =
2133            Merger::with_localtime(&syncer, &interrupt_scope, ServerTimestamp(0), now.into());
2134
2135        let tree = merger.fetch_local_tree()?;
2136
2137        // should be each user root, plus the real root, plus the bookmark we added.
2138        assert_eq!(tree.guids().count(), USER_CONTENT_ROOTS.len() + 2);
2139
2140        let node = tree
2141            .node_for_guid(&"bookmark1___".into())
2142            .expect("should exist");
2143        assert!(node.needs_merge);
2144        assert_eq!(node.level(), 2);
2145        assert!(node.is_syncable());
2146        assert_eq!(node.age, 10000);
2147
2148        let node = tree
2149            .node_for_guid(&BookmarkRootGuid::Unfiled.as_guid().as_str().into())
2150            .expect("should exist");
2151        assert!(node.needs_merge);
2152        assert_eq!(node.level(), 1);
2153        assert!(node.is_syncable());
2154
2155        let node = tree
2156            .node_for_guid(&BookmarkRootGuid::Menu.as_guid().as_str().into())
2157            .expect("should exist");
2158        assert!(!node.needs_merge);
2159        assert_eq!(node.level(), 1);
2160        assert!(node.is_syncable());
2161
2162        let node = tree
2163            .node_for_guid(&BookmarkRootGuid::Root.as_guid().as_str().into())
2164            .expect("should exist");
2165        assert!(!node.needs_merge);
2166        assert_eq!(node.level(), 0);
2167        assert!(!node.is_syncable());
2168        // hard to know the exact age of the root, but we know the max.
2169        let max_dur = SystemTime::now().duration_since(now).unwrap();
2170        let max_age = max_dur.as_secs() as i64 * 1000 + i64::from(max_dur.subsec_millis());
2171        assert!(node.age <= max_age);
2172
2173        // We should have changes.
2174        assert!(db_has_changes(&syncer).unwrap());
2175        Ok(())
2176    }
2177
2178    #[test]
2179    fn test_apply_bookmark() {
2180        let api = new_mem_api();
2181        assert_incoming_creates_local_tree(
2182            &api,
2183            json!([{
2184                "id": "bookmark1___",
2185                "type": "bookmark",
2186                "parentid": "unfiled",
2187                "parentName": "Unfiled Bookmarks",
2188                "dateAdded": 1_381_542_355_843u64,
2189                "title": "Some bookmark",
2190                "bmkUri": "http://example.com",
2191            },
2192            {
2193                "id": "unfiled",
2194                "type": "folder",
2195                "parentid": "places",
2196                "dateAdded": 1_381_542_355_843u64,
2197                "title": "Unfiled",
2198                "children": ["bookmark1___"],
2199            }]),
2200            &BookmarkRootGuid::Unfiled.as_guid(),
2201            json!({"children" : [{"guid": "bookmark1___", "url": "http://example.com"}]}),
2202        );
2203        let reader = api
2204            .open_connection(ConnectionType::ReadOnly)
2205            .expect("Should open read-only connection");
2206        assert!(
2207            frecency_stale_at(&reader, &Url::parse("http://example.com").unwrap())
2208                .expect("Should check stale frecency")
2209                .is_some(),
2210            "Should mark frecency for bookmark URL as stale"
2211        );
2212
2213        let writer = api
2214            .open_connection(ConnectionType::ReadWrite)
2215            .expect("Should open read-write connection");
2216        insert_local_json_tree(
2217            &writer,
2218            json!({
2219                "guid": &BookmarkRootGuid::Menu.as_guid(),
2220                "children": [
2221                    {
2222                        "guid": "bookmark2___",
2223                        "title": "2",
2224                        "url": "http://example.com/2",
2225                    }
2226                ],
2227            }),
2228        );
2229        assert_incoming_creates_local_tree(
2230            &api,
2231            json!([{
2232                "id": "menu",
2233                "type": "folder",
2234                "parentid": "places",
2235                "parentName": "",
2236                "dateAdded": 0,
2237                "title": "menu",
2238                "children": ["bookmark2___"],
2239            }, {
2240                "id": "bookmark2___",
2241                "type": "bookmark",
2242                "parentid": "menu",
2243                "parentName": "menu",
2244                "dateAdded": 1_381_542_355_843u64,
2245                "title": "2",
2246                "bmkUri": "http://example.com/2-remote",
2247            }]),
2248            &BookmarkRootGuid::Menu.as_guid(),
2249            json!({"children" : [{"guid": "bookmark2___", "url": "http://example.com/2-remote"}]}),
2250        );
2251        assert!(
2252            frecency_stale_at(&reader, &Url::parse("http://example.com/2").unwrap())
2253                .expect("Should check stale frecency for old URL")
2254                .is_some(),
2255            "Should mark frecency for old URL as stale"
2256        );
2257        assert!(
2258            frecency_stale_at(&reader, &Url::parse("http://example.com/2-remote").unwrap())
2259                .expect("Should check stale frecency for new URL")
2260                .is_some(),
2261            "Should mark frecency for new URL as stale"
2262        );
2263
2264        let sync_db = api.get_sync_connection().unwrap();
2265        let syncer = sync_db.lock();
2266        let interrupt_scope = syncer.begin_interrupt_scope().unwrap();
2267
2268        update_frecencies(&syncer, &interrupt_scope).expect("Should update frecencies");
2269
2270        assert!(
2271            frecency_stale_at(&reader, &Url::parse("http://example.com").unwrap())
2272                .expect("Should check stale frecency")
2273                .is_none(),
2274            "Should recalculate frecency for first bookmark"
2275        );
2276        assert!(
2277            frecency_stale_at(&reader, &Url::parse("http://example.com/2").unwrap())
2278                .expect("Should check stale frecency for old URL")
2279                .is_none(),
2280            "Should recalculate frecency for old URL"
2281        );
2282        assert!(
2283            frecency_stale_at(&reader, &Url::parse("http://example.com/2-remote").unwrap())
2284                .expect("Should check stale frecency for new URL")
2285                .is_none(),
2286            "Should recalculate frecency for new URL"
2287        );
2288    }
2289
2290    #[test]
2291    fn test_apply_complex_bookmark_tags() -> Result<()> {
2292        let api = new_mem_api();
2293        let writer = api.open_connection(ConnectionType::ReadWrite)?;
2294
2295        // Insert two local bookmarks with the same URL A (so they'll have
2296        // identical tags) and a third with a different URL B, but one same
2297        // tag as A.
2298        let local_bookmarks = vec![
2299            InsertableBookmark {
2300                parent_guid: BookmarkRootGuid::Unfiled.as_guid(),
2301                position: BookmarkPosition::Append,
2302                date_added: None,
2303                last_modified: None,
2304                guid: Some("bookmarkAAA1".into()),
2305                url: Url::parse("http://example.com/a").unwrap(),
2306                title: Some("A1".into()),
2307            }
2308            .into(),
2309            InsertableBookmark {
2310                parent_guid: BookmarkRootGuid::Menu.as_guid(),
2311                position: BookmarkPosition::Append,
2312                date_added: None,
2313                last_modified: None,
2314                guid: Some("bookmarkAAA2".into()),
2315                url: Url::parse("http://example.com/a").unwrap(),
2316                title: Some("A2".into()),
2317            }
2318            .into(),
2319            InsertableBookmark {
2320                parent_guid: BookmarkRootGuid::Unfiled.as_guid(),
2321                position: BookmarkPosition::Append,
2322                date_added: None,
2323                last_modified: None,
2324                guid: Some("bookmarkBBBB".into()),
2325                url: Url::parse("http://example.com/b").unwrap(),
2326                title: Some("B".into()),
2327            }
2328            .into(),
2329        ];
2330        let local_tags = &[
2331            ("http://example.com/a", vec!["one", "two"]),
2332            (
2333                "http://example.com/b",
2334                // Local duplicate tags should be ignored.
2335                vec!["two", "three", "three", "four"],
2336            ),
2337        ];
2338        for bm in local_bookmarks.into_iter() {
2339            insert_bookmark(&writer, bm)?;
2340        }
2341        for (url, tags) in local_tags {
2342            let url = Url::parse(url)?;
2343            for t in tags.iter() {
2344                tags::tag_url(&writer, &url, t)?;
2345            }
2346        }
2347
2348        // Now for some fun server data. Only B, C, and F2 have problems;
2349        // D and E are fine, and shouldn't be reuploaded.
2350        let remote_records = json!([{
2351            // Change B's tags on the server, and duplicate `two` for good
2352            // measure. We should reupload B with only one `two` tag.
2353            "id": "bookmarkBBBB",
2354            "type": "bookmark",
2355            "parentid": "unfiled",
2356            "parentName": "Unfiled",
2357            "dateAdded": 1_381_542_355_843u64,
2358            "title": "B",
2359            "bmkUri": "http://example.com/b",
2360            "tags": ["two", "two", "three", "eight"],
2361        }, {
2362            // C is an example of bad data on the server: bookmarks with the
2363            // same URL should have the same tags, but C1/C2 have different tags
2364            // than C3. We should reupload all of them.
2365            "id": "bookmarkCCC1",
2366            "type": "bookmark",
2367            "parentid": "unfiled",
2368            "parentName": "Unfiled",
2369            "dateAdded": 1_381_542_355_843u64,
2370            "title": "C1",
2371            "bmkUri": "http://example.com/c",
2372            "tags": ["four", "five", "six"],
2373        }, {
2374            "id": "bookmarkCCC2",
2375            "type": "bookmark",
2376            "parentid": "menu",
2377            "parentName": "Menu",
2378            "dateAdded": 1_381_542_355_843u64,
2379            "title": "C2",
2380            "bmkUri": "http://example.com/c",
2381            "tags": ["four", "five", "six"],
2382        }, {
2383            "id": "bookmarkCCC3",
2384            "type": "bookmark",
2385            "parentid": "menu",
2386            "parentName": "Menu",
2387            "dateAdded": 1_381_542_355_843u64,
2388            "title": "C3",
2389            "bmkUri": "http://example.com/c",
2390            "tags": ["six", "six", "seven"],
2391        }, {
2392            // D has the same tags as C1/2, but a different URL. This is
2393            // perfectly fine, since URLs and tags are many-many! D also
2394            // isn't duplicated, so it'll be filtered out by the
2395            // `HAVING COUNT(*) > 1` clause.
2396            "id": "bookmarkDDDD",
2397            "type": "bookmark",
2398            "parentid": "unfiled",
2399            "parentName": "Unfiled",
2400            "dateAdded": 1_381_542_355_843u64,
2401            "title": "D",
2402            "bmkUri": "http://example.com/d",
2403            "tags": ["four", "five", "six"],
2404        }, {
2405            // E1 and E2 have the same URLs and the same tags, so we shouldn't
2406            // reupload either.
2407            "id": "bookmarkEEE1",
2408            "type": "bookmark",
2409            "parentid": "toolbar",
2410            "parentName": "Toolbar",
2411            "dateAdded": 1_381_542_355_843u64,
2412            "title": "E1",
2413            "bmkUri": "http://example.com/e",
2414            "tags": ["nine", "ten", "eleven"],
2415        }, {
2416            "id": "bookmarkEEE2",
2417            "type": "bookmark",
2418            "parentid": "mobile",
2419            "parentName": "Mobile",
2420            "dateAdded": 1_381_542_355_843u64,
2421            "title": "E2",
2422            "bmkUri": "http://example.com/e",
2423            "tags": ["nine", "ten", "eleven"],
2424        }, {
2425            // F1 and F2 have mismatched tags, but with a twist: F2 doesn't
2426            // have _any_ tags! We should only reupload F2.
2427            "id": "bookmarkFFF1",
2428            "type": "bookmark",
2429            "parentid": "toolbar",
2430            "parentName": "Toolbar",
2431            "dateAdded": 1_381_542_355_843u64,
2432            "title": "F1",
2433            "bmkUri": "http://example.com/f",
2434            "tags": ["twelve"],
2435        }, {
2436            "id": "bookmarkFFF2",
2437            "type": "bookmark",
2438            "parentid": "mobile",
2439            "parentName": "Mobile",
2440            "dateAdded": 1_381_542_355_843u64,
2441            "title": "F2",
2442            "bmkUri": "http://example.com/f",
2443        }, {
2444            "id": "unfiled",
2445            "type": "folder",
2446            "parentid": "places",
2447            "dateAdded": 1_381_542_355_843u64,
2448            "title": "Unfiled",
2449            "children": ["bookmarkBBBB", "bookmarkCCC1", "bookmarkDDDD"],
2450        }, {
2451            "id": "menu",
2452            "type": "folder",
2453            "parentid": "places",
2454            "dateAdded": 1_381_542_355_843u64,
2455            "title": "Menu",
2456            "children": ["bookmarkCCC2", "bookmarkCCC3"],
2457        }, {
2458            "id": "toolbar",
2459            "type": "folder",
2460            "parentid": "places",
2461            "dateAdded": 1_381_542_355_843u64,
2462            "title": "Toolbar",
2463            "children": ["bookmarkEEE1", "bookmarkFFF1"],
2464        }, {
2465            "id": "mobile",
2466            "type": "folder",
2467            "parentid": "places",
2468            "dateAdded": 1_381_542_355_843u64,
2469            "title": "Mobile",
2470            "children": ["bookmarkEEE2", "bookmarkFFF2"],
2471        }]);
2472
2473        // Boilerplate to apply incoming records, since we want to check
2474        // outgoing record contents.
2475        let engine = create_sync_engine(&api);
2476        let incoming = if let Value::Array(records) = remote_records {
2477            records
2478                .into_iter()
2479                .map(IncomingBso::from_test_content)
2480                .collect()
2481        } else {
2482            unreachable!("JSON records must be an array");
2483        };
2484        let mut outgoing = engine_apply_incoming(&engine, incoming);
2485        outgoing.sort_by(|a, b| a.envelope.id.cmp(&b.envelope.id));
2486
2487        // Verify that we applied all incoming records correctly.
2488        assert_local_json_tree(
2489            &writer,
2490            &BookmarkRootGuid::Root.as_guid(),
2491            json!({
2492                "guid": &BookmarkRootGuid::Root.as_guid(),
2493                "children": [{
2494                    "guid": &BookmarkRootGuid::Menu.as_guid(),
2495                    "children": [{
2496                        "guid": "bookmarkCCC2",
2497                        "title": "C2",
2498                        "url": "http://example.com/c",
2499                    }, {
2500                        "guid": "bookmarkCCC3",
2501                        "title": "C3",
2502                        "url": "http://example.com/c",
2503                    }, {
2504                        "guid": "bookmarkAAA2",
2505                        "title": "A2",
2506                        "url": "http://example.com/a",
2507                    }],
2508                }, {
2509                    "guid": &BookmarkRootGuid::Toolbar.as_guid(),
2510                    "children": [{
2511                        "guid": "bookmarkEEE1",
2512                        "title": "E1",
2513                        "url": "http://example.com/e",
2514                    }, {
2515                        "guid": "bookmarkFFF1",
2516                        "title": "F1",
2517                        "url": "http://example.com/f",
2518                    }],
2519                }, {
2520                    "guid": &BookmarkRootGuid::Unfiled.as_guid(),
2521                    "children": [{
2522                        "guid": "bookmarkBBBB",
2523                        "title": "B",
2524                        "url": "http://example.com/b",
2525                    }, {
2526                        "guid": "bookmarkCCC1",
2527                        "title": "C1",
2528                        "url": "http://example.com/c",
2529                    }, {
2530                        "guid": "bookmarkDDDD",
2531                        "title": "D",
2532                        "url": "http://example.com/d",
2533                    }, {
2534                        "guid": "bookmarkAAA1",
2535                        "title": "A1",
2536                        "url": "http://example.com/a",
2537                    }],
2538                }, {
2539                    "guid": &BookmarkRootGuid::Mobile.as_guid(),
2540                    "children": [{
2541                        "guid": "bookmarkEEE2",
2542                        "title": "E2",
2543                        "url": "http://example.com/e",
2544                    }, {
2545                        "guid": "bookmarkFFF2",
2546                        "title": "F2",
2547                        "url": "http://example.com/f",
2548                    }],
2549                }],
2550            }),
2551        );
2552        // And verify our local tags are correct, too.
2553        let expected_local_tags = &[
2554            ("http://example.com/a", vec!["one", "two"]),
2555            ("http://example.com/b", vec!["eight", "three", "two"]),
2556            ("http://example.com/c", vec!["five", "four", "seven", "six"]),
2557            ("http://example.com/d", vec!["five", "four", "six"]),
2558            ("http://example.com/e", vec!["eleven", "nine", "ten"]),
2559            ("http://example.com/f", vec!["twelve"]),
2560        ];
2561        for (href, expected) in expected_local_tags {
2562            let mut actual = tags::get_tags_for_url(&writer, &Url::parse(href).unwrap())?;
2563            actual.sort();
2564            assert_eq!(&actual, expected);
2565        }
2566
2567        let expected_outgoing_ids = &[
2568            "bookmarkAAA1", // A is new locally.
2569            "bookmarkAAA2",
2570            "bookmarkBBBB", // B has a duplicate tag.
2571            "bookmarkCCC1", // C has mismatched tags.
2572            "bookmarkCCC2",
2573            "bookmarkCCC3",
2574            "bookmarkFFF2", // F2 is missing tags.
2575            "menu",         // Roots always get uploaded on the first sync.
2576            "mobile",
2577            "toolbar",
2578            "unfiled",
2579        ];
2580        assert_eq!(
2581            outgoing
2582                .iter()
2583                .map(|p| p.envelope.id.as_str())
2584                .collect::<Vec<_>>(),
2585            expected_outgoing_ids,
2586            "Should upload new bookmarks and fix up tags",
2587        );
2588
2589        // Now push the records back to the engine, so we can check what we're
2590        // uploading.
2591        engine
2592            .set_uploaded(
2593                ServerTimestamp(0),
2594                expected_outgoing_ids.iter().map(SyncGuid::from).collect(),
2595            )
2596            .expect("Should push synced changes back to the engine");
2597        engine.sync_finished().expect("should work");
2598
2599        // A and C should have the same URL and tags, and should be valid now.
2600        // Because the builder methods take a `&mut SyncedBookmarkItem`, and we
2601        // want to hang on to our base items for cloning later, we can't use
2602        // one-liners to create them.
2603        let mut synced_item_for_a = SyncedBookmarkItem::new();
2604        synced_item_for_a
2605            .validity(SyncedBookmarkValidity::Valid)
2606            .kind(SyncedBookmarkKind::Bookmark)
2607            .url(Some("http://example.com/a"))
2608            .tags(["one", "two"].iter().map(|&tag| tag.into()).collect());
2609        let mut synced_item_for_b = SyncedBookmarkItem::new();
2610        synced_item_for_b
2611            .validity(SyncedBookmarkValidity::Valid)
2612            .kind(SyncedBookmarkKind::Bookmark)
2613            .url(Some("http://example.com/b"))
2614            .tags(
2615                ["eight", "three", "two"]
2616                    .iter()
2617                    .map(|&tag| tag.into())
2618                    .collect(),
2619            )
2620            .parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
2621            .title(Some("B"));
2622        let mut synced_item_for_c = SyncedBookmarkItem::new();
2623        synced_item_for_c
2624            .validity(SyncedBookmarkValidity::Valid)
2625            .kind(SyncedBookmarkKind::Bookmark)
2626            .url(Some("http://example.com/c"))
2627            .tags(
2628                ["five", "four", "seven", "six"]
2629                    .iter()
2630                    .map(|&tag| tag.into())
2631                    .collect(),
2632            );
2633        let mut synced_item_for_f = SyncedBookmarkItem::new();
2634        synced_item_for_f
2635            .validity(SyncedBookmarkValidity::Valid)
2636            .kind(SyncedBookmarkKind::Bookmark)
2637            .url(Some("http://example.com/f"))
2638            .tags(vec!["twelve".into()]);
2639        // A table-driven test to clean up some of the boilerplate. We clone
2640        // the base item for each test, and pass it to the boxed closure to set
2641        // additional properties.
2642        let expected_synced_items = &[
2643            ExpectedSyncedItem::with_properties("bookmarkAAA1", &synced_item_for_a, |a| {
2644                a.parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
2645                    .title(Some("A1"))
2646            }),
2647            ExpectedSyncedItem::with_properties("bookmarkAAA2", &synced_item_for_a, |a| {
2648                a.parent_guid(Some(&BookmarkRootGuid::Menu.as_guid()))
2649                    .title(Some("A2"))
2650            }),
2651            ExpectedSyncedItem::new("bookmarkBBBB", &synced_item_for_b),
2652            ExpectedSyncedItem::with_properties("bookmarkCCC1", &synced_item_for_c, |c| {
2653                c.parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
2654                    .title(Some("C1"))
2655            }),
2656            ExpectedSyncedItem::with_properties("bookmarkCCC2", &synced_item_for_c, |c| {
2657                c.parent_guid(Some(&BookmarkRootGuid::Menu.as_guid()))
2658                    .title(Some("C2"))
2659            }),
2660            ExpectedSyncedItem::with_properties("bookmarkCCC3", &synced_item_for_c, |c| {
2661                c.parent_guid(Some(&BookmarkRootGuid::Menu.as_guid()))
2662                    .title(Some("C3"))
2663            }),
2664            ExpectedSyncedItem::with_properties(
2665                // We didn't reupload F1, but let's make sure it's still valid.
2666                "bookmarkFFF1",
2667                &synced_item_for_f,
2668                |f| {
2669                    f.parent_guid(Some(&BookmarkRootGuid::Toolbar.as_guid()))
2670                        .title(Some("F1"))
2671                },
2672            ),
2673            ExpectedSyncedItem::with_properties("bookmarkFFF2", &synced_item_for_f, |f| {
2674                f.parent_guid(Some(&BookmarkRootGuid::Mobile.as_guid()))
2675                    .title(Some("F2"))
2676            }),
2677        ];
2678        for item in expected_synced_items {
2679            item.check(&writer)?;
2680        }
2681
2682        Ok(())
2683    }
2684
2685    #[test]
2686    fn test_apply_bookmark_tags() -> Result<()> {
2687        let api = new_mem_api();
2688        let writer = api.open_connection(ConnectionType::ReadWrite)?;
2689
2690        // Insert local item with tagged URL.
2691        insert_bookmark(
2692            &writer,
2693            InsertableBookmark {
2694                parent_guid: BookmarkRootGuid::Unfiled.as_guid(),
2695                position: BookmarkPosition::Append,
2696                date_added: None,
2697                last_modified: None,
2698                guid: Some("bookmarkAAAA".into()),
2699                url: Url::parse("http://example.com/a").unwrap(),
2700                title: Some("A".into()),
2701            }
2702            .into(),
2703        )?;
2704        tags::tag_url(&writer, &Url::parse("http://example.com/a").unwrap(), "one")?;
2705
2706        let mut tags_for_a =
2707            tags::get_tags_for_url(&writer, &Url::parse("http://example.com/a").unwrap())?;
2708        tags_for_a.sort();
2709        assert_eq!(tags_for_a, vec!["one".to_owned()]);
2710
2711        assert_incoming_creates_local_tree(
2712            &api,
2713            json!([{
2714                "id": "bookmarkBBBB",
2715                "type": "bookmark",
2716                "parentid": "unfiled",
2717                "parentName": "Unfiled",
2718                "dateAdded": 1_381_542_355_843u64,
2719                "title": "B",
2720                "bmkUri": "http://example.com/b",
2721                "tags": ["one", "two"],
2722            }, {
2723                "id": "bookmarkCCCC",
2724                "type": "bookmark",
2725                "parentid": "unfiled",
2726                "parentName": "Unfiled",
2727                "dateAdded": 1_381_542_355_843u64,
2728                "title": "C",
2729                "bmkUri": "http://example.com/c",
2730                "tags": ["three"],
2731            }, {
2732                "id": "unfiled",
2733                "type": "folder",
2734                "parentid": "places",
2735                "dateAdded": 1_381_542_355_843u64,
2736                "title": "Unfiled",
2737                "children": ["bookmarkBBBB", "bookmarkCCCC"],
2738            }]),
2739            &BookmarkRootGuid::Unfiled.as_guid(),
2740            json!({"children" : [
2741                  {"guid": "bookmarkBBBB", "url": "http://example.com/b"},
2742                  {"guid": "bookmarkCCCC", "url": "http://example.com/c"},
2743                  {"guid": "bookmarkAAAA", "url": "http://example.com/a"},
2744            ]}),
2745        );
2746
2747        let mut tags_for_a =
2748            tags::get_tags_for_url(&writer, &Url::parse("http://example.com/a").unwrap())?;
2749        tags_for_a.sort();
2750        assert_eq!(tags_for_a, vec!["one".to_owned()]);
2751
2752        let mut tags_for_b =
2753            tags::get_tags_for_url(&writer, &Url::parse("http://example.com/b").unwrap())?;
2754        tags_for_b.sort();
2755        assert_eq!(tags_for_b, vec!["one".to_owned(), "two".to_owned()]);
2756
2757        let mut tags_for_c =
2758            tags::get_tags_for_url(&writer, &Url::parse("http://example.com/c").unwrap())?;
2759        tags_for_c.sort();
2760        assert_eq!(tags_for_c, vec!["three".to_owned()]);
2761
2762        let synced_item_for_a = SyncedBookmarkItem::get(&writer, &"bookmarkAAAA".into())
2763            .expect("Should fetch A")
2764            .expect("A should exist");
2765        assert_eq!(
2766            synced_item_for_a,
2767            *SyncedBookmarkItem::new()
2768                .validity(SyncedBookmarkValidity::Valid)
2769                .kind(SyncedBookmarkKind::Bookmark)
2770                .parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
2771                .title(Some("A"))
2772                .url(Some("http://example.com/a"))
2773                .tags(vec!["one".into()])
2774        );
2775
2776        let synced_item_for_b = SyncedBookmarkItem::get(&writer, &"bookmarkBBBB".into())
2777            .expect("Should fetch B")
2778            .expect("B should exist");
2779        assert_eq!(
2780            synced_item_for_b,
2781            *SyncedBookmarkItem::new()
2782                .validity(SyncedBookmarkValidity::Valid)
2783                .kind(SyncedBookmarkKind::Bookmark)
2784                .parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
2785                .title(Some("B"))
2786                .url(Some("http://example.com/b"))
2787                .tags(vec!["one".into(), "two".into()])
2788        );
2789
2790        Ok(())
2791    }
2792
2793    #[test]
2794    fn test_apply_bookmark_keyword() -> Result<()> {
2795        let api = new_mem_api();
2796
2797        let records = json!([{
2798            "id": "bookmarkAAAA",
2799            "type": "bookmark",
2800            "parentid": "unfiled",
2801            "parentName": "Unfiled",
2802            "dateAdded": 1_381_542_355_843u64,
2803            "title": "A",
2804            "bmkUri": "http://example.com/a?b=c&d=%s",
2805            "keyword": "ex",
2806        },
2807        {
2808            "id": "unfiled",
2809            "type": "folder",
2810            "parentid": "places",
2811            "dateAdded": 1_381_542_355_843u64,
2812            "title": "Unfiled",
2813            "children": ["bookmarkAAAA"],
2814        }]);
2815
2816        let db_mutex = api.get_sync_connection().unwrap();
2817        let db = db_mutex.lock();
2818        let tx = db.begin_transaction()?;
2819        let applicator = IncomingApplicator::new(&db);
2820
2821        if let Value::Array(records) = records {
2822            for record in records {
2823                applicator.apply_bso(IncomingBso::from_test_content(record))?;
2824            }
2825        } else {
2826            unreachable!("JSON records must be an array");
2827        }
2828
2829        tx.commit()?;
2830
2831        // Flag the bookmark with the keyword for reupload, so that we can
2832        // ensure the keyword is round-tripped correctly.
2833        db.execute(
2834            "UPDATE moz_bookmarks_synced SET
2835                 validity = :validity
2836             WHERE guid = :guid",
2837            rusqlite::named_params! {
2838                ":validity": SyncedBookmarkValidity::Reupload,
2839                ":guid": SyncGuid::from("bookmarkAAAA"),
2840            },
2841        )?;
2842
2843        let interrupt_scope = db.begin_interrupt_scope()?;
2844
2845        let mut merger = Merger::new(&db, &interrupt_scope, ServerTimestamp(0));
2846        merger.merge()?;
2847
2848        assert_local_json_tree(
2849            &db,
2850            &BookmarkRootGuid::Unfiled.as_guid(),
2851            json!({"children" : [{"guid": "bookmarkAAAA", "url": "http://example.com/a?b=c&d=%s"}]}),
2852        );
2853
2854        let outgoing = fetch_outgoing_records(&db, &interrupt_scope)?;
2855        let record_for_a = outgoing
2856            .iter()
2857            .find(|payload| payload.envelope.id == "bookmarkAAAA")
2858            .expect("Should reupload A");
2859        let bk = record_for_a.to_test_incoming_t::<BookmarkRecord>();
2860        assert_eq!(bk.url.unwrap(), "http://example.com/a?b=c&d=%s");
2861        assert_eq!(bk.keyword.unwrap(), "ex");
2862
2863        Ok(())
2864    }
2865
2866    #[test]
2867    fn test_apply_query() {
2868        // should we add some more query variations here?
2869        let api = new_mem_api();
2870        assert_incoming_creates_local_tree(
2871            &api,
2872            json!([{
2873                "id": "query1______",
2874                "type": "query",
2875                "parentid": "unfiled",
2876                "parentName": "Unfiled Bookmarks",
2877                "dateAdded": 1_381_542_355_843u64,
2878                "title": "Some query",
2879                "bmkUri": "place:tag=foo",
2880            },
2881            {
2882                "id": "unfiled",
2883                "type": "folder",
2884                "parentid": "places",
2885                "dateAdded": 1_381_542_355_843u64,
2886                "title": "Unfiled",
2887                "children": ["query1______"],
2888            }]),
2889            &BookmarkRootGuid::Unfiled.as_guid(),
2890            json!({"children" : [{"guid": "query1______", "url": "place:tag=foo"}]}),
2891        );
2892        let reader = api
2893            .open_connection(ConnectionType::ReadOnly)
2894            .expect("Should open read-only connection");
2895        assert!(
2896            frecency_stale_at(&reader, &Url::parse("place:tag=foo").unwrap())
2897                .expect("Should check stale frecency")
2898                .is_none(),
2899            "Should not mark frecency for queries as stale"
2900        );
2901    }
2902
2903    #[test]
2904    fn test_apply() -> Result<()> {
2905        let api = new_mem_api();
2906        let writer = api.open_connection(ConnectionType::ReadWrite)?;
2907        let db = api.get_sync_connection().unwrap();
2908        let syncer = db.lock();
2909
2910        syncer
2911            .execute("UPDATE moz_bookmarks SET syncChangeCounter = 0", [])
2912            .expect("should work");
2913
2914        insert_local_json_tree(
2915            &writer,
2916            json!({
2917                "guid": &BookmarkRootGuid::Unfiled.as_guid(),
2918                "children": [
2919                    {
2920                        "guid": "bookmarkAAAA",
2921                        "title": "A",
2922                        "url": "http://example.com/a",
2923                    },
2924                    {
2925                        "guid": "bookmarkBBBB",
2926                        "title": "B",
2927                        "url": "http://example.com/b",
2928                    },
2929                ]
2930            }),
2931        );
2932        tags::tag_url(
2933            &writer,
2934            &Url::parse("http://example.com/a").expect("Should parse URL for A"),
2935            "baz",
2936        )
2937        .expect("Should tag A");
2938
2939        let records = vec![
2940            json!({
2941                "id": "bookmarkCCCC",
2942                "type": "bookmark",
2943                "parentid": "menu",
2944                "parentName": "menu",
2945                "dateAdded": 1_552_183_116_885u64,
2946                "title": "C",
2947                "bmkUri": "http://example.com/c",
2948                "tags": ["foo", "bar"],
2949            }),
2950            json!({
2951                "id": "menu",
2952                "type": "folder",
2953                "parentid": "places",
2954                "parentName": "",
2955                "dateAdded": 0,
2956                "title": "menu",
2957                "children": ["bookmarkCCCC"],
2958            }),
2959        ];
2960
2961        // Drop the sync connection to avoid a deadlock when the sync engine locks the mutex
2962        drop(syncer);
2963        let engine = create_sync_engine(&api);
2964
2965        let incoming = records
2966            .into_iter()
2967            .map(IncomingBso::from_test_content)
2968            .collect();
2969
2970        let mut outgoing = engine_apply_incoming(&engine, incoming);
2971        outgoing.sort_by(|a, b| a.envelope.id.cmp(&b.envelope.id));
2972        assert_eq!(
2973            outgoing
2974                .iter()
2975                .map(|p| p.envelope.id.as_str())
2976                .collect::<Vec<_>>(),
2977            vec!["bookmarkAAAA", "bookmarkBBBB", "unfiled",]
2978        );
2979        let record_for_a = outgoing
2980            .iter()
2981            .find(|p| p.envelope.id == "bookmarkAAAA")
2982            .expect("Should upload A");
2983        let content_for_a = record_for_a.to_test_incoming_t::<BookmarkRecord>();
2984        assert_eq!(content_for_a.tags, vec!["baz".to_string()]);
2985
2986        assert_local_json_tree(
2987            &writer,
2988            &BookmarkRootGuid::Root.as_guid(),
2989            json!({
2990                "guid": &BookmarkRootGuid::Root.as_guid(),
2991                "children": [
2992                    {
2993                        "guid": &BookmarkRootGuid::Menu.as_guid(),
2994                        "children": [
2995                            {
2996                                "guid": "bookmarkCCCC",
2997                                "title": "C",
2998                                "url": "http://example.com/c",
2999                                "date_added": Timestamp(1_552_183_116_885),
3000                            },
3001                        ],
3002                    },
3003                    {
3004                        "guid": &BookmarkRootGuid::Toolbar.as_guid(),
3005                        "children": [],
3006                    },
3007                    {
3008                        "guid": &BookmarkRootGuid::Unfiled.as_guid(),
3009                        "children": [
3010                            {
3011                                "guid": "bookmarkAAAA",
3012                                "title": "A",
3013                                "url": "http://example.com/a",
3014                            },
3015                            {
3016                                "guid": "bookmarkBBBB",
3017                                "title": "B",
3018                                "url": "http://example.com/b",
3019                            },
3020                        ],
3021                    },
3022                    {
3023                        "guid": &BookmarkRootGuid::Mobile.as_guid(),
3024                        "children": [],
3025                    },
3026                ],
3027            }),
3028        );
3029
3030        // We haven't finished the sync yet, so all local change counts for
3031        // items to upload should still be > 0.
3032        let guid_for_a: SyncGuid = "bookmarkAAAA".into();
3033        let info_for_a = get_raw_bookmark(&writer, &guid_for_a)
3034            .expect("Should fetch info for A")
3035            .unwrap();
3036        assert_eq!(info_for_a._sync_change_counter, 2);
3037        let info_for_unfiled = get_raw_bookmark(&writer, &BookmarkRootGuid::Unfiled.as_guid())
3038            .expect("Should fetch info for unfiled")
3039            .unwrap();
3040        assert_eq!(info_for_unfiled._sync_change_counter, 2);
3041
3042        engine
3043            .set_uploaded(
3044                ServerTimestamp(0),
3045                vec![
3046                    "bookmarkAAAA".into(),
3047                    "bookmarkBBBB".into(),
3048                    "unfiled".into(),
3049                ],
3050            )
3051            .expect("Should push synced changes back to the engine");
3052        engine.sync_finished().expect("finish always works");
3053
3054        let info_for_a = get_raw_bookmark(&writer, &guid_for_a)
3055            .expect("Should fetch info for A")
3056            .unwrap();
3057        assert_eq!(info_for_a._sync_change_counter, 0);
3058        let info_for_unfiled = get_raw_bookmark(&writer, &BookmarkRootGuid::Unfiled.as_guid())
3059            .expect("Should fetch info for unfiled")
3060            .unwrap();
3061        assert_eq!(info_for_unfiled._sync_change_counter, 0);
3062
3063        let mut tags_for_c = tags::get_tags_for_url(
3064            &writer,
3065            &Url::parse("http://example.com/c").expect("Should parse URL for C"),
3066        )
3067        .expect("Should return tags for C");
3068        tags_for_c.sort();
3069        assert_eq!(tags_for_c, &["bar", "foo"]);
3070
3071        Ok(())
3072    }
3073
3074    #[test]
3075    fn test_apply_invalid_url() -> Result<()> {
3076        let api = new_mem_api();
3077        let db = api.get_sync_connection().unwrap();
3078        let syncer = db.lock();
3079
3080        syncer
3081            .execute("UPDATE moz_bookmarks SET syncChangeCounter = 0", [])
3082            .expect("should work");
3083
3084        let records = vec![
3085            json!({
3086                "id": "bookmarkXXXX",
3087                "type": "bookmark",
3088                "parentid": "menu",
3089                "parentName": "menu",
3090                "dateAdded": 1_552_183_116_885u64,
3091                "title": "Invalid",
3092                "bmkUri": "invalid url",
3093            }),
3094            json!({
3095                "id": "menu",
3096                "type": "folder",
3097                "parentid": "places",
3098                "parentName": "",
3099                "dateAdded": 0,
3100                "title": "menu",
3101                "children": ["bookmarkXXXX"],
3102            }),
3103        ];
3104
3105        // Drop the sync connection to avoid a deadlock when the sync engine locks the mutex
3106        drop(syncer);
3107        let engine = create_sync_engine(&api);
3108
3109        let incoming = records
3110            .into_iter()
3111            .map(IncomingBso::from_test_content)
3112            .collect();
3113
3114        let mut outgoing = engine_apply_incoming(&engine, incoming);
3115        outgoing.sort_by(|a, b| a.envelope.id.cmp(&b.envelope.id));
3116        assert_eq!(
3117            outgoing
3118                .iter()
3119                .map(|p| p.envelope.id.as_str())
3120                .collect::<Vec<_>>(),
3121            vec!["bookmarkXXXX", "menu",]
3122        );
3123
3124        let record_for_invalid = outgoing
3125            .iter()
3126            .find(|p| p.envelope.id == "bookmarkXXXX")
3127            .expect("Should re-upload the invalid record");
3128
3129        assert!(
3130            matches!(
3131                record_for_invalid
3132                    .to_test_incoming()
3133                    .into_content::<BookmarkRecord>()
3134                    .kind,
3135                IncomingKind::Tombstone
3136            ),
3137            "is invalid record"
3138        );
3139
3140        let record_for_menu = outgoing
3141            .iter()
3142            .find(|p| p.envelope.id == "menu")
3143            .expect("Should upload menu");
3144        let content_for_menu = record_for_menu.to_test_incoming_t::<FolderRecord>();
3145        assert!(
3146            content_for_menu.children.is_empty(),
3147            "should have been removed from the parent"
3148        );
3149        Ok(())
3150    }
3151
3152    #[test]
3153    fn test_apply_tombstones() -> Result<()> {
3154        let local_modified = Timestamp::now();
3155        let api = new_mem_api();
3156        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3157        insert_local_json_tree(
3158            &writer,
3159            json!({
3160                "guid": &BookmarkRootGuid::Unfiled.as_guid(),
3161                "children": [{
3162                    "guid": "bookmarkAAAA",
3163                    "title": "A",
3164                    "url": "http://example.com/a",
3165                    "date_added": local_modified,
3166                    "last_modified": local_modified,
3167                }, {
3168                    "guid": "separatorAAA",
3169                    "type": BookmarkType::Separator as u8,
3170                    "date_added": local_modified,
3171                    "last_modified": local_modified,
3172                }, {
3173                    "guid": "folderAAAAAA",
3174                    "children": [{
3175                        "guid": "bookmarkBBBB",
3176                        "title": "b",
3177                        "url": "http://example.com/b",
3178                        "date_added": local_modified,
3179                        "last_modified": local_modified,
3180                    }],
3181                }],
3182            }),
3183        );
3184        // a first sync, which will populate our mirror.
3185        let engine = create_sync_engine(&api);
3186        let outgoing = engine_apply_incoming(&engine, vec![]);
3187        let outgoing_ids = outgoing
3188            .iter()
3189            .map(|p| p.envelope.id.clone())
3190            .collect::<Vec<_>>();
3191        // 4 roots + 4 items
3192        assert_eq!(outgoing_ids.len(), 8, "{:?}", outgoing_ids);
3193
3194        engine
3195            .set_uploaded(ServerTimestamp(0), outgoing_ids)
3196            .expect("should work");
3197        engine.sync_finished().expect("should work");
3198
3199        // Now the next sync with incoming tombstones.
3200        let remote_unfiled = json!({
3201            "id": "unfiled",
3202            "type": "folder",
3203            "parentid": "places",
3204            "title": "Unfiled",
3205            "children": [],
3206        });
3207
3208        let incoming = vec![
3209            IncomingBso::new_test_tombstone(Guid::new("bookmarkAAAA")),
3210            IncomingBso::new_test_tombstone(Guid::new("separatorAAA")),
3211            IncomingBso::new_test_tombstone(Guid::new("folderAAAAAA")),
3212            IncomingBso::new_test_tombstone(Guid::new("bookmarkBBBB")),
3213            IncomingBso::from_test_content(remote_unfiled),
3214        ];
3215
3216        let outgoing = engine_apply_incoming(&engine, incoming);
3217        let outgoing_ids = outgoing
3218            .iter()
3219            .map(|p| p.envelope.id.clone())
3220            .collect::<Vec<_>>();
3221        assert_eq!(outgoing_ids.len(), 0, "{:?}", outgoing_ids);
3222
3223        engine
3224            .set_uploaded(ServerTimestamp(0), outgoing_ids)
3225            .expect("should work");
3226        engine.sync_finished().expect("should work");
3227
3228        // We deleted everything from unfiled.
3229        assert_local_json_tree(
3230            &api.get_sync_connection().unwrap().lock(),
3231            &BookmarkRootGuid::Unfiled.as_guid(),
3232            json!({"children" : []}),
3233        );
3234        Ok(())
3235    }
3236
3237    #[test]
3238    fn test_keywords() -> Result<()> {
3239        use crate::storage::bookmarks::bookmarks_get_url_for_keyword;
3240
3241        let api = new_mem_api();
3242        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3243
3244        let records = vec![
3245            json!({
3246                "id": "toolbar",
3247                "type": "folder",
3248                "parentid": "places",
3249                "parentName": "",
3250                "dateAdded": 0,
3251                "title": "toolbar",
3252                "children": ["bookmarkAAAA"],
3253            }),
3254            json!({
3255                "id": "bookmarkAAAA",
3256                "type": "bookmark",
3257                "parentid": "toolbar",
3258                "parentName": "toolbar",
3259                "dateAdded": 1_552_183_116_885u64,
3260                "title": "A",
3261                "bmkUri": "http://example.com/a/%s",
3262                "keyword": "a",
3263            }),
3264        ];
3265
3266        let engine = create_sync_engine(&api);
3267
3268        let incoming = records
3269            .into_iter()
3270            .map(IncomingBso::from_test_content)
3271            .collect();
3272
3273        let outgoing = engine_apply_incoming(&engine, incoming);
3274        let mut outgoing_ids = outgoing
3275            .iter()
3276            .map(|p| p.envelope.id.clone())
3277            .collect::<Vec<_>>();
3278        outgoing_ids.sort();
3279        assert_eq!(outgoing_ids, &["menu", "mobile", "toolbar", "unfiled"],);
3280
3281        assert_eq!(
3282            bookmarks_get_url_for_keyword(&writer, "a")?,
3283            Some(Url::parse("http://example.com/a/%s")?)
3284        );
3285
3286        engine
3287            .set_uploaded(ServerTimestamp(0), outgoing_ids)
3288            .expect("Should push synced changes back to the engine");
3289        engine.sync_finished().expect("should work");
3290
3291        update_bookmark(
3292            &writer,
3293            &"bookmarkAAAA".into(),
3294            &UpdatableBookmark {
3295                title: Some("A (local)".into()),
3296                ..UpdatableBookmark::default()
3297            }
3298            .into(),
3299        )?;
3300
3301        let outgoing = engine_apply_incoming(&engine, vec![]);
3302        assert_eq!(outgoing.len(), 1);
3303        let bk = outgoing[0].to_test_incoming_t::<BookmarkRecord>();
3304        assert_eq!(bk.record_id.as_guid(), "bookmarkAAAA");
3305        assert_eq!(bk.keyword.unwrap(), "a");
3306        assert_eq!(bk.url.unwrap(), "http://example.com/a/%s");
3307
3308        // URLs with keywords should have a foreign count of 3 (one for the
3309        // local bookmark, one for the synced bookmark, and one for the
3310        // keyword), and we shouldn't allow deleting them until the keyword
3311        // is removed.
3312        let foreign_count = writer
3313            .try_query_row(
3314                "SELECT foreign_count FROM moz_places
3315             WHERE url_hash = hash(:url) AND
3316                   url = :url",
3317                &[(":url", &"http://example.com/a/%s")],
3318                |row| -> rusqlite::Result<_> { row.get::<_, i64>(0) },
3319                false,
3320            )?
3321            .expect("Should fetch foreign count for URL A");
3322        assert_eq!(foreign_count, 3);
3323        let err = writer
3324            .execute(
3325                "DELETE FROM moz_places
3326             WHERE url_hash = hash(:url) AND
3327                   url = :url",
3328                rusqlite::named_params! {
3329                    ":url": "http://example.com/a/%s",
3330                },
3331            )
3332            .expect_err("Should fail to delete URL A with keyword");
3333        match err {
3334            RusqlError::SqliteFailure(e, _) => assert_eq!(e.code, ErrorCode::ConstraintViolation),
3335            _ => panic!("Wanted constraint violation error; got {:?}", err),
3336        }
3337
3338        Ok(())
3339    }
3340
3341    #[test]
3342    fn test_apply_complex_bookmark_keywords() -> Result<()> {
3343        use crate::storage::bookmarks::bookmarks_get_url_for_keyword;
3344
3345        // We don't provide an API for setting keywords locally, but we'll
3346        // still round-trip and fix up keywords on the server.
3347
3348        let api = new_mem_api();
3349        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3350
3351        // Let's add some remote bookmarks with keywords.
3352        let remote_records = json!([{
3353            // A1 and A2 have the same URL and keyword, so we shouldn't
3354            // reupload them.
3355            "id": "bookmarkAAA1",
3356            "type": "bookmark",
3357            "parentid": "unfiled",
3358            "parentName": "Unfiled",
3359            "title": "A1",
3360            "bmkUri": "http://example.com/a",
3361            "keyword": "one",
3362        }, {
3363            "id": "bookmarkAAA2",
3364            "type": "bookmark",
3365            "parentid": "menu",
3366            "parentName": "Menu",
3367            "title": "A2",
3368            "bmkUri": "http://example.com/a",
3369            "keyword": "one",
3370        }, {
3371            // B1 and B2 have mismatched keywords, and we should reupload
3372            // both of them. It's not specified which keyword wins, but
3373            // reuploading both means we make them consistent.
3374            "id": "bookmarkBBB1",
3375            "type": "bookmark",
3376            "parentid": "unfiled",
3377            "parentName": "Unfiled",
3378            "title": "B1",
3379            "bmkUri": "http://example.com/b",
3380            "keyword": "two",
3381        }, {
3382            "id": "bookmarkBBB2",
3383            "type": "bookmark",
3384            "parentid": "menu",
3385            "parentName": "Menu",
3386            "title": "B2",
3387            "bmkUri": "http://example.com/b",
3388            "keyword": "three",
3389        }, {
3390            // C1 has a keyword; C2 doesn't. As with B, which one wins
3391            // depends on which record we apply last, and how SQLite
3392            // processes the rows, but we should reupload both.
3393            "id": "bookmarkCCC1",
3394            "type": "bookmark",
3395            "parentid": "unfiled",
3396            "parentName": "Unfiled",
3397            "title": "C1",
3398            "bmkUri": "http://example.com/c",
3399            "keyword": "four",
3400        }, {
3401            "id": "bookmarkCCC2",
3402            "type": "bookmark",
3403            "parentid": "menu",
3404            "parentName": "Menu",
3405            "title": "C2",
3406            "bmkUri": "http://example.com/c",
3407        }, {
3408            // D has a keyword that needs to be cleaned up before
3409            // inserting. In this case, we intentionally don't reupload.
3410            "id": "bookmarkDDDD",
3411            "type": "bookmark",
3412            "parentid": "unfiled",
3413            "parentName": "Unfiled",
3414            "title": "D",
3415            "bmkUri": "http://example.com/d",
3416            "keyword": " FIVE ",
3417        }, {
3418            "id": "unfiled",
3419            "type": "folder",
3420            "parentid": "places",
3421            "title": "Unfiled",
3422            "children": ["bookmarkAAA1", "bookmarkBBB1", "bookmarkCCC1", "bookmarkDDDD"],
3423        }, {
3424            "id": "menu",
3425            "type": "folder",
3426            "parentid": "places",
3427            "title": "Menu",
3428            "children": ["bookmarkAAA2", "bookmarkBBB2", "bookmarkCCC2"],
3429        }]);
3430
3431        let engine = create_sync_engine(&api);
3432        let incoming = if let Value::Array(records) = remote_records {
3433            records
3434                .into_iter()
3435                .map(IncomingBso::from_test_content)
3436                .collect()
3437        } else {
3438            unreachable!("JSON records must be an array");
3439        };
3440        let mut outgoing = engine_apply_incoming(&engine, incoming);
3441        outgoing.sort_by(|a, b| a.envelope.id.cmp(&b.envelope.id));
3442
3443        assert_local_json_tree(
3444            &writer,
3445            &BookmarkRootGuid::Root.as_guid(),
3446            json!({
3447                "guid": &BookmarkRootGuid::Root.as_guid(),
3448                "children": [{
3449                    "guid": &BookmarkRootGuid::Menu.as_guid(),
3450                    "children": [{
3451                        "guid": "bookmarkAAA2",
3452                        "title": "A2",
3453                        "url": "http://example.com/a",
3454                    }, {
3455                        "guid": "bookmarkBBB2",
3456                        "title": "B2",
3457                        "url": "http://example.com/b",
3458                    }, {
3459                        "guid": "bookmarkCCC2",
3460                        "title": "C2",
3461                        "url": "http://example.com/c",
3462                    }],
3463                }, {
3464                    "guid": &BookmarkRootGuid::Toolbar.as_guid(),
3465                    "children": [],
3466                }, {
3467                    "guid": &BookmarkRootGuid::Unfiled.as_guid(),
3468                    "children": [{
3469                        "guid": "bookmarkAAA1",
3470                        "title": "A1",
3471                        "url": "http://example.com/a",
3472                    }, {
3473                        "guid": "bookmarkBBB1",
3474                        "title": "B1",
3475                        "url": "http://example.com/b",
3476                    }, {
3477                        "guid": "bookmarkCCC1",
3478                        "title": "C1",
3479                        "url": "http://example.com/c",
3480                    }, {
3481                        "guid": "bookmarkDDDD",
3482                        "title": "D",
3483                        "url": "http://example.com/d",
3484                    }],
3485                }, {
3486                    "guid": &BookmarkRootGuid::Mobile.as_guid(),
3487                    "children": [],
3488                }],
3489            }),
3490        );
3491        // And verify our local keywords are correct, too.
3492        let url_for_one = bookmarks_get_url_for_keyword(&writer, "one")?
3493            .expect("Should have URL for keyword `one`");
3494        assert_eq!(url_for_one.as_str(), "http://example.com/a");
3495
3496        let keyword_for_b = match (
3497            bookmarks_get_url_for_keyword(&writer, "two")?,
3498            bookmarks_get_url_for_keyword(&writer, "three")?,
3499        ) {
3500            (Some(url), None) => {
3501                assert_eq!(url.as_str(), "http://example.com/b");
3502                "two".to_string()
3503            }
3504            (None, Some(url)) => {
3505                assert_eq!(url.as_str(), "http://example.com/b");
3506                "three".to_string()
3507            }
3508            (Some(_), Some(_)) => panic!("Should pick `two` or `three`, not both"),
3509            (None, None) => panic!("Should have URL for either `two` or `three`"),
3510        };
3511
3512        let keyword_for_c = match bookmarks_get_url_for_keyword(&writer, "four")? {
3513            Some(url) => {
3514                assert_eq!(url.as_str(), "http://example.com/c");
3515                Some("four".to_string())
3516            }
3517            None => None,
3518        };
3519
3520        let url_for_five = bookmarks_get_url_for_keyword(&writer, "five")?
3521            .expect("Should have URL for keyword `five`");
3522        assert_eq!(url_for_five.as_str(), "http://example.com/d");
3523
3524        let expected_outgoing_keywords = &[
3525            ("bookmarkBBB1", Some(keyword_for_b.clone())),
3526            ("bookmarkBBB2", Some(keyword_for_b.clone())),
3527            ("bookmarkCCC1", keyword_for_c.clone()),
3528            ("bookmarkCCC2", keyword_for_c.clone()),
3529            ("menu", None), // Roots always get uploaded on the first sync.
3530            ("mobile", None),
3531            ("toolbar", None),
3532            ("unfiled", None),
3533        ];
3534        assert_eq!(
3535            outgoing
3536                .iter()
3537                .map(|p| (
3538                    p.envelope.id.as_str(),
3539                    p.to_test_incoming_t::<BookmarkRecord>().keyword
3540                ))
3541                .collect::<Vec<_>>(),
3542            expected_outgoing_keywords,
3543            "Should upload new bookmarks and fix up keywords",
3544        );
3545
3546        // Now push the records back to the engine, so we can check what we're
3547        // uploading.
3548        engine
3549            .set_uploaded(
3550                ServerTimestamp(0),
3551                expected_outgoing_keywords
3552                    .iter()
3553                    .map(|(id, _)| SyncGuid::from(id))
3554                    .collect(),
3555            )
3556            .expect("Should push synced changes back to the engine");
3557        engine.sync_finished().expect("should work");
3558
3559        let mut synced_item_for_b = SyncedBookmarkItem::new();
3560        synced_item_for_b
3561            .validity(SyncedBookmarkValidity::Valid)
3562            .kind(SyncedBookmarkKind::Bookmark)
3563            .url(Some("http://example.com/b"))
3564            .keyword(Some(&keyword_for_b));
3565        let mut synced_item_for_c = SyncedBookmarkItem::new();
3566        synced_item_for_c
3567            .validity(SyncedBookmarkValidity::Valid)
3568            .kind(SyncedBookmarkKind::Bookmark)
3569            .url(Some("http://example.com/c"))
3570            .keyword(Some(keyword_for_c.unwrap().as_str()));
3571        let expected_synced_items = &[
3572            ExpectedSyncedItem::with_properties("bookmarkBBB1", &synced_item_for_b, |a| {
3573                a.parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
3574                    .title(Some("B1"))
3575            }),
3576            ExpectedSyncedItem::with_properties("bookmarkBBB2", &synced_item_for_b, |a| {
3577                a.parent_guid(Some(&BookmarkRootGuid::Menu.as_guid()))
3578                    .title(Some("B2"))
3579            }),
3580            ExpectedSyncedItem::with_properties("bookmarkCCC1", &synced_item_for_c, |a| {
3581                a.parent_guid(Some(&BookmarkRootGuid::Unfiled.as_guid()))
3582                    .title(Some("C1"))
3583            }),
3584            ExpectedSyncedItem::with_properties("bookmarkCCC2", &synced_item_for_c, |a| {
3585                a.parent_guid(Some(&BookmarkRootGuid::Menu.as_guid()))
3586                    .title(Some("C2"))
3587            }),
3588        ];
3589        for item in expected_synced_items {
3590            item.check(&writer)?;
3591        }
3592
3593        Ok(())
3594    }
3595
3596    #[test]
3597    fn test_wipe() -> Result<()> {
3598        let api = new_mem_api();
3599        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3600
3601        let records = vec![
3602            json!({
3603                "id": "toolbar",
3604                "type": "folder",
3605                "parentid": "places",
3606                "parentName": "",
3607                "dateAdded": 0,
3608                "title": "toolbar",
3609                "children": ["folderAAAAAA"],
3610            }),
3611            json!({
3612                "id": "folderAAAAAA",
3613                "type": "folder",
3614                "parentid": "toolbar",
3615                "parentName": "toolbar",
3616                "dateAdded": 0,
3617                "title": "A",
3618                "children": ["bookmarkBBBB"],
3619            }),
3620            json!({
3621                "id": "bookmarkBBBB",
3622                "type": "bookmark",
3623                "parentid": "folderAAAAAA",
3624                "parentName": "A",
3625                "dateAdded": 0,
3626                "title": "A",
3627                "bmkUri": "http://example.com/a",
3628            }),
3629            json!({
3630                "id": "menu",
3631                "type": "folder",
3632                "parentid": "places",
3633                "parentName": "",
3634                "dateAdded": 0,
3635                "title": "menu",
3636                "children": ["folderCCCCCC"],
3637            }),
3638            json!({
3639                "id": "folderCCCCCC",
3640                "type": "folder",
3641                "parentid": "menu",
3642                "parentName": "menu",
3643                "dateAdded": 0,
3644                "title": "A",
3645                "children": ["bookmarkDDDD", "folderEEEEEE"],
3646            }),
3647            json!({
3648                "id": "bookmarkDDDD",
3649                "type": "bookmark",
3650                "parentid": "folderCCCCCC",
3651                "parentName": "C",
3652                "dateAdded": 0,
3653                "title": "D",
3654                "bmkUri": "http://example.com/d",
3655            }),
3656            json!({
3657                "id": "folderEEEEEE",
3658                "type": "folder",
3659                "parentid": "folderCCCCCC",
3660                "parentName": "C",
3661                "dateAdded": 0,
3662                "title": "E",
3663                "children": ["bookmarkFFFF"],
3664            }),
3665            json!({
3666                "id": "bookmarkFFFF",
3667                "type": "bookmark",
3668                "parentid": "folderEEEEEE",
3669                "parentName": "E",
3670                "dateAdded": 0,
3671                "title": "F",
3672                "bmkUri": "http://example.com/f",
3673            }),
3674        ];
3675
3676        let engine = create_sync_engine(&api);
3677
3678        let incoming = records
3679            .into_iter()
3680            .map(IncomingBso::from_test_content)
3681            .collect();
3682
3683        let outgoing = engine_apply_incoming(&engine, incoming);
3684        let mut outgoing_ids = outgoing
3685            .iter()
3686            .map(|p| p.envelope.id.clone())
3687            .collect::<Vec<_>>();
3688        outgoing_ids.sort();
3689        assert_eq!(outgoing_ids, &["menu", "mobile", "toolbar", "unfiled"],);
3690
3691        engine
3692            .set_uploaded(ServerTimestamp(0), outgoing_ids)
3693            .expect("Should push synced changes back to the engine");
3694        engine.sync_finished().expect("should work");
3695
3696        engine.wipe().expect("Should wipe the store");
3697
3698        // Wiping the store should delete all items except for the roots.
3699        assert_local_json_tree(
3700            &writer,
3701            &BookmarkRootGuid::Root.as_guid(),
3702            json!({
3703                "guid": &BookmarkRootGuid::Root.as_guid(),
3704                "children": [
3705                    {
3706                        "guid": &BookmarkRootGuid::Menu.as_guid(),
3707                        "children": [],
3708                    },
3709                    {
3710                        "guid": &BookmarkRootGuid::Toolbar.as_guid(),
3711                        "children": [],
3712                    },
3713                    {
3714                        "guid": &BookmarkRootGuid::Unfiled.as_guid(),
3715                        "children": [],
3716                    },
3717                    {
3718                        "guid": &BookmarkRootGuid::Mobile.as_guid(),
3719                        "children": [],
3720                    },
3721                ],
3722            }),
3723        );
3724
3725        // Now pretend that F changed remotely between the time we called `wipe`
3726        // and the next sync.
3727        let record_for_f = json!({
3728            "id": "bookmarkFFFF",
3729            "type": "bookmark",
3730            "parentid": "folderEEEEEE",
3731            "parentName": "E",
3732            "dateAdded": 0,
3733            "title": "F (remote)",
3734            "bmkUri": "http://example.com/f-remote",
3735        });
3736
3737        let incoming = vec![IncomingBso::from_test_content_ts(
3738            record_for_f,
3739            ServerTimestamp(1000),
3740        )];
3741
3742        let outgoing = engine_apply_incoming(&engine, incoming);
3743        let (outgoing_tombstones, outgoing_records): (Vec<_>, Vec<_>) =
3744            outgoing.iter().partition(|record| {
3745                matches!(
3746                    record
3747                        .to_test_incoming()
3748                        .into_content::<BookmarkRecord>()
3749                        .kind,
3750                    IncomingKind::Tombstone
3751                )
3752            });
3753        let mut outgoing_record_ids = outgoing_records
3754            .iter()
3755            .map(|p| p.envelope.id.as_str())
3756            .collect::<Vec<_>>();
3757        outgoing_record_ids.sort_unstable();
3758        assert_eq!(
3759            outgoing_record_ids,
3760            &["bookmarkFFFF", "menu", "mobile", "toolbar", "unfiled"],
3761        );
3762        let mut outgoing_tombstone_ids = outgoing_tombstones
3763            .iter()
3764            .map(|p| p.envelope.id.clone())
3765            .collect::<Vec<_>>();
3766        outgoing_tombstone_ids.sort();
3767        assert_eq!(
3768            outgoing_tombstone_ids,
3769            &[
3770                "bookmarkBBBB",
3771                "bookmarkDDDD",
3772                "folderAAAAAA",
3773                "folderCCCCCC",
3774                "folderEEEEEE"
3775            ]
3776        );
3777
3778        // F should move to the closest surviving ancestor, which, in this case,
3779        // is the menu.
3780        assert_local_json_tree(
3781            &writer,
3782            &BookmarkRootGuid::Root.as_guid(),
3783            json!({
3784                "guid": &BookmarkRootGuid::Root.as_guid(),
3785                "children": [
3786                    {
3787                        "guid": &BookmarkRootGuid::Menu.as_guid(),
3788                        "children": [
3789                            {
3790                                "guid": "bookmarkFFFF",
3791                                "title": "F (remote)",
3792                                "url": "http://example.com/f-remote",
3793                            },
3794                        ],
3795                    },
3796                    {
3797                        "guid": &BookmarkRootGuid::Toolbar.as_guid(),
3798                        "children": [],
3799                    },
3800                    {
3801                        "guid": &BookmarkRootGuid::Unfiled.as_guid(),
3802                        "children": [],
3803                    },
3804                    {
3805                        "guid": &BookmarkRootGuid::Mobile.as_guid(),
3806                        "children": [],
3807                    },
3808                ],
3809            }),
3810        );
3811
3812        Ok(())
3813    }
3814
3815    #[test]
3816    fn test_reset() -> anyhow::Result<()> {
3817        let api = new_mem_api();
3818        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3819
3820        insert_local_json_tree(
3821            &writer,
3822            json!({
3823                "guid": &BookmarkRootGuid::Menu.as_guid(),
3824                "children": [
3825                    {
3826                        "guid": "bookmark2___",
3827                        "title": "2",
3828                        "url": "http://example.com/2",
3829                    }
3830                ],
3831            }),
3832        );
3833
3834        {
3835            // scope to kill our sync connection.
3836            let engine = create_sync_engine(&api);
3837
3838            assert_eq!(
3839                engine.get_sync_assoc()?,
3840                EngineSyncAssociation::Disconnected
3841            );
3842
3843            let outgoing = engine_apply_incoming(&engine, vec![]);
3844            let synced_ids: Vec<Guid> = outgoing.into_iter().map(|c| c.envelope.id).collect();
3845            assert_eq!(synced_ids.len(), 5, "should be 4 roots + 1 outgoing item");
3846            engine.set_uploaded(ServerTimestamp(2_000), synced_ids)?;
3847            engine.sync_finished().expect("should work");
3848
3849            let db = api.get_sync_connection().unwrap();
3850            let syncer = db.lock();
3851            assert_eq!(get_meta::<i64>(&syncer, LAST_SYNC_META_KEY)?, Some(2_000));
3852
3853            let sync_ids = CollSyncIds {
3854                global: Guid::random(),
3855                coll: Guid::random(),
3856            };
3857            // Temporarily drop the sync connection to avoid a deadlock when the sync engine locks
3858            // the mutex
3859            drop(syncer);
3860            engine.reset(&EngineSyncAssociation::Connected(sync_ids.clone()))?;
3861            let syncer = db.lock();
3862            assert_eq!(
3863                get_meta::<Guid>(&syncer, GLOBAL_SYNCID_META_KEY)?,
3864                Some(sync_ids.global)
3865            );
3866            assert_eq!(
3867                get_meta::<Guid>(&syncer, COLLECTION_SYNCID_META_KEY)?,
3868                Some(sync_ids.coll)
3869            );
3870            assert_eq!(get_meta::<i64>(&syncer, LAST_SYNC_META_KEY)?, Some(0));
3871        }
3872        // do it all again - after the reset we should get the same results.
3873        {
3874            let engine = create_sync_engine(&api);
3875
3876            let outgoing = engine_apply_incoming(&engine, vec![]);
3877            let synced_ids: Vec<Guid> = outgoing.into_iter().map(|c| c.envelope.id).collect();
3878            assert_eq!(synced_ids.len(), 5, "should be 4 roots + 1 outgoing item");
3879            engine.set_uploaded(ServerTimestamp(2_000), synced_ids)?;
3880            engine.sync_finished().expect("should work");
3881
3882            let db = api.get_sync_connection().unwrap();
3883            let syncer = db.lock();
3884            assert_eq!(get_meta::<i64>(&syncer, LAST_SYNC_META_KEY)?, Some(2_000));
3885
3886            // Temporarily drop the sync connection to avoid a deadlock when the sync engine locks
3887            // the mutex
3888            drop(syncer);
3889            engine.reset(&EngineSyncAssociation::Disconnected)?;
3890            let syncer = db.lock();
3891            assert_eq!(
3892                get_meta::<Option<String>>(&syncer, GLOBAL_SYNCID_META_KEY)?,
3893                None
3894            );
3895            assert_eq!(
3896                get_meta::<Option<String>>(&syncer, COLLECTION_SYNCID_META_KEY)?,
3897                None
3898            );
3899            assert_eq!(get_meta::<i64>(&syncer, LAST_SYNC_META_KEY)?, Some(0));
3900        }
3901
3902        Ok(())
3903    }
3904
3905    #[test]
3906    fn test_dedupe_local_newer() -> anyhow::Result<()> {
3907        let api = new_mem_api();
3908        let writer = api.open_connection(ConnectionType::ReadWrite)?;
3909
3910        let local_modified = Timestamp::now();
3911        let remote_modified = local_modified.as_millis() as f64 / 1000f64 - 5f64;
3912
3913        // Start with merged items.
3914        apply_incoming(
3915            &api,
3916            ServerTimestamp::from_float_seconds(remote_modified),
3917            json!([{
3918                "id": "menu",
3919                "type": "folder",
3920                "parentid": "places",
3921                "parentName": "",
3922                "title": "menu",
3923                "children": ["bookmarkAAA5"],
3924                "modified": remote_modified,
3925            }, {
3926                "id": "bookmarkAAA5",
3927                "type": "bookmark",
3928                "parentid": "menu",
3929                "parentName": "menu",
3930                "title": "A",
3931                "bmkUri": "http://example.com/a",
3932                "modified": remote_modified,
3933            }]),
3934        );
3935
3936        // Add newer local dupes.
3937        insert_local_json_tree(
3938            &writer,
3939            json!({
3940                "guid": &BookmarkRootGuid::Menu.as_guid(),
3941                "children": [{
3942                    "guid": "bookmarkAAA1",
3943                    "title": "A",
3944                    "url": "http://example.com/a",
3945                    "date_added": local_modified,
3946                    "last_modified": local_modified,
3947                }, {
3948                    "guid": "bookmarkAAA2",
3949                    "title": "A",
3950                    "url": "http://example.com/a",
3951                    "date_added": local_modified,
3952                    "last_modified": local_modified,
3953                }, {
3954                    "guid": "bookmarkAAA3",
3955                    "title": "A",
3956                    "url": "http://example.com/a",
3957                    "date_added": local_modified,
3958                    "last_modified": local_modified,
3959                }],
3960            }),
3961        );
3962
3963        // Add older remote dupes.
3964        apply_incoming(
3965            &api,
3966            ServerTimestamp(local_modified.as_millis() as i64),
3967            json!([{
3968                "id": "menu",
3969                "type": "folder",
3970                "parentid": "places",
3971                "parentName": "",
3972                "title": "menu",
3973                "children": ["bookmarkAAAA", "bookmarkAAA4", "bookmarkAAA5"],
3974            }, {
3975                "id": "bookmarkAAAA",
3976                "type": "bookmark",
3977                "parentid": "menu",
3978                "parentName": "menu",
3979                "title": "A",
3980                "bmkUri": "http://example.com/a",
3981                "modified": remote_modified,
3982            }, {
3983                "id": "bookmarkAAA4",
3984                "type": "bookmark",
3985                "parentid": "menu",
3986                "parentName": "menu",
3987                "title": "A",
3988                "bmkUri": "http://example.com/a",
3989                "modified": remote_modified,
3990            }]),
3991        );
3992
3993        assert_local_json_tree(
3994            &writer,
3995            &BookmarkRootGuid::Menu.as_guid(),
3996            json!({
3997                "guid": &BookmarkRootGuid::Menu.as_guid(),
3998                "children": [{
3999                    "guid": "bookmarkAAAA",
4000                    "title": "A",
4001                    "url": "http://example.com/a",
4002                }, {
4003                    "guid": "bookmarkAAA4",
4004                    "title": "A",
4005                    "url": "http://example.com/a",
4006                }, {
4007                    "guid": "bookmarkAAA5",
4008                    "title": "A",
4009                    "url": "http://example.com/a",
4010                }, {
4011                    "guid": "bookmarkAAA3",
4012                    "title": "A",
4013                    "url": "http://example.com/a",
4014                }],
4015            }),
4016        );
4017
4018        Ok(())
4019    }
4020
4021    #[test]
4022    fn test_deduping_remote_newer() -> anyhow::Result<()> {
4023        let api = new_mem_api();
4024        let writer = api.open_connection(ConnectionType::ReadWrite)?;
4025
4026        let local_modified = Timestamp::from(Timestamp::now().as_millis() - 5000);
4027        let remote_modified = local_modified.as_millis() as f64 / 1000f64;
4028
4029        // Start with merged items.
4030        apply_incoming(
4031            &api,
4032            ServerTimestamp::from_float_seconds(remote_modified),
4033            json!([{
4034                "id": "menu",
4035                "type": "folder",
4036                "parentid": "places",
4037                "parentName": "",
4038                "title": "menu",
4039                "children": ["folderAAAAAA"],
4040                "modified": remote_modified,
4041            }, {
4042                // Shouldn't dedupe to `folderA11111` because it's been applied.
4043                "id": "folderAAAAAA",
4044                "type": "folder",
4045                "parentid": "menu",
4046                "parentName": "menu",
4047                "title": "A",
4048                "children": ["bookmarkGGGG"],
4049                "modified": remote_modified,
4050            }, {
4051                // Shouldn't dedupe to `bookmarkG111`.
4052                "id": "bookmarkGGGG",
4053                "type": "bookmark",
4054                "parentid": "folderAAAAAA",
4055                "parentName": "A",
4056                "title": "G",
4057                "bmkUri": "http://example.com/g",
4058                "modified": remote_modified,
4059            }]),
4060        );
4061
4062        // Add older local dupes.
4063        insert_local_json_tree(
4064            &writer,
4065            json!({
4066                "guid": "folderAAAAAA",
4067                "children": [{
4068                    // Not a candidate for `bookmarkH111` because we didn't dupe `folderAAAAAA`.
4069                    "guid": "bookmarkHHHH",
4070                    "title": "H",
4071                    "url": "http://example.com/h",
4072                    "date_added": local_modified,
4073                    "last_modified": local_modified,
4074                }]
4075            }),
4076        );
4077        insert_local_json_tree(
4078            &writer,
4079            json!({
4080                "guid": &BookmarkRootGuid::Menu.as_guid(),
4081                "children": [{
4082                    // Should dupe to `folderB11111`.
4083                    "guid": "folderBBBBBB",
4084                    "type": BookmarkType::Folder as u8,
4085                    "title": "B",
4086                    "date_added": local_modified,
4087                    "last_modified": local_modified,
4088                    "children": [{
4089                        // Should dupe to `bookmarkC222`.
4090                        "guid": "bookmarkC111",
4091                        "title": "C",
4092                        "url": "http://example.com/c",
4093                        "date_added": local_modified,
4094                        "last_modified": local_modified,
4095                    }, {
4096                        // Should dupe to `separatorF11` because the positions are the same.
4097                        "guid": "separatorFFF",
4098                        "type": BookmarkType::Separator as u8,
4099                        "date_added": local_modified,
4100                        "last_modified": local_modified,
4101                    }],
4102                }, {
4103                    // Shouldn't dupe to `separatorE11`, because the positions are different.
4104                    "guid": "separatorEEE",
4105                    "type": BookmarkType::Separator as u8,
4106                    "date_added": local_modified,
4107                    "last_modified": local_modified,
4108                }, {
4109                    // Shouldn't dupe to `bookmarkC222` because the parents are different.
4110                    "guid": "bookmarkCCCC",
4111                    "title": "C",
4112                    "url": "http://example.com/c",
4113                    "date_added": local_modified,
4114                    "last_modified": local_modified,
4115                }, {
4116                    // Should dupe to `queryD111111`.
4117                    "guid": "queryDDDDDDD",
4118                    "title": "Most Visited",
4119                    "url": "place:maxResults=10&sort=8",
4120                    "date_added": local_modified,
4121                    "last_modified": local_modified,
4122                }],
4123            }),
4124        );
4125
4126        // Add newer remote items.
4127        apply_incoming(
4128            &api,
4129            ServerTimestamp::from_float_seconds(remote_modified),
4130            json!([{
4131                "id": "menu",
4132                "type": "folder",
4133                "parentid": "places",
4134                "parentName": "",
4135                "title": "menu",
4136                "children": ["folderAAAAAA", "folderB11111", "folderA11111", "separatorE11", "queryD111111"],
4137                "dateAdded": local_modified.as_millis(),
4138                "modified": remote_modified + 5f64,
4139            }, {
4140                "id": "folderB11111",
4141                "type": "folder",
4142                "parentid": "menu",
4143                "parentName": "menu",
4144                "title": "B",
4145                "children": ["bookmarkC222", "separatorF11"],
4146                "dateAdded": local_modified.as_millis(),
4147                "modified": remote_modified + 5f64,
4148            }, {
4149                "id": "bookmarkC222",
4150                "type": "bookmark",
4151                "parentid": "folderB11111",
4152                "parentName": "B",
4153                "title": "C",
4154                "bmkUri": "http://example.com/c",
4155                "dateAdded": local_modified.as_millis(),
4156                "modified": remote_modified + 5f64,
4157            }, {
4158                "id": "separatorF11",
4159                "type": "separator",
4160                "parentid": "folderB11111",
4161                "parentName": "B",
4162                "dateAdded": local_modified.as_millis(),
4163                "modified": remote_modified + 5f64,
4164            }, {
4165                "id": "folderA11111",
4166                "type": "folder",
4167                "parentid": "menu",
4168                "parentName": "menu",
4169                "title": "A",
4170                "children": ["bookmarkG111"],
4171                "dateAdded": local_modified.as_millis(),
4172                "modified": remote_modified + 5f64,
4173            }, {
4174                "id": "bookmarkG111",
4175                "type": "bookmark",
4176                "parentid": "folderA11111",
4177                "parentName": "A",
4178                "title": "G",
4179                "bmkUri": "http://example.com/g",
4180                "dateAdded": local_modified.as_millis(),
4181                "modified": remote_modified + 5f64,
4182            }, {
4183                "id": "separatorE11",
4184                "type": "separator",
4185                "parentid": "folderB11111",
4186                "parentName": "B",
4187                "dateAdded": local_modified.as_millis(),
4188                "modified": remote_modified + 5f64,
4189            }, {
4190                "id": "queryD111111",
4191                "type": "query",
4192                "parentid": "menu",
4193                "parentName": "menu",
4194                "title": "Most Visited",
4195                "bmkUri": "place:maxResults=10&sort=8",
4196                "dateAdded": local_modified.as_millis(),
4197                "modified": remote_modified + 5f64,
4198            }]),
4199        );
4200
4201        assert_local_json_tree(
4202            &writer,
4203            &BookmarkRootGuid::Menu.as_guid(),
4204            json!({
4205                "guid": &BookmarkRootGuid::Menu.as_guid(),
4206                "children": [{
4207                    "guid": "folderAAAAAA",
4208                    "children": [{
4209                        "guid": "bookmarkGGGG",
4210                        "title": "G",
4211                        "url": "http://example.com/g",
4212                    }, {
4213                        "guid": "bookmarkHHHH",
4214                        "title": "H",
4215                        "url": "http://example.com/h",
4216                    }]
4217                }, {
4218                    "guid": "folderB11111",
4219                    "children": [{
4220                        "guid": "bookmarkC222",
4221                        "title": "C",
4222                        "url": "http://example.com/c",
4223                    }, {
4224                        "guid": "separatorF11",
4225                        "type": BookmarkType::Separator as u8,
4226                    }],
4227                }, {
4228                    "guid": "folderA11111",
4229                    "children": [{
4230                        "guid": "bookmarkG111",
4231                        "title": "G",
4232                        "url": "http://example.com/g",
4233                    }]
4234                }, {
4235                    "guid": "separatorE11",
4236                    "type": BookmarkType::Separator as u8,
4237                }, {
4238                    "guid": "queryD111111",
4239                    "title": "Most Visited",
4240                    "url": "place:maxResults=10&sort=8",
4241                }, {
4242                    "guid": "separatorEEE",
4243                    "type": BookmarkType::Separator as u8,
4244                }, {
4245                    "guid": "bookmarkCCCC",
4246                    "title": "C",
4247                    "url": "http://example.com/c",
4248                }],
4249            }),
4250        );
4251
4252        Ok(())
4253    }
4254
4255    #[test]
4256    fn test_reconcile_sync_metadata() -> anyhow::Result<()> {
4257        let api = new_mem_api();
4258        let writer = api.open_connection(ConnectionType::ReadWrite)?;
4259
4260        let local_modified = Timestamp::from(Timestamp::now().as_millis() - 5000);
4261        let remote_modified = local_modified.as_millis() as f64 / 1000f64;
4262
4263        insert_local_json_tree(
4264            &writer,
4265            json!({
4266                "guid": &BookmarkRootGuid::Menu.as_guid(),
4267                "children": [{
4268                    // this folder is going to reconcile exactly
4269                    "guid": "folderAAAAAA",
4270                    "type": BookmarkType::Folder as u8,
4271                    "title": "A",
4272                    "date_added": local_modified,
4273                    "last_modified": local_modified,
4274                    "children": [{
4275                        "guid": "bookmarkBBBB",
4276                        "title": "B",
4277                        "url": "http://example.com/b",
4278                        "date_added": local_modified,
4279                        "last_modified": local_modified,
4280                    }]
4281                }, {
4282                    // this folder's existing child isn't on the server (so will be
4283                    // outgoing) and also will take a new child from the server.
4284                    "guid": "folderCCCCCC",
4285                    "type": BookmarkType::Folder as u8,
4286                    "title": "C",
4287                    "date_added": local_modified,
4288                    "last_modified": local_modified,
4289                    "children": [{
4290                        "guid": "bookmarkEEEE",
4291                        "title": "E",
4292                        "url": "http://example.com/e",
4293                        "date_added": local_modified,
4294                        "last_modified": local_modified,
4295                    }]
4296                }, {
4297                    // This bookmark is going to take the remote title.
4298                    "guid": "bookmarkFFFF",
4299                    "title": "f",
4300                    "url": "http://example.com/f",
4301                    "date_added": local_modified,
4302                    "last_modified": local_modified,
4303                }],
4304            }),
4305        );
4306
4307        let outgoing = apply_incoming(
4308            &api,
4309            ServerTimestamp::from_float_seconds(remote_modified),
4310            json!([{
4311                "id": "menu",
4312                "type": "folder",
4313                "parentid": "places",
4314                "parentName": "",
4315                "title": "menu",
4316                "children": ["folderAAAAAA", "folderCCCCCC", "bookmarkFFFF"],
4317                "dateAdded": local_modified.as_millis(),
4318                "modified": remote_modified,
4319            }, {
4320                "id": "folderAAAAAA",
4321                "type": "folder",
4322                "parentid": "menu",
4323                "parentName": "menu",
4324                "title": "A",
4325                "children": ["bookmarkBBBB"],
4326                "dateAdded": local_modified.as_millis(),
4327                "modified": remote_modified,
4328            }, {
4329                "id": "bookmarkBBBB",
4330                "type": "bookmark",
4331                "parentid": "folderAAAAAA",
4332                "parentName": "A",
4333                "title": "B",
4334                "bmkUri": "http://example.com/b",
4335                "dateAdded": local_modified.as_millis(),
4336                "modified": remote_modified,
4337            }, {
4338                "id": "folderCCCCCC",
4339                "type": "folder",
4340                "parentid": "menu",
4341                "parentName": "menu",
4342                "title": "C",
4343                "children": ["bookmarkDDDD"],
4344                "dateAdded": local_modified.as_millis(),
4345                "modified": remote_modified,
4346            }, {
4347                "id": "bookmarkDDDD",
4348                "type": "bookmark",
4349                "parentid": "folderCCCCCC",
4350                "parentName": "C",
4351                "title": "D",
4352                "bmkUri": "http://example.com/d",
4353                "dateAdded": local_modified.as_millis(),
4354                "modified": remote_modified,
4355            }, {
4356                "id": "bookmarkFFFF",
4357                "type": "bookmark",
4358                "parentid": "menu",
4359                "parentName": "menu",
4360                "title": "F",
4361                "bmkUri": "http://example.com/f",
4362                "dateAdded": local_modified.as_millis(),
4363                "modified": remote_modified + 5f64,
4364            },]),
4365        );
4366
4367        // Assert the tree is correct even though that's not really the point
4368        // of this test.
4369        assert_local_json_tree(
4370            &writer,
4371            &BookmarkRootGuid::Menu.as_guid(),
4372            json!({
4373                "guid": &BookmarkRootGuid::Menu.as_guid(),
4374                "children": [{
4375                    // this folder is going to reconcile exactly
4376                    "guid": "folderAAAAAA",
4377                    "type": BookmarkType::Folder as u8,
4378                    "title": "A",
4379                    "children": [{
4380                        "guid": "bookmarkBBBB",
4381                        "title": "B",
4382                        "url": "http://example.com/b",
4383                    }]
4384                }, {
4385                    "guid": "folderCCCCCC",
4386                    "type": BookmarkType::Folder as u8,
4387                    "title": "C",
4388                    "children": [{
4389                        "guid": "bookmarkDDDD",
4390                        "title": "D",
4391                        "url": "http://example.com/d",
4392                    },{
4393                        "guid": "bookmarkEEEE",
4394                        "title": "E",
4395                        "url": "http://example.com/e",
4396                    }]
4397                }, {
4398                    "guid": "bookmarkFFFF",
4399                    "title": "F",
4400                    "url": "http://example.com/f",
4401                }],
4402            }),
4403        );
4404
4405        // After application everything should have SyncStatus::Normal and
4406        // a change counter of zero.
4407        for guid in &[
4408            "folderAAAAAA",
4409            "bookmarkBBBB",
4410            "folderCCCCCC",
4411            "bookmarkDDDD",
4412            "bookmarkFFFF",
4413        ] {
4414            let bm = get_raw_bookmark(&writer, &guid.into())
4415                .expect("must work")
4416                .expect("must exist");
4417            assert_eq!(bm._sync_status, SyncStatus::Normal, "{}", guid);
4418            assert_eq!(bm._sync_change_counter, 0, "{}", guid);
4419        }
4420        // And bookmarkEEEE wasn't on the server, so should be outgoing, and
4421        // it's parent too.
4422        assert!(outgoing.contains(&"bookmarkEEEE".into()));
4423        assert!(outgoing.contains(&"folderCCCCCC".into()));
4424        Ok(())
4425    }
4426
4427    /*
4428     * Due to bug 1935797, Users were running into a state where in itemsToApply
4429     * localID = None/Null, but mergedGuid was something already locally in the
4430     * tree -- this lead to an uptick of guid collision issues in `apply_remote_items`
4431     * below is an example of a 'user' going into this state and the new code fixing it
4432     */
4433    #[test]
4434    fn test_handle_unique_guid_violation() -> Result<()> {
4435        let api = new_mem_api();
4436        let db = api.get_sync_connection().unwrap();
4437        let conn = db.lock();
4438
4439        conn.execute_batch(
4440            r#"
4441            INSERT INTO moz_places(url, guid, title, frecency)
4442            VALUES
4443                ('http://example.com/', 'testPlaceGuidAAAA', 'Example site', 0)
4444            "#,
4445        )?;
4446
4447        // Insert a local row in moz_bookmarks with guid="collisionGUI"
4448        // so we already have that GUID in the table.
4449        conn.execute_batch(&format!(
4450            r#"
4451        INSERT INTO moz_bookmarks(guid, parent, fk, position, type)
4452        VALUES (
4453            'collisionGUI',
4454            (SELECT id FROM moz_bookmarks WHERE guid = '{menu}'),
4455            (SELECT id FROM moz_places WHERE guid = 'testPlaceGuidAAAA'),
4456            0,
4457            1  -- type=1 => bookmark
4458        );
4459        "#,
4460            menu = BookmarkRootGuid::Menu.as_guid(),
4461        ))?;
4462
4463        // Insert a row into itemsToApply that will cause an insert
4464        // with the same guid="collisionGUI".
4465        // localId is NULL, so the engine sees it as a "new" local item,
4466        // and remoteId could be any integer. We set newKind=1 => "bookmark."
4467        conn.execute(
4468            r#"
4469        INSERT INTO itemsToApply(
4470            mergedGuid,
4471            localId,
4472            remoteId,
4473            remoteGuid,
4474            newKind,
4475            newLevel,
4476            newTitle,
4477            newPlaceId,
4478            oldPlaceId,
4479            localDateAdded,
4480            remoteDateAdded,
4481            lastModified
4482        )
4483        VALUES (
4484            ?1,        -- mergedGuid
4485            NULL,      -- localId => so it doesn't unify
4486            999,       -- remoteId => arbitrary
4487            ?1,        -- remoteGuid
4488            1,         -- newKind=1 => bookmark
4489            0,         -- level
4490            'New Title',   -- newTitle
4491            1,             -- newPlaceId
4492            NULL,          -- oldPlaceId
4493            1000,          -- localDateAdded
4494            2000,          -- remoteDateAdded
4495            2000           -- lastModified
4496        )
4497        "#,
4498            [&"collisionGUI"],
4499        )?;
4500
4501        // Call apply_remote_items directly.
4502        // This tries "INSERT INTO moz_bookmarks(guid='collisionGUI')"
4503        // and should NOT fail with a unique constraint.
4504        let scope = conn.begin_interrupt_scope()?;
4505        apply_remote_items(&conn, &scope, Timestamp(999))?;
4506
4507        // Assert the tree still looks valid after applying
4508        assert_local_json_tree(
4509            &conn,
4510            &BookmarkRootGuid::Menu.as_guid(),
4511            json!({
4512                "guid": &BookmarkRootGuid::Menu.as_guid(),
4513                // should only be one child
4514                "children": [{
4515                    "guid": "collisionGUI",
4516                    "title": "New Title", // title was updated from remote
4517                    "url": "http://example.com/",
4518                }],
4519            }),
4520        );
4521        Ok(())
4522    }
4523}