lib/PublicInbox/AutoReap.pm
lib/PublicInbox/Cgit.pm
lib/PublicInbox/CmdIPC4.pm
+lib/PublicInbox/CodeSearch.pm
+lib/PublicInbox/CodeSearchIdx.pm
lib/PublicInbox/CompressNoop.pm
lib/PublicInbox/Config.pm
lib/PublicInbox/ConfigIter.pm
sa_config/root/etc/spamassassin/public-inbox.pre
sa_config/user/.spamassassin/user_prefs
script/lei
+script/public-inbox-cindex
script/public-inbox-clone
script/public-inbox-compact
script/public-inbox-convert
t/altid_v2.t
t/cgi.t
t/check-www-inbox.perl
+t/cindex.t
t/clone-coderepo-puh1.sh
t/clone-coderepo-puh2.sh
t/clone-coderepo.psgi
--- /dev/null
+# Copyright (C) all contributors <meta@public-inbox.org>
+# License: AGPL-3.0+ <https://www.gnu.org/licenses/agpl-3.0.txt>
+
+# read-only external index for coderepos
+# currently, it only indexes commits and repository metadata
+# (pathname, root commits); not blob contents
+package PublicInbox::CodeSearch;
+use v5.12;
+use parent qw(PublicInbox::Search);
+use PublicInbox::Search qw(retry_reopen int_val xap_terms);
+use constant {
+ AT => 0, # author time YYYYMMDDHHMMSS, dt: for mail)
+ CT => 1, # commit time (Unix time stamp, like TS/rt: in mail)
+ CIDX_SCHEMA_VER => 1, # brand new schema for code search
+ # for repos (`Tr'), CT(col=1) is used for the latest tip commit time
+ # in refs/{heads,tags}. AT(col=0) may be used to store disk usage
+ # in the future, but disk usage calculation is espensive w/ alternates
+};
+
+# note: the non-X term prefix allocations are shared with Xapian omega,
+# see xapian-applications/omega/docs/termprefixes.rst
+# bool_pfx_internal:
+# type => 'T', # 'c' - commit, 'r' - repo GIT_DIR
+# tags are not indexed, only normal branches (refs/heads/*), not hidden
+# 'P' # (pathname) GIT_DIR # uniq
+# 'G' # (group) root commit (may have multiple roots)
+my %bool_pfx_external = (
+ oid => 'Q', # type:commit - git OID hex (40|64)-byte SHA-(1|256)
+ # type:repo - rel2abs_collapsed(GIT_DIR)
+ parent => 'XP',
+ %PublicInbox::Search::PATCH_BOOL_COMMON,
+);
+
+my %prob_prefix = ( # copied from PublicInbox::Search
+ # do we care about committer? or partial commit OID via Xapian?
+ # o => 'XQ', # 'oid:' (bool) is exact, 'o:' (prob) can do partial
+ %PublicInbox::Search::PATCH_PROB_COMMON,
+
+ # default:
+ '' => 'S A XQUOT XFN ' . $PublicInbox::Search::NON_QUOTED_BODY
+);
+
+sub new {
+ my ($cls, $dir) = @_;
+ bless { xpfx => "$dir/cidx".CIDX_SCHEMA_VER }, $cls;
+}
+
+sub cqparse_new ($) {
+ my ($self) = @_;
+ my $qp = $self->qp_init_common;
+ my $cb = $qp->can('add_valuerangeprocessor') //
+ $qp->can('add_rangeprocessor'); # Xapian 1.5.0+
+ $cb->($qp, $PublicInbox::Search::NVRP->new(AT, 'd:')); # mairix compat
+ $cb->($qp, $PublicInbox::Search::NVRP->new(AT, 'dt:')); # mail compat
+ $cb->($qp, $PublicInbox::Search::NVRP->new(CT, 'ct:'));
+
+ while (my ($name, $pfx) = each %bool_pfx_external) {
+ $qp->add_boolean_prefix($name, $_) for split(/ /, $pfx);
+ }
+ while (my ($name, $pfx) = each %prob_prefix) {
+ $qp->add_prefix($name, $_) for split(/ /, $pfx);
+ }
+ $qp;
+}
+
+# returns a Xapian::Query to filter by roots
+sub roots_filter { # retry_reopen callback
+ my ($self, $git_dir) = @_;
+ my $xdb = $self->xdb;
+ my $P = 'P'.$git_dir;
+ my ($cur, $end) = ($xdb->postlist_begin($P), $xdb->postlist_end($P));
+ if ($cur == $end) {
+ warn "W: $git_dir not indexed?\n";
+ return;
+ }
+ my @roots = xap_terms('G', $xdb, $cur->get_docid);
+ if (!@roots) {
+ warn "W: $git_dir has no root commits?\n";
+ return;
+ }
+ my $q = $PublicInbox::Search::X{Query}->new('G'.shift(@roots));
+ for my $r (@roots) {
+ $q = $PublicInbox::Search::X{Query}->new(
+ PublicInbox::Search::OP_OR(),
+ $q, 'G'.$r);
+ }
+ $q;
+}
+
+sub mset {
+ my ($self, $qry_str, $opt) = @_;
+ my $qp = $self->{qp} //= cqparse_new($self);
+ my $qry = $qp->parse_query($qry_str, $self->{qp_flags});
+
+ # limit to commits with shared roots
+ if (defined(my $git_dir = $opt->{git_dir})) {
+ my $rf = retry_reopen($self, \&roots_filter, $git_dir)
+ or return;
+
+ $qry = $PublicInbox::Search::X{Query}->new(
+ PublicInbox::Search::OP_FILTER(),
+ $qry, $rf);
+ }
+
+ # we only want commits:
+ $qry = $PublicInbox::Search::X{Query}->new(
+ PublicInbox::Search::OP_FILTER(),
+ $qry, 'T'.'c');
+
+ my $enq = $PublicInbox::Search::X{Enquire}->new($self->xdb);
+ $enq->set_query($qry);
+ if ($opt->{relevance}) {
+ $enq->set_sort_by_relevance_then_value(CT, !$opt->{asc});
+ } else {
+ $enq->set_sort_by_value_then_relevance(CT, !$opt->{asc});
+ }
+ $self->retry_reopen($self->can('enquire_once'), $enq,
+ $opt->{offset} || 0, $opt->{limit} || 50);
+}
+
+1;
--- /dev/null
+# Copyright (C) all contributors <meta@public-inbox.org>
+# License: AGPL-3.0+ <https://www.gnu.org/licenses/agpl-3.0.txt>
+#
+# indexer for git coderepos, just commits and repo paths for now
+# this stores normalized absolute paths of indexed GIT_DIR inside
+# the DB itself and is designed to handle forks by designating roots
+#
+# Unlike mail search, docid isn't tied to NNTP artnum or IMAP UID,
+# there's no serial number dependency at all. The first 32-bits of
+# the commit SHA-(1|256) is used to select a shard.
+#
+# We shard repos using the first 32-bits of sha256($ABS_GIT_DIR)
+#
+# See PublicInbox::CodeSearch (read-only API) for more
+package PublicInbox::CodeSearchIdx;
+use v5.12;
+use parent qw(PublicInbox::Lock PublicInbox::CodeSearch PublicInbox::SearchIdx);
+use PublicInbox::Eml;
+use PublicInbox::DS ();
+use PublicInbox::IPC qw(nproc_shards);
+use PublicInbox::Admin;
+use POSIX qw(WNOHANG SEEK_SET);
+use File::Path ();
+use File::Spec ();
+use PublicInbox::SHA qw(sha256_hex);
+use PublicInbox::Search qw(xap_terms);
+use PublicInbox::SearchIdx qw(add_val);
+use PublicInbox::Config;
+use PublicInbox::Spawn qw(run_die);
+
+# stop walking history if we see >$SEEN_MAX existing commits, this assumes
+# branches don't diverge by more than this number of commits...
+# git walks commits quickly if it doesn't have to read trees
+our $SEEN_MAX = 100000;
+
+# TODO: do we care about committer name + email? or tree OID?
+my @FMT = qw(H P ct an ae at s b); # (b)ody must be last
+my @LOG_STDIN = (qw(log --no-decorate --no-color --no-notes -p --stat -M
+ --stdin --no-walk=unsorted), '--pretty=format:%n%x00'.
+ join('%n', map { "%$_" } @FMT));
+
+sub new {
+ my (undef, $dir, $opt) = @_;
+ my $l = $opt->{indexlevel} // 'full';
+ $l !~ $PublicInbox::SearchIdx::INDEXLEVELS and
+ die "invalid indexlevel=$l\n";
+ $l eq 'basic' and die "E: indexlevel=basic not supported\n";
+ my $self = bless {
+ xpfx => "$dir/cidx". PublicInbox::CodeSearch::CIDX_SCHEMA_VER,
+ cidx_dir => $dir,
+ creat => 1, # TODO: get rid of this, should be implicit
+ indexlevel => $l,
+ transact_bytes => 0, # for checkpoint
+ total_bytes => 0, # for lock_release
+ current_info => '',
+ parallel => 1,
+ -opt => $opt,
+ lock_path => "$dir/cidx.lock",
+ }, __PACKAGE__;
+ $self->{nshard} = count_shards($self) ||
+ nproc_shards({nproc => $opt->{jobs}});
+ $self->{-no_fsync} = 1 if !$opt->{fsync};
+ $self->{-dangerous} = 1 if $opt->{dangerous};
+ $self;
+}
+
+# TODO: may be used for reshard/compact
+sub count_shards { scalar($_[0]->xdb_shards_flat) }
+
+sub add_commit ($$) {
+ my ($self, $cmt) = @_; # fields from @FMT
+ my $x = 'Q'.$cmt->{H};
+ for (docids_by_postlist($self, $x)) {
+ $self->{xdb}->delete_document($_)
+ }
+ my $doc = $PublicInbox::Search::X{Document}->new;
+ $doc->add_boolean_term($x);
+ $doc->add_boolean_term('G'.$_) for @{$self->{roots}};
+ $doc->add_boolean_term('XP'.$_) for split(/ /, $cmt->{P});
+ $doc->add_boolean_term('T'.'c');
+
+ # Author-Time is compatible with dt: for mail search schema_version=15
+ add_val($doc, PublicInbox::CodeSearch::AT,
+ POSIX::strftime('%Y%m%d%H%M%S', gmtime($cmt->{at})));
+
+ # Commit-Time is the fallback used by rt: (TS) for mail search:
+ add_val($doc, PublicInbox::CodeSearch::CT, $cmt->{ct});
+
+ $self->term_generator->set_document($doc);
+
+ # email address is always indexed with positional data for usability
+ $self->index_phrase("$cmt->{an} <$cmt->{ae}>", 1, 'A');
+
+ $x = $cmt->{'s'};
+ $self->index_text($x, 1, 'S') if $x =~ /\S/s;
+ $doc->set_data($x); # subject is the first (and currently only) line
+
+ $x = delete $cmt->{b};
+ $self->index_body_text($doc, \$x) if $x =~ /\S/s;
+ $self->{xdb}->add_document($doc);
+}
+
+sub progress {
+ my ($self, @msg) = @_;
+ my $pr = $self->{-opt}->{-progress} or return;
+ $pr->($self->{git} ? ("$self->{git}->{git_dir}: ") : (), @msg, "\n");
+}
+
+sub store_repo ($$) {
+ my ($self, $repo) = @_;
+ my $xdb = delete($repo->{shard})->idx_acquire;
+ $xdb->begin_transaction;
+ if (defined $repo->{id}) {
+ my $doc = $xdb->get_document($repo->{id}) //
+ die "$self->{git}->{git_dir} doc #$repo->{id} gone";
+ add_val($doc, PublicInbox::CodeSearch::CT, $repo->{ct});
+ my %new = map { $_ => undef } @{$self->{roots}};
+ my $old = xap_terms('G', $doc);
+ delete @new{keys %$old};
+ $doc->add_boolean_term('G'.$_) for keys %new;
+ delete @$old{@{$self->{roots}}};
+ $doc->remove_term('G'.$_) for keys %$old;
+ $doc->set_data($repo->{fp});
+ $xdb->replace_document($repo->{id}, $doc);
+ } else {
+ my $new = $PublicInbox::Search::X{Document}->new;
+ add_val($new, PublicInbox::CodeSearch::CT, $repo->{ct});
+ $new->add_boolean_term("P$self->{git}->{git_dir}");
+ $new->add_boolean_term('T'.'r');
+ $new->add_boolean_term('G'.$_) for @{$repo->{roots}};
+ $new->set_data($repo->{fp}); # \n delimited
+ $xdb->add_document($new);
+ }
+ $xdb->commit_transaction;
+}
+
+# sharded reader for `git log --pretty=format: --stdin'
+sub shard_worker ($$$) {
+ my ($self, $r, $sigset) = @_;
+ my ($quit, $cmt);
+ my $batch_bytes = $self->{-opt}->{batch_size} //
+ $PublicInbox::SearchIdx::BATCH_BYTES;
+ my $max = $batch_bytes;
+ $SIG{USR1} = sub { $max = -1 }; # similar to `git fast-import'
+ $SIG{QUIT} = $SIG{TERM} = $SIG{INT} = sub { $quit = shift };
+ PublicInbox::DS::sig_setmask($sigset);
+
+ # the parent process of this shard process writes directly to
+ # the stdin of `git log', we consume git log's stdout:
+ my $rd = $self->{git}->popen(@LOG_STDIN, undef, { 0 => $r });
+ close $r or die "close: $!";
+ my $nr = 0;
+
+ # a patch may have \0, see c4201214cbf10636e2c1ab9131573f735b42c8d4
+ # in linux.git, so we use $/ = "\n\0" to check end-of-patch
+ my $FS = "\n\0";
+ local $/ = $FS;
+ my $buf = <$rd> // return; # leading $FS
+ $buf eq $FS or die "BUG: not LF-NUL: $buf\n";
+ my $xdb = $self->idx_acquire;
+ $xdb->begin_transaction;
+ while (defined($buf = <$rd>)) {
+ chomp($buf);
+ $max -= length($buf);
+ @$cmt{@FMT} = split(/\n/, $buf, scalar(@FMT));
+ $/ = "\n";
+ add_commit($self, $cmt);
+ last if $quit; # likely SIGPIPE
+ ++$nr;
+ if ($max <= 0 && !$PublicInbox::Search::X{CLOEXEC_UNSET}) {
+ progress($self, $nr);
+ $xdb->commit_transaction;
+ $max = $batch_bytes;
+ $xdb->begin_transaction;
+ }
+ $/ = $FS;
+ }
+ close($rd);
+ if (!$? || ($quit && ($? & 127) == POSIX::SIGPIPE)) {
+ $xdb->commit_transaction;
+ } else {
+ warn "E: git @LOG_STDIN: \$?=$?\n";
+ $xdb->cancel_transaction;
+ }
+}
+
+sub seen ($$) {
+ my ($xdb, $q) = @_; # $q = "Q$COMMIT_HASH"
+ $xdb->postlist_begin($q) != $xdb->postlist_end($q)
+}
+
+# used to select the shard for a GIT_DIR
+sub git_dir_hash ($) { hex(substr(sha256_hex($_[0]), 0, 8)) }
+
+sub docids_by_postlist ($$) { # consider moving to PublicInbox::Search
+ my ($self, $q) = @_;
+ my $cur = $self->{xdb}->postlist_begin($q);
+ my $end = $self->{xdb}->postlist_end($q);
+ my @ids;
+ for (; $cur != $end; $cur++) { push(@ids, $cur->get_docid) };
+ @ids;
+}
+
+sub get_roots ($$) {
+ my ($self, $refs) = @_;
+ my @roots = $self->{git}->qx([qw(rev-list --stdin --max-parents=0)],
+ undef, { 0 => $refs });
+ die "git rev-list \$?=$?" if $?;
+ sysseek($refs, 0, SEEK_SET) or die "seek: $!"; # for rev-list --stdin
+ chomp(@roots);
+ scalar(@roots) ? \@roots : undef;
+}
+
+# this is different from the grokmirror-compatible fingerprint since we
+# only care about --heads (branches) and --tags, and not even their names
+sub cidx_fp ($) {
+ my ($self) = @_;
+ open my $refs, '+>', undef or die "open: $!";
+ run_die(['git', "--git-dir=$self->{git}->{git_dir}",
+ qw(show-ref --heads --tags --hash)], undef, { 1 => $refs });
+ seek($refs, 0, SEEK_SET) or die "seek: $!";
+ my $buf;
+ my $dig = PublicInbox::SHA->new(256);
+ while (read($refs, $buf, 65536)) { $dig->add($buf) }
+ sysseek($refs, 0, SEEK_SET) or die "seek: $!"; # for rev-list --stdin
+ ($dig->hexdigest, $refs);
+}
+
+# TODO: should we also index gitweb.owner and the full fingerprint for grokmirror?
+sub prep_git_dir ($) {
+ my ($self) = @_;
+ my $git_dir = $self->{git}->{git_dir};
+ my $ct = $self->{git}->qx([qw[for-each-ref
+ --sort=-committerdate --format=%(committerdate:raw) --count=1
+ refs/heads/ refs/tags/]]);
+ my $repo = {};
+ @$repo{qw(fp refs)} = cidx_fp($self);
+ $repo->{roots} = get_roots($self, $repo->{refs});
+ if (!$repo->{roots} || !defined($ct)) {
+ warn "W: $git_dir has no root commits, skipping\n";
+ return;
+ }
+ $ct =~ s/ .*\z//s; # drop TZ
+ $repo->{ct} = $ct + 0;
+ my $n = git_dir_hash($git_dir) % $self->{nshard};
+ my $shard = $repo->{shard} = bless { %$self, shard => $n }, ref($self);
+ delete @$shard{qw(lockfh lock_path)};
+ local $shard->{xdb};
+ my $xdb = $shard->idx_acquire;
+ my @docids = docids_by_postlist($shard, 'P'.$git_dir);
+ my $docid = shift(@docids) // return $repo;
+ if (@docids) {
+ warn "BUG: $git_dir indexed multiple times, culling\n";
+ $xdb->begin_transaction;
+ for (@docids) { $xdb->delete_document($_) }
+ $xdb->commit_transaction;
+ }
+ my $doc = $xdb->get_document($docid) //
+ die "BUG: no #$docid ($git_dir)";
+ my $old_fp = $doc->get_data;
+ if ($old_fp eq $repo->{fp}) { # no change
+ progress($self, 'unchanged');
+ return;
+ }
+ $repo->{id} = $docid;
+ $repo;
+}
+
+sub partition_refs ($$) {
+ my ($self, $refs) = @_; # show-ref --heads --tags --hash output
+ my $fh = $self->{git}->popen(qw(rev-list --stdin), undef,
+ { 0 => $refs });
+ close $refs or die "close: $!";
+ local $self->{xdb};
+ my $xdb = $self->{-opt}->{reindex} ? undef : $self->xdb;
+ my ($seen, $nchange, $nshard) = (0, 0, $self->{nshard});
+ my @shard_in;
+ for (0..($nshard - 1)) {
+ open $shard_in[$_], '+>', undef or die "open: $!";
+ }
+ while (defined(my $cmt = <$fh>)) {
+ chomp $cmt;
+ if ($xdb && seen($xdb, 'Q'.$cmt)) {
+ last if ++$seen > $SEEN_MAX;
+ } else {
+ my $n = hex(substr($cmt, 0, 8)) % $nshard;
+ say { $shard_in[$n] } $cmt or die "say: $!";
+ ++$nchange;
+ $seen = 0;
+ }
+ }
+ close($fh);
+ if (!$? || (($? & 127) == POSIX::SIGPIPE && $seen > $SEEN_MAX)) {
+ $self->{nchange} += $nchange;
+ progress($self, "$nchange commits");
+ for my $fh (@shard_in) {
+ $fh->flush or die "flush: $!";
+ sysseek($fh, 0, SEEK_SET) or die "seek: $!";
+ }
+ return @shard_in;
+ }
+ die "git-rev-list: \$?=$?\n";
+}
+
+sub index_git_dir ($$) {
+ my ($self, $git_dir) = @_;
+ local $self->{git} = PublicInbox::Git->new($git_dir); # for ->patch_id
+ my $repo = prep_git_dir($self) or return;
+ local $self->{current_info} = $git_dir;
+ my @shard_in = partition_refs($self, delete($repo->{refs}));
+ my %pids;
+ my $fwd_kill = sub {
+ my ($sig) = @_;
+ kill($sig, $_) for keys %pids;
+ };
+ local $SIG{USR1} = $fwd_kill;
+ local $SIG{QUIT} = $fwd_kill;
+ local $SIG{INT} = $fwd_kill;
+ local $SIG{TERM} = $fwd_kill;
+ my $sigset = PublicInbox::DS::block_signals();
+ for (my $n = 0; $n <= $#shard_in; $n++) {
+ -s $shard_in[$n] or next;
+ my $pid = fork // die "fork: $!";
+ if ($pid == 0) { # no RNG use, here
+ $0 = "code index [$n]";
+ $self->{shard} = $n;
+ $self->{current_info} = "$self->{current_info} [$n]";
+ delete @$self{qw(lockfh lock_path)};
+ my $in = $shard_in[$n];
+ @shard_in = ();
+ $self->{roots} = delete $repo->{roots};
+ undef $repo;
+ eval { shard_worker($self, $in, $sigset) };
+ warn "E: $@" if $@;
+ POSIX::_exit($@ ? 1 : 0);
+ } else {
+ $pids{$pid} = "code index [$n]";
+ }
+ }
+ PublicInbox::DS::sig_setmask($sigset);
+ @shard_in = ();
+ my $err;
+ while (keys %pids) {
+ my $pid = waitpid(-1, 0) or last;
+ my $j = delete $pids{$pid} // "unknown PID:$pid";
+ next if $? == 0;
+ warn "PID:$pid $j exited with \$?=$?\n";
+ $err = 1;
+ }
+ die "subprocess(es) failed\n" if $err;
+ store_repo($self, $repo);
+ progress($self, 'done');
+ # TODO: check fp afterwards?
+}
+
+# for PublicInbox::SearchIdx::patch_id and with_umask
+sub git { $_[0]->{git} }
+
+sub load_existing ($) { # for -u/--update
+ my ($self) = @_;
+ my $dirs = $self->{git_dirs} // [];
+ if ($self->{-opt}->{update}) {
+ local $self->{xdb};
+ $self->xdb or
+ die "E: $self->{cidx_dir} non-existent for --update\n";
+ my @cur = $self->all_terms('P');
+ push @$dirs, @cur;
+ }
+ my %uniq; # List::Util::uniq requires Perl 5.26+
+ @$dirs = grep { !$uniq{$_}++ } @$dirs;
+}
+
+sub cidx_init ($) {
+ my ($self) = @_;
+ my $dir = $self->{cidx_dir};
+ unless (-d $dir) {
+ warn "# creating $dir\n" if !$self->{-opt}->{quiet};
+ File::Path::mkpath($dir);
+ }
+ for my $n (0..($self->{nshard} - 1)) {
+ my $shard = bless { %$self, shard => $n }, ref($self);
+ $shard->idx_acquire;
+ }
+ # this warning needs to happen after idx_acquire
+ state $once;
+ warn <<EOM if $PublicInbox::Search::X{CLOEXEC_UNSET} && !$once++;
+W: Xapian v1.2.21..v1.2.24 were missing close-on-exec on OFD locks,
+W: memory usage may be high for large indexing runs
+EOM
+}
+
+sub cidx_run {
+ my ($self) = @_;
+ cidx_init($self);
+ local $self->{current_info} = '';
+ my $cb = $SIG{__WARN__} || \&CORE::warn;
+ local $SIG{__WARN__} = sub {
+ my $m = shift @_;
+ $self->{current_info} eq '' or
+ $m =~ s/\A(#?\s*)/$1$self->{current_info}: /;
+ $cb->($m, @_);
+ };
+ $self->lock_acquire;
+ load_existing($self);
+ my @nc = grep { File::Spec->canonpath($_) ne $_ } @{$self->{git_dirs}};
+ if (@nc) {
+ warn "E: BUG? paths in $self->{cidx_dir} not canonicalized:\n";
+ for my $d (@{$self->{git_dirs}}) {
+ my $c = File::Spec->canonpath($_);
+ warn "E: $d => $c\n";
+ $d = $c;
+ }
+ warn "E: canonicalized and attempting to continue\n";
+ }
+ local $self->{nchange} = 0;
+ # do_prune($self) if $self->{-opt}->{prune}; TODO
+ if ($self->{-opt}->{scan} // 1) {
+ for my $gd (@{$self->{git_dirs}}) {
+ index_git_dir($self, $gd);
+ }
+ }
+ $self->lock_release(!!$self->{nchange});
+}
+
+1;
# Things indexed include:
# * inboxes themselves
# * epoch information
-# * (maybe) git code repository information
+# * (maybe) git code repository information (not commits)
# Expect ~100K-1M documents with no parallelism opportunities,
# so no sharding, here.
#
# a prefix common in patch emails
our $LANG = 'english';
+our %PATCH_BOOL_COMMON = (
+ dfpre => 'XDFPRE',
+ dfpost => 'XDFPOST',
+ dfblob => 'XDFPRE XDFPOST',
+ patchid => 'XDFID',
+);
+
# note: the non-X term prefix allocations are shared with
# Xapian omega, see xapian-applications/omega/docs/termprefixes.rst
my %bool_pfx_external = (
mid => 'Q', # Message-ID (full/exact), this is mostly uniQue
lid => 'G', # newsGroup (or similar entity), just inside <>
- dfpre => 'XDFPRE',
- dfpost => 'XDFPOST',
- dfblob => 'XDFPRE XDFPOST',
- patchid => 'XDFID',
+ %PATCH_BOOL_COMMON
);
-my $non_quoted_body = 'XNQ XDFN XDFA XDFB XDFHH XDFCTX XDFPRE XDFPOST XDFID';
-my %prob_prefix = (
- # for mairix compatibility
+# for mairix compatibility
+our $NON_QUOTED_BODY = 'XNQ XDFN XDFA XDFB XDFHH XDFCTX XDFPRE XDFPOST XDFID';
+our %PATCH_PROB_COMMON = (
s => 'S',
- m => 'XM', # 'mid:' (bool) is exact, 'm:' (prob) can do partial
- l => 'XL', # 'lid:' (bool) is exact, 'l:' (prob) can do partial
f => 'A',
- t => 'XTO',
- tc => 'XTO XCC',
- c => 'XCC',
- tcf => 'XTO XCC A',
- a => 'XTO XCC A',
- b => $non_quoted_body . ' XQUOT',
- bs => $non_quoted_body . ' XQUOT S',
+ b => $NON_QUOTED_BODY . ' XQUOT',
+ bs => $NON_QUOTED_BODY . ' XQUOT S',
n => 'XFN',
q => 'XQUOT',
- nq => $non_quoted_body,
+ nq => $NON_QUOTED_BODY,
dfn => 'XDFN',
dfa => 'XDFA',
dfb => 'XDFB',
dfhh => 'XDFHH',
dfctx => 'XDFCTX',
+);
+my %prob_prefix = (
+ m => 'XM', # 'mid:' (bool) is exact, 'm:' (prob) can do partial
+ l => 'XL', # 'lid:' (bool) is exact, 'l:' (prob) can do partial
+ t => 'XTO',
+ tc => 'XTO XCC',
+ c => 'XCC',
+ tcf => 'XTO XCC A',
+ a => 'XTO XCC A',
+ %PATCH_PROB_COMMON,
# default:
- '' => 'XM S A XQUOT XFN ' . $non_quoted_body,
+ '' => 'XM S A XQUOT XFN ' . $NON_QUOTED_BODY,
);
# not documenting m: and mid: for now, the using the URLs works w/o Xapian
$x = "\0%Y%m%d%H%M%S$#$to_parse\0";
}
}
- } else { # "rt", let git interpret "YYYY", deal with Y10K later :P
+ } else { # (rt|ct), let git interpret "YYYY", deal with Y10K later :P
for my $x (@r) {
next if $x eq '' || $x =~ /\A[0-9]{5,}\z/;
push @$to_parse, $x;
# read-write
sub stemmer { $X{Stem}->new($LANG) }
-# read-only
-sub qparse_new {
+sub qp_init_common {
my ($self) = @_;
-
- my $xdb = xdb($self);
my $qp = $X{QueryParser}->new;
$qp->set_default_op(OP_AND());
- $qp->set_database($xdb);
+ $qp->set_database(xdb($self));
$qp->set_stemmer(stemmer($self));
$qp->set_stemming_strategy(STEM_SOME());
my $cb = $qp->can('set_max_wildcard_expansion') //
$qp->can('set_max_expansion'); # Xapian 1.5.0+
$cb->($qp, 100);
- $cb = $qp->can('add_valuerangeprocessor') //
+ $qp;
+}
+
+# read-only
+sub qparse_new {
+ my ($self) = @_;
+ my $qp = qp_init_common($self);
+ my $cb = $qp->can('add_valuerangeprocessor') //
$qp->can('add_rangeprocessor'); # Xapian 1.5.0+
$cb->($qp, $NVRP->new(YYYYMMDD, 'd:'));
$cb->($qp, $NVRP->new(DT, 'dt:'));
}
# get combined docid from over.num:
-# (not generic Xapian, only works with our sharding scheme)
+# (not generic Xapian, only works with our sharding scheme for mail)
sub num2docid ($$) {
my ($self, $num) = @_;
my $nshard = $self->{nshard};
load_xapian_writable();
$flag = $self->{creat} ? $DB_CREATE_OR_OPEN : $DB_OPEN;
}
+ my $owner = $self->{ibx} // $self->{eidx} // $self;
if ($self->{creat}) {
require File::Path;
$self->lock_acquire;
File::Path::mkpath($dir);
require PublicInbox::Syscall;
PublicInbox::Syscall::nodatacow_dir($dir);
- $self->{-set_has_threadid_once} = 1;
- if (($self->{ibx} // $self->{eidx})->{-dangerous}) {
- $flag |= $DB_DANGEROUS;
- }
+ # owner == self for CodeSearchIdx
+ $self->{-set_has_threadid_once} = 1 if $owner != $self;
+ $flag |= $DB_DANGEROUS if $owner->{-dangerous};
}
}
return unless defined $flag;
- $flag |= $DB_NO_SYNC if ($self->{ibx} // $self->{eidx})->{-no_fsync};
+ $flag |= $DB_NO_SYNC if $owner->{-no_fsync};
my $xdb = eval { ($X->{WritableDatabase})->new($dir, $flag) };
croak "Failed opening $dir: $@" if $@;
$self->{xdb} = $xdb;
index_text($self, join("\n", @$xnq), 1, 'XNQ');
}
-sub index_xapian { # msg_iter callback
- my $part = $_[0]->[0]; # ignore $depth and $idx
- my ($self, $doc) = @{$_[1]};
- my $ct = $part->content_type || 'text/plain';
- my $fn = $part->filename;
- if (defined $fn && $fn ne '') {
- index_phrase($self, $fn, 1, 'XFN');
- }
- if ($part->{is_submsg}) {
- my $mids = mids_for_index($part);
- index_ids($self, $doc, $part, $mids);
- my $smsg = bless {}, 'PublicInbox::Smsg';
- $smsg->populate($part);
- index_headers($self, $smsg);
- }
-
- my ($s, undef) = msg_part_text($part, $ct);
- defined $s or return;
- $_[0]->[0] = $part = undef; # free memory
+sub patch_id {
+ my ($self) = @_; # $_[1] is the diff (may be huge)
+ open(my $fh, '+>:utf8', undef) or die "open: $!";
+ open(my $eh, '+>', undef) or die "open: $!";
+ $fh->autoflush(1);
+ print $fh $_[1] or die "print: $!";
+ sysseek($fh, 0, SEEK_SET) or die "sysseek: $!";
+ my $id = ($self->{ibx} // $self->{eidx} // $self)->git->qx(
+ [qw(patch-id --stable)], {}, { 0 => $fh, 2 => $eh });
+ seek($eh, 0, SEEK_SET) or die "seek: $!";
+ while (<$eh>) { warn $_ }
+ $id =~ /\A([a-f0-9]{40,})/ ? $1 : undef;
+}
- if ($s =~ /^(?:diff|---|\+\+\+) /ms) {
- open(my $fh, '+>:utf8', undef) or die "open: $!";
- open(my $eh, '+>', undef) or die "open: $!";
- $fh->autoflush(1);
- print $fh $s or die "print: $!";
- sysseek($fh, 0, SEEK_SET) or die "sysseek: $!";
- my $id = ($self->{ibx} // $self->{eidx})->git->qx(
- [qw(patch-id --stable)],
- {}, { 0 => $fh, 2 => $eh });
- $id =~ /\A([a-f0-9]{40,})/ and $doc->add_term('XDFID'.$1);
- seek($eh, 0, SEEK_SET) or die "seek: $!";
- while (<$eh>) { warn $_ }
+sub index_body_text {
+ my ($self, $doc, $sref) = @_;
+ if ($$sref =~ /^(?:diff|---|\+\+\+) /ms) {
+ my $id = patch_id($self, $$sref);
+ $doc->add_term('XDFID'.$id) if defined($id);
}
# split off quoted and unquoted blocks:
- my @sections = PublicInbox::MsgIter::split_quotes($s);
- undef $s; # free memory
+ my @sections = PublicInbox::MsgIter::split_quotes($$sref);
+ undef $$sref; # free memory
for my $txt (@sections) {
if ($txt =~ /\A>/) {
if ($txt =~ /^[>\t ]+GIT binary patch\r?/sm) {
(?:[>\h]+$BASE85\h*\r?\n)+/$1/gsmx;
}
index_text($self, $txt, 0, 'XQUOT');
- } else {
- # does it look like a diff?
+ } else { # does it look like a diff?
if ($txt =~ /^(?:diff|---|\+\+\+) /ms) {
index_diff($self, \$txt, $doc);
} else {
}
}
+sub index_xapian { # msg_iter callback
+ my $part = $_[0]->[0]; # ignore $depth and $idx
+ my ($self, $doc) = @{$_[1]};
+ my $ct = $part->content_type || 'text/plain';
+ my $fn = $part->filename;
+ if (defined $fn && $fn ne '') {
+ index_phrase($self, $fn, 1, 'XFN');
+ }
+ if ($part->{is_submsg}) {
+ my $mids = mids_for_index($part);
+ index_ids($self, $doc, $part, $mids);
+ my $smsg = bless {}, 'PublicInbox::Smsg';
+ $smsg->populate($part);
+ index_headers($self, $smsg);
+ }
+
+ my ($s, undef) = msg_part_text($part, $ct);
+ defined $s or return;
+ $_[0]->[0] = $part = undef; # free memory
+ index_body_text($self, $doc, \$s);
+}
+
sub index_list_id ($$$) {
my ($self, $doc, $hdr) = @_;
for my $l ($hdr->header_raw('List-Id')) {
@EXPORT = qw(tmpdir tcp_server tcp_connect require_git require_mods
run_script start_script key2sub xsys xsys_e xqx eml_load tick
have_xapian_compact json_utf8 setup_public_inboxes create_inbox
+ create_coderepo
tcp_host_port test_lei lei lei_ok $lei_out $lei_err $lei_opt
test_httpd xbail require_cmd is_xdeeply tail_f
ignore_inline_c_missing);
}
}
my $tail = @tail_paths ? tail_f(@tail_paths) : undef;
- if ($key =~ /-(index|convert|extindex|convert|xcpdb)\z/) {
+ if ($key =~ /-(index|cindex|extindex|convert|xcpdb)\z/) {
unshift @argv, '--no-fsync';
}
if ($run_mode == 0) {
@ret;
}
+our %COMMIT_ENV = (
+ GIT_AUTHOR_NAME => 'A U Thor',
+ GIT_COMMITTER_NAME => 'C O Mitter',
+ GIT_AUTHOR_EMAIL => 'a@example.com',
+ GIT_COMMITTER_EMAIL => 'c@example.com',
+);
+
+sub create_coderepo ($$;@) {
+ my $ident = shift;
+ my $cb = pop;
+ my %opt = @_;
+ require PublicInbox::Lock;
+ require PublicInbox::Import;
+ my ($base) = ($0 =~ m!\b([^/]+)\.[^\.]+\z!);
+ my ($db) = (PublicInbox::Import::default_branch() =~ m!([^/]+)\z!);
+ my $dir = "t/data-gen/$base.$ident-$db";
+ my $new = !-d $dir;
+ if ($new && !mkdir($dir)) {
+ my $err = $!;
+ -d $dir or xbail "mkdir($dir): $err";
+ }
+ my $lk = bless { lock_path => "$dir/creat.lock" }, 'PublicInbox::Lock';
+ my $scope = $lk->lock_for_scope;
+ my $tmpdir = delete $opt{tmpdir};
+ if (!-f "$dir/creat.stamp") {
+ opendir(my $dfh, '.') or xbail "opendir .: $!";
+ chdir($dir) or xbail "chdir($dir): $!";
+ local %ENV = (%ENV, %COMMIT_ENV);
+ $cb->($dir);
+ chdir($dfh) or xbail "cd -: $!";
+ open my $s, '>', "$dir/creat.stamp" or
+ BAIL_OUT "error creating $dir/creat.stamp: $!";
+ }
+ return $dir if !defined($tmpdir);
+ xsys_e([qw(/bin/cp -Rp), $dir, $tmpdir]);
+ $tmpdir;
+}
+
sub create_inbox ($$;@) {
my $ident = shift;
my $cb = pop;
--- /dev/null
+#!perl -w
+# Copyright (C) all contributors <meta@public-inbox.org>
+# License: AGPL-3.0+ <https://www.gnu.org/licenses/agpl-3.0.txt>
+use v5.12;
+use Getopt::Long qw(:config gnu_getopt no_ignore_case auto_abbrev);
+my $help = <<EOF; # the following should fit w/o scrolling in 80x24 term:
+usage: public-inbox-cindex [options] GIT_DIR...
+usage: public-inbox-cindex [options] --project-list=FILE PROJECT_ROOT
+
+ Create and update search indices for code repos
+
+ -d EXTDIR use EXTDIR instead of GIT_DIR/public-inbox-cindex
+ --no-fsync speed up indexing, risk corruption on power outage
+ -L LEVEL `medium', or `full' (default: medium)
+ --project-list=FILE use a cgit/gitweb-compatible list of projects
+ --update | -u update previously-indexed code repos with `-d'
+ --jobs=NUM set or disable parallelization (NUM=0)
+ --batch-size=BYTES flush changes to OS after a given number of bytes
+ --prune prune old repos and commits
+ --reindex reindex previously indexed repos
+ --verbose | -v increase verbosity (may be repeated)
+
+BYTES may use `k', `m', and `g' suffixes (e.g. `10m' for 10 megabytes)
+See public-inbox-cindex(1) man page for full documentation.
+EOF
+my $opt = { fsync => 1, scan => 1 }; # --no-scan is hidden
+GetOptions($opt, qw(quiet|q verbose|v+ reindex jobs|j=i fsync|sync! dangerous
+ indexlevel|index-level|L=s batch_size|batch-size=s
+ project-list=s
+ d=s update|u scan! prune dry-run|n C=s@ help|h))
+ or die $help;
+if ($opt->{help}) { print $help; exit 0 };
+die "--jobs must be >= 0\n" if defined $opt->{jobs} && $opt->{jobs} < 0;
+require IO::Handle;
+STDOUT->autoflush(1);
+STDERR->autoflush(1);
+local $SIG{USR1} = 'IGNORE'; # to be overridden in cidx_sync
+# require lazily to speed up --help
+require PublicInbox::Admin;
+PublicInbox::Admin::do_chdir(delete $opt->{C});
+my $cfg = PublicInbox::Config->new;
+my $cidx_dir = $opt->{d};
+PublicInbox::Admin::require_or_die('Search::Xapian');
+PublicInbox::Admin::progress_prepare($opt);
+my $env = PublicInbox::Admin::index_prepare($opt, $cfg);
+%ENV = (%ENV, %$env) if $env;
+
+require PublicInbox::CodeSearchIdx; # unstable internal API
+my @git_dirs;
+if (defined(my $pl = $opt->{'project-list'})) {
+ my $pfx = shift @ARGV // die <<EOM;
+PROJECTS_ROOT required for --project-list
+EOM
+ open my $fh, '<', $pl or die "open($pl): $!\n";
+ chomp(@git_dirs = <$fh>);
+ $_ = PublicInbox::Admin::resolve_git_dir("$pfx/$_") for @git_dirs;
+} else {
+ @git_dirs = map { PublicInbox::Admin::resolve_git_dir($_) } @ARGV;
+}
+if (defined $cidx_dir) { # external index
+ die "`%' is not allowed in $cidx_dir\n" if $cidx_dir =~ /\%/;
+ my $cidx = PublicInbox::CodeSearchIdx->new($cidx_dir, $opt);
+ @{$cidx->{git_dirs}} = @git_dirs; # may be empty
+ $cidx->cidx_run;
+} elsif (!@git_dirs) {
+ die $help
+} else {
+ for my $gd (@git_dirs) {
+ my $cd = "$gd/public-inbox-cindex";
+ my $cidx = PublicInbox::CodeSearchIdx->new($cd, { %$opt });
+ $cidx->{-internal} = 1;
+ @{$cidx->{git_dirs}} = ($gd);
+ $cidx->cidx_run;
+ }
+}
--- /dev/null
+#!perl -w
+# Copyright (C) all contributors <meta@public-inbox.org>
+# License: AGPL-3.0+ <https://www.gnu.org/licenses/agpl-3.0.txt>
+use v5.12;
+use PublicInbox::TestCommon;
+use Cwd qw(getcwd abs_path);
+require_mods(qw(json Search::Xapian));
+use_ok 'PublicInbox::CodeSearchIdx';
+require PublicInbox::Import;
+my ($tmp, $for_destroy) = tmpdir();
+my $pwd = getcwd();
+
+# I reworked CodeSearchIdx->shard_worker to handle empty trees
+# in the initial commit generated by cvs2svn for xapian.git
+create_coderepo 'empty-tree-root', tmpdir => "$tmp/wt0", sub {
+ xsys_e([qw(/bin/sh -c), <<'EOM']);
+git init -q &&
+tree=$(git mktree </dev/null) &&
+head=$(git symbolic-ref HEAD) &&
+cmt=$(echo 'empty root' | git commit-tree $tree) &&
+git update-ref $head $cmt &&
+echo hi >f &&
+git add f &&
+git commit -q -m hi &&
+git gc -q
+EOM
+}; # /create_coderepo
+
+ok(run_script([qw(-cindex --dangerous -q), "$tmp/wt0"]), 'cindex internal');
+ok(-e "$tmp/wt0/.git/public-inbox-cindex/cidx.lock", 'internal dir created');
+
+
+# it's possible for git to emit NUL characters in diffs
+# (see c4201214cbf10636e2c1ab9131573f735b42c8d4 in linux.git)
+my $zp = create_coderepo 'NUL in patch', sub {
+ require PublicInbox::Git;
+ my $src = PublicInbox::Git::try_cat("$pwd/COPYING");
+ xsys_e([qw(git init -q)]);
+
+ # needs to be further than FIRST_FEW_BYTES (8000) in git.git
+ $src =~ s/\b(Limitation of Liability\.)\n\n/$1\n\0\n/s or
+ xbail "BUG: no `\\n\\n' in $pwd/COPYING";
+
+ open my $fh, '>', 'f' or xbail "open: $!";
+ print $fh $src or xbail "print: $!";
+ close $fh or xbail "close: $!";
+ xsys_e([qw(/bin/sh -c), <<'EOM']);
+git add f &&
+git commit -q -m 'initial with NUL character'
+EOM
+ $src =~ s/\n\0\n/\n\n/ or xbail "BUG: no `\\n\\0\\n'";
+ open $fh, '>', 'f' or xbail "open: $!";
+ print $fh $src or xbail "print: $!";
+ close $fh or xbail "close: $!";
+ xsys_e([qw(/bin/sh -c), <<'EOM']);
+git add f &&
+git commit -q -m 'remove NUL character' &&
+git gc -q
+EOM
+}; # /create_coderepo
+
+ok(run_script([qw(-cindex --dangerous -q -d), "$tmp/ext", $zp, "$tmp/wt0"]),
+ 'cindex external');
+ok(-e "$tmp/ext/cidx.lock", 'external dir created');
+ok(!-d "$zp/.git/public-inbox-cindex", 'no cindex in original coderepo');
+
+use_ok 'PublicInbox::CodeSearch';
+if ('multi-repo search') {
+ my $csrch = PublicInbox::CodeSearch->new("$tmp/ext");
+ my $mset = $csrch->mset('NUL');
+ is(scalar($mset->items), 2, 'got results');
+ my $exp = [ 'initial with NUL character', 'remove NUL character' ];
+ my @have = sort(map { $_->get_document->get_data } $mset->items);
+ is_xdeeply(\@have, $exp, 'got expected subjects');
+
+ $mset = $csrch->mset('NUL', { git_dir => "$tmp/wt0/.git" });
+ is(scalar($mset->items), 0, 'no results with other GIT_DIR');
+
+ $mset = $csrch->mset('NUL', { git_dir => abs_path("$zp/.git") });
+ @have = sort(map { $_->get_document->get_data } $mset->items);
+ is_xdeeply(\@have, $exp, 'got expected subjects w/ GIT_DIR filter');
+}
+
+if ('--update') {
+ my $csrch = PublicInbox::CodeSearch->new("$tmp/ext");
+ my $mset = $csrch->mset('dfn:for-update');
+ is(scalar($mset->items), 0, 'no result before update');
+
+ my $e = \%PublicInbox::TestCommon::COMMIT_ENV;
+ xsys_e([qw(/bin/sh -c), <<'EOM'], $e, { -C => "$tmp/wt0" });
+>for-update && git add for-update && git commit -q -m updated
+EOM
+ ok(run_script([qw(-cindex -qu -d), "$tmp/ext"]), '-cindex -u');
+ $mset = $csrch->reopen->mset('dfn:for-update');
+ is(scalar($mset->items), 1, 'got updated result');
+}
+
+done_testing;