segmented_changelog/tailer: remove boilerplate code from logging

Summary: Writing `repo: {}` everywhere is quite annoying when we can just set a value in the logger instead.

Reviewed By: farnz

Differential Revision: D33845555

fbshipit-source-id: 0e6773549c3913df68719362fa1c926f205802fe
This commit is contained in:
Ilia Medianikov 2022-01-28 10:10:51 -08:00 committed by Facebook GitHub Bot
parent a54653df2f
commit f29f4fb675
12 changed files with 117 additions and 129 deletions

View File

@ -15,7 +15,7 @@ use bytes::Bytes;
use clap::Arg;
use futures::future::join_all;
use futures::stream;
use slog::{error, info};
use slog::{error, info, o};
use changesets::deserialize_cs_entries;
use cmdlib::{
@ -145,6 +145,8 @@ async fn run<'a>(ctx: CoreContext, matches: &'a MononokeMatches<'a>) -> Result<(
let blobrepo: BlobRepo =
args::open_repo_with_repo_id(ctx.fb, ctx.logger(), repo_id, matches).await?;
let ctx = ctx.clone_with_logger(ctx.logger().new(o!("repo_id" => repo_id.to_string())));
let prefetched_commits = stream::iter(prefetched_commits.iter().filter_map(|entry| {
if entry.repo_id == repo_id {
Some(Ok(entry.clone()))
@ -160,9 +162,9 @@ async fn run<'a>(ctx: CoreContext, matches: &'a MononokeMatches<'a>) -> Result<(
let head = helpers::csid_resolve(&ctx, blobrepo.clone(), head_arg)
.await
.with_context(|| {
format!("repo {}: resolving head csid for '{}'", repo_id, head_arg)
format!("resolving head csid '{}' for repo {}", head_arg, repo_id)
})?;
info!(ctx.logger(), "repo {}: using '{}' for head", repo_id, head);
info!(ctx.logger(), "using '{}' for head", head);
heads.push(head.into());
}
heads
@ -181,20 +183,14 @@ async fn run<'a>(ctx: CoreContext, matches: &'a MononokeMatches<'a>) -> Result<(
)
.await?;
info!(
ctx.logger(),
"repo {}: SegmentedChangelogTailer initialized", repo_id
);
info!(ctx.logger(), "SegmentedChangelogTailer initialized",);
if matches.is_present(ONCE_ARG) || matches.is_present(FORCE_RESEED_ARG) {
segmented_changelog_tailer
.once(&ctx, matches.is_present(FORCE_RESEED_ARG))
.await
.with_context(|| format!("repo {}: incrementally building repo", repo_id))?;
info!(
ctx.logger(),
"repo {}: SegmentedChangelogTailer is done", repo_id,
);
.with_context(|| format!("incrementally building repo {}", repo_id))?;
info!(ctx.logger(), "SegmentedChangelogTailer is done",);
} else if let Some(period) = config.segmented_changelog_config.tailer_update_period {
// spread out update operations, start updates on another repo after 7 seconds
let wait_to_start = Duration::from_secs(7 * index as u64);

View File

@ -124,7 +124,7 @@ impl CloneHints {
) -> Result<()> {
info!(
ctx.logger(),
"Adding hints for repo {} idmap_version {}", self.inner.repo_id, idmap_version.0
"Adding hints for idmap_version {}", idmap_version.0
);
let universal_ids: Vec<_> = namedag
.dag()
@ -142,8 +142,7 @@ impl CloneHints {
if new_ids.len() < HINTS_PER_CHUNK {
info!(
ctx.logger(),
"repo {} idmap_version {} has a full set of hints ({} unhinted IDs is less than chunk size of {})",
self.inner.repo_id,
"idmap_version {} has a full set of hints ({} unhinted IDs is less than chunk size of {})",
idmap_version.0,
new_ids.len(),
HINTS_PER_CHUNK
@ -229,10 +228,7 @@ impl CloneHints {
)
.await?;
info!(
ctx.logger(),
"Hints uploaded for repo {}", self.inner.repo_id
);
info!(ctx.logger(), "Hints uploaded",);
Ok(())
}
}

View File

@ -7,7 +7,7 @@
use async_trait::async_trait;
use context::CoreContext;
use mononoke_types::{ChangesetId, RepositoryId};
use mononoke_types::ChangesetId;
use slog::info;
use std::collections::HashMap;
use std::sync::Arc;
@ -55,8 +55,6 @@ pub fn vertex_name_from_cs_id(cs_id: &ChangesetId) -> VertexName {
#[derive(Clone)]
pub struct IdMapMemWrites {
/// Needed for logging
repo_id: RepositoryId,
/// The actual IdMap
inner: Arc<dyn IdMap>,
/// Stores recent writes that haven't yet been persisted to the backing store
@ -64,19 +62,17 @@ pub struct IdMapMemWrites {
}
impl IdMapMemWrites {
pub fn new(repo_id: RepositoryId, inner: Arc<dyn IdMap>) -> Self {
pub fn new(inner: Arc<dyn IdMap>) -> Self {
Self {
inner,
mem: Arc::new(ConcurrentMemIdMap::new()),
repo_id,
}
}
pub async fn flush_writes(&self, ctx: &CoreContext) -> anyhow::Result<()> {
info!(
ctx.logger(),
"repo {}: flushing {} in-memory IdMap entries to SQL",
self.repo_id,
"flushing {} in-memory IdMap entries to SQL",
self.mem.len(),
);
let writes = self.mem.drain();
@ -116,8 +112,7 @@ impl IdMap for IdMapMemWrites {
if new_size / sampling_rate != old_size / sampling_rate {
info!(
ctx.logger(),
"repo {}: {} entries inserted into in-memory IdMap, new size: {}",
self.repo_id,
"{} entries inserted into in-memory IdMap, new size: {}",
mappings_size,
new_size,
);
@ -235,8 +230,8 @@ pub struct IdMapWrapper {
impl IdMapWrapper {
/// Create a new wrapper around the server IdMap, using CoreContext
/// for calling update functions
pub fn new(ctx: CoreContext, repo_id: RepositoryId, idmap: Arc<dyn IdMap>) -> Self {
let idmap_memwrites = IdMapMemWrites::new(repo_id, idmap);
pub fn new(ctx: CoreContext, idmap: Arc<dyn IdMap>) -> Self {
let idmap_memwrites = IdMapMemWrites::new(idmap);
Self {
verlink: VerLink::new(),
inner: idmap_memwrites,

View File

@ -35,8 +35,7 @@ pub fn log_new_segmented_changelog_version(
) {
slog::info!(
ctx.logger(),
"repo {}: segmented changelog version saved, idmap_version: {}, iddag_version: {}",
repo_id,
"segmented changelog version saved, idmap_version: {}, iddag_version: {}",
sc_version.idmap_version,
sc_version.iddag_version,
);

View File

@ -110,7 +110,7 @@ impl OnDemandUpdateSegmentedChangelog {
seed_heads: Vec<SeedHead>,
clone_hints: Option<CloneHints>,
) -> Result<Self> {
let namedag = server_namedag(ctx, repo_id, iddag, idmap)?;
let namedag = server_namedag(ctx, iddag, idmap)?;
let namedag = Arc::new(RwLock::new(namedag));
Ok(Self {
repo_id,

View File

@ -142,13 +142,13 @@ impl SegmentedChangelogTailer {
ReadOnlyStorage(false),
)
.await
.with_context(|| format!("repo {}: constructing metadata sql factory", repo_id))?;
.with_context(|| format!("constructing metadata sql factory for repo {}", repo_id))?;
let segmented_changelog_sql_connections = sql_factory
.open::<SegmentedChangelogSqlConnections>()
.with_context(|| {
format!(
"repo {}: error constructing segmented changelog sql connections",
"error constructing segmented changelog sql connections for repo {}",
repo_id
)
})?;
@ -190,7 +190,7 @@ impl SegmentedChangelogTailer {
let mut interval = tokio::time::interval(period);
loop {
let _ = interval.tick().await;
debug!(ctx.logger(), "repo {}: woke up to update", self.repo_id,);
debug!(ctx.logger(), "woke up to update");
STATS::count.add_value(1);
STATS::count_per_repo.add_value(1, (self.repo_id.id(),));
@ -219,9 +219,7 @@ impl SegmentedChangelogTailer {
STATS::failure_per_repo.add_value(1, (self.repo_id.id(),));
error!(
ctx.logger(),
"repo {}: failed to incrementally update segmented changelog: {:?}",
self.repo_id,
err
"failed to incrementally update segmented changelog: {:?}", err
);
Some(format!("{:?}", err))
}
@ -237,13 +235,13 @@ impl SegmentedChangelogTailer {
) -> Result<OwnedSegmentedChangelog> {
info!(
ctx.logger(),
"repo {}: starting incremental update to segmented changelog", self.repo_id,
"starting incremental update to segmented changelog",
);
let (seeding, idmap_version, iddag) = {
let sc_version = self.sc_version_store.get(&ctx).await.with_context(|| {
format!(
"repo {}: error loading segmented changelog version",
"error loading segmented changelog version for repo {}",
self.repo_id
)
})?;
@ -262,7 +260,7 @@ impl SegmentedChangelogTailer {
.load(&ctx, sc_version.iddag_version)
.await
.with_context(|| {
format!("repo {}: failed to load iddag", self.repo_id)
format!("failed to load iddag for repo {}", self.repo_id)
})?;
(false, sc_version.idmap_version, iddag)
}
@ -274,14 +272,13 @@ impl SegmentedChangelogTailer {
if let Ok(set) = iddag.all() {
info!(
ctx.logger(),
"repo {}: iddag initialized, it covers {} ids",
self.repo_id,
"iddag initialized, it covers {} ids",
set.count(),
);
}
let idmap = self.idmap_factory.for_writer(ctx, idmap_version);
let mut namedag = server_namedag(ctx.clone(), self.repo_id, iddag, idmap)?;
let mut namedag = server_namedag(ctx.clone(), iddag, idmap)?;
let heads =
vertexlist_from_seedheads(&ctx, &self.seed_heads, self.bookmarks.as_ref()).await?;
@ -325,10 +322,7 @@ impl SegmentedChangelogTailer {
.bulk_fetch
.get_repo_bounds_after_commits(ctx, head_commits)
.await?;
info!(
ctx.logger(),
"repo {}: prefetching changeset entries", self.repo_id,
);
info!(ctx.logger(), "prefetching changeset entries",);
let mut counter = 0usize;
// This has the potential to cause OOM by fetching a large
// chunk of the repo
@ -347,9 +341,7 @@ impl SegmentedChangelogTailer {
if counter % sampling_rate == 0 {
info!(
ctx.logger(),
"repo {}: fetched {} changeset entries in total",
self.repo_id,
counter,
"fetched {} changeset entries in total", counter,
);
}
res
@ -362,10 +354,7 @@ impl SegmentedChangelogTailer {
let parent_fetcher = FetchParents::new(ctx.clone(), changeset_fetcher);
info!(
ctx.logger(),
"repo {}: starting the actual update", self.repo_id,
);
info!(ctx.logger(), "starting the actual update");
// Note on memory use: we do not flush the changes out in the middle
// of writing to the IdMap.
// Thus, if OOMs happen here, the IdMap may need to flush writes to the DB
@ -387,24 +376,20 @@ impl SegmentedChangelogTailer {
if !changed {
info!(
ctx.logger(),
"repo {}: segmented changelog already up to date, skipping update to iddag",
self.repo_id
"segmented changelog already up to date, skipping update to iddag",
);
let owned = OwnedSegmentedChangelog::new(iddag, idmap);
return Ok(owned);
}
info!(
ctx.logger(),
"repo {}: IdMap updated, IdDag updated", self.repo_id
);
info!(ctx.logger(), "IdMap updated, IdDag updated",);
// Save the IdDag
let iddag_version = self
.iddag_save_store
.save(&ctx, &iddag)
.await
.with_context(|| format!("repo {}: error saving iddag", self.repo_id))?;
.with_context(|| format!("error saving iddag for repo {}", self.repo_id))?;
// Update SegmentedChangelogVersion
let sc_version = SegmentedChangelogVersion::new(iddag_version, idmap_version);
@ -414,27 +399,24 @@ impl SegmentedChangelogTailer {
.await
.with_context(|| {
format!(
"repo {}: error updating segmented changelog version store",
"error updating segmented changelog version store for repo {}",
self.repo_id
)
})?;
info!(
ctx.logger(),
"repo {}: successfully seeded segmented changelog", self.repo_id,
);
info!(ctx.logger(), "successfully seeded segmented changelog",);
} else {
self.sc_version_store
.update(&ctx, sc_version)
.await
.with_context(|| {
format!(
"repo {}: error updating segmented changelog version store",
"error updating segmented changelog version store for repo {}",
self.repo_id
)
})?;
info!(
ctx.logger(),
"repo {}: successful incremental update to segmented changelog", self.repo_id,
"successful incremental update to segmented changelog",
);
}

View File

@ -17,7 +17,7 @@ use bookmarks::{
};
use context::CoreContext;
use metaconfig_types::SegmentedChangelogConfig;
use mononoke_types::{ChangesetId, RepositoryId};
use mononoke_types::ChangesetId;
use crate::dag::{NameDagBuilder, VertexListWithOptions, VertexName, VertexOptions};
use crate::idmap::{vertex_name_from_cs_id, IdMap, IdMapWrapper};
@ -129,11 +129,10 @@ pub type ServerNameDag = crate::dag::namedag::AbstractNameDag<InProcessIdDag, Id
/// to write out updates to the IdMap
pub fn server_namedag(
ctx: CoreContext,
repo_id: RepositoryId,
iddag: InProcessIdDag,
idmap: Arc<dyn IdMap>,
) -> Result<ServerNameDag> {
let idmap = IdMapWrapper::new(ctx, repo_id, idmap);
let idmap = IdMapWrapper::new(ctx, idmap);
NameDagBuilder::new_with_idmap_dag(idmap, iddag)
.build()
.map_err(anyhow::Error::from)

View File

@ -73,6 +73,14 @@ impl CoreContext {
}
}
pub fn clone_with_logger(&self, logger: Logger) -> Self {
Self {
fb: self.fb,
session: self.session.clone(),
logging: self.logging.clone_with_logger(logger),
}
}
pub fn with_mutated_scuba(
&self,
mutator: impl FnOnce(MononokeScubaSampleBuilder) -> MononokeScubaSampleBuilder,

View File

@ -74,6 +74,16 @@ impl LoggingContainer {
}
}
pub fn clone_with_logger(&self, logger: Logger) -> Self {
Self {
logger,
scuba: self.scuba.clone(),
perf_counters: self.perf_counters.clone(),
sampling_key: self.sampling_key.clone(),
scribe: self.scribe.clone(),
}
}
pub fn with_scribe(&mut self, scribe: Scribe) -> &mut Self {
self.scribe = scribe;
self

View File

@ -40,20 +40,21 @@ Setup repository
Run Segmented Changelog Tailer. This seeds the segmented changelog.
$ quiet segmented_changelog_tailer_once --head master_bookmark --repo repo
$ grep -e "repo 0" -e "segmented_changelog_tailer" "$TESTTMP/quiet.last.log"
$ grep -e "repo_id: 0" -e "segmented_changelog_tailer" "$TESTTMP/quiet.last.log"
* repo name 'repo' translates to id 0 (glob)
* repo 0: using * for head (glob)
* repo 0: SegmentedChangelogTailer initialized (glob)
* repo 0: starting incremental update to segmented changelog (glob)
* repo 0: iddag initialized, it covers 0 ids (glob)
* repo 0: starting the actual update (glob)
* Adding hints for repo 0 idmap_version 1 (glob)
* repo 0 idmap_version 1 has a full set of hints * (glob)
* repo 0: flushing 3 in-memory IdMap entries to SQL (glob)
* repo 0: IdMap updated, IdDag updated (glob)
* repo 0: segmented changelog version saved, idmap_version: 1, iddag_version: * (glob)
* repo 0: successfully seeded segmented changelog (glob)
* repo 0: SegmentedChangelogTailer is done (glob)
* changeset resolved as: *, repo_id: 0 (glob)
* using * for head, repo_id: 0 (glob)
* SegmentedChangelogTailer initialized, repo_id: 0 (glob)
* starting incremental update to segmented changelog, repo_id: 0 (glob)
* iddag initialized, it covers 0 ids, repo_id: 0 (glob)
* starting the actual update, repo_id: 0 (glob)
* Adding hints for idmap_version 1, repo_id: 0 (glob)
* idmap_version 1 has a full set of hints *, repo_id: 0 (glob)
* flushing 3 in-memory IdMap entries to SQL, repo_id: 0 (glob)
* IdMap updated, IdDag updated, repo_id: 0 (glob)
* segmented changelog version saved, idmap_version: 1, iddag_version: *, repo_id: 0 (glob)
* successfully seeded segmented changelog, repo_id: 0 (glob)
* SegmentedChangelogTailer is done, repo_id: 0 (glob)
Now test without head option (tailer will fetch it from config) and with prefetched commits
@ -63,20 +64,21 @@ Now test without head option (tailer will fetch it from config) and with prefetc
> CONFIG
$ dump_public_changeset_entries --out-filename "$TESTTMP/prefetched_commits" &> /dev/null
$ quiet segmented_changelog_tailer_reseed --repo repo --prefetched-commits-path "$TESTTMP/prefetched_commits"
$ grep -e "repo 0" -e "segmented_changelog_tailer" "$TESTTMP/quiet.last.log"
$ grep -e "repo_id: 0" -e "segmented_changelog_tailer" "$TESTTMP/quiet.last.log"
* reading prefetched commits from $TESTTMP/prefetched_commits (glob)
* repo name 'repo' translates to id 0 (glob)
* repo 0: SegmentedChangelogTailer initialized (glob)
* repo 0: starting incremental update to segmented changelog (glob)
* repo 0: iddag initialized, it covers 0 ids (glob)
* repo 0: starting the actual update (glob)
* Adding hints for repo 0 idmap_version 2 (glob)
* repo 0 idmap_version 2 has a full set of hints * (glob)
* repo 0: flushing 3 in-memory IdMap entries to SQL (glob)
* repo 0: IdMap updated, IdDag updated (glob)
* repo 0: segmented changelog version saved, idmap_version: 2, iddag_version: * (glob)
* repo 0: successfully seeded segmented changelog (glob)
* repo 0: SegmentedChangelogTailer is done (glob)
* using 'Bookmark master_bookmark' for head, repo_id: 0 (glob)
* SegmentedChangelogTailer initialized, repo_id: 0 (glob)
* starting incremental update to segmented changelog, repo_id: 0 (glob)
* iddag initialized, it covers 0 ids, repo_id: 0 (glob)
* starting the actual update, repo_id: 0 (glob)
* Adding hints for idmap_version 2, repo_id: 0 (glob)
* idmap_version 2 has a full set of hints *, repo_id: 0 (glob)
* flushing 3 in-memory IdMap entries to SQL, repo_id: 0 (glob)
* IdMap updated, IdDag updated, repo_id: 0 (glob)
* segmented changelog version saved, idmap_version: 2, iddag_version: *, repo_id: 0 (glob)
* successfully seeded segmented changelog, repo_id: 0 (glob)
* SegmentedChangelogTailer is done, repo_id: 0 (glob)
Add a new commit, and see the tailer tail it in properly
$ cd repo-hg || exit 1
@ -86,21 +88,22 @@ Add a new commit, and see the tailer tail it in properly
$ cd ..
$ blobimport repo-hg/.hg repo --derived-data-type fsnodes
$ quiet segmented_changelog_tailer_once --repo repo
$ grep "repo 0" "$TESTTMP/quiet.last.log"
* repo 0: SegmentedChangelogTailer initialized (glob)
* repo 0: starting incremental update to segmented changelog (glob)
* repo 0: iddag initialized, it covers 3 ids (glob)
* repo 0: starting the actual update (glob)
* Adding hints for repo 0 idmap_version 2 (glob)
* repo 0 idmap_version 2 has a full set of hints * (glob)
* repo 0: flushing 1 in-memory IdMap entries to SQL (glob)
* repo 0: IdMap updated, IdDag updated (glob)
* repo 0: segmented changelog version saved, idmap_version: 2, iddag_version: * (glob)
* repo 0: successful incremental update to segmented changelog (glob)
* repo 0: SegmentedChangelogTailer is done (glob)
$ grep "repo_id: 0" "$TESTTMP/quiet.last.log"
* using 'Bookmark master_bookmark' for head, repo_id: 0 (glob)
* SegmentedChangelogTailer initialized, repo_id: 0 (glob)
* starting incremental update to segmented changelog, repo_id: 0 (glob)
* iddag initialized, it covers 3 ids, repo_id: 0 (glob)
* starting the actual update, repo_id: 0 (glob)
* Adding hints for idmap_version 2, repo_id: 0 (glob)
* idmap_version 2 has a full set of hints *, repo_id: 0 (glob)
* flushing 1 in-memory IdMap entries to SQL, repo_id: 0 (glob)
* IdMap updated, IdDag updated, repo_id: 0 (glob)
* segmented changelog version saved, idmap_version: 2, iddag_version: *, repo_id: 0 (glob)
* successful incremental update to segmented changelog, repo_id: 0 (glob)
* SegmentedChangelogTailer is done, repo_id: 0 (glob)
Run Segmented Changelog Tailer. Nothing to do.
$ quiet segmented_changelog_tailer_once --repo repo
$ grep 'already up to date' "$TESTTMP/quiet.last.log"
* repo 0: segmented changelog already up to date, skipping update to iddag (glob)
* segmented changelog already up to date, skipping update to iddag, repo_id: 0 (glob)

View File

@ -17,20 +17,20 @@ Run Segmented Changelog Tailer to seed the repo.
> CONFIG
$ quiet segmented_changelog_tailer_reseed --repo repo
$ grep 'successfully' "$TESTTMP/quiet.last.log"
* repo 0: successfully seeded segmented changelog (glob)
* successfully seeded segmented changelog, repo_id: 0 (glob)
Run Segmented Changelog Tailer. Nothing to do.
$ quiet segmented_changelog_tailer_once --repo repo
$ grep 'already up to date' "$TESTTMP/quiet.last.log"
* repo 0: segmented changelog already up to date, skipping update to iddag (glob)
* segmented changelog already up to date, skipping update to iddag, repo_id: 0 (glob)
Truncate down to 1 changeset and then tail in the missing two
$ quiet mononoke_admin truncate-segmented-changelog $(hg log -T'{node}' -r 'limit(::master_bookmark, 1)')
$ grep 'segmented changelog version saved' "$TESTTMP/quiet.last.log"
* repo 0: segmented changelog version saved, idmap_version: 2, iddag_version: 5fd1e81c* (glob)
* segmented changelog version saved, idmap_version: 2, iddag_version: 5fd1e81c* (glob)
Run the tailer again, and see it pull in the commits we removed
$ quiet segmented_changelog_tailer_once --repo repo
$ grep 'successful incremental update' "$TESTTMP/quiet.last.log"
* repo 0: successful incremental update to segmented changelog (glob)
* successful incremental update to segmented changelog, repo_id: 0 (glob)

View File

@ -102,9 +102,9 @@
> CONFIG
$ segmented_changelog_tailer_reseed --repo repo 2>&1 | grep -e successfully -e segmented_changelog_tailer
* repo name 'repo' translates to id 0 (glob)
* repo 0: SegmentedChangelogTailer initialized (glob)
* repo 0: successfully seeded segmented changelog (glob)
* repo 0: SegmentedChangelogTailer is done (glob)
* SegmentedChangelogTailer initialized, repo_id: 0 (glob)
* successfully seeded segmented changelog, repo_id: 0 (glob)
* SegmentedChangelogTailer is done, repo_id: 0 (glob)
# Import the repo
# Segmented changelog should be rebuild for newly imported commits along the way.
@ -138,17 +138,17 @@
* Finished deriving data types (glob)
* Start tailing segmented changelog (glob)
* using 'Bookmark master_bookmark' for head (glob)
* repo 0: SegmentedChangelogTailer initialized (glob)
* repo 0: starting incremental update to segmented changelog (glob)
* repo 0: iddag initialized, it covers 3 ids (glob)
* repo 0: starting the actual update (glob)
* Adding hints for repo 0 idmap_version 1 (glob)
* repo 0 idmap_version 1 has a full set of hints * (glob)
* repo 0: flushing 2 in-memory IdMap entries to SQL (glob)
* repo 0: IdMap updated, IdDag updated (glob)
* repo 0: segmented changelog version saved, idmap_version: 1, iddag_version: * (glob)
* repo 0: successful incremental update to segmented changelog (glob)
* repo 0: SegmentedChangelogTailer is done (glob)
* SegmentedChangelogTailer initialized (glob)
* starting incremental update to segmented changelog (glob)
* iddag initialized, it covers 3 ids (glob)
* starting the actual update (glob)
* Adding hints for idmap_version 1 (glob)
* idmap_version 1 has a full set of hints * (glob)
* flushing 2 in-memory IdMap entries to SQL (glob)
* IdMap updated, IdDag updated (glob)
* segmented changelog version saved, idmap_version: 1, iddag_version: * (glob)
* successful incremental update to segmented changelog (glob)
* SegmentedChangelogTailer is done (glob)
* Finished tailing segmented changelog (glob)
* Start moving the bookmark (glob)
* Created bookmark BookmarkName { bookmark: "repo_import_new_repo" } pointing to * (glob)