aboutsummaryrefslogtreecommitdiff
path: root/icing
diff options
context:
space:
mode:
authorTerry Wang <tytytyww@google.com>2020-10-01 18:53:44 -0700
committerTerry Wang <tytytyww@google.com>2020-10-01 18:53:44 -0700
commit5abfe5bcac00f4f188d3d8041fa97bf77206b577 (patch)
tree69376254e2e5f886cb0d26cdb547001f8e45f372 /icing
parente15b6b66f871a71b73278c34d5c54f648f880c29 (diff)
downloadicing-5abfe5bcac00f4f188d3d8041fa97bf77206b577.tar.gz
Pull upstream changes.
Change-Id: I794757716961569b5c02171cfc82785efb2cf106
Diffstat (limited to 'icing')
-rw-r--r--icing/index/index.cc2
-rw-r--r--icing/index/lite/doc-hit-info-iterator-term-lite.cc2
-rw-r--r--icing/index/lite/lite-index.cc50
-rw-r--r--icing/index/lite/lite-index.h126
-rw-r--r--icing/index/lite/term-id-hit-pair.h80
-rw-r--r--icing/index/main/main-index-merger.cc225
-rw-r--r--icing/index/main/main-index-merger.h49
-rw-r--r--icing/index/main/main-index-merger_test.cc367
-rw-r--r--icing/index/main/main-index.cc184
-rw-r--r--icing/index/main/main-index.h63
-rw-r--r--icing/index/main/main-index_test.cc536
-rw-r--r--icing/store/document-store.cc58
-rw-r--r--icing/store/document-store.h19
-rw-r--r--icing/store/usage-store.cc193
-rw-r--r--icing/store/usage-store.h160
-rw-r--r--icing/store/usage-store_test.cc389
-rw-r--r--icing/tokenization/ios/ios-language-segmenter-factory.cc51
-rw-r--r--icing/tokenization/ios/ios-language-segmenter.h88
-rw-r--r--icing/tokenization/ios/ios-language-segmenter_test.cc1265
19 files changed, 2362 insertions, 1545 deletions
diff --git a/icing/index/index.cc b/icing/index/index.cc
index e7f2fbc..0b014d9 100644
--- a/icing/index/index.cc
+++ b/icing/index/index.cc
@@ -159,7 +159,7 @@ libtextclassifier3::Status Index::Editor::AddHit(const char* term,
Hit::Score score) {
// Step 1: See if this term is already in the lexicon
uint32_t tvi;
- auto tvi_or = lite_index_->FindTerm(term);
+ auto tvi_or = lite_index_->GetTermId(term);
// Step 2: Update the lexicon, either add the term or update its properties
if (tvi_or.ok()) {
diff --git a/icing/index/lite/doc-hit-info-iterator-term-lite.cc b/icing/index/lite/doc-hit-info-iterator-term-lite.cc
index a975f86..1f1c296 100644
--- a/icing/index/lite/doc-hit-info-iterator-term-lite.cc
+++ b/icing/index/lite/doc-hit-info-iterator-term-lite.cc
@@ -61,7 +61,7 @@ libtextclassifier3::Status DocHitInfoIteratorTermLite::Advance() {
libtextclassifier3::Status DocHitInfoIteratorTermLiteExact::RetrieveMoreHits() {
// Exact match only. All hits in lite lexicon are exact.
- ICING_ASSIGN_OR_RETURN(uint32_t tvi, lite_index_->FindTerm(term_));
+ ICING_ASSIGN_OR_RETURN(uint32_t tvi, lite_index_->GetTermId(term_));
ICING_ASSIGN_OR_RETURN(uint32_t term_id,
term_id_codec_->EncodeTvi(tvi, TviType::LITE));
lite_index_->AppendHits(term_id, section_restrict_mask_,
diff --git a/icing/index/lite/lite-index.cc b/icing/index/lite/lite-index.cc
index a72402e..89240ee 100644
--- a/icing/index/lite/lite-index.cc
+++ b/icing/index/lite/lite-index.cc
@@ -65,8 +65,8 @@ size_t header_size() { return sizeof(IcingLiteIndex_HeaderImpl::HeaderData); }
} // namespace
-const LiteIndex::Element::Value LiteIndex::Element::kInvalidValue =
- LiteIndex::Element(0, Hit()).value();
+const TermIdHitPair::Value TermIdHitPair::kInvalidValue =
+ TermIdHitPair(0, Hit()).value();
libtextclassifier3::StatusOr<std::unique_ptr<LiteIndex>> LiteIndex::Create(
const LiteIndex::Options& options, const IcingFilesystem* filesystem) {
@@ -163,7 +163,7 @@ libtextclassifier3::Status LiteIndex::Initialize() {
header_->Reset();
if (!hit_buffer_.Init(hit_buffer_fd_.get(), header_padded_size, true,
- sizeof(Element::Value), header_->cur_size(),
+ sizeof(TermIdHitPair::Value), header_->cur_size(),
options_.hit_buffer_size, &hit_buffer_crc_, true)) {
status = absl_ports::InternalError("Failed to initialize new hit buffer");
goto error;
@@ -177,7 +177,7 @@ libtextclassifier3::Status LiteIndex::Initialize() {
header_mmap_.address()));
if (!hit_buffer_.Init(hit_buffer_fd_.get(), header_padded_size, true,
- sizeof(Element::Value), header_->cur_size(),
+ sizeof(TermIdHitPair::Value), header_->cur_size(),
options_.hit_buffer_size, &hit_buffer_crc_, true)) {
status = absl_ports::InternalError(
"Failed to re-initialize existing hit buffer");
@@ -312,20 +312,21 @@ libtextclassifier3::Status LiteIndex::AddHit(uint32_t term_id, const Hit& hit) {
header_->set_last_added_docid(hit.document_id());
- Element elt(term_id, hit);
+ TermIdHitPair term_id_hit_pair(term_id, hit);
uint32_t cur_size = header_->cur_size();
- Element::Value* valp = hit_buffer_.GetMutableMem<Element::Value>(cur_size, 1);
+ TermIdHitPair::Value* valp =
+ hit_buffer_.GetMutableMem<TermIdHitPair::Value>(cur_size, 1);
if (valp == nullptr) {
return absl_ports::ResourceExhaustedError(
"Allocating more space in hit buffer failed!");
}
- *valp = elt.value();
+ *valp = term_id_hit_pair.value();
header_->set_cur_size(cur_size + 1);
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::StatusOr<uint32_t> LiteIndex::FindTerm(
+libtextclassifier3::StatusOr<uint32_t> LiteIndex::GetTermId(
const std::string& term) const {
char dummy;
uint32_t tvi;
@@ -336,16 +337,17 @@ libtextclassifier3::StatusOr<uint32_t> LiteIndex::FindTerm(
return tvi;
}
-uint32_t LiteIndex::AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
- bool only_from_prefix_sections,
- std::vector<DocHitInfo>* hits_out) {
- uint32_t count = 0;
+int LiteIndex::AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
+ bool only_from_prefix_sections,
+ std::vector<DocHitInfo>* hits_out) {
+ int count = 0;
DocumentId last_document_id = kInvalidDocumentId;
for (uint32_t idx = Seek(term_id); idx < header_->cur_size(); idx++) {
- Element elt(hit_buffer_.array_cast<Element>()[idx]);
- if (elt.term_id() != term_id) break;
+ TermIdHitPair term_id_hit_pair(
+ hit_buffer_.array_cast<TermIdHitPair>()[idx]);
+ if (term_id_hit_pair.term_id() != term_id) break;
- const Hit& hit = elt.hit();
+ const Hit& hit = term_id_hit_pair.hit();
// Check sections.
if (((1u << hit.section_id()) & section_id_mask) == 0) {
continue;
@@ -356,7 +358,7 @@ uint32_t LiteIndex::AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
}
DocumentId document_id = hit.document_id();
if (document_id != last_document_id) {
- count++;
+ ++count;
if (hits_out != nullptr) {
hits_out->push_back(DocHitInfo(document_id));
}
@@ -369,7 +371,7 @@ uint32_t LiteIndex::AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
return count;
}
-uint32_t LiteIndex::CountHits(uint32_t term_id) {
+int LiteIndex::CountHits(uint32_t term_id) {
return AppendHits(term_id, kSectionIdMaskAll,
/*only_from_prefix_sections=*/false,
/*hits_out=*/nullptr);
@@ -421,8 +423,8 @@ uint32_t LiteIndex::Seek(uint32_t term_id) {
IcingTimer timer;
auto* array_start =
- hit_buffer_.GetMutableMem<Element::Value>(0, header_->cur_size());
- Element::Value* sort_start = array_start + header_->searchable_end();
+ hit_buffer_.GetMutableMem<TermIdHitPair::Value>(0, header_->cur_size());
+ TermIdHitPair::Value* sort_start = array_start + header_->searchable_end();
std::sort(sort_start, array_start + header_->cur_size());
// Now merge with previous region. Since the previous region is already
@@ -445,11 +447,13 @@ uint32_t LiteIndex::Seek(uint32_t term_id) {
// Binary search for our term_id. Make sure we get the first
// element. Using kBeginSortValue ensures this for the hit value.
- Element elt(term_id, Hit(Hit::kMaxDocumentIdSortValue, Hit::kMaxHitScore));
+ TermIdHitPair term_id_hit_pair(
+ term_id, Hit(Hit::kMaxDocumentIdSortValue, Hit::kMaxHitScore));
- const Element::Value* array = hit_buffer_.array_cast<Element::Value>();
- const Element::Value* ptr =
- std::lower_bound(array, array + header_->cur_size(), elt.value());
+ const TermIdHitPair::Value* array =
+ hit_buffer_.array_cast<TermIdHitPair::Value>();
+ const TermIdHitPair::Value* ptr = std::lower_bound(
+ array, array + header_->cur_size(), term_id_hit_pair.value());
return ptr - array;
}
diff --git a/icing/index/lite/lite-index.h b/icing/index/lite/lite-index.h
index b60a947..27ccf33 100644
--- a/icing/index/lite/lite-index.h
+++ b/icing/index/lite/lite-index.h
@@ -30,6 +30,7 @@
#include "icing/file/filesystem.h"
#include "icing/index/hit/doc-hit-info.h"
#include "icing/index/hit/hit.h"
+#include "icing/index/lite/term-id-hit-pair.h"
#include "icing/legacy/index/icing-array-storage.h"
#include "icing/legacy/index/icing-dynamic-trie.h"
#include "icing/legacy/index/icing-filesystem.h"
@@ -49,49 +50,6 @@ namespace lib {
class LiteIndex {
public:
// An entry in the hit buffer.
- class Element {
- public:
- // Layout bits: 24 termid + 32 hit value + 8 hit score.
- using Value = uint64_t;
-
- static constexpr int kTermIdBits = 24;
- static constexpr int kHitValueBits = sizeof(Hit::Value) * 8;
- static constexpr int kHitScoreBits = sizeof(Hit::Score) * 8;
-
- static const Value kInvalidValue;
-
- explicit Element(Value v = kInvalidValue) : value_(v) {}
-
- Element(uint32_t term_id, const Hit& hit) {
- static_assert(
- kTermIdBits + kHitValueBits + kHitScoreBits <= sizeof(Value) * 8,
- "LiteIndexElementTooBig");
-
- value_ = 0;
- // Term id goes into the most significant bits because it takes
- // precedent in sorts.
- bit_util::BitfieldSet(term_id, kHitValueBits + kHitScoreBits, kTermIdBits,
- &value_);
- bit_util::BitfieldSet(hit.value(), kHitScoreBits, kHitValueBits, &value_);
- bit_util::BitfieldSet(hit.score(), 0, kHitScoreBits, &value_);
- }
-
- uint32_t term_id() const {
- return bit_util::BitfieldGet(value_, kHitValueBits + kHitScoreBits,
- kTermIdBits);
- }
-
- Hit hit() const {
- return Hit(bit_util::BitfieldGet(value_, kHitScoreBits, kHitValueBits),
- bit_util::BitfieldGet(value_, 0, kHitScoreBits));
- }
-
- Value value() const { return value_; }
-
- private:
- Value value_;
- };
-
using Options = IcingLiteIndexOptions;
// Updates checksum of subcomponents.
@@ -126,7 +84,7 @@ class LiteIndex {
Crc32 ComputeChecksum();
// Returns term_id if term found, NOT_FOUND otherwise.
- libtextclassifier3::StatusOr<uint32_t> FindTerm(
+ libtextclassifier3::StatusOr<uint32_t> GetTermId(
const std::string& term) const;
// Returns an iterator for all terms for which 'prefix' is a prefix.
@@ -170,25 +128,89 @@ class LiteIndex {
NamespaceId namespace_id);
// Append hit to buffer. term_id must be encoded using the same term_id_codec
- // supplied to the index constructor. Returns non-OK if hit cannot be added
- // (either due to hit buffer or file system capacity reached).
+ // supplied to the index constructor.
+ // RETURNS:
+ // - OK if hit was successfully added
+ // - RESOURCE_EXHAUSTED if hit could not be added (either due to hit buffer
+ // or file system capacity reached).
libtextclassifier3::Status AddHit(uint32_t term_id, const Hit& hit);
// Add all hits with term_id from the sections specified in section_id_mask,
// skipping hits in non-prefix sections if only_from_prefix_sections is true,
- // to hits_out.
- uint32_t AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
- bool only_from_prefix_sections,
- std::vector<DocHitInfo>* hits_out);
+ // to hits_out. If hits_out is nullptr, no hits will be added.
+ //
+ // Returns the number of hits that would be added to hits_out.
+ int AppendHits(uint32_t term_id, SectionIdMask section_id_mask,
+ bool only_from_prefix_sections,
+ std::vector<DocHitInfo>* hits_out);
// Returns the hit count of the term.
- uint32_t CountHits(uint32_t term_id);
+ int CountHits(uint32_t term_id);
// Check if buffer has reached its capacity.
bool is_full() const;
+ bool empty() const { return size() == 0; }
+
+ uint32_t size() const { return header_->cur_size(); }
+
+ class const_iterator {
+ friend class LiteIndex;
+
+ public:
+ using iterator_category = std::forward_iterator_tag;
+ using value_type = TermIdHitPair;
+ using reference = const value_type&;
+ using pointer = const value_type*;
+
+ const_iterator() : const_iterator(nullptr, -1, -1) {}
+
+ reference operator*() const { return start_[position_]; }
+
+ pointer operator->() const { return start_ + position_; }
+
+ const_iterator& operator++() {
+ if (++position_ >= end_position_) {
+ start_ = nullptr;
+ position_ = -1;
+ end_position_ = -1;
+ }
+ return *this;
+ }
+
+ const_iterator operator++(int) {
+ auto tmp = *this;
+ ++*this;
+ return tmp;
+ }
+
+ bool operator!=(const const_iterator& rhs) { return !(*this == rhs); }
+
+ bool operator==(const const_iterator& rhs) {
+ return start_ == rhs.start_ && position_ == rhs.position_;
+ }
+
+ private:
+ explicit const_iterator(const TermIdHitPair* start, int position,
+ int end_position)
+ : start_(start), position_(position), end_position_(end_position) {}
+
+ const TermIdHitPair* start_;
+ int position_;
+ int end_position_;
+ };
+
+ const_iterator begin() const {
+ // If the LiteIndex is empty, just return end().
+ return empty() ? end()
+ : const_iterator(hit_buffer_.array_cast<TermIdHitPair>(), 0,
+ header_->cur_size());
+ }
+
+ const_iterator end() const { return const_iterator(); }
+
constexpr static uint32_t max_hit_buffer_size() {
- return std::numeric_limits<uint32_t>::max() / sizeof(LiteIndex::Element);
+ return std::numeric_limits<uint32_t>::max() / sizeof(TermIdHitPair);
}
// We keep track of the last added document_id. This is always the largest
diff --git a/icing/index/lite/term-id-hit-pair.h b/icing/index/lite/term-id-hit-pair.h
new file mode 100644
index 0000000..191f766
--- /dev/null
+++ b/icing/index/lite/term-id-hit-pair.h
@@ -0,0 +1,80 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_INDEX_TERM_ID_HIT_PAIR_H_
+#define ICING_INDEX_TERM_ID_HIT_PAIR_H_
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "icing/index/hit/hit.h"
+#include "icing/util/bit-util.h"
+
+namespace icing {
+namespace lib {
+
+class TermIdHitPair {
+ public:
+ // Layout bits: 24 termid + 32 hit value + 8 hit score.
+ using Value = uint64_t;
+
+ static constexpr int kTermIdBits = 24;
+ static constexpr int kHitValueBits = sizeof(Hit::Value) * 8;
+ static constexpr int kHitScoreBits = sizeof(Hit::Score) * 8;
+
+ static const Value kInvalidValue;
+
+ explicit TermIdHitPair(Value v = kInvalidValue) : value_(v) {}
+
+ TermIdHitPair(uint32_t term_id, const Hit& hit) {
+ static_assert(
+ kTermIdBits + kHitValueBits + kHitScoreBits <= sizeof(Value) * 8,
+ "TermIdHitPairTooBig");
+
+ value_ = 0;
+ // Term id goes into the most significant bits because it takes
+ // precedent in sorts.
+ bit_util::BitfieldSet(term_id, kHitValueBits + kHitScoreBits, kTermIdBits,
+ &value_);
+ bit_util::BitfieldSet(hit.value(), kHitScoreBits, kHitValueBits, &value_);
+ bit_util::BitfieldSet(hit.score(), 0, kHitScoreBits, &value_);
+ }
+
+ uint32_t term_id() const {
+ return bit_util::BitfieldGet(value_, kHitValueBits + kHitScoreBits,
+ kTermIdBits);
+ }
+
+ Hit hit() const {
+ return Hit(bit_util::BitfieldGet(value_, kHitScoreBits, kHitValueBits),
+ bit_util::BitfieldGet(value_, 0, kHitScoreBits));
+ }
+
+ Value value() const { return value_; }
+
+ bool operator==(const TermIdHitPair& rhs) const {
+ return value_ == rhs.value_;
+ }
+
+ private:
+ Value value_;
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_INDEX_TERM_ID_HIT_PAIR_H_
diff --git a/icing/index/main/main-index-merger.cc b/icing/index/main/main-index-merger.cc
new file mode 100644
index 0000000..724cf43
--- /dev/null
+++ b/icing/index/main/main-index-merger.cc
@@ -0,0 +1,225 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/index/main/main-index-merger.h"
+
+#include <cstring>
+#include <memory>
+
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/index/lite/term-id-hit-pair.h"
+#include "icing/index/term-id-codec.h"
+#include "icing/legacy/core/icing-string-util.h"
+#include "icing/util/status-macros.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+class HitSelector {
+ public:
+ // Returns whether or not term_id_hit_pair has the same term_id, document_id and section_id
+ // as the previously selected hits.
+ bool IsEquivalentHit(const TermIdHitPair& term_id_hit_pair) {
+ return prev_.term_id() == term_id_hit_pair.term_id() &&
+ prev_.hit().document_id() == term_id_hit_pair.hit().document_id() &&
+ prev_.hit().section_id() == term_id_hit_pair.hit().section_id();
+ }
+
+ // Merges term_id_hit_pair with previously added hits.
+ void SelectIfBetter(const TermIdHitPair& term_id_hit_pair) {
+ if (term_id_hit_pair.hit().is_prefix_hit()) {
+ SelectPrefixHitIfBetter(term_id_hit_pair);
+ } else {
+ SelectExactHitIfBetter(term_id_hit_pair);
+ }
+ prev_ = term_id_hit_pair;
+ }
+
+ // Adds all valid, selected hits to hits starting at position pos in hits.
+ // Returns the offset in hits after the position of the last added hit.
+ // This function may add between 0-2 hits depending on whether the HitSelector
+ // holds both a valid exact hit and a valid prefix hit, one of those or none.
+ size_t InsertSelectedHits(size_t pos, std::vector<TermIdHitPair>* hits) {
+ // Given highest scoring prefix/exact hits for a given
+ // term+docid+sectionid, push needed hits into hits array at offset
+ // pos. Return new pos.
+ if (best_prefix_hit_.hit().is_valid() && best_exact_hit_.hit().is_valid()) {
+ // Output both if scores are unequal. Otherwise only exact hit is
+ // sufficient because 1) they have the same scores and 2) any prefix query
+ // will also accept an exact hit.
+ (*hits)[pos++] = best_exact_hit_;
+ if (best_prefix_hit_.hit().score() != best_exact_hit_.hit().score()) {
+ (*hits)[pos++] = best_prefix_hit_;
+ // Ensure sorted.
+ if (best_prefix_hit_.hit() < best_exact_hit_.hit()) {
+ std::swap((*hits)[pos - 1], (*hits)[pos - 2]);
+ }
+ }
+ } else if (best_prefix_hit_.hit().is_valid()) {
+ (*hits)[pos++] = best_prefix_hit_;
+ } else if (best_exact_hit_.hit().is_valid()) {
+ (*hits)[pos++] = best_exact_hit_;
+ }
+
+ return pos;
+ }
+
+ void Reset() {
+ best_prefix_hit_ = TermIdHitPair();
+ best_exact_hit_ = TermIdHitPair();
+ prev_ = TermIdHitPair();
+ }
+
+ private:
+ void SelectPrefixHitIfBetter(const TermIdHitPair& term_id_hit_pair) {
+ if (!best_prefix_hit_.hit().is_valid() ||
+ best_prefix_hit_.hit().score() < term_id_hit_pair.hit().score()) {
+ best_prefix_hit_ = term_id_hit_pair;
+ }
+ }
+
+ void SelectExactHitIfBetter(const TermIdHitPair& term_id_hit_pair) {
+ if (!best_exact_hit_.hit().is_valid() ||
+ best_exact_hit_.hit().score() < term_id_hit_pair.hit().score()) {
+ best_exact_hit_ = term_id_hit_pair;
+ }
+ }
+
+ TermIdHitPair best_prefix_hit_;
+ TermIdHitPair best_exact_hit_;
+ TermIdHitPair prev_;
+};
+
+// A helper function to dedupe hits stored in hits. Suppose that the lite index
+// contained a single document with two hits in a single prefix section: "foot"
+// and "fool". When expanded, there would be four hits:
+// {"fo", docid0, sectionid0}
+// {"fo", docid0, sectionid0}
+// {"foot", docid0, sectionid0}
+// {"fool", docid0, sectionid0}
+//
+// The first two are duplicates of each other. So, this function will dedupe
+// and shrink hits to be:
+// {"fo", docid0, sectionid0}
+// {"foot", docid0, sectionid0}
+// {"fool", docid0, sectionid0}
+//
+// When duplicates are encountered, we prefer the hit with the highest hit
+// score. If there is both an exact and prefix hit for the same term, we prefer
+// the exact hit, unless they have different scores, in which case we keep both
+// them.
+void DedupeHits(std::vector<TermIdHitPair>* hits) {
+ // Now all terms are grouped together and all hits for a term are sorted.
+ // Merge equivalent hits into one.
+ std::sort(hits->begin(), hits->end(),
+ [](const TermIdHitPair& lhs, const TermIdHitPair& rhs) {
+ return lhs.value() < rhs.value();
+ });
+ size_t current_offset = 0;
+ HitSelector hit_selector;
+ for (const TermIdHitPair& term_id_hit_pair : *hits) {
+ if (!hit_selector.IsEquivalentHit(term_id_hit_pair)) {
+ // We've reached a new hit. Insert the previously selected hits that we
+ // had accumulated and reset to add this new hit.
+ current_offset = hit_selector.InsertSelectedHits(current_offset, hits);
+ hit_selector.Reset();
+ }
+ // Update best exact and prefix hit.
+ hit_selector.SelectIfBetter(term_id_hit_pair);
+ }
+
+ // Push last.
+ current_offset = hit_selector.InsertSelectedHits(current_offset, hits);
+
+ hits->resize(current_offset);
+}
+
+// Based on experiments with full prefix expansion, the multiplier
+// is ~4x.
+constexpr int kAvgPrefixesPerTerm = 4;
+
+} // namespace
+
+libtextclassifier3::StatusOr<std::vector<TermIdHitPair>>
+MainIndexMerger::TranslateAndExpandLiteHits(
+ const LiteIndex& lite_index, const TermIdCodec& term_id_codec,
+ const MainIndex::LexiconMergeOutputs& lexicon_merge_outputs) {
+ std::vector<TermIdHitPair> hits;
+ if (lite_index.empty()) {
+ return hits;
+ }
+ // Reserve enough space for the average number of prefixes per term and the
+ // terms themselves.
+ hits.reserve(lite_index.size() * (kAvgPrefixesPerTerm + 1));
+
+ // Translate lite tvis to main tvis.
+ for (const TermIdHitPair& term_id_hit_pair : lite_index) {
+ uint32_t cur_term_id = term_id_hit_pair.term_id();
+ ICING_ASSIGN_OR_RETURN(TermIdCodec::DecodedTermInfo cur_decoded_term,
+ term_id_codec.DecodeTermInfo(cur_term_id));
+ Hit hit(term_id_hit_pair.hit());
+
+ // 1. Translate and push original.
+ auto itr =
+ lexicon_merge_outputs.other_tvi_to_main_tvi.find(cur_decoded_term.tvi);
+ if (itr == lexicon_merge_outputs.other_tvi_to_main_tvi.cend()) {
+ // b/37273773
+ return absl_ports::InternalError(IcingStringUtil::StringPrintf(
+ "Trying to translate lite tvi %u that was never added to the lexicon",
+ cur_decoded_term.tvi));
+ }
+ ICING_ASSIGN_OR_RETURN(uint32_t term_id,
+ term_id_codec.EncodeTvi(itr->second, TviType::MAIN));
+ hits.emplace_back(term_id, hit);
+
+ // 2. Expand hits in prefix sections.
+ if (hit.is_in_prefix_section()) {
+ // Hit was in a prefix section. Push prefixes. Turn on prefix bit.
+ auto itr_prefixes =
+ lexicon_merge_outputs.other_tvi_to_prefix_main_tvis.find(
+ cur_decoded_term.tvi);
+ if (itr_prefixes ==
+ lexicon_merge_outputs.other_tvi_to_prefix_main_tvis.end()) {
+ ICING_VLOG(1) << "No necessary prefix expansion for " << cur_decoded_term.tvi;
+ continue;
+ }
+ // The tvis of all prefixes of this hit's term that appear in the main
+ // lexicon are between [prefix_tvis_buf[offset],
+ // prefix_tvis_buf[offset+len]).
+ size_t offset = itr_prefixes->second.first;
+ size_t len = itr_prefixes->second.second;
+ Hit prefix_hit(hit.section_id(), hit.document_id(), hit.score(),
+ /*is_in_prefix_section=*/true, /*is_prefix_hit=*/true);
+ for (; offset < len; ++offset) {
+ // Take the tvi (in the main lexicon) of each prefix term.
+ uint32_t prefix_main_tvi =
+ lexicon_merge_outputs.prefix_tvis_buf[offset];
+ // Convert it to a term_id.
+ ICING_ASSIGN_OR_RETURN(
+ uint32_t prefix_term_id,
+ term_id_codec.EncodeTvi(prefix_main_tvi, TviType::MAIN));
+ // Create add an element for this prefix TermId and prefix Hit to hits.
+ hits.emplace_back(prefix_term_id, prefix_hit);
+ }
+ }
+ }
+ // 3. Remove any duplicate hits.
+ DedupeHits(&hits);
+ return hits;
+}
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/index/main/main-index-merger.h b/icing/index/main/main-index-merger.h
new file mode 100644
index 0000000..1413a8f
--- /dev/null
+++ b/icing/index/main/main-index-merger.h
@@ -0,0 +1,49 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_INDEX_MAIN_MAIN_INDEX_MERGER_H_
+#define ICING_INDEX_MAIN_MAIN_INDEX_MERGER_H_
+
+#include <memory>
+
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/index/lite/lite-index.h"
+#include "icing/index/main/main-index.h"
+#include "icing/index/term-id-codec.h"
+
+namespace icing {
+namespace lib {
+
+// Class used to merge hits from the lite_index and lite_lexicon into main_index
+// and main_lexicon.
+class MainIndexMerger {
+ public:
+ // Retrieves all hits in the lite index, translates the term ids of each
+ // LiteIndex::Element and expands prefix hits based on the mapping from
+ // lexicon_merge_outputs.other_tvi_to_prefix_main_tvis.
+ //
+ // RETURNS:
+ // - OK on success
+ // - INVALID_ARGUMENT if one of the elements in the lite index has a term_id
+ // that exceeds the max TermId
+ static libtextclassifier3::StatusOr<std::vector<TermIdHitPair>>
+ TranslateAndExpandLiteHits(
+ const LiteIndex& lite_index, const TermIdCodec& term_id_codec,
+ const MainIndex::LexiconMergeOutputs& lexicon_merge_outputs);
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_INDEX_MAIN_MAIN_INDEX_MERGER_H_
diff --git a/icing/index/main/main-index-merger_test.cc b/icing/index/main/main-index-merger_test.cc
new file mode 100644
index 0000000..42b3266
--- /dev/null
+++ b/icing/index/main/main-index-merger_test.cc
@@ -0,0 +1,367 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+#include "icing/index/main/main-index-merger.h"
+
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/filesystem.h"
+#include "icing/index/iterator/doc-hit-info-iterator.h"
+#include "icing/index/main/doc-hit-info-iterator-term-main.h"
+#include "icing/index/main/main-index-merger.h"
+#include "icing/index/main/main-index.h"
+#include "icing/index/term-id-codec.h"
+#include "icing/index/term-property-id.h"
+#include "icing/legacy/index/icing-dynamic-trie.h"
+#include "icing/legacy/index/icing-filesystem.h"
+#include "icing/schema/section.h"
+#include "icing/store/namespace-id.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/tmp-directory.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+using ::testing::UnorderedElementsAre;
+
+class MainIndexMergerTest : public testing::Test {
+ protected:
+ void SetUp() override {
+ index_dir_ = GetTestTempDir() + "/test_dir";
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(index_dir_.c_str()));
+
+ std::string lite_index_file_name = index_dir_ + "/test_file.lite-idx.index";
+ LiteIndex::Options options(lite_index_file_name,
+ /*hit_buffer_want_merge_bytes=*/1024 * 1024);
+ ICING_ASSERT_OK_AND_ASSIGN(lite_index_,
+ LiteIndex::Create(options, &icing_filesystem_));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ term_id_codec_,
+ TermIdCodec::Create(
+ IcingDynamicTrie::max_value_index(IcingDynamicTrie::Options()),
+ IcingDynamicTrie::max_value_index(options.lexicon_options)));
+ }
+
+ void TearDown() override {
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(index_dir_.c_str()));
+ }
+
+ std::string index_dir_;
+ Filesystem filesystem_;
+ IcingFilesystem icing_filesystem_;
+ std::unique_ptr<LiteIndex> lite_index_;
+ std::unique_ptr<TermIdCodec> term_id_codec_;
+};
+
+constexpr NamespaceId kNamespace0 = 0;
+
+TEST_F(MainIndexMergerTest, TranslateTermNotAdded) {
+ // 1. Index two docs in the Lite Index:
+ // - Doc0 {"foot" is_in_prefix_section=FALSE}
+ // - Doc1 {"fool", is_in_prefix_section=FALSE}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(foot_tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_tvi,
+ lite_index_->InsertTerm("fool", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_term_id,
+ term_id_codec_->EncodeTvi(fool_tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc1_hit));
+
+ // 2. Build up a fake LexiconMergeOutputs
+ // This is some made up number that doesn't matter for this test.
+ uint32_t foot_main_tvi = 5;
+
+ // Only create a mapping for 'foot'. Leave out the mapping for 'fool'
+ MainIndex::LexiconMergeOutputs lexicon_outputs;
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foot_tvi, foot_main_tvi);
+
+ // 3. TranslateAndExpand should fail because 'fool' doesn't have a main tvi
+ // mapping.
+ ASSERT_THAT(MainIndexMerger::TranslateAndExpandLiteHits(
+ *lite_index_, *term_id_codec_, lexicon_outputs),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(MainIndexMergerTest, PrefixExpansion) {
+ // 1. Index two docs in the Lite Index:
+ // - Doc0 {"foot" is_in_prefix_section=FALSE}
+ // - Doc1 {"fool", is_in_prefix_section=TRUE}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(foot_tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_tvi,
+ lite_index_->InsertTerm("fool", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_term_id,
+ term_id_codec_->EncodeTvi(fool_tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc1_hit));
+
+ // 2. Build up a fake LexiconMergeOutputs
+ // This is some made up number that doesn't matter for this test.
+ uint32_t foo_main_tvi = 12;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(foo_main_tvi, TviType::MAIN));
+ Hit doc1_prefix_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true, /*is_prefix_hit=*/true);
+
+ uint32_t foot_main_tvi = 5;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_main_term_id,
+ term_id_codec_->EncodeTvi(foot_main_tvi, TviType::MAIN));
+ uint32_t fool_main_tvi = 10;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_main_term_id,
+ term_id_codec_->EncodeTvi(fool_main_tvi, TviType::MAIN));
+
+ MainIndex::LexiconMergeOutputs lexicon_outputs;
+ // Map "fool" to it's prefix hit for "foo".
+ lexicon_outputs.other_tvi_to_prefix_main_tvis.emplace(fool_tvi,
+ std::make_pair(0, 1));
+ lexicon_outputs.prefix_tvis_buf.push_back(foo_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foot_tvi, foot_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(fool_tvi, fool_main_tvi);
+
+ // 3. TranslateAndExpand should;
+ // a. Translate lite term ids to main term ids based on the map
+ // b. Expand 'fool' to have a hit for 'foo'
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<TermIdHitPair> expanded_elts,
+ MainIndexMerger::TranslateAndExpandLiteHits(*lite_index_, *term_id_codec_,
+ lexicon_outputs));
+ EXPECT_THAT(expanded_elts, UnorderedElementsAre(
+ TermIdHitPair(foot_main_term_id, doc0_hit),
+ TermIdHitPair(fool_main_term_id, doc1_hit),
+ TermIdHitPair(foo_term_id, doc1_prefix_hit)));
+}
+
+TEST_F(MainIndexMergerTest, DedupePrefixAndExactWithDifferentScores) {
+ // 1. Index one doc in the Lite Index:
+ // - Doc0 {"foot" "foo" is_in_prefix_section=TRUE}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(foot_tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_tvi,
+ lite_index_->InsertTerm("foo", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(foo_tvi, TviType::LITE));
+
+ Hit foot_doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, foot_doc0_hit));
+ Hit foo_doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foo_term_id, foo_doc0_hit));
+
+ // 2. Build up a fake LexiconMergeOutputs
+ // This is some made up number that doesn't matter for this test.
+ uint32_t foo_main_tvi = 12;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_main_term_id,
+ term_id_codec_->EncodeTvi(foo_main_tvi, TviType::MAIN));
+ // The prefix hit for 'foot' should have the same score as the exact hit for
+ // 'foot'.
+ Hit doc0_prefix_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/true, /*is_prefix_hit=*/true);
+
+ uint32_t foot_main_tvi = 5;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_main_term_id,
+ term_id_codec_->EncodeTvi(foot_main_tvi, TviType::MAIN));
+
+ MainIndex::LexiconMergeOutputs lexicon_outputs;
+ // Map "foot" to it's prefix hit for "foo".
+ lexicon_outputs.other_tvi_to_prefix_main_tvis.emplace(foot_tvi,
+ std::make_pair(0, 1));
+ lexicon_outputs.prefix_tvis_buf.push_back(foo_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foot_tvi, foot_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foo_tvi, foo_main_tvi);
+
+ // 3. TranslateAndExpand should;
+ // a. Translate lite term ids to main term ids based on the map
+ // b. Expand 'foot' to have a hit for 'foo'
+ // c. Keep both the exact hit for 'foo' and the prefix hit for 'foot'
+ // because they have different scores.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<TermIdHitPair> expanded_elts,
+ MainIndexMerger::TranslateAndExpandLiteHits(*lite_index_, *term_id_codec_,
+ lexicon_outputs));
+ EXPECT_THAT(
+ expanded_elts,
+ UnorderedElementsAre(TermIdHitPair(foot_main_term_id, foot_doc0_hit),
+ TermIdHitPair(foo_main_term_id, foo_doc0_hit),
+ TermIdHitPair(foo_main_term_id, doc0_prefix_hit)));
+}
+
+TEST_F(MainIndexMergerTest, DedupeWithExactSameScores) {
+ // 1. Index one doc in the Lite Index:
+ // - Doc0 {"foot" "foo" is_in_prefix_section=TRUE}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(foot_tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_tvi,
+ lite_index_->InsertTerm("foo", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(foo_tvi, TviType::LITE));
+
+ Hit foot_doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, foot_doc0_hit));
+ Hit foo_doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foo_term_id, foo_doc0_hit));
+
+ // 2. Build up a fake LexiconMergeOutputs
+ // This is some made up number that doesn't matter for this test.
+ uint32_t foo_main_tvi = 12;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_main_term_id,
+ term_id_codec_->EncodeTvi(foo_main_tvi, TviType::MAIN));
+
+ uint32_t foot_main_tvi = 5;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_main_term_id,
+ term_id_codec_->EncodeTvi(foot_main_tvi, TviType::MAIN));
+
+ MainIndex::LexiconMergeOutputs lexicon_outputs;
+ // Map "foot" to it's prefix hit for "foo".
+ lexicon_outputs.other_tvi_to_prefix_main_tvis.emplace(foot_tvi,
+ std::make_pair(0, 1));
+ lexicon_outputs.prefix_tvis_buf.push_back(foo_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foot_tvi, foot_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foo_tvi, foo_main_tvi);
+
+ // 3. TranslateAndExpand should;
+ // a. Translate lite term ids to main term ids based on the map
+ // b. Expand 'foot' to have a hit for 'foo'
+ // c. Keep only the exact hit for 'foo' since they both have the same hit
+ // score.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<TermIdHitPair> expanded_elts,
+ MainIndexMerger::TranslateAndExpandLiteHits(*lite_index_, *term_id_codec_,
+ lexicon_outputs));
+ EXPECT_THAT(
+ expanded_elts,
+ UnorderedElementsAre(TermIdHitPair(foot_main_term_id, foot_doc0_hit),
+ TermIdHitPair(foo_main_term_id, foo_doc0_hit)));
+}
+
+TEST_F(MainIndexMergerTest, DedupePrefixExpansion) {
+ // 1. Index one doc in the Lite Index:
+ // - Doc0 {"foot" "fool" is_in_prefix_section=TRUE}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(foot_tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_tvi,
+ lite_index_->InsertTerm("fool", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_term_id,
+ term_id_codec_->EncodeTvi(fool_tvi, TviType::LITE));
+
+ Hit foot_doc0_hit(/*section_id=*/0, /*document_id=*/0, /*score=*/57,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, foot_doc0_hit));
+ Hit fool_doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, fool_doc0_hit));
+
+ // 2. Build up a fake LexiconMergeOutputs
+ // This is some made up number that doesn't matter for this test.
+ uint32_t foo_main_tvi = 12;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(foo_main_tvi, TviType::MAIN));
+ // The prefix hit should take the best score - MaxHitScore when merging these
+ // two.
+ Hit doc0_prefix_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true, /*is_prefix_hit=*/true);
+
+ uint32_t foot_main_tvi = 5;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t foot_main_term_id,
+ term_id_codec_->EncodeTvi(foot_main_tvi, TviType::MAIN));
+ uint32_t fool_main_tvi = 10;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t fool_main_term_id,
+ term_id_codec_->EncodeTvi(fool_main_tvi, TviType::MAIN));
+
+ MainIndex::LexiconMergeOutputs lexicon_outputs;
+ // Map "fool" to it's prefix hit for "foo" and "foot" to it's prefix hit for
+ // "foo".
+ lexicon_outputs.other_tvi_to_prefix_main_tvis.emplace(fool_tvi,
+ std::make_pair(0, 1));
+ lexicon_outputs.prefix_tvis_buf.push_back(foo_main_tvi);
+ lexicon_outputs.other_tvi_to_prefix_main_tvis.emplace(foot_tvi,
+ std::make_pair(1, 1));
+ lexicon_outputs.prefix_tvis_buf.push_back(foo_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(foot_tvi, foot_main_tvi);
+ lexicon_outputs.other_tvi_to_main_tvi.emplace(fool_tvi, fool_main_tvi);
+
+ // 3. TranslateAndExpand should;
+ // a. Translate lite term ids to main term ids based on the map
+ // b. Expand 'foot' and 'fool' to have hits for 'foo'
+ // c. Merge the prefix hits from 'foot' and 'fool', taking the best hit
+ // score.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<TermIdHitPair> expanded_elts,
+ MainIndexMerger::TranslateAndExpandLiteHits(*lite_index_, *term_id_codec_,
+ lexicon_outputs));
+ EXPECT_THAT(
+ expanded_elts,
+ UnorderedElementsAre(TermIdHitPair(foot_main_term_id, foot_doc0_hit),
+ TermIdHitPair(fool_main_term_id, fool_doc0_hit),
+ TermIdHitPair(foo_term_id, doc0_prefix_hit)));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/index/main/main-index.cc b/icing/index/main/main-index.cc
index 878038f..6e45760 100644
--- a/icing/index/main/main-index.cc
+++ b/icing/index/main/main-index.cc
@@ -28,8 +28,8 @@ namespace lib {
namespace {
-// Finds the best prefix term in lexicon for which "prefix" is a prefix.
-// 'Best' is defined as the shortest term that holds a valid posting list id.
+// Finds the shortest,valid prefix term with prefix hits in lexicon for which
+// "prefix" is a prefix.
// Returns a valid FindTermResult with found=true if either:
// 1. prefix exists as a term in lexicon.
// 2. the shortest, valid prefix in the lexicon exists and contains prefix
@@ -82,8 +82,10 @@ FindTermResult FindShortestValidTermWithPrefixHits(
} // namespace
libtextclassifier3::StatusOr<MainIndex> MainIndex::Create(
- const string& index_filename, const Filesystem* filesystem,
+ const std::string& index_filename, const Filesystem* filesystem,
const IcingFilesystem* icing_filesystem) {
+ ICING_RETURN_ERROR_IF_NULL(filesystem);
+ ICING_RETURN_ERROR_IF_NULL(icing_filesystem);
MainIndex main_index;
ICING_RETURN_IF_ERROR(
main_index.Init(index_filename, filesystem, icing_filesystem));
@@ -92,13 +94,14 @@ libtextclassifier3::StatusOr<MainIndex> MainIndex::Create(
// TODO(b/139087650) : Migrate off of IcingFilesystem.
libtextclassifier3::Status MainIndex::Init(
- const string& index_filename, const Filesystem* filesystem,
+ const std::string& index_filename, const Filesystem* filesystem,
const IcingFilesystem* icing_filesystem) {
std::string flash_index_file = index_filename + "-main-index";
ICING_ASSIGN_OR_RETURN(
FlashIndexStorage flash_index,
FlashIndexStorage::Create(flash_index_file, filesystem));
- flash_index_ = std::make_unique<FlashIndexStorage>(std::move(flash_index));
+ flash_index_storage_ =
+ std::make_unique<FlashIndexStorage>(std::move(flash_index));
std::string lexicon_file = index_filename + "-main-lexicon";
IcingDynamicTrie::RuntimeOptions runtime_options;
@@ -121,7 +124,7 @@ MainIndex::GetAccessorForExactTerm(const std::string& term) {
}
ICING_ASSIGN_OR_RETURN(PostingListAccessor accessor,
PostingListAccessor::CreateFromExisting(
- flash_index_.get(), posting_list_id));
+ flash_index_storage_.get(), posting_list_id));
return std::make_unique<PostingListAccessor>(std::move(accessor));
}
@@ -153,7 +156,7 @@ MainIndex::GetAccessorForPrefixTerm(const std::string& prefix) {
memcpy(&posting_list_id, main_itr.GetValue(), sizeof(posting_list_id));
ICING_ASSIGN_OR_RETURN(PostingListAccessor pl_accessor,
PostingListAccessor::CreateFromExisting(
- flash_index_.get(), posting_list_id));
+ flash_index_storage_.get(), posting_list_id));
GetPrefixAccessorResult result = {std::make_unique<PostingListAccessor>(std::move(pl_accessor)), exact};
return result;
}
@@ -218,7 +221,7 @@ libtextclassifier3::StatusOr<MainIndex::LexiconMergeOutputs>
MainIndex::AddTerms(const IcingDynamicTrie& other_lexicon,
LexiconMergeOutputs&& outputs) {
IcingDynamicTrie::PropertyReadersAll new_term_prop_readers(other_lexicon);
- for (IcingDynamicTrie::Iterator other_term_itr(other_lexicon, "");
+ for (IcingDynamicTrie::Iterator other_term_itr(other_lexicon, /*prefix=*/"");
other_term_itr.IsValid(); other_term_itr.Advance()) {
uint32_t new_main_tvi;
PostingListIdentifier posting_list_id = PostingListIdentifier::kInvalid;
@@ -233,7 +236,8 @@ MainIndex::AddTerms(const IcingDynamicTrie& other_lexicon,
uint32_t other_tvi = other_term_itr.GetValueIndex();
if (!CopyProperties(new_term_prop_readers, other_lexicon, other_tvi,
new_main_tvi)) {
- return absl_ports::InternalError("Could not insert term");
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Could not insert term: ", other_term_itr.GetKey()));
}
// Add other to main mapping.
@@ -251,7 +255,7 @@ MainIndex::AddBranchPoints(const IcingDynamicTrie& other_lexicon,
return outputs;
}
std::string prefix;
- for (IcingDynamicTrie::Iterator other_term_itr(other_lexicon, "");
+ for (IcingDynamicTrie::Iterator other_term_itr(other_lexicon, /*prefix=*/"");
other_term_itr.IsValid(); other_term_itr.Advance()) {
// Only expand terms that have hits in prefix sections.
if (!has_prefix_prop_reader.HasProperty(other_term_itr.GetValueIndex())) {
@@ -277,7 +281,8 @@ MainIndex::AddBranchPoints(const IcingDynamicTrie& other_lexicon,
PostingListIdentifier::kInvalid;
if (!main_lexicon_->Insert(prefix.c_str(), &posting_list_identifier,
&prefix_tvi, /*replace=*/false, &new_key)) {
- return absl_ports::InternalError("Could not insert prefix");
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Could not insert prefix: ", prefix));
}
// Prefix tvi will have hits in prefix section.
@@ -319,13 +324,14 @@ bool MainIndex::CopyProperties(
// HasNoExactHits in the main_lexicon should not be modified.
if (!prop_reader.HasProperty(property_id, other_tvi) &&
!main_lexicon_->ClearProperty(new_main_tvi, property_id)) {
- LOG(ERROR) << "Clearing prefix prop failed";
+ ICING_LOG(ERROR) << "Clearing HasNoExactHitsProperty failed";
return false;
}
} else {
// If other_lexicon has this property set for this term, then that
// property needs to be set for the main_lexicon. If other_lexicon
- // doesn't have this property set, then
+ // doesn't have this property set, then the property in the main lexicon
+ // should not be modified.
if (prop_reader.HasProperty(property_id, other_tvi) &&
!main_lexicon_->SetProperty(new_main_tvi, property_id)) {
return false;
@@ -335,5 +341,157 @@ bool MainIndex::CopyProperties(
return true;
}
+libtextclassifier3::Status MainIndex::AddHits(
+ const TermIdCodec& term_id_codec,
+ std::unordered_map<uint32_t, uint32_t>&& backfill_map,
+ std::vector<TermIdHitPair>&& hits) {
+ if (hits.empty()) {
+ return libtextclassifier3::Status::OK;
+ }
+ uint32_t cur_term_id = hits[0].term_id();
+ ICING_ASSIGN_OR_RETURN(TermIdCodec::DecodedTermInfo cur_decoded_term,
+ term_id_codec.DecodeTermInfo(cur_term_id));
+ // Iterate through all hits. If these hits are for a term that also needs
+ // backfill, then backfill first and then add the new hits.
+ size_t k_start = 0;
+ size_t k_end = 0;
+ while (k_start < hits.size()) {
+ uint32_t term_id = hits[k_end].term_id();
+ while (term_id == cur_term_id && ++k_end < hits.size()) {
+ term_id = hits[k_end].term_id();
+ }
+
+ // Look for backfill.
+ PostingListIdentifier backfill_posting_list_id =
+ PostingListIdentifier::kInvalid;
+ auto itr = backfill_map.find(cur_decoded_term.tvi);
+ if (itr != backfill_map.end()) {
+ const void* value = main_lexicon_->GetValueAtIndex(itr->second);
+ memcpy(&backfill_posting_list_id, value,
+ sizeof(backfill_posting_list_id));
+ backfill_map.erase(itr);
+ }
+ ICING_RETURN_IF_ERROR(AddHitsForTerm(cur_decoded_term.tvi,
+ backfill_posting_list_id,
+ &hits[k_start], k_end - k_start));
+ cur_term_id = term_id;
+ ICING_ASSIGN_OR_RETURN(cur_decoded_term,
+ term_id_codec.DecodeTermInfo(cur_term_id));
+ k_start = k_end;
+ }
+
+ // Now copy remaining backfills.
+ ICING_VLOG(2) << IcingStringUtil::StringPrintf("Remaining backfills %zu",
+ backfill_map.size());
+ for (auto other_tvi_main_tvi_pair : backfill_map) {
+ PostingListIdentifier backfill_posting_list_id =
+ PostingListIdentifier::kInvalid;
+ memcpy(&backfill_posting_list_id,
+ main_lexicon_->GetValueAtIndex(other_tvi_main_tvi_pair.second),
+ sizeof(backfill_posting_list_id));
+ ICING_ASSIGN_OR_RETURN(
+ PostingListAccessor hit_accum,
+ PostingListAccessor::Create(flash_index_storage_.get()));
+ ICING_RETURN_IF_ERROR(
+ AddPrefixBackfillHits(backfill_posting_list_id, &hit_accum));
+ PostingListAccessor::FinalizeResult result =
+ PostingListAccessor::Finalize(std::move(hit_accum));
+ if (result.id.is_valid()) {
+ main_lexicon_->SetValueAtIndex(other_tvi_main_tvi_pair.first, &result.id);
+ }
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status MainIndex::AddHitsForTerm(
+ uint32_t tvi, PostingListIdentifier backfill_posting_list_id,
+ const TermIdHitPair* hit_elements, size_t len) {
+ // 1. Create a PostingListAccessor - either from the pre-existing block, if
+ // one exists, or from scratch.
+ PostingListIdentifier posting_list_id = PostingListIdentifier::kInvalid;
+ memcpy(&posting_list_id, main_lexicon_->GetValueAtIndex(tvi),
+ sizeof(posting_list_id));
+ std::unique_ptr<PostingListAccessor> pl_accessor;
+ if (posting_list_id.is_valid()) {
+ if (posting_list_id.block_index() >= flash_index_storage_->num_blocks()) {
+ ICING_LOG(ERROR) << IcingStringUtil::StringPrintf(
+ "Index dropped hits. Invalid block index %u >= %u",
+ posting_list_id.block_index(), flash_index_storage_->num_blocks());
+ // TODO(b/159918304) : Consider revising the checksumming strategy in the
+ // main index. Providing some mechanism to check for corruption - either
+ // during initialization or some later time would allow us to avoid
+ // whack-a-mole with odd corruption issues like this one (b/62820689).
+ return absl_ports::InternalError(
+ "Valid posting list has an invalid block index!");
+ }
+ ICING_ASSIGN_OR_RETURN(PostingListAccessor tmp,
+ PostingListAccessor::CreateFromExisting(
+ flash_index_storage_.get(), posting_list_id));
+ pl_accessor = std::make_unique<PostingListAccessor>(std::move(tmp));
+ } else {
+ // New posting list.
+ ICING_ASSIGN_OR_RETURN(
+ PostingListAccessor tmp,
+ PostingListAccessor::Create(flash_index_storage_.get()));
+ pl_accessor = std::make_unique<PostingListAccessor>(std::move(tmp));
+ }
+
+ // 2. Backfill any hits if necessary.
+ if (backfill_posting_list_id.is_valid()) {
+ ICING_RETURN_IF_ERROR(
+ AddPrefixBackfillHits(backfill_posting_list_id, pl_accessor.get()));
+ }
+
+ // 3. Add all the new hits.
+ for (int i = len - 1; i >= 0; --i) {
+ Hit hit = hit_elements[i].hit();
+ ICING_RETURN_IF_ERROR(pl_accessor->PrependHit(hit));
+ }
+
+ // 4. Finalize this posting list and put its identifier in the lexicon.
+ PostingListAccessor::FinalizeResult result =
+ PostingListAccessor::Finalize(std::move(*pl_accessor));
+ if (result.id.is_valid()) {
+ main_lexicon_->SetValueAtIndex(tvi, &result.id);
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status MainIndex::AddPrefixBackfillHits(
+ PostingListIdentifier backfill_posting_list_id,
+ PostingListAccessor* hit_accum) {
+ ICING_ASSIGN_OR_RETURN(
+ PostingListAccessor backfill_accessor,
+ PostingListAccessor::CreateFromExisting(flash_index_storage_.get(),
+ backfill_posting_list_id));
+ std::vector<Hit> backfill_hits;
+ ICING_ASSIGN_OR_RETURN(std::vector<Hit> tmp,
+ backfill_accessor.GetNextHitsBatch());
+ while (!tmp.empty()) {
+ std::copy(tmp.begin(), tmp.end(), std::back_inserter(backfill_hits));
+ ICING_ASSIGN_OR_RETURN(tmp, backfill_accessor.GetNextHitsBatch());
+ }
+
+ Hit last_added_hit;
+ for (const Hit& hit : backfill_hits) {
+ // Skip hits from non-prefix-enabled sections.
+ if (!hit.is_in_prefix_section()) {
+ continue;
+ }
+
+ // A backfill hit is a prefix hit in a prefix section.
+ const Hit backfill_hit(hit.section_id(), hit.document_id(), hit.score(),
+ /*is_in_prefix_section=*/true,
+ /*is_prefix_hit=*/true);
+ if (backfill_hit == last_added_hit) {
+ // Skip duplicate values due to overriding of the is_prefix flag.
+ continue;
+ }
+ last_added_hit = backfill_hit;
+ ICING_RETURN_IF_ERROR(hit_accum->PrependHit(backfill_hit));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
} // namespace lib
} // namespace icing
diff --git a/icing/index/main/main-index.h b/icing/index/main/main-index.h
index 15bec1f..79378ea 100644
--- a/icing/index/main/main-index.h
+++ b/icing/index/main/main-index.h
@@ -20,7 +20,7 @@
#include "icing/text_classifier/lib3/utils/base/status.h"
#include "icing/text_classifier/lib3/utils/base/statusor.h"
#include "icing/file/filesystem.h"
-#include "icing/index/lite/lite-index.h"
+#include "icing/index/lite/term-id-hit-pair.h"
#include "icing/index/main/flash-index-storage.h"
#include "icing/index/main/posting-list-accessor.h"
#include "icing/index/term-id-codec.h"
@@ -33,8 +33,11 @@ namespace lib {
class MainIndex {
public:
+ // RETURNS:
+ // - valid instance of MainIndex, on success.
+ // - INTERNAL error if unable to create the lexicon or flash storage.
static libtextclassifier3::StatusOr<MainIndex> Create(
- const string& index_filename, const Filesystem* filesystem,
+ const std::string& index_filename, const Filesystem* filesystem,
const IcingFilesystem* icing_filesystem);
// Get a PostingListAccessor that holds the posting list chain for 'term'.
@@ -106,6 +109,11 @@ class MainIndex {
// Add hits to the main index and backfill from existing posting lists to new
// backfill branch points.
//
+ // The backfill_map maps from main_lexicon tvi for a newly added branching
+ // point to the main_lexicon tvi for the posting list whose hits must be
+ // backfilled. backfill_map should be populated as part of LexiconMergeOutputs
+ // in MergeLexicon and be blindly passed to this function.
+ //
// RETURNS:
// - OK on success
// - INVALID_ARGUMENT if one of the elements in the lite index has a term_id
@@ -116,10 +124,10 @@ class MainIndex {
libtextclassifier3::Status AddHits(
const TermIdCodec& term_id_codec,
std::unordered_map<uint32_t, uint32_t>&& backfill_map,
- std::vector<LiteIndex::Element>&& hits);
+ std::vector<TermIdHitPair>&& hits);
private:
- libtextclassifier3::Status Init(const string& index_filename,
+ libtextclassifier3::Status Init(const std::string& index_filename,
const Filesystem* filesystem,
const IcingFilesystem* icing_filesystem);
@@ -172,7 +180,52 @@ class MainIndex {
const IcingDynamicTrie& other_lexicon, uint32_t other_tvi,
uint32_t new_main_tvi);
- std::unique_ptr<FlashIndexStorage> flash_index_;
+ // Add all hits between [hit_elements, hit_elements + len) to main_index,
+ // updating the entry in the main lexicon at trie_value_index to point to the
+ // resulting posting list. Hits are sorted in descending document id order, so
+ // they should be to posting lists in reverse (starting at hit_elements
+ // + len - 1) and working backwards. Therefore, hit_elements must be in sorted
+ // order.
+ //
+ // trie_value_index may point to a valid posting list id if there is a
+ // pre-existing posting list to append to.
+ //
+ // If backfill_posting_list_id is valid, then the hits from the posting list
+ // identified by backfill_posting_list_id should be added to the new posting
+ // list before the hits in hit_elements.
+ //
+ // RETURNS:
+ // - OK on success
+ // - INVALID_ARGUMENT if posting_list_id stored at trie_value_index is valid
+ // but points out of bounds in the IndexBlock referred to by
+ // id.block_index(), if one of the hits from [hit_elements,hit_elements+len)
+ // is not valid, or if one of the hits from [hit_elements,hit_elements+len)
+ // is not less than the previously added hits.
+ // - INTERNAL_ERROR if posting_list_id stored at trie_value_index is valid
+ // but points to an invalid block index or if unable to mmap the IndexBlock.
+ // - RESOURCE_EXHAUSTED error if unable to grow the index to allocate a new
+ // posting list.
+ libtextclassifier3::Status AddHitsForTerm(
+ uint32_t tvi, PostingListIdentifier backfill_posting_list_id,
+ const TermIdHitPair* hit_elements, size_t len);
+
+ // Adds all prefix hits or hits from prefix sections present on the posting
+ // list identified by backfill_posting_list_id to hit_accum.
+ //
+ // RETURNS:
+ // - OK, on success
+ // - INVALID_ARGUMENT if backfill_posting_list_id points out of bounds in the
+ // IndexBlock referred to by id.block_index()
+ // - INTERNAL_ERROR if unable to mmap the block identified by
+ // backfill_posting_list_id or if the posting list identified by
+ // backfill_posting_list_id has been corrupted.
+ // - RESOURCE_EXHAUSTED error if unable to grow the index to allocate a new
+ // posting list.
+ libtextclassifier3::Status AddPrefixBackfillHits(
+ PostingListIdentifier backfill_posting_list_id,
+ PostingListAccessor* hit_accum);
+
+ std::unique_ptr<FlashIndexStorage> flash_index_storage_;
std::unique_ptr<IcingDynamicTrie> main_lexicon_;
};
diff --git a/icing/index/main/main-index_test.cc b/icing/index/main/main-index_test.cc
new file mode 100644
index 0000000..019b588
--- /dev/null
+++ b/icing/index/main/main-index_test.cc
@@ -0,0 +1,536 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/index/main/main-index.h"
+
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/filesystem.h"
+#include "icing/index/iterator/doc-hit-info-iterator.h"
+#include "icing/index/lite/term-id-hit-pair.h"
+#include "icing/index/main/doc-hit-info-iterator-term-main.h"
+#include "icing/index/main/main-index-merger.h"
+#include "icing/index/main/main-index.h"
+#include "icing/index/term-id-codec.h"
+#include "icing/index/term-property-id.h"
+#include "icing/legacy/index/icing-dynamic-trie.h"
+#include "icing/legacy/index/icing-filesystem.h"
+#include "icing/legacy/index/icing-mock-filesystem.h"
+#include "icing/schema/section.h"
+#include "icing/store/namespace-id.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/tmp-directory.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+using ::testing::ElementsAre;
+using ::testing::IsEmpty;
+using ::testing::NiceMock;
+using ::testing::SizeIs;
+
+std::vector<DocHitInfo> GetHits(std::unique_ptr<DocHitInfoIterator> iterator) {
+ std::vector<DocHitInfo> infos;
+ while (iterator->Advance().ok()) {
+ infos.push_back(iterator->doc_hit_info());
+ }
+ return infos;
+}
+
+std::vector<DocHitInfo> GetExactHits(
+ MainIndex* main_index, const std::string& term,
+ SectionIdMask section_mask = kSectionIdMaskAll) {
+ auto iterator = std::make_unique<DocHitInfoIteratorTermMainExact>(
+ main_index, term, section_mask);
+ return GetHits(std::move(iterator));
+}
+
+std::vector<DocHitInfo> GetPrefixHits(
+ MainIndex* main_index, const std::string& term,
+ SectionIdMask section_mask = kSectionIdMaskAll) {
+ auto iterator = std::make_unique<DocHitInfoIteratorTermMainPrefix>(
+ main_index, term, section_mask);
+ return GetHits(std::move(iterator));
+}
+
+libtextclassifier3::Status Merge(const LiteIndex& lite_index,
+ const TermIdCodec& term_id_codec,
+ MainIndex* main_index) {
+ ICING_ASSIGN_OR_RETURN(MainIndex::LexiconMergeOutputs outputs,
+ main_index->MergeLexicon(lite_index.lexicon()));
+ ICING_ASSIGN_OR_RETURN(std::vector<TermIdHitPair> elts,
+ MainIndexMerger::TranslateAndExpandLiteHits(
+ lite_index, term_id_codec, outputs));
+ return main_index->AddHits(term_id_codec, std::move(outputs.backfill_map),
+ std::move(elts));
+}
+
+class MainIndexTest : public testing::Test {
+ protected:
+ void SetUp() override {
+ index_dir_ = GetTestTempDir() + "/test_dir";
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(index_dir_.c_str()));
+
+ std::string lite_index_file_name = index_dir_ + "/test_file.lite-idx.index";
+ LiteIndex::Options options(lite_index_file_name,
+ /*hit_buffer_want_merge_bytes=*/1024 * 1024);
+ ICING_ASSERT_OK_AND_ASSIGN(lite_index_,
+ LiteIndex::Create(options, &icing_filesystem_));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ term_id_codec_,
+ TermIdCodec::Create(
+ IcingDynamicTrie::max_value_index(IcingDynamicTrie::Options()),
+ IcingDynamicTrie::max_value_index(options.lexicon_options)));
+ }
+
+ void TearDown() override {
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(index_dir_.c_str()));
+ }
+
+ std::string index_dir_;
+ Filesystem filesystem_;
+ IcingFilesystem icing_filesystem_;
+ std::unique_ptr<LiteIndex> lite_index_;
+ std::unique_ptr<TermIdCodec> term_id_codec_;
+};
+
+constexpr NamespaceId kNamespace0 = 0;
+
+TEST_F(MainIndexTest, MainIndexCreateIOFailure) {
+ // Create the index with mock filesystem. By default, Mock will return false,
+ // so the first attempted file operation will fail.
+ NiceMock<IcingMockFilesystem> mock_filesystem;
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ EXPECT_THAT(
+ MainIndex::Create(main_index_file_name, &filesystem_, &mock_filesystem),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(MainIndexTest, MainIndexGetAccessorForPrefixTermNotFound) {
+ // Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+ EXPECT_THAT(main_index.GetAccessorForPrefixTerm("foo"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(MainIndexTest, MainIndexGetAccessorForPrefixReturnsValidAccessor) {
+ // 1. Index one doc in the Lite Index:
+ // - Doc0 {"foot" is_in_prefix_section=true}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the index. The main index should contain "foo".
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ // GetAccessorForPrefixTerm should return a valid accessor for "foo".
+ EXPECT_THAT(main_index.GetAccessorForPrefixTerm("foo"), IsOk());
+}
+
+TEST_F(MainIndexTest, MainIndexGetAccessorForExactTermNotFound) {
+ // Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+ EXPECT_THAT(main_index.GetAccessorForExactTerm("foo"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(MainIndexTest, MainIndexGetAccessorForExactReturnsValidAccessor) {
+ // 1. Index one doc in the Lite Index:
+ // - Doc0 {"foo" is_in_prefix_section=false}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foo", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the index. The main index should contain "foo".
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ // GetAccessorForPrefixTerm should return a valid accessor for "foo".
+ EXPECT_THAT(main_index.GetAccessorForExactTerm("foo"), IsOk());
+}
+
+TEST_F(MainIndexTest, MergeIndexToEmpty) {
+ // 1. Index three docs in the Lite Index:
+ // - Doc0 {"foot", "fool", "far" is_in_prefix_section=false}
+ // - Doc1 {"foot", "fool" is_in_prefix_section=true}
+ // - Doc2 {"fool", "far" is_in_prefix_section=false}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi, lite_index_->InsertTerm("fool", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t fool_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi,
+ lite_index_->InsertTerm("far", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t far_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc0_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(far_term_id, doc0_hit));
+
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc1_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc1_hit));
+
+ Hit doc2_hit(/*section_id=*/0, /*document_id=*/2, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc2_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(far_term_id, doc2_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ std::vector<DocHitInfo> hits = GetExactHits(&main_index, "foot");
+ EXPECT_THAT(hits, IsEmpty());
+ hits = GetPrefixHits(&main_index, "fo");
+ EXPECT_THAT(hits, IsEmpty());
+
+ // 3. Merge the index. The main index should contain "fool", "foot"
+ // and "far" as well as a branch points for "foo" and "f". "fa" and "fo"
+ // should not be present because it is not a branch point.
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ // Get hits from an exact posting list.
+ hits = GetExactHits(&main_index, "foot");
+ // We should get hits for "foot" in doc1 and doc0
+ EXPECT_THAT(
+ hits,
+ ElementsAre(
+ EqualsDocHitInfo(doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()}),
+ EqualsDocHitInfo(doc0_hit.document_id(),
+ std::vector<SectionId>{doc0_hit.section_id()})));
+
+ // Get hits from a branching point posting list. "fo" should redirect to "foo"
+ hits = GetPrefixHits(&main_index, "fo");
+ // We should get hits for "foot" in doc1 and "fool" in doc1. We shouldn't get
+ // the hits for "foot" in doc0 and "fool" in doc0 and doc2 because they
+ // weren't hits in prefix sections.
+ EXPECT_THAT(hits, ElementsAre(EqualsDocHitInfo(
+ doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()})));
+}
+
+TEST_F(MainIndexTest, MergeIndexToPreexisting) {
+ // 1. Index three docs in the Lite Index:
+ // - Doc0 {"foot", "fool", "far" is_in_prefix_section=false}
+ // - Doc1 {"foot", "fool" is_in_prefix_section=true}
+ // - Doc2 {"fool", "far" is_in_prefix_section=false}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi, lite_index_->InsertTerm("fool", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t fool_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi,
+ lite_index_->InsertTerm("far", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t far_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc0_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(far_term_id, doc0_hit));
+
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc1_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc1_hit));
+
+ Hit doc2_hit(/*section_id=*/0, /*document_id=*/2, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(fool_term_id, doc2_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(far_term_id, doc2_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the index. The main index should contain "fool", "foot"
+ // and "far" as well as a branch points for "foo" and "f". "fa" and "fo"
+ // should not be present because it is not a branch point.
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+
+ // 4. Index two docs in a new Lite Index:
+ // - Doc3 {"foot", "four", "foul", "fall" is_in_prefix_section=false}
+ // - Doc4 {"four", "foul" is_in_prefix_section=true}
+ std::string lite_index_file_name2 = index_dir_ + "/test_file.lite-idx.index2";
+ LiteIndex::Options options(lite_index_file_name2,
+ /*hit_buffer_want_merge_bytes=*/1024 * 1024);
+ ICING_ASSERT_OK_AND_ASSIGN(lite_index_,
+ LiteIndex::Create(options, &icing_filesystem_));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi, lite_index_->InsertTerm("four", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t four_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi, lite_index_->InsertTerm("foul", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foul_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi,
+ lite_index_->InsertTerm("fall", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t fall_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc3_hit(/*section_id=*/0, /*document_id=*/3, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc3_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(four_term_id, doc3_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(foul_term_id, doc3_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(fall_term_id, doc3_hit));
+
+ Hit doc4_hit(/*section_id=*/0, /*document_id=*/4, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(four_term_id, doc4_hit));
+ ICING_ASSERT_OK(lite_index_->AddHit(foul_term_id, doc4_hit));
+
+ // 3. Merge the index. The main index should now contain "foul", "four"
+ // and "fall", a branch points for "fou" and backfill points for "fo".
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ // Get hits from an exact posting list the existed before the merge.
+ std::vector<DocHitInfo> hits = GetExactHits(&main_index, "foot");
+
+ // We should get hits for "foot" in doc3, doc1 and doc0
+ EXPECT_THAT(
+ hits,
+ ElementsAre(
+ EqualsDocHitInfo(doc3_hit.document_id(),
+ std::vector<SectionId>{doc3_hit.section_id()}),
+ EqualsDocHitInfo(doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()}),
+ EqualsDocHitInfo(doc0_hit.document_id(),
+ std::vector<SectionId>{doc0_hit.section_id()})));
+ // Get hits from backfill posting list.
+ hits = GetPrefixHits(&main_index, "fo");
+ // We should get hits for "four" and "foul" in doc4 and hits for "foot" and
+ // "fool" in doc1. We shouldn't get the hits for "foot" in doc0 and doc3,
+ // "fool" in doc0 and doc2 or the hits for "four" and "foul" in doc4 because
+ // they weren't hits in prefix sections.
+ EXPECT_THAT(
+ hits,
+ ElementsAre(
+ EqualsDocHitInfo(doc4_hit.document_id(),
+ std::vector<SectionId>{doc4_hit.section_id()}),
+ EqualsDocHitInfo(doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()})));
+}
+
+TEST_F(MainIndexTest, ExactRetrievedInPrefixSearch) {
+ // 1. Index two docs in the Lite Index:
+ // - Doc0 {"foot" is_in_prefix_section=true}
+ // - Doc1 {"foo" is_in_prefix_section=false}
+ // - Doc2 {"foot" is_in_prefix_section=false}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi,
+ lite_index_->InsertTerm("foo", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foo_term_id, doc1_hit));
+
+ Hit doc2_hit(/*section_id=*/0, /*document_id=*/2, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc2_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the lite lexicon. The main lexicon should contain "foot" and
+ // "foo".
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ std::vector<DocHitInfo> hits = GetPrefixHits(&main_index, "foo");
+ // We should get hits for "foo" in doc1 and doc0, but not in doc2 because it
+ // is not a prefix hit.
+ EXPECT_THAT(
+ hits,
+ ElementsAre(
+ EqualsDocHitInfo(doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()}),
+ EqualsDocHitInfo(doc0_hit.document_id(),
+ std::vector<SectionId>{doc0_hit.section_id()})));
+}
+
+TEST_F(MainIndexTest, PrefixNotRetrievedInExactSearch) {
+ // 1. Index two docs in the Lite Index:
+ // - Doc0 {"foot" is_in_prefix_section=true}
+ // - Doc1 {"foo" is_in_prefix_section=false}
+ // - Doc1 {"foo" is_in_prefix_section=true}
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ tvi, lite_index_->InsertTerm("foo", TermMatchType::PREFIX, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foo_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ Hit doc0_hit(/*section_id=*/0, /*document_id=*/0, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc0_hit));
+
+ Hit doc1_hit(/*section_id=*/0, /*document_id=*/1, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foo_term_id, doc1_hit));
+
+ Hit doc2_hit(/*section_id=*/0, /*document_id=*/2, Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/true);
+ ICING_ASSERT_OK(lite_index_->AddHit(foo_term_id, doc2_hit));
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the lite lexicon. The main lexicon should contain "foot" and
+ // "foo".
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ std::vector<DocHitInfo> hits = GetExactHits(&main_index, "foo");
+
+ // We should get hits for "foo" in doc2 and doc1, but not in doc0 because it
+ // is not an exact hit.
+ EXPECT_THAT(
+ hits,
+ ElementsAre(
+ EqualsDocHitInfo(doc2_hit.document_id(),
+ std::vector<SectionId>{doc2_hit.section_id()}),
+ EqualsDocHitInfo(doc1_hit.document_id(),
+ std::vector<SectionId>{doc1_hit.section_id()})));
+}
+
+TEST_F(MainIndexTest, SearchChainedPostingLists) {
+ // Index 2048 document with 3 hits in each document. When merged into the main
+ // index, this will 1) lead to a chained posting list and 2) split at least
+ // one document's hits across multiple posting lists.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ uint32_t tvi,
+ lite_index_->InsertTerm("foot", TermMatchType::EXACT_ONLY, kNamespace0));
+ ICING_ASSERT_OK_AND_ASSIGN(uint32_t foot_term_id,
+ term_id_codec_->EncodeTvi(tvi, TviType::LITE));
+
+ for (DocumentId document_id = 0; document_id < 2048; ++document_id) {
+ Hit doc_hit0(/*section_id=*/0, /*document_id=*/document_id,
+ Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc_hit0));
+
+ Hit doc_hit1(/*section_id=*/1, /*document_id=*/document_id,
+ Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc_hit1));
+
+ Hit doc_hit2(/*section_id=*/2, /*document_id=*/document_id,
+ Hit::kMaxHitScore,
+ /*is_in_prefix_section=*/false);
+ ICING_ASSERT_OK(lite_index_->AddHit(foot_term_id, doc_hit2));
+ }
+
+ // 2. Create the main index. It should have no entries in its lexicon.
+ std::string main_index_file_name = index_dir_ + "/test_file.idx.index";
+ ICING_ASSERT_OK_AND_ASSIGN(
+ MainIndex main_index,
+ MainIndex::Create(main_index_file_name, &filesystem_,
+ &icing_filesystem_));
+
+ // 3. Merge the lite index.
+ ICING_ASSERT_OK(Merge(*lite_index_, *term_id_codec_, &main_index));
+ // Get hits for all documents containing "foot" - which should be all of them.
+ std::vector<DocHitInfo> hits = GetExactHits(&main_index, "foot");
+
+ EXPECT_THAT(hits, SizeIs(2048));
+ EXPECT_THAT(hits.front(),
+ EqualsDocHitInfo(2047, std::vector<SectionId>{0, 1, 2}));
+ EXPECT_THAT(hits.back(),
+ EqualsDocHitInfo(0, std::vector<SectionId>{0, 1, 2}));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/store/document-store.cc b/icing/store/document-store.cc
index 79b91df..7577f6b 100644
--- a/icing/store/document-store.cc
+++ b/icing/store/document-store.cc
@@ -335,8 +335,7 @@ libtextclassifier3::Status DocumentStore::RegenerateDerivedFiles() {
if (absl_ports::IsNotFound(document_wrapper_or.status())) {
// The erased document still occupies 1 document id.
DocumentId new_document_id = document_id_mapper_->num_elements();
- ICING_RETURN_IF_ERROR(
- ClearDerivedData(/*name_space=*/"", /*uri=*/"", new_document_id));
+ ICING_RETURN_IF_ERROR(ClearDerivedData(new_document_id));
iterator_status = iterator.Advance();
continue;
} else if (!document_wrapper_or.ok()) {
@@ -889,7 +888,7 @@ libtextclassifier3::Status DocumentStore::Delete(
return SoftDelete(name_space, uri, document_id);
} else {
uint64_t document_log_offset = file_offset_or.ValueOrDie();
- return HardDelete(name_space, uri, document_id, document_log_offset);
+ return HardDelete(document_id, document_log_offset);
}
}
@@ -898,22 +897,21 @@ libtextclassifier3::Status DocumentStore::Delete(DocumentId document_id,
// Copy out the document to get namespace and uri.
ICING_ASSIGN_OR_RETURN(int64_t document_log_offset,
DoesDocumentExistAndGetFileOffset(document_id));
- auto document_wrapper_or = document_log_->ReadProto(document_log_offset);
- if (!document_wrapper_or.ok()) {
- ICING_LOG(ERROR) << document_wrapper_or.status().error_message()
- << "Failed to read from document log";
- return document_wrapper_or.status();
- }
- DocumentWrapper document_wrapper =
- std::move(document_wrapper_or).ValueOrDie();
if (soft_delete) {
+ auto document_wrapper_or = document_log_->ReadProto(document_log_offset);
+ if (!document_wrapper_or.ok()) {
+ ICING_LOG(ERROR) << document_wrapper_or.status().error_message()
+ << "Failed to read from document log";
+ return document_wrapper_or.status();
+ }
+ DocumentWrapper document_wrapper =
+ std::move(document_wrapper_or).ValueOrDie();
+
return SoftDelete(document_wrapper.document().namespace_(),
document_wrapper.document().uri(), document_id);
} else {
- return HardDelete(document_wrapper.document().namespace_(),
- document_wrapper.document().uri(), document_id,
- document_log_offset);
+ return HardDelete(document_id, document_log_offset);
}
}
@@ -937,11 +935,10 @@ libtextclassifier3::Status DocumentStore::SoftDelete(
}
libtextclassifier3::Status DocumentStore::HardDelete(
- std::string_view name_space, std::string_view uri, DocumentId document_id,
- uint64_t document_log_offset) {
+ DocumentId document_id, uint64_t document_log_offset) {
// Erases document proto.
ICING_RETURN_IF_ERROR(document_log_->EraseProto(document_log_offset));
- return ClearDerivedData(name_space, uri, document_id);
+ return ClearDerivedData(document_id);
}
libtextclassifier3::StatusOr<NamespaceId> DocumentStore::GetNamespaceId(
@@ -1112,23 +1109,15 @@ libtextclassifier3::StatusOr<int> DocumentStore::BatchDelete(
ICING_RETURN_IF_ERROR(
document_id_mapper_->Set(document_id, kDocDeletedFlag));
} else {
- // Hard delete. Try to copy out the document to get namespace and uri.
- // Getting namespace and uri is necessary to delete entries in
- // document_key_mapper_.
- auto document_or = Get(document_id);
- if (absl_ports::IsNotFound(document_or.status())) {
- // Document not found.
+ // Hard delete.
+ libtextclassifier3::Status delete_status =
+ Delete(document_id, /*soft_delete=*/false);
+ if (absl_ports::IsNotFound(delete_status)) {
continue;
- } else if (!document_or.ok()) {
+ } else if (!delete_status.ok()) {
// Real error, pass up.
- return document_or.status();
+ return delete_status;
}
- DocumentProto document_copy = std::move(document_or).ValueOrDie();
-
- // Erase from the ground truth. Delete() won't return NOT_FOUND because
- // NOT_FOUND should have been caught by Get() above.
- ICING_RETURN_IF_ERROR(Delete(document_copy.namespace_(),
- document_copy.uri(), /*soft_delete=*/false));
++num_updated_documents;
}
}
@@ -1424,11 +1413,10 @@ libtextclassifier3::Status DocumentStore::UpdateFilterCache(
}
libtextclassifier3::Status DocumentStore::ClearDerivedData(
- const std::string_view name_space, const std::string_view uri,
DocumentId document_id) {
- if (!name_space.empty() && !uri.empty()) {
- document_key_mapper_->Delete(MakeFingerprint(name_space, uri));
- }
+ // We intentionally leave the data in key_mapper_ because locating that data
+ // requires fetching namespace and uri. Leaving data in key_mapper_ should be
+ // fine because the data is hashed.
ICING_RETURN_IF_ERROR(document_id_mapper_->Set(document_id, kDocDeletedFlag));
diff --git a/icing/store/document-store.h b/icing/store/document-store.h
index 52ea176..2ac1c71 100644
--- a/icing/store/document-store.h
+++ b/icing/store/document-store.h
@@ -170,8 +170,9 @@ class DocumentStore {
// otherwise the document proto will be erased immediately.
//
// NOTE:
- // 1. The soft deletion uses less CPU power, it can be applied on
- // non-sensitive data.
+ // 1. If possible, please use the other method Delete(name_space, uri,
+ // soft_delete) for soft deletes because we need namespace and uri to
+ // perform soft deletes.
// 2. Space is not reclaimed for deleted documents until Optimize() is
// called.
//
@@ -488,16 +489,14 @@ class DocumentStore {
std::string_view uri,
DocumentId document_id);
- // Erases the document identified by the given name_space, uri and document_id
- // from the document_log and erases its uri from the document_key_mapper_, the
- // space will be reclaimed later during Optimize().
+ // Erases the document at the given document_log_offset from the document_log
+ // and clears the derived data identified by the given document_id. The space
+ // will be reclaimed later during Optimize().
//
// Returns:
// OK on success
// INTERNAL_ERROR on IO error
- libtextclassifier3::Status HardDelete(std::string_view name_space,
- std::string_view uri,
- DocumentId document_id,
+ libtextclassifier3::Status HardDelete(DocumentId document_id,
uint64_t document_log_offset);
// Helper method to find a DocumentId that is associated with the given
@@ -539,9 +538,7 @@ class DocumentStore {
DocumentId document_id, const DocumentFilterData& filter_data);
// Helper method to clear the derived data of a document
- libtextclassifier3::Status ClearDerivedData(std::string_view name_space,
- std::string_view uri,
- DocumentId document_id);
+ libtextclassifier3::Status ClearDerivedData(DocumentId document_id);
};
} // namespace lib
diff --git a/icing/store/usage-store.cc b/icing/store/usage-store.cc
new file mode 100644
index 0000000..911c45a
--- /dev/null
+++ b/icing/store/usage-store.cc
@@ -0,0 +1,193 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/store/usage-store.h"
+
+#include "icing/file/file-backed-vector.h"
+#include "icing/proto/usage.pb.h"
+#include "icing/store/document-id.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+std::string MakeUsageScoreCacheFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/usage-scores");
+}
+} // namespace
+
+libtextclassifier3::StatusOr<std::unique_ptr<UsageStore>> UsageStore::Create(
+ const Filesystem* filesystem, const std::string& base_dir) {
+ ICING_RETURN_ERROR_IF_NULL(filesystem);
+
+ auto usage_score_cache_or = FileBackedVector<UsageScores>::Create(
+ *filesystem, MakeUsageScoreCacheFilename(base_dir),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC);
+
+ if (!usage_score_cache_or.ok()) {
+ ICING_LOG(ERROR) << usage_score_cache_or.status().error_message()
+ << "Failed to initialize usage_score_cache";
+ return usage_score_cache_or.status();
+ }
+
+ return std::unique_ptr<UsageStore>(new UsageStore(
+ std::move(usage_score_cache_or).ValueOrDie(), *filesystem, base_dir));
+}
+
+libtextclassifier3::Status UsageStore::AddUsageReport(const UsageReport& report,
+ DocumentId document_id) {
+ if (!IsDocumentIdValid(document_id)) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "Document id %d is invalid.", document_id));
+ }
+
+ auto usage_scores_or = usage_score_cache_->Get(document_id);
+
+ // OutOfRange means that the mapper hasn't seen this document id before, it's
+ // not an error here.
+ UsageScores usage_scores;
+ if (usage_scores_or.ok()) {
+ usage_scores = *std::move(usage_scores_or).ValueOrDie();
+ } else if (!absl_ports::IsOutOfRange(usage_scores_or.status())) {
+ // Real error
+ return usage_scores_or.status();
+ }
+
+ // Update last used timestamps and type counts. The counts won't be
+ // incremented if they are already the maximum values. The timestamp from
+ // UsageReport is in milliseconds, we need to convert it to seconds.
+ int64_t report_timestamp_s = report.usage_timestamp_ms() / 1000;
+
+ switch (report.usage_type()) {
+ case UsageReport::USAGE_TYPE1:
+ if (report_timestamp_s > std::numeric_limits<uint32_t>::max()) {
+ usage_scores.usage_type1_last_used_timestamp_s =
+ std::numeric_limits<uint32_t>::max();
+ } else if (report_timestamp_s >
+ usage_scores.usage_type1_last_used_timestamp_s) {
+ usage_scores.usage_type1_last_used_timestamp_s = report_timestamp_s;
+ }
+
+ if (usage_scores.usage_type1_count < std::numeric_limits<int>::max()) {
+ ++usage_scores.usage_type1_count;
+ }
+ break;
+ case UsageReport::USAGE_TYPE2:
+ if (report_timestamp_s > std::numeric_limits<uint32_t>::max()) {
+ usage_scores.usage_type2_last_used_timestamp_s =
+ std::numeric_limits<uint32_t>::max();
+ } else if (report_timestamp_s >
+ usage_scores.usage_type2_last_used_timestamp_s) {
+ usage_scores.usage_type2_last_used_timestamp_s = report_timestamp_s;
+ }
+
+ if (usage_scores.usage_type2_count < std::numeric_limits<int>::max()) {
+ ++usage_scores.usage_type2_count;
+ }
+ break;
+ case UsageReport::USAGE_TYPE3:
+ if (report_timestamp_s > std::numeric_limits<uint32_t>::max()) {
+ usage_scores.usage_type3_last_used_timestamp_s =
+ std::numeric_limits<uint32_t>::max();
+ } else if (report_timestamp_s >
+ usage_scores.usage_type3_last_used_timestamp_s) {
+ usage_scores.usage_type3_last_used_timestamp_s = report_timestamp_s;
+ }
+
+ if (usage_scores.usage_type3_count < std::numeric_limits<int>::max()) {
+ ++usage_scores.usage_type3_count;
+ }
+ }
+
+ // Write updated usage scores to file.
+ ICING_RETURN_IF_ERROR(usage_score_cache_->Set(document_id, usage_scores));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status UsageStore::DeleteUsageScores(
+ DocumentId document_id) {
+ if (!IsDocumentIdValid(document_id)) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "Document id %d is invalid.", document_id));
+ }
+
+ // Clear all the scores of the document.
+ ICING_RETURN_IF_ERROR(usage_score_cache_->Set(document_id, UsageScores()));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<UsageStore::UsageScores>
+UsageStore::GetUsageScores(DocumentId document_id) {
+ if (!IsDocumentIdValid(document_id)) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "Document id %d is invalid.", document_id));
+ }
+
+ auto usage_scores_or = usage_score_cache_->Get(document_id);
+ if (absl_ports::IsOutOfRange(usage_scores_or.status())) {
+ // No usage scores found. Return the default scores.
+ return UsageScores();
+ } else if (!usage_scores_or.ok()) {
+ // Pass up any other errors.
+ return usage_scores_or.status();
+ }
+
+ return *std::move(usage_scores_or).ValueOrDie();
+}
+
+libtextclassifier3::Status UsageStore::SetUsageScores(
+ DocumentId document_id, UsageScores usage_scores) {
+ if (!IsDocumentIdValid(document_id)) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "Document id %d is invalid.", document_id));
+ }
+
+ ICING_RETURN_IF_ERROR(usage_score_cache_->Set(document_id, usage_scores));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status UsageStore::PersistToDisk() {
+ ICING_RETURN_IF_ERROR(usage_score_cache_->PersistToDisk());
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status UsageStore::Reset() {
+ // We delete all the scores by deleting the whole file.
+ libtextclassifier3::Status status = FileBackedVector<int64_t>::Delete(
+ filesystem_, MakeUsageScoreCacheFilename(base_dir_));
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete usage_score_cache";
+ return status;
+ }
+
+ // Create a new usage_score_cache
+ auto usage_score_cache_or = FileBackedVector<UsageScores>::Create(
+ filesystem_, MakeUsageScoreCacheFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC);
+ if (!usage_score_cache_or.ok()) {
+ ICING_LOG(ERROR) << usage_score_cache_or.status().error_message()
+ << "Failed to re-create usage_score_cache";
+ return usage_score_cache_or.status();
+ }
+ usage_score_cache_ = std::move(usage_score_cache_or).ValueOrDie();
+
+ return libtextclassifier3::Status::OK;
+}
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/store/usage-store.h b/icing/store/usage-store.h
new file mode 100644
index 0000000..9a8c286
--- /dev/null
+++ b/icing/store/usage-store.h
@@ -0,0 +1,160 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <cstdint>
+
+#include "icing/file/file-backed-vector.h"
+#include "icing/proto/usage.pb.h"
+#include "icing/store/document-id.h"
+
+#ifndef ICING_STORE_USAGE_STORE_H_
+#define ICING_STORE_USAGE_STORE_H_
+
+namespace icing {
+namespace lib {
+
+// A storage class that maintains scores that are calculated based on usage
+// reports.
+class UsageStore {
+ public:
+ // Factory function to create a UsageStore instance. The base directory is
+ // used to persist usage scores. If a usage store was previously created with
+ // this directory, it will reload the files saved by the last instance.
+ //
+ // TODO(b/169594617): consider returning StatusOr<UsageStore>
+ //
+ // Returns:
+ // A UsageStore on success
+ // FAILED_PRECONDITION on any null pointer input
+ // INTERNAL_ERROR on I/O error
+ static libtextclassifier3::StatusOr<std::unique_ptr<UsageStore>> Create(
+ const Filesystem* filesystem, const std::string& base_dir);
+
+ // The scores here reflect the timestamps and usage types defined in
+ // usage.proto.
+ struct UsageScores {
+ // The latest timestamp in seconds reported with custom usage type 1.
+ uint32_t usage_type1_last_used_timestamp_s = 0;
+
+ // The latest timestamp in seconds reported with custom usage type 2.
+ uint32_t usage_type2_last_used_timestamp_s = 0;
+
+ // The latest timestamp in seconds reported with custom usage type 3.
+ uint32_t usage_type3_last_used_timestamp_s = 0;
+
+ // Count of reports with custom usage type 1
+ int usage_type1_count = 0;
+
+ // Count of reports with custom usage type 2
+ int usage_type2_count = 0;
+
+ // Count of reports with custom usage type 3
+ int usage_type3_count = 0;
+
+ bool operator==(const UsageScores& other) const {
+ return usage_type1_last_used_timestamp_s ==
+ other.usage_type1_last_used_timestamp_s &&
+ usage_type2_last_used_timestamp_s ==
+ other.usage_type2_last_used_timestamp_s &&
+ usage_type3_last_used_timestamp_s ==
+ other.usage_type3_last_used_timestamp_s &&
+ usage_type1_count == other.usage_type1_count &&
+ usage_type2_count == other.usage_type2_count &&
+ usage_type3_count == other.usage_type3_count;
+ }
+ };
+
+ // Adds one usage report. The corresponding usage scores of the specified
+ // document will be updated.
+ //
+ // Note: changes are written to disk automatically, callers can also call
+ // PersistToDisk() to flush changes immediately.
+ //
+ // Returns:
+ // OK on success
+ // INVALID_ARGUMENT if document_id is invalid
+ // INTERNAL_ERROR on I/O errors.
+ libtextclassifier3::Status AddUsageReport(const UsageReport& report,
+ DocumentId document_id);
+
+ // Deletes the usage scores of a document.
+ //
+ // Note: changes are written to disk automatically, callers can also call
+ // PersistToDisk() to flush changes immediately.
+ //
+ // Returns:
+ // OK on success
+ // INVALID_ARGUMENT if document_id is invalid
+ // INTERNAL_ERROR on I/O errors
+ libtextclassifier3::Status DeleteUsageScores(DocumentId document_id);
+
+ // Gets the usage scores of a document.
+ //
+ // Returns:
+ // UsageScores on success
+ // INVALID_ARGUMENT if document_id is invalid
+ // NOT_FOUND if no scores are found for the document
+ // INTERNAL_ERROR on I/O errors
+ //
+ // TODO(b/169433395): return a pointer instead of an object.
+ libtextclassifier3::StatusOr<UsageScores> GetUsageScores(
+ DocumentId document_id);
+
+ // Sets the usage scores of a document.
+ //
+ // Note: changes are written to disk automatically, callers can also call
+ // PersistToDisk() to flush changes immediately.
+ //
+ // Returns:
+ // OK on success
+ // INVALID_ARGUMENT if document_id is invalid
+ // INTERNAL_ERROR on I/O errors
+ libtextclassifier3::Status SetUsageScores(DocumentId document_id,
+ UsageScores usage_scores);
+
+ // Syncs data to disk.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL on I/O error
+ libtextclassifier3::Status PersistToDisk();
+
+ // Deletes all usage data and re-initialize the storage.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status Reset();
+
+ private:
+ explicit UsageStore(std::unique_ptr<FileBackedVector<UsageScores>>
+ document_id_to_scores_mapper,
+ const Filesystem& filesystem, std::string base_dir)
+ : filesystem_(filesystem),
+ base_dir_(std::move(base_dir)),
+ usage_score_cache_(std::move(document_id_to_scores_mapper)) {}
+
+ const Filesystem& filesystem_;
+
+ // Base directory where the files are located.
+ const std::string base_dir_;
+
+ // Used to store the usage scores of documents.
+ std::unique_ptr<FileBackedVector<UsageScores>> usage_score_cache_;
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_USAGE_STORE_H_
diff --git a/icing/store/usage-store_test.cc b/icing/store/usage-store_test.cc
new file mode 100644
index 0000000..39985f0
--- /dev/null
+++ b/icing/store/usage-store_test.cc
@@ -0,0 +1,389 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/store/usage-store.h"
+
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/tmp-directory.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+using ::testing::Eq;
+using ::testing::Not;
+
+class UsageStoreTest : public testing::Test {
+ protected:
+ UsageStoreTest() : test_dir_(GetTestTempDir() + "/usage-store-test") {}
+
+ void SetUp() override {
+ filesystem_.CreateDirectoryRecursively(test_dir_.c_str());
+ }
+
+ void TearDown() override {
+ filesystem_.DeleteDirectoryRecursively(test_dir_.c_str());
+ }
+
+ const Filesystem filesystem_;
+ const std::string test_dir_;
+};
+
+UsageReport CreateUsageReport(std::string name_space, std::string uri,
+ int64 timestamp_ms,
+ UsageReport::UsageType usage_type) {
+ UsageReport usage_report;
+ usage_report.set_document_namespace(name_space);
+ usage_report.set_document_uri(uri);
+ usage_report.set_usage_timestamp_ms(timestamp_ms);
+ usage_report.set_usage_type(usage_type);
+ return usage_report;
+}
+
+TEST_F(UsageStoreTest, CreationShouldSucceed) {
+ EXPECT_THAT(UsageStore::Create(&filesystem_, test_dir_), IsOk());
+}
+
+TEST_F(UsageStoreTest, CreationShouldFailOnNullPointer) {
+ EXPECT_THAT(UsageStore::Create(nullptr, test_dir_),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+}
+
+TEST_F(UsageStoreTest, UsageScoresShouldBeComparable) {
+ UsageStore::UsageScores scores1;
+ UsageStore::UsageScores scores2;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type1_last_used_timestamp_s.
+ ++scores1.usage_type1_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type1_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type2_last_used_timestamp_s.
+ ++scores1.usage_type2_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type2_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type3_last_used_timestamp_s.
+ ++scores1.usage_type3_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type3_last_used_timestamp_s;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type1_count.
+ ++scores1.usage_type1_count;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type1_count;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type2_count.
+ ++scores1.usage_type2_count;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type2_count;
+ EXPECT_THAT(scores1, Eq(scores2));
+
+ // operator== should compare usage_type3_count.
+ ++scores1.usage_type3_count;
+ EXPECT_THAT(scores1, Not(Eq(scores2)));
+ ++scores2.usage_type3_count;
+ EXPECT_THAT(scores1, Eq(scores2));
+}
+
+TEST_F(UsageStoreTest, InvalidDocumentIdShouldReturnError) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ DocumentId invalid_document_id = -1;
+
+ EXPECT_THAT(usage_store->AddUsageReport(UsageReport(), invalid_document_id),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ EXPECT_THAT(usage_store->DeleteUsageScores(invalid_document_id),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ EXPECT_THAT(usage_store->GetUsageScores(invalid_document_id),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ EXPECT_THAT(usage_store->SetUsageScores(invalid_document_id,
+ UsageStore::UsageScores()),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(UsageStoreTest, AddUsageReportShouldUpdateLastUsedTimestamp) {
+ // Create 3 reports with different timestamps.
+ UsageReport usage_report_time1 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/1000, UsageReport::USAGE_TYPE1);
+ UsageReport usage_report_time5 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/5000, UsageReport::USAGE_TYPE1);
+ UsageReport usage_report_time10 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/10000, UsageReport::USAGE_TYPE1);
+
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Report a usage with timestamp 5.
+ usage_store->AddUsageReport(usage_report_time5, /*document_id=*/1);
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_last_used_timestamp_s = 5;
+ expected_scores.usage_type1_count = 1;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+
+ // Report a usage with timestamp 1. The timestamp won't be updated.
+ usage_store->AddUsageReport(usage_report_time1, /*document_id=*/1);
+ ++expected_scores.usage_type1_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+
+ // Report a usage with timestamp 10. The timestamp should be updated.
+ usage_store->AddUsageReport(usage_report_time10, /*document_id=*/1);
+ expected_scores.usage_type1_last_used_timestamp_s = 10;
+ ++expected_scores.usage_type1_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, AddUsageReportShouldUpdateCounts) {
+ // Create 3 reports with different usage types.
+ UsageReport usage_report_type1 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/0, UsageReport::USAGE_TYPE1);
+ UsageReport usage_report_type2 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/0, UsageReport::USAGE_TYPE2);
+ UsageReport usage_report_type3 = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/0, UsageReport::USAGE_TYPE3);
+
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Report a usage with type 1.
+ usage_store->AddUsageReport(usage_report_type1, /*document_id=*/1);
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_count = 1;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+ // Report another usage with type 1.
+ usage_store->AddUsageReport(usage_report_type1, /*document_id=*/1);
+ ++expected_scores.usage_type1_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+
+ // Report a usage with type 2.
+ usage_store->AddUsageReport(usage_report_type2, /*document_id=*/1);
+ ++expected_scores.usage_type2_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+ // Report another usage with type 2.
+ usage_store->AddUsageReport(usage_report_type2, /*document_id=*/1);
+ ++expected_scores.usage_type2_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+
+ // Report a usage with type 3.
+ usage_store->AddUsageReport(usage_report_type3, /*document_id=*/1);
+ ++expected_scores.usage_type3_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+ // Report another usage with type 3.
+ usage_store->AddUsageReport(usage_report_type3, /*document_id=*/1);
+ ++expected_scores.usage_type3_count;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, GetNonExistingDocumentShouldReturnDefaultScores) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(UsageStore::UsageScores()));
+}
+
+TEST_F(UsageStoreTest, SetAndGetUsageScores) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Create usage scores with some random numbers.
+ UsageStore::UsageScores scores;
+ scores.usage_type1_last_used_timestamp_s = 7;
+ scores.usage_type2_last_used_timestamp_s = 9;
+ scores.usage_type3_last_used_timestamp_s = 11;
+ scores.usage_type1_count = 3;
+ scores.usage_type2_count = 4;
+ scores.usage_type3_count = 9;
+
+ // Verify that set and get results are consistent.
+ ICING_EXPECT_OK(usage_store->SetUsageScores(/*document_id=*/1, scores));
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(scores));
+}
+
+TEST_F(UsageStoreTest, ImplicitlyInitializedScoresShouldBeZero) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Explicitly set scores for document 2.
+ ICING_ASSERT_OK(usage_store->SetUsageScores(/*document_id=*/2,
+ UsageStore::UsageScores()));
+
+ // Now the scores of document 1 have been implicitly initialized. The scores
+ // should all be 0.
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_last_used_timestamp_s = 0;
+ expected_scores.usage_type2_last_used_timestamp_s = 0;
+ expected_scores.usage_type3_last_used_timestamp_s = 0;
+ expected_scores.usage_type1_count = 0;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, DeleteUsageScores) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Create usage scores with some random numbers.
+ UsageStore::UsageScores scores;
+ scores.usage_type1_last_used_timestamp_s = 7;
+ scores.usage_type2_last_used_timestamp_s = 9;
+ scores.usage_type3_last_used_timestamp_s = 11;
+ scores.usage_type1_count = 3;
+ scores.usage_type2_count = 4;
+ scores.usage_type3_count = 9;
+ ICING_EXPECT_OK(usage_store->SetUsageScores(/*document_id=*/1, scores));
+
+ // Delete the usage scores of document 1, all the scores of document 1 should
+ // be 0.
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_last_used_timestamp_s = 0;
+ expected_scores.usage_type2_last_used_timestamp_s = 0;
+ expected_scores.usage_type3_last_used_timestamp_s = 0;
+ expected_scores.usage_type1_count = 0;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ ICING_EXPECT_OK(usage_store->DeleteUsageScores(/*document_id=*/1));
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, PersistToDisk) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Create usage scores with some random numbers.
+ UsageStore::UsageScores scores;
+ scores.usage_type1_last_used_timestamp_s = 7;
+ scores.usage_type2_last_used_timestamp_s = 9;
+ scores.usage_type3_last_used_timestamp_s = 11;
+ scores.usage_type1_count = 3;
+ scores.usage_type2_count = 4;
+ scores.usage_type3_count = 9;
+ ICING_EXPECT_OK(usage_store->SetUsageScores(/*document_id=*/1, scores));
+
+ EXPECT_THAT(usage_store->PersistToDisk(), IsOk());
+}
+
+TEST_F(UsageStoreTest, Reset) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Create usage scores with some random numbers.
+ UsageStore::UsageScores scores;
+ scores.usage_type1_last_used_timestamp_s = 7;
+ scores.usage_type2_last_used_timestamp_s = 9;
+ scores.usage_type3_last_used_timestamp_s = 11;
+ scores.usage_type1_count = 3;
+ scores.usage_type2_count = 4;
+ scores.usage_type3_count = 9;
+
+ // Set scores for document 1 and document 2.
+ ICING_EXPECT_OK(usage_store->SetUsageScores(/*document_id=*/1, scores));
+ ICING_EXPECT_OK(usage_store->SetUsageScores(/*document_id=*/2, scores));
+
+ EXPECT_THAT(usage_store->Reset(), IsOk());
+
+ // After resetting, all the scores are cleared.
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_last_used_timestamp_s = 0;
+ expected_scores.usage_type2_last_used_timestamp_s = 0;
+ expected_scores.usage_type3_last_used_timestamp_s = 0;
+ expected_scores.usage_type1_count = 0;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/2),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, TimestampInSecondsShouldNotOverflow) {
+ // Create a report with the max value of timestamps.
+ UsageReport usage_report = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/std::numeric_limits<int64>::max(),
+ UsageReport::USAGE_TYPE1);
+
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // The stored timestamp in seconds should be the max value of uint32.
+ usage_store->AddUsageReport(usage_report, /*document_id=*/1);
+ UsageStore::UsageScores expected_scores;
+ expected_scores.usage_type1_last_used_timestamp_s =
+ std::numeric_limits<uint32_t>::max();
+ expected_scores.usage_type1_count = 1;
+ expected_scores.usage_type2_count = 0;
+ expected_scores.usage_type3_count = 0;
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(expected_scores));
+}
+
+TEST_F(UsageStoreTest, CountsShouldNotOverflow) {
+ ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<UsageStore> usage_store,
+ UsageStore::Create(&filesystem_, test_dir_));
+
+ // Create usage scores with the max value of int.
+ UsageStore::UsageScores scores;
+ scores.usage_type1_last_used_timestamp_s = 0;
+ scores.usage_type2_last_used_timestamp_s = 0;
+ scores.usage_type3_last_used_timestamp_s = 0;
+ scores.usage_type1_count = std::numeric_limits<int>::max();
+ scores.usage_type2_count = 0;
+ scores.usage_type3_count = 0;
+
+ ICING_ASSERT_OK(usage_store->SetUsageScores(/*document_id=*/1, scores));
+ ASSERT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(scores));
+
+ // Report another usage with type 1.
+ UsageReport usage_report = CreateUsageReport(
+ "namespace", "uri", /*timestamp_ms=*/0, UsageReport::USAGE_TYPE1);
+ usage_store->AddUsageReport(usage_report, /*document_id=*/1);
+
+ // usage_type1_count should not change because it's already the max value.
+ EXPECT_THAT(usage_store->GetUsageScores(/*document_id=*/1),
+ IsOkAndHolds(scores));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/tokenization/ios/ios-language-segmenter-factory.cc b/icing/tokenization/ios/ios-language-segmenter-factory.cc
deleted file mode 100644
index 3af7914..0000000
--- a/icing/tokenization/ios/ios-language-segmenter-factory.cc
+++ /dev/null
@@ -1,51 +0,0 @@
-// Copyright (C) 2019 Google LLC
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include "icing/tokenization/ios/ios-language-segmenter.h"
-#include "icing/tokenization/language-segmenter-factory.h"
-#include "icing/util/logging.h"
-
-namespace icing {
-namespace lib {
-
-namespace language_segmenter_factory {
-
-namespace {
-constexpr std::string_view kLocaleAmericanEnglishComputer = "en_US_POSIX";
-} // namespace
-
-// Creates a language segmenter with the given locale.
-//
-// Returns:
-// A LanguageSegmenter on success
-// INVALID_ARGUMENT if locale string is invalid
-libtextclassifier3::StatusOr<std::unique_ptr<LanguageSegmenter>> Create(
- SegmenterOptions options) {
- // Word connector rules for "en_US_POSIX" (American English (Computer)) are
- // different from other locales. E.g. "email.subject" will be split into 3
- // terms in "en_US_POSIX": "email", ".", and "subject", while it's just one
- // term in other locales. Our current LanguageSegmenter doesn't handle this
- // special rule, so we replace it with "en_US".
- if (options.locale == kLocaleAmericanEnglishComputer) {
- ICING_LOG(WARNING) << "Locale " << kLocaleAmericanEnglishComputer
- << " not supported. Converting to locale en_US";
- options.locale = "en_US";
- }
- return std::make_unique<IosLanguageSegmenter>(std::move(options.locale));
-}
-
-} // namespace language_segmenter_factory
-
-} // namespace lib
-} // namespace icing
diff --git a/icing/tokenization/ios/ios-language-segmenter.h b/icing/tokenization/ios/ios-language-segmenter.h
deleted file mode 100644
index 1aa1f1b..0000000
--- a/icing/tokenization/ios/ios-language-segmenter.h
+++ /dev/null
@@ -1,88 +0,0 @@
-// Copyright (C) 2019 Google LLC
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#ifndef ICING_TOKENIZATION_IOS_IOS_LANGUAGE_SEGMENTER_H_
-#define ICING_TOKENIZATION_IOS_IOS_LANGUAGE_SEGMENTER_H_
-
-#include <memory>
-#include <string>
-#include <string_view>
-#include <vector>
-
-#include "icing/text_classifier/lib3/utils/base/statusor.h"
-#include "icing/tokenization/language-segmenter.h"
-
-namespace icing {
-namespace lib {
-
-// This class is used to segment sentences into words based on rules from
-// CFStringTokenizer, some extra rules are applied in this class:
-//
-// 1. All ASCII terms will be returned.
-// 2. For non-ASCII terms, only the alphabetic terms are returned, which means
-// non-ASCII punctuation and special characters are left out.
-// 3. Multiple continuous whitespaces are treated as one.
-//
-// The rules above are common to the high-level tokenizers that might use this
-// class. Other special tokenization logic will be in each tokenizer.
-//
-// This implementation has a few notable deviations from the ICU-based
-// implementations:
-// 1. This implementation doesn't treat ':' as a word connector. ICU does.
-// 2. When the locale is Japanese, this implementation treats internal periods
-// are as word breaks rather than connectors. "N.B.A." becomes {"N", ".",
-// "B", ".", "A", "."} rather than {"N.B.A", "."} (which is what ICU and
-// all other locales do.
-// 3. Locale can have other effects on segmentation - this is often when the
-// wrong locale is specified for CJKT text.
-// 4. Some CJKT segmentation deviates from ICU results even when the correct
-// locale is specified.
-class IosLanguageSegmenter : public LanguageSegmenter {
- public:
- explicit IosLanguageSegmenter(std::string locale)
- : locale_(std::move(locale)) {}
-
- IosLanguageSegmenter(const IosLanguageSegmenter&) = delete;
- IosLanguageSegmenter& operator=(const IosLanguageSegmenter&) = delete;
-
- // The segmentation depends on the language detected in the input text.
- //
- // Note: It could happen that the language detected from text is wrong, then
- // there would be a small chance that the text is segmented incorrectly.
- //
- // Returns:
- // An iterator of terms on success
- // INTERNAL_ERROR if any error occurs
- libtextclassifier3::StatusOr<std::unique_ptr<LanguageSegmenter::Iterator>>
- Segment(std::string_view text) const override;
-
- // The segmentation depends on the language detected in the input text.
- //
- // Note: It could happen that the language detected from text is wrong, then
- // there would be a small chance that the text is segmented incorrectly.
- //
- // Returns:
- // A list of terms on success
- // INTERNAL_ERROR if any error occurs
- libtextclassifier3::StatusOr<std::vector<std::string_view>> GetAllTerms(
- std::string_view text) const override;
-
- private:
- std::string locale_;
-};
-
-} // namespace lib
-} // namespace icing
-
-#endif // ICING_TOKENIZATION_IOS_IOS_LANGUAGE_SEGMENTER_H_
diff --git a/icing/tokenization/ios/ios-language-segmenter_test.cc b/icing/tokenization/ios/ios-language-segmenter_test.cc
deleted file mode 100644
index b6831e2..0000000
--- a/icing/tokenization/ios/ios-language-segmenter_test.cc
+++ /dev/null
@@ -1,1265 +0,0 @@
-// Copyright (C) 2019 Google LLC
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include "gmock/gmock.h"
-#include "gtest/gtest.h"
-#include "icing/absl_ports/str_cat.h"
-#include "icing/testing/common-matchers.h"
-#include "icing/testing/icu-i18n-test-utils.h"
-#include "icing/tokenization/language-segmenter-factory.h"
-#include "icing/tokenization/language-segmenter.h"
-#include "unicode/uloc.h"
-
-namespace icing {
-namespace lib {
-namespace {
-using ::testing::ElementsAre;
-using ::testing::ElementsAreArray;
-using ::testing::Eq;
-using ::testing::IsEmpty;
-
-// Returns a vector containing all terms retrieved by Advancing on the iterator.
-std::vector<std::string_view> GetAllTermsAdvance(
- LanguageSegmenter::Iterator* itr) {
- std::vector<std::string_view> terms;
- while (itr->Advance()) {
- terms.push_back(itr->GetTerm());
- }
- return terms;
-}
-
-// Returns a vector containing all terms retrieved by calling
-// ResetToStart/ResetAfter with the current position to simulate Advancing on
-// the iterator.
-std::vector<std::string_view> GetAllTermsResetAfter(
- LanguageSegmenter::Iterator* itr) {
- std::vector<std::string_view> terms;
- if (!itr->ResetToStart().ok()) {
- return terms;
- }
- terms.push_back(itr->GetTerm());
- const char* text_begin = itr->GetTerm().data();
- // Calling ResetToTermStartingAfter with the current position should get the
- // very next term in the sequence.
- for (int current_pos = 0; itr->ResetToTermStartingAfter(current_pos).ok();
- current_pos = itr->GetTerm().data() - text_begin) {
- terms.push_back(itr->GetTerm());
- }
- return terms;
-}
-
-// Returns a vector containing all terms retrieved by alternating calls to
-// Advance and calls to ResetAfter with the current position to simulate
-// Advancing.
-std::vector<std::string_view> GetAllTermsAdvanceAndResetAfter(
- LanguageSegmenter::Iterator* itr) {
- const char* text_begin = itr->GetTerm().data();
- std::vector<std::string_view> terms;
-
- bool is_ok = true;
- int current_pos = 0;
- while (is_ok) {
- // Alternate between using Advance and ResetToTermAfter.
- if (terms.size() % 2 == 0) {
- is_ok = itr->Advance();
- } else {
- // Calling ResetToTermStartingAfter with the current position should get
- // the very next term in the sequence.
- current_pos = itr->GetTerm().data() - text_begin;
- is_ok = itr->ResetToTermStartingAfter(current_pos).ok();
- }
- if (is_ok) {
- terms.push_back(itr->GetTerm());
- }
- }
- return terms;
-}
-
-// Returns a vector containing all terms retrieved by calling ResetBefore with
-// the current position, starting at the end of the text. This vector should be
-// in reverse order of GetAllTerms and missing the last term.
-std::vector<std::string_view> GetAllTermsResetBefore(
- LanguageSegmenter::Iterator* itr) {
- const char* text_begin = itr->GetTerm().data();
- int last_pos = 0;
- while (itr->Advance()) {
- last_pos = itr->GetTerm().data() - text_begin;
- }
- std::vector<std::string_view> terms;
- // Calling ResetToTermEndingBefore with the current position should get the
- // previous term in the sequence.
- for (int current_pos = last_pos;
- itr->ResetToTermEndingBefore(current_pos).ok();
- current_pos = itr->GetTerm().data() - text_begin) {
- terms.push_back(itr->GetTerm());
- }
- return terms;
-}
-
-class IosLanguageSegmenterAllLocalesTest
- : public testing::TestWithParam<const char*> {
- protected:
- static std::string GetLocale() { return GetParam(); }
- static language_segmenter_factory::SegmenterOptions GetOptions() {
- return language_segmenter_factory::SegmenterOptions(GetLocale());
- }
-};
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, EmptyText) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- EXPECT_THAT(language_segmenter->GetAllTerms(""), IsOkAndHolds(IsEmpty()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, SimpleText) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- EXPECT_THAT(language_segmenter->GetAllTerms("Hello World"),
- IsOkAndHolds(ElementsAre("Hello", " ", "World")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ASCII_Punctuation) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // ASCII punctuation marks are kept
- EXPECT_THAT(
- language_segmenter->GetAllTerms("Hello, World!!!"),
- IsOkAndHolds(ElementsAre("Hello", ",", " ", "World", "!", "!", "!")));
- EXPECT_THAT(language_segmenter->GetAllTerms("Open-source project"),
- IsOkAndHolds(ElementsAre("Open", "-", "source", " ", "project")));
- EXPECT_THAT(language_segmenter->GetAllTerms("100%"),
- IsOkAndHolds(ElementsAre("100", "%")));
- EXPECT_THAT(language_segmenter->GetAllTerms("A&B"),
- IsOkAndHolds(ElementsAre("A", "&", "B")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ASCII_SpecialCharacter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // ASCII special characters are kept
- EXPECT_THAT(language_segmenter->GetAllTerms("Pay $1000"),
- IsOkAndHolds(ElementsAre("Pay", " ", "$", "1000")));
- EXPECT_THAT(language_segmenter->GetAllTerms("A+B"),
- IsOkAndHolds(ElementsAre("A", "+", "B")));
- // 0x0009 is the unicode for tab (within ASCII range).
- std::string text_with_tab = absl_ports::StrCat(
- "Hello", UCharToString(0x0009), UCharToString(0x0009), "World");
- EXPECT_THAT(language_segmenter->GetAllTerms(text_with_tab),
- IsOkAndHolds(ElementsAre("Hello", UCharToString(0x0009),
- UCharToString(0x0009), "World")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Non_ASCII_Non_Alphabetic) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Full-width (non-ASCII) punctuation marks and special characters are left
- // out.
- ICING_ASSERT_OK_AND_ASSIGN(std::vector<std::string_view> terms,
- language_segmenter->GetAllTerms("。?·Hello!×"));
- EXPECT_THAT(terms, ElementsAre("Hello"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Acronym) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // LOCALE DEVIATION!! When the locale is Japanese, internal periods are
- // considered word breaks.
- std::vector<std::string> exp_terms;
- if (GetOptions().locale == ULOC_JAPAN) {
- exp_terms = {"U", ".", "S", ".", " ", "Bank"};
- } else {
- exp_terms = {"U.S", ".", " ", "Bank"};
- }
- EXPECT_THAT(language_segmenter->GetAllTerms("U.S. Bank"),
- IsOkAndHolds(ElementsAreArray(exp_terms)));
-
- // LOCALE DEVIATION!! When the locale is Japanese, internal periods are
- // considered word breaks.
- if (GetOptions().locale == ULOC_JAPAN) {
- exp_terms = {"I", ".", "B", ".", "M", "."};
- } else {
- exp_terms = {"I.B.M", "."};
- }
- EXPECT_THAT(language_segmenter->GetAllTerms("I.B.M."),
- IsOkAndHolds(ElementsAreArray(exp_terms)));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("I,B,M"),
- IsOkAndHolds(ElementsAre("I", ",", "B", ",", "M")));
- EXPECT_THAT(language_segmenter->GetAllTerms("I B M"),
- IsOkAndHolds(ElementsAre("I", " ", "B", " ", "M")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, WordConnector) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // According to unicode word break rules
- // WB6(https://unicode.org/reports/tr29/#WB6),
- // WB7(https://unicode.org/reports/tr29/#WB7), and a few others, some
- // punctuation characters are used as word connecters. That is, words don't
- // break before and after them. Here we just test some that we care about.
-
- // Word connecters
- EXPECT_THAT(language_segmenter->GetAllTerms("com.google.android"),
- IsOkAndHolds(ElementsAre("com.google.android")));
- // DIFFERENCE!! iOS doesn't agree that ':' is a word connector
- ICING_ASSERT_OK_AND_ASSIGN(
- std::vector<std::string_view> term,
- language_segmenter->GetAllTerms("com:google:android"));
- EXPECT_THAT(term, ElementsAre("com", ":", "google", ":", "android"));
- EXPECT_THAT(language_segmenter->GetAllTerms("com'google'android"),
- IsOkAndHolds(ElementsAre("com'google'android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com_google_android"),
- IsOkAndHolds(ElementsAre("com_google_android")));
-
- // Word connecters can be mixed
- // DIFFERENCE!! iOS doesn't agree that ':' is a word connector
- // TODO(b/157565185) resolve the handling of ':' as a connector.
- EXPECT_THAT(language_segmenter->GetAllTerms("com.google.android:icing"),
- IsOkAndHolds(ElementsAre("com.google.android", ":", "icing")));
-
- // Any heading and trailing characters are not connecters
- EXPECT_THAT(language_segmenter->GetAllTerms(".com.google.android."),
- IsOkAndHolds(ElementsAre(".", "com.google.android", ".")));
-
- // Not word connecters
- EXPECT_THAT(language_segmenter->GetAllTerms("com,google,android"),
- IsOkAndHolds(ElementsAre("com", ",", "google", ",", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com-google-android"),
- IsOkAndHolds(ElementsAre("com", "-", "google", "-", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com+google+android"),
- IsOkAndHolds(ElementsAre("com", "+", "google", "+", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com*google*android"),
- IsOkAndHolds(ElementsAre("com", "*", "google", "*", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com@google@android"),
- IsOkAndHolds(ElementsAre("com", "@", "google", "@", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com^google^android"),
- IsOkAndHolds(ElementsAre("com", "^", "google", "^", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com&google&android"),
- IsOkAndHolds(ElementsAre("com", "&", "google", "&", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com|google|android"),
- IsOkAndHolds(ElementsAre("com", "|", "google", "|", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com/google/android"),
- IsOkAndHolds(ElementsAre("com", "/", "google", "/", "android")));
- EXPECT_THAT(language_segmenter->GetAllTerms("com;google;android"),
- IsOkAndHolds(ElementsAre("com", ";", "google", ";", "android")));
- EXPECT_THAT(
- language_segmenter->GetAllTerms("com\"google\"android"),
- IsOkAndHolds(ElementsAre("com", "\"", "google", "\"", "android")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Apostrophes) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- EXPECT_THAT(language_segmenter->GetAllTerms("It's ok."),
- IsOkAndHolds(ElementsAre("It's", " ", "ok", ".")));
- EXPECT_THAT(language_segmenter->GetAllTerms("He'll be back."),
- IsOkAndHolds(ElementsAre("He'll", " ", "be", " ", "back", ".")));
- EXPECT_THAT(language_segmenter->GetAllTerms("'Hello 'World."),
- IsOkAndHolds(ElementsAre("'", "Hello", " ", "'", "World", ".")));
- EXPECT_THAT(language_segmenter->GetAllTerms("The dogs' bone"),
- IsOkAndHolds(ElementsAre("The", " ", "dogs", "'", " ", "bone")));
- // 0x2019 is the single right quote, should be treated the same as "'"
- std::string token_with_quote =
- absl_ports::StrCat("He", UCharToString(0x2019), "ll");
- std::string text_with_quote =
- absl_ports::StrCat(token_with_quote, " be back.");
- EXPECT_THAT(
- language_segmenter->GetAllTerms(text_with_quote),
- IsOkAndHolds(ElementsAre(token_with_quote, " ", "be", " ", "back", ".")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Parentheses) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("(Hello)"),
- IsOkAndHolds(ElementsAre("(", "Hello", ")")));
-
- EXPECT_THAT(language_segmenter->GetAllTerms(")Hello("),
- IsOkAndHolds(ElementsAre(")", "Hello", "(")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Quotes) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("\"Hello\""),
- IsOkAndHolds(ElementsAre("\"", "Hello", "\"")));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("'Hello'"),
- IsOkAndHolds(ElementsAre("'", "Hello", "'")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Alphanumeric) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
-
- // Alphanumeric terms are allowed
- EXPECT_THAT(language_segmenter->GetAllTerms("Se7en A4 3a"),
- IsOkAndHolds(ElementsAre("Se7en", " ", "A4", " ", "3a")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, Number) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
-
- // Alphanumeric terms are allowed
- EXPECT_THAT(
- language_segmenter->GetAllTerms("3.141592653589793238462643383279"),
- IsOkAndHolds(ElementsAre("3.141592653589793238462643383279")));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("3,456.789"),
- IsOkAndHolds(ElementsAre("3,456.789")));
-
- EXPECT_THAT(language_segmenter->GetAllTerms("-123"),
- IsOkAndHolds(ElementsAre("-", "123")));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ContinuousWhitespaces) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Multiple continuous whitespaces are treated as one.
- const int kNumSeparators = 256;
- std::string text_with_spaces =
- absl_ports::StrCat("Hello", std::string(kNumSeparators, ' '), "World");
- EXPECT_THAT(language_segmenter->GetAllTerms(text_with_spaces),
- IsOkAndHolds(ElementsAre("Hello", " ", "World")));
-
- // Multiple continuous whitespaces are treated as one. Whitespace at the
- // beginning of the text doesn't affect the results of GetTerm() after the
- // iterator is done.
- text_with_spaces = absl_ports::StrCat(std::string(kNumSeparators, ' '),
- "Hello", " ", "World");
- ICING_ASSERT_OK_AND_ASSIGN(auto itr,
- language_segmenter->Segment(text_with_spaces));
- std::vector<std::string_view> terms;
- while (itr->Advance()) {
- terms.push_back(itr->GetTerm());
- }
- EXPECT_THAT(terms, ElementsAre(" ", "Hello", " ", "World"));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, CJKT) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // CJKT (Chinese, Japanese, Khmer, Thai) are the 4 main languages that don't
- // have whitespaces as word delimiter.
-
- // Chinese
- // DIFFERENCE/LOCALE DEVIATION!! SIMPLISTIC_CHINESE agrees with ICU that
- // "每天" should be treated as a single token. All other locales split it into
- // two tokens.
- std::vector<std::string> exp_terms;
- if (GetOptions().locale == ULOC_SIMPLIFIED_CHINESE) {
- exp_terms = {"我", "每天", "走路", "去", "上班"};
- } else if (GetOptions().locale == ULOC_JAPAN) {
- // LOCALE DEVIATION!! JAPANESE groups "去上" and leaves "班" on its own.
- // All other locales which, like ICU, breaks the text into "去" and "上班".
- exp_terms = {"我", "每", "天", "走路", "去上", "班"};
- } else {
- exp_terms = {"我", "每", "天", "走路", "去", "上班"};
- }
- ICING_ASSERT_OK_AND_ASSIGN(
- std::vector<std::string_view> terms,
- language_segmenter->GetAllTerms("我每天走路去上班。"));
- EXPECT_THAT(terms, ElementsAreArray(exp_terms));
-
- // Japanese
- // DIFFERENCE!! Disagreement over how to segment "歩い" (iOS groups) and
- // "てい" (iOS splits). This difference persists even when locale is set to
- // JAPAN.
- if (GetOptions().locale == ULOC_SIMPLIFIED_CHINESE ||
- GetOptions().locale == ULOC_TRADITIONAL_CHINESE) {
- // LOCALE DEVIATION!! There is also disagreement when locale is CHINESE
- // about how to tokenize "毎日", "仕事", "歩い", which are all split, and
- // "てい" which is grouped.
- exp_terms = {"私", "は", "毎", "日", "仕", "事",
- "に", "歩", "い", "てい", "ます"};
- } else {
- exp_terms = {"私", "は", "毎日", "仕事", "に", "歩い", "て", "い", "ます"};
- }
- ICING_ASSERT_OK_AND_ASSIGN(
- terms, language_segmenter->GetAllTerms("私は毎日仕事に歩いています。"));
- EXPECT_THAT(terms, ElementsAreArray(exp_terms));
-
- // Khmer
- ICING_ASSERT_OK_AND_ASSIGN(
- terms, language_segmenter->GetAllTerms("ញុំដើរទៅធ្វើការរាល់ថ្ងៃ។"));
- EXPECT_THAT(terms, ElementsAre("ញុំ", "ដើរទៅ", "ធ្វើការ", "រាល់ថ្ងៃ"));
-
- // Thai
- // DIFFERENCE!! Disagreement over how to segment "ทุกวัน" (iOS groups).
- // This difference persists even when locale is set to THAI
- ICING_ASSERT_OK_AND_ASSIGN(
- terms, language_segmenter->GetAllTerms("ฉันเดินไปทำงานทุกวัน"));
- EXPECT_THAT(terms, ElementsAre("ฉัน", "เดิน", "ไป", "ทำงาน", "ทุกวัน"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, LatinLettersWithAccents) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- EXPECT_THAT(language_segmenter->GetAllTerms("āăąḃḅḇčćç"),
- IsOkAndHolds(ElementsAre("āăąḃḅḇčćç")));
-}
-
-// TODO(samzheng): test cases for more languages (e.g. top 20 in the world)
-TEST_P(IosLanguageSegmenterAllLocalesTest, WhitespaceSplitLanguages) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Turkish
- ICING_ASSERT_OK_AND_ASSIGN(std::vector<std::string_view> terms,
- language_segmenter->GetAllTerms("merhaba dünya"));
- EXPECT_THAT(terms, ElementsAre("merhaba", " ", "dünya"));
- // Korean
- ICING_ASSERT_OK_AND_ASSIGN(
- terms, language_segmenter->GetAllTerms("나는 매일 출근합니다."));
- EXPECT_THAT(terms, ElementsAre("나는", " ", "매일", " ", "출근합니다", "."));
-}
-
-// TODO(samzheng): more mixed languages test cases
-TEST_P(IosLanguageSegmenterAllLocalesTest, MixedLanguages) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // DIFFERENCE/LOCALE DEVIATION!! JAPANESE agrees with ICU that "你好" should
- // be treated as a single token. All other locales other than
- // SIMPLIFIED_CHINESE split it into two tokens.
- std::vector<std::string> exp_terms;
- if (GetOptions().locale == ULOC_JAPAN) {
- exp_terms = {"How", " ", "are", " ", "you", "你好",
- "吗", "お", "元気", "です", "か"};
- } else if (GetOptions().locale == ULOC_TRADITIONAL_CHINESE) {
- // LOCALE DEVIATION!! TRADITIONAL_CHINESE disagrees over tokenization of
- // "你好" and "元気", both of which it breaks up.
- exp_terms = {"How", " ", "are", " ", "you", "你", "好",
- "吗", "お", "元", "気", "です", "か"};
- } else if (GetOptions().locale == ULOC_SIMPLIFIED_CHINESE) {
- // LOCALE DEVIATION!! SIMPLIFIED_CHINESE disagrees over tokenization of
- // "元気", which it breaks up.
- exp_terms = {"How", " ", "are", " ", "you", "你好",
- "吗", "お", "元", "気", "です", "か"};
- } else {
- // LOCALE DEVIATION!! All other locales disagree over the tokenization of
- // "你好", which it breaks up.
- exp_terms = {"How", " ", "are", " ", "you", "你",
- "好", "吗", "お", "元気", "です", "か"};
- }
- ICING_ASSERT_OK_AND_ASSIGN(
- std::vector<std::string_view> terms,
- language_segmenter->GetAllTerms("How are you你好吗お元気ですか"));
- EXPECT_THAT(terms, ElementsAreArray(exp_terms));
-
- ICING_ASSERT_OK_AND_ASSIGN(
- terms, language_segmenter->GetAllTerms("나는 California에 산다"));
- EXPECT_THAT(terms, ElementsAre("나는", " ", "California", "에", " ", "산다"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, NotCopyStrings) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Validates that the input strings are not copied
- const std::string text = "Hello World";
- const char* word1_address = text.c_str();
- const char* word2_address = text.c_str() + 6;
- ICING_ASSERT_OK_AND_ASSIGN(std::vector<std::string_view> terms,
- language_segmenter->GetAllTerms(text));
- ASSERT_THAT(terms, ElementsAre("Hello", " ", "World"));
- const char* word1_result_address = terms.at(0).data();
- const char* word2_result_address = terms.at(2).data();
-
- // The underlying char* should be the same
- EXPECT_THAT(word1_address, Eq(word1_result_address));
- EXPECT_THAT(word2_address, Eq(word2_result_address));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, NewIteratorResetToStart) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- EXPECT_THAT(itr->ResetToStart(), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("How"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, IteratorOneAdvanceResetToStart) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- ASSERT_TRUE(itr->Advance()); // itr points to 'How'
- EXPECT_THAT(itr->ResetToStart(), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("How"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- IteratorMultipleAdvancesResetToStart) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- ASSERT_TRUE(itr->Advance());
- ASSERT_TRUE(itr->Advance());
- ASSERT_TRUE(itr->Advance());
- ASSERT_TRUE(itr->Advance()); // itr points to ' '
- EXPECT_THAT(itr->ResetToStart(), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("How"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, IteratorDoneResetToStart) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- while (itr->Advance()) {
- // Do nothing.
- }
- EXPECT_THAT(itr->ResetToStart(), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("How"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ResetToTermAfterOutOfBounds) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- auto position_or = itr->ResetToTermStartingAfter(7);
- EXPECT_THAT(position_or, IsOk());
- EXPECT_THAT(position_or.ValueOrDie(), Eq(8));
- ASSERT_THAT(itr->GetTerm(), Eq("you"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(-1),
- StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
- EXPECT_THAT(itr->GetTerm(), Eq("you"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(kText.length()),
- StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
- EXPECT_THAT(itr->GetTerm(), Eq("you"));
-}
-
-// Tests that ResetToTermAfter and Advance produce the same output. With the
-// exception of the first term which is inacessible via ResetToTermAfter,
-// the stream of terms produced by Advance calls should exacly match the
-// terms produced by ResetToTermAfter calls with the current position
-// provided as the argument.
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- MixedLanguagesResetToTermAfterEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are𡔖 you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetAfter(reset_to_term_itr.get());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- ThaiResetToTermAfterEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kThai = "ฉันเดินไปทำงานทุกวัน";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetAfter(reset_to_term_itr.get());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- KoreanResetToTermAfterEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kKorean = "나는 매일 출근합니다.";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetAfter(reset_to_term_itr.get());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-// Tests that ResetToTermAfter and Advance can be used in conjunction. Just as
-// ResetToTermAfter(current_position) can be used to simulate Advance, users
-// should be able to mix ResetToTermAfter(current_position) calls and Advance
-// calls to mimic calling Advance.
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- MixedLanguagesResetToTermAfterInteroperableWithAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are𡔖 you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_and_reset_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> advance_and_reset_terms =
- GetAllTermsAdvanceAndResetAfter(advance_and_reset_itr.get());
-
- EXPECT_THAT(advance_and_reset_terms,
- testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(advance_and_reset_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- ThaiResetToTermAfterInteroperableWithAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kThai = "ฉันเดินไปทำงานทุกวัน";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_and_reset_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> advance_and_reset_terms =
- GetAllTermsAdvanceAndResetAfter(advance_and_reset_itr.get());
-
- EXPECT_THAT(advance_and_reset_terms,
- testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(advance_and_reset_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- KoreanResetToTermAfterInteroperableWithAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kKorean = "나는 매일 출근합니다.";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_and_reset_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> advance_and_reset_terms =
- GetAllTermsAdvanceAndResetAfter(advance_and_reset_itr.get());
-
- EXPECT_THAT(advance_and_reset_terms,
- testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(advance_and_reset_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, MixedLanguagesResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment("How are you你好吗お元気ですか"));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^^ ^ ^ ^ ^
- // Bytes: 0 3 4 78 1114172023 29 35
- EXPECT_THAT(itr->ResetToTermStartingAfter(2), IsOkAndHolds(Eq(3)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- // DIFFERENCE/LOCALE DEVIATION!! JAPANESE and SIMPLIFIED_CHINESE agrees with
- // ICU that "你好" should be treated as a single token. All other locales
- // other than SIMPLIFIED_CHINESE split it into two tokens.
- std::string exp_token;
- if (GetLocale() == ULOC_JAPAN || GetLocale() == ULOC_SIMPLIFIED_CHINESE) {
- exp_token = "你好";
- } else {
- exp_token = "你";
- }
- EXPECT_THAT(itr->ResetToTermStartingAfter(10), IsOkAndHolds(Eq(11)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(7), IsOkAndHolds(Eq(8)));
- EXPECT_THAT(itr->GetTerm(), Eq("you"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(32), IsOkAndHolds(Eq(35)));
- EXPECT_THAT(itr->GetTerm(), Eq("か"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(14), IsOkAndHolds(Eq(17)));
- EXPECT_THAT(itr->GetTerm(), Eq("吗"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(0), IsOkAndHolds(Eq(3)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(35),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- ContinuousWhitespacesResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Multiple continuous whitespaces are treated as one.
- constexpr std::string_view kTextWithSpace = "Hello World";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kTextWithSpace));
-
- // String: "Hello World"
- // ^ ^ ^
- // Bytes: 0 5 15
- auto offset_or = itr->ResetToTermStartingAfter(0);
- EXPECT_THAT(offset_or.status(), IsOk());
- EXPECT_THAT(offset_or.ValueOrDie(), Eq(5));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(2), IsOkAndHolds(Eq(5)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(10), IsOkAndHolds(Eq(15)));
- EXPECT_THAT(itr->GetTerm(), Eq("World"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(5), IsOkAndHolds(Eq(15)));
- EXPECT_THAT(itr->GetTerm(), Eq("World"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(15),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(17),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(19),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ChineseResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // CJKT (Chinese, Japanese, Khmer, Thai) are the 4 main languages that
- // don't have whitespaces as word delimiter. Chinese
- constexpr std::string_view kChinese = "我每天走路去上班。";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kChinese));
- // String: "我每天走路去上班。"
- // ^ ^^ ^ ^^
- // Bytes: 0 3 6 9 15 18
- std::string exp_token;
- // DIFFERENCE/LOCALE DEVIATION!! SIMPLISTIC_CHINESE agrees with ICU that
- // "每天" should be treated as a single token. All other locales split it into
- // two tokens.
- if (GetLocale() == ULOC_SIMPLIFIED_CHINESE) {
- exp_token = "每天";
- } else {
- exp_token = "每";
- }
- EXPECT_THAT(itr->ResetToTermStartingAfter(0), IsOkAndHolds(Eq(3)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(7), IsOkAndHolds(Eq(9)));
- EXPECT_THAT(itr->GetTerm(), Eq("走路"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(21),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, JapaneseResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Japanese
- constexpr std::string_view kJapanese = "私は毎日仕事に歩いています。";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kJapanese));
- // String: "私は毎日仕事に歩いています。"
- // ^ ^ ^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 6 12 18212427 33
- EXPECT_THAT(itr->ResetToTermStartingAfter(0), IsOkAndHolds(Eq(3)));
- EXPECT_THAT(itr->GetTerm(), Eq("は"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(33),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- // LOCALE DEVIATION!! There is disagreement when locale is CHINESE about how
- // to tokenize "毎日", "仕事", "歩い", which are all split, and "てい" which
- // is grouped.
- std::string exp_term;
- int exp_offset;
- if (GetLocale() == ULOC_SIMPLIFIED_CHINESE ||
- GetLocale() == ULOC_TRADITIONAL_CHINESE) {
- // Since "毎日" is broken up when the locale is CHINESE, ResetAfter(7) will
- // point to "日" instead of the next segment ("仕事") like other locales.
- exp_term = "日";
- exp_offset = 9;
- } else {
- exp_term = "仕事";
- exp_offset = 12;
- }
- EXPECT_THAT(itr->ResetToTermStartingAfter(7), IsOkAndHolds(Eq(exp_offset)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_term));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, KhmerResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kKhmer = "ញុំដើរទៅធ្វើការរាល់ថ្ងៃ។";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kKhmer));
- // String: "ញុំដើរទៅធ្វើការរាល់ថ្ងៃ។"
- // ^ ^ ^ ^
- // Bytes: 0 9 24 45
- EXPECT_THAT(itr->ResetToTermStartingAfter(0), IsOkAndHolds(Eq(9)));
- EXPECT_THAT(itr->GetTerm(), Eq("ដើរទៅ"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(47),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(14), IsOkAndHolds(Eq(24)));
- EXPECT_THAT(itr->GetTerm(), Eq("ធ្វើការ"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ThaiResetToTermAfter) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Thai
- constexpr std::string_view kThai = "ฉันเดินไปทำงานทุกวัน";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kThai));
- // String: "ฉันเดินไปทำงานทุกวัน"
- // ^ ^ ^ ^ ^
- // Bytes: 0 9 21 27 42
- EXPECT_THAT(itr->ResetToTermStartingAfter(0), IsOkAndHolds(Eq(9)));
- EXPECT_THAT(itr->GetTerm(), Eq("เดิน"));
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(51),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermStartingAfter(13), IsOkAndHolds(Eq(21)));
- EXPECT_THAT(itr->GetTerm(), Eq("ไป"));
-
- // DIFFERENCE!! Disagreement over how to segment "ทุกวัน" (iOS groups).
- // This difference persists even when locale is set to THAI
- EXPECT_THAT(itr->ResetToTermStartingAfter(34), IsOkAndHolds(Eq(42)));
- EXPECT_THAT(itr->GetTerm(), Eq("ทุกวัน"));
-}
-TEST_P(IosLanguageSegmenterAllLocalesTest, ResetToTermBeforeOutOfBounds) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- segmenter->Segment(kText));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- ASSERT_THAT(itr->ResetToTermEndingBefore(7), IsOkAndHolds(Eq(4)));
- ASSERT_THAT(itr->GetTerm(), Eq("are"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(-1),
- StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
- EXPECT_THAT(itr->GetTerm(), Eq("are"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(kText.length()),
- StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
- EXPECT_THAT(itr->GetTerm(), Eq("are"));
-}
-
-// Tests that ResetToTermBefore and Advance produce the same output. With the
-// exception of the last term which is inacessible via ResetToTermBefore,
-// the stream of terms produced by Advance calls should exacly match the
-// terms produced by ResetToTermBefore calls with the current position
-// provided as the argument (after their order has been reversed).
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- MixedLanguagesResetToTermBeforeEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kText = "How are𡔖 you你好吗お元気ですか";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
- // Can't produce the last term via calls to ResetToTermBefore. So skip
- // past that one.
- auto itr = advance_terms.begin();
- std::advance(itr, advance_terms.size() - 1);
- advance_terms.erase(itr);
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kText));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetBefore(reset_to_term_itr.get());
- std::reverse(reset_terms.begin(), reset_terms.end());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), IsEmpty());
- EXPECT_THAT(advance_itr->GetTerm(), IsEmpty());
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- ThaiResetToTermBeforeEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kThai = "ฉันเดินไปทำงานทุกวัน";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
- // Can't produce the last term via calls to ResetToTermBefore. So skip
- // past that one.
- auto itr = advance_terms.begin();
- std::advance(itr, advance_terms.size() - 1);
- advance_terms.erase(itr);
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kThai));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetBefore(reset_to_term_itr.get());
- std::reverse(reset_terms.begin(), reset_terms.end());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- KoreanResetToTermBeforeEquivalentToAdvance) {
- ICING_ASSERT_OK_AND_ASSIGN(auto segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kKorean = "나는 매일 출근합니다.";
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> advance_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> advance_terms =
- GetAllTermsAdvance(advance_itr.get());
- // Can't produce the last term via calls to ResetToTermBefore. So skip
- // past that one.
- auto itr = advance_terms.begin();
- std::advance(itr, advance_terms.size() - 1);
- advance_terms.erase(itr);
-
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> reset_to_term_itr,
- segmenter->Segment(kKorean));
- std::vector<std::string_view> reset_terms =
- GetAllTermsResetBefore(reset_to_term_itr.get());
- std::reverse(reset_terms.begin(), reset_terms.end());
-
- EXPECT_THAT(reset_terms, testing::ElementsAreArray(advance_terms));
- EXPECT_THAT(reset_to_term_itr->GetTerm(), Eq(advance_itr->GetTerm()));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, MixedLanguagesResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment("How are you你好吗お元気ですか"));
-
- // String: "How are you你好吗お元気ですか"
- // ^ ^^ ^^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 4 7 8 11 172023 29 35
- EXPECT_THAT(itr->ResetToTermEndingBefore(2),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(10), IsOkAndHolds(Eq(7)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(7), IsOkAndHolds(Eq(4)));
- EXPECT_THAT(itr->GetTerm(), Eq("are"));
-
- std::string exp_token;
- int exp_offset;
- if (GetOptions().locale == ULOC_TRADITIONAL_CHINESE ||
- GetOptions().locale == ULOC_SIMPLIFIED_CHINESE) {
- // LOCALE DEVIATION!! SIMPLIFIED_CHINESE disagrees over tokenization of
- // "元気", which it breaks up.
- exp_offset = 26;
- exp_token = "気";
- } else {
- exp_offset = 23;
- exp_token = "元気";
- }
- EXPECT_THAT(itr->ResetToTermEndingBefore(32), IsOkAndHolds(Eq(exp_offset)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-
- // DIFFERENCE/LOCALE DEVIATION!! JAPANESE and SIMPLIFIED_CHINESE agrees with
- // ICU that "你好" should be treated as a single token. All other locales
- // split it into two tokens.
- if (GetLocale() == ULOC_JAPAN || GetLocale() == ULOC_SIMPLIFIED_CHINESE) {
- exp_offset = 8;
- exp_token = "you";
- } else {
- exp_offset = 11;
- exp_token = "你";
- }
- EXPECT_THAT(itr->ResetToTermEndingBefore(14), IsOkAndHolds(Eq(exp_offset)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(35), IsOkAndHolds(Eq(29)));
- EXPECT_THAT(itr->GetTerm(), Eq("です"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest,
- ContinuousWhitespacesResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Multiple continuous whitespaces are treated as one.
- constexpr std::string_view kTextWithSpace = "Hello World";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kTextWithSpace));
-
- // String: "Hello World"
- // ^ ^ ^
- // Bytes: 0 5 15
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(2),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(10), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("Hello"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(5), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("Hello"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(15), IsOkAndHolds(Eq(5)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(17), IsOkAndHolds(Eq(5)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(19), IsOkAndHolds(Eq(5)));
- EXPECT_THAT(itr->GetTerm(), Eq(" "));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ChineseResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // CJKT (Chinese, Japanese, Khmer, Thai) are the 4 main languages that
- // don't have whitespaces as word delimiter. Chinese
- constexpr std::string_view kChinese = "我每天走路去上班。";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kChinese));
- // String: "我每天走路去上班。"
- // ^ ^^ ^ ^ ^
- // Bytes: 0 3 6 9 15 18
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- std::string exp_token;
- int exp_offset;
- // DIFFERENCE/LOCALE DEVIATION!! SIMPLISTIC_CHINESE agrees with ICU that
- // "每天" should be treated as a single token. All other locales split it into
- // two tokens.
- if (GetLocale() == ULOC_SIMPLIFIED_CHINESE) {
- exp_offset = 0;
- exp_token = "我";
- } else {
- exp_offset = 3;
- exp_token = "每";
- }
- EXPECT_THAT(itr->ResetToTermEndingBefore(7), IsOkAndHolds(Eq(exp_offset)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-
- if (GetOptions().locale == ULOC_JAPAN) {
- // LOCALE DEVIATION!! JAPANESE groups "去上" and leaves "班" on its own.
- // All other locales which, like ICU, breaks the text into "去" and "上班".
- exp_offset = 9;
- exp_token = "走路";
- } else {
- exp_offset = 15;
- exp_token = "去";
- }
- EXPECT_THAT(itr->ResetToTermEndingBefore(19), IsOkAndHolds(Eq(exp_offset)));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_token));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, JapaneseResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Japanese
- constexpr std::string_view kJapanese = "私は毎日仕事に歩いています。";
- // String: "私は毎日仕事に歩いています。"
- // ^ ^ ^ ^ ^ ^ ^ ^ ^
- // Bytes: 0 3 6 12 18212427 33
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kJapanese));
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- // LOCALE DEVIATION!! There is disagreement when locale is CHINESE about how
- // to tokenize "毎日", "仕事", "歩い", which are all split, and "てい" which
- // is grouped.
- std::string exp_term;
- int exp_offset;
- if (GetLocale() == ULOC_SIMPLIFIED_CHINESE ||
- GetLocale() == ULOC_TRADITIONAL_CHINESE) {
- // TODO(b/157565185) For some reason, CFStringTokenizerGoToTokenAtIndex
- // believes that "いています" is one token when locale is
- // SIMPLIFIED/TRADITIONAL CHINESE, but CFStringTokenizerAdvanceToNextToken
- // thinks that it is three: "い" "てい", "ます". Other locales and ICU agree
- // that that segment should be "歩い", "て", "い", "ます".
- // This is the only case where CFStringTokenizerGoToTokenAtIndex and
- // CFStringTokenizerAdvanceToNextToken disagree. Find a way around this
- // (such as rewinding past the desired segment and then advancing to it) if
- // this is still an issue after adding language detection.
- exp_term = "歩";
- exp_offset = 21;
- } else {
- // Since "てい" is broken up when the locale is not CHINESE,
- // ResetBefore(33) will point to "い" at offset 30.
- exp_term = "い";
- exp_offset = 30;
- }
- auto offset_or = itr->ResetToTermEndingBefore(33);
- EXPECT_THAT(offset_or, IsOk());
- EXPECT_THAT(offset_or.ValueOrDie(), Eq(exp_offset));
- EXPECT_THAT(itr->GetTerm(), Eq(exp_term));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(7), IsOkAndHolds(Eq(3)));
- EXPECT_THAT(itr->GetTerm(), Eq("は"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, KhmerResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- constexpr std::string_view kKhmer = "ញុំដើរទៅធ្វើការរាល់ថ្ងៃ។";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kKhmer));
- // String: "ញុំដើរទៅធ្វើការរាល់ថ្ងៃ។"
- // ^ ^ ^ ^
- // Bytes: 0 9 24 45
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(47), IsOkAndHolds(Eq(24)));
- EXPECT_THAT(itr->GetTerm(), Eq("ធ្វើការ"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(14), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("ញុំ"));
-}
-
-TEST_P(IosLanguageSegmenterAllLocalesTest, ThaiResetToTermBefore) {
- ICING_ASSERT_OK_AND_ASSIGN(auto language_segmenter,
- language_segmenter_factory::Create(GetOptions()));
- // Thai
- constexpr std::string_view kThai = "ฉันเดินไปทำงานทุกวัน";
- ICING_ASSERT_OK_AND_ASSIGN(std::unique_ptr<LanguageSegmenter::Iterator> itr,
- language_segmenter->Segment(kThai));
- // String: "ฉันเดินไปทำงานทุกวัน"
- // ^ ^ ^ ^ ^
- // Bytes: 0 9 21 27 42
- EXPECT_THAT(itr->ResetToTermEndingBefore(0),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
- EXPECT_THAT(itr->GetTerm(), IsEmpty());
-
- // DIFFERENCE!! Disagreement over how to segment "ทุกวัน" (iOS groups).
- // This difference persists even when locale is set to THAI
- EXPECT_THAT(itr->ResetToTermEndingBefore(51), IsOkAndHolds(Eq(27)));
- EXPECT_THAT(itr->GetTerm(), Eq("ทำงาน"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(13), IsOkAndHolds(Eq(0)));
- EXPECT_THAT(itr->GetTerm(), Eq("ฉัน"));
-
- EXPECT_THAT(itr->ResetToTermEndingBefore(34), IsOkAndHolds(Eq(21)));
- EXPECT_THAT(itr->GetTerm(), Eq("ไป"));
-}
-
-INSTANTIATE_TEST_SUITE_P(
- LocaleName, IosLanguageSegmenterAllLocalesTest,
- testing::Values(ULOC_US, ULOC_UK, ULOC_CANADA, ULOC_CANADA_FRENCH,
- ULOC_FRANCE, ULOC_GERMANY, ULOC_ITALY, ULOC_JAPAN,
- ULOC_KOREA,
- ULOC_SIMPLIFIED_CHINESE,
- ULOC_TRADITIONAL_CHINESE,
- "es_ES", // Spanish
- "hi_IN", // Hindi
- "th_TH", // Thai
- "lo_LA", // Lao
- "km_KH", // Khmer
- "ar_DZ", // Arabic
- "ru_RU", // Russian
- "pt_PT", // Portuguese
- "en_US_POSIX" // American English (Computer)
- "wrong_locale" // Will fall back to ICU default locale
- "" // Will fall back to ICU default locale
- ));
-
-} // namespace
-} // namespace lib
-} // namespace icing