aboutsummaryrefslogtreecommitdiff
path: root/icing/join
diff options
context:
space:
mode:
Diffstat (limited to 'icing/join')
-rw-r--r--icing/join/document-id-to-join-info.h67
-rw-r--r--icing/join/join-processor.cc128
-rw-r--r--icing/join/join-processor_test.cc399
-rw-r--r--icing/join/posting-list-join-data-accessor.h211
-rw-r--r--icing/join/posting-list-join-data-accessor_test.cc435
-rw-r--r--icing/join/posting-list-join-data-serializer.h803
-rw-r--r--icing/join/posting-list-join-data-serializer_test.cc653
-rw-r--r--icing/join/qualified-id-join-index-impl-v1.cc (renamed from icing/join/qualified-id-join-index.cc)81
-rw-r--r--icing/join/qualified-id-join-index-impl-v1.h327
-rw-r--r--icing/join/qualified-id-join-index-impl-v1_test.cc (renamed from icing/join/qualified-id-join-index_test.cc)421
-rw-r--r--icing/join/qualified-id-join-index-impl-v2.cc681
-rw-r--r--icing/join/qualified-id-join-index-impl-v2.h369
-rw-r--r--icing/join/qualified-id-join-index-impl-v2_test.cc1414
-rw-r--r--icing/join/qualified-id-join-index.h275
-rw-r--r--icing/join/qualified-id-join-indexing-handler-v1_test.cc558
-rw-r--r--icing/join/qualified-id-join-indexing-handler.cc117
-rw-r--r--icing/join/qualified-id-join-indexing-handler.h12
-rw-r--r--icing/join/qualified-id-join-indexing-handler_test.cc591
18 files changed, 6875 insertions, 667 deletions
diff --git a/icing/join/document-id-to-join-info.h b/icing/join/document-id-to-join-info.h
new file mode 100644
index 0000000..dee4885
--- /dev/null
+++ b/icing/join/document-id-to-join-info.h
@@ -0,0 +1,67 @@
+// Copyright (C) 2023 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_JOIN_DOCUMENT_ID_TO_JOIN_INFO_H_
+#define ICING_JOIN_DOCUMENT_ID_TO_JOIN_INFO_H_
+
+#include <utility>
+
+#include "icing/store/document-id.h"
+
+namespace icing {
+namespace lib {
+
+// DocumentIdToJoinInfo is composed of document_id and its join info.
+// - QualifiedId join: join info is the referenced document's namespace_id +
+// fingerprint(uri).
+// - String join: join info is the term id.
+// - Integer join: join info is the integer.
+//
+// DocumentIdToJoinInfo will be stored in posting list.
+template <typename JoinInfoType>
+class DocumentIdToJoinInfo {
+ public:
+ static DocumentIdToJoinInfo<JoinInfoType> GetInvalid() {
+ return DocumentIdToJoinInfo<JoinInfoType>(kInvalidDocumentId,
+ JoinInfoType());
+ }
+
+ explicit DocumentIdToJoinInfo(DocumentId document_id, JoinInfoType join_info)
+ : document_id_(document_id), join_info_(std::move(join_info)) {}
+
+ DocumentId document_id() const { return document_id_; }
+ const JoinInfoType& join_info() const { return join_info_; }
+
+ bool is_valid() const { return IsDocumentIdValid(document_id_); }
+
+ bool operator<(const DocumentIdToJoinInfo<JoinInfoType>& other) const {
+ if (document_id_ != other.document_id_) {
+ return document_id_ < other.document_id_;
+ }
+ return join_info_ < other.join_info_;
+ }
+
+ bool operator==(const DocumentIdToJoinInfo<JoinInfoType>& other) const {
+ return document_id_ == other.document_id_ && join_info_ == other.join_info_;
+ }
+
+ private:
+ DocumentId document_id_;
+ JoinInfoType join_info_;
+} __attribute__((packed));
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_JOIN_DOCUMENT_ID_TO_JOIN_INFO_H_
diff --git a/icing/join/join-processor.cc b/icing/join/join-processor.cc
index e27b1ea..1b7ca0d 100644
--- a/icing/join/join-processor.cc
+++ b/icing/join/join-processor.cc
@@ -29,6 +29,7 @@
#include "icing/join/aggregation-scorer.h"
#include "icing/join/doc-join-info.h"
#include "icing/join/join-children-fetcher.h"
+#include "icing/join/qualified-id-join-index.h"
#include "icing/join/qualified-id.h"
#include "icing/proto/schema.pb.h"
#include "icing/proto/scoring.pb.h"
@@ -37,6 +38,7 @@
#include "icing/scoring/scored-document-hit.h"
#include "icing/store/document-filter-data.h"
#include "icing/store/document-id.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
#include "icing/util/status-macros.h"
namespace icing {
@@ -53,17 +55,121 @@ JoinProcessor::GetChildrenFetcher(
"Parent property expression must be ", kQualifiedIdExpr));
}
- std::sort(
- child_scored_document_hits.begin(), child_scored_document_hits.end(),
- ScoredDocumentHitComparator(
- /*is_descending=*/join_spec.nested_spec().scoring_spec().order_by() ==
- ScoringSpecProto::Order::DESC));
-
- // TODO(b/256022027):
- // - Optimization
- // - Cache property to speed up property retrieval.
- // - If there is no cache, then we still have the flexibility to fetch it
- // from actual docs via DocumentStore.
+ ScoredDocumentHitComparator score_comparator(
+ /*is_descending=*/join_spec.nested_spec().scoring_spec().order_by() ==
+ ScoringSpecProto::Order::DESC);
+
+ if (qualified_id_join_index_->is_v2()) {
+ // v2
+ // Step 1a: sort child ScoredDocumentHits in document id descending order.
+ std::sort(child_scored_document_hits.begin(),
+ child_scored_document_hits.end(),
+ [](const ScoredDocumentHit& lhs, const ScoredDocumentHit& rhs) {
+ return lhs.document_id() > rhs.document_id();
+ });
+
+ // Step 1b: group all child ScoredDocumentHits by the document's
+ // schema_type_id.
+ std::unordered_map<SchemaTypeId, std::vector<ScoredDocumentHit>>
+ schema_to_child_scored_doc_hits_map;
+ for (const ScoredDocumentHit& child_scored_document_hit :
+ child_scored_document_hits) {
+ std::optional<DocumentFilterData> child_doc_filter_data =
+ doc_store_->GetAliveDocumentFilterData(
+ child_scored_document_hit.document_id(), current_time_ms_);
+ if (!child_doc_filter_data) {
+ continue;
+ }
+
+ schema_to_child_scored_doc_hits_map[child_doc_filter_data
+ ->schema_type_id()]
+ .push_back(child_scored_document_hit);
+ }
+
+ // Step 1c: for each schema_type_id, lookup QualifiedIdJoinIndexImplV2 to
+ // fetch all child join data from posting list(s). Convert all
+ // child join data to referenced parent document ids and bucketize
+ // child ScoredDocumentHits by it.
+ std::unordered_map<DocumentId, std::vector<ScoredDocumentHit>>
+ parent_to_child_docs_map;
+ for (auto& [schema_type_id, grouped_child_scored_doc_hits] :
+ schema_to_child_scored_doc_hits_map) {
+ // Get joinable_property_id of this schema.
+ ICING_ASSIGN_OR_RETURN(
+ const JoinablePropertyMetadata* metadata,
+ schema_store_->GetJoinablePropertyMetadata(
+ schema_type_id, join_spec.child_property_expression()));
+ if (metadata == nullptr ||
+ metadata->value_type != JoinableConfig::ValueType::QUALIFIED_ID) {
+ // Currently we only support qualified id, so skip other types.
+ continue;
+ }
+
+ // Lookup QualifiedIdJoinIndexImplV2.
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<QualifiedIdJoinIndex::JoinDataIteratorBase>
+ join_index_iter,
+ qualified_id_join_index_->GetIterator(
+ schema_type_id, /*joinable_property_id=*/metadata->id));
+
+ // - Join index contains all join data of schema_type_id and
+ // join_index_iter will return all of them in (child) document id
+ // descending order.
+ // - But we only need join data of child document ids which appear in
+ // grouped_child_scored_doc_hits. Also grouped_child_scored_doc_hits
+ // contain ScoredDocumentHits in (child) document id descending order.
+ // - Therefore, we advance 2 iterators to intersect them and get desired
+ // join data.
+ auto child_scored_doc_hits_iter = grouped_child_scored_doc_hits.cbegin();
+ while (join_index_iter->Advance().ok() &&
+ child_scored_doc_hits_iter !=
+ grouped_child_scored_doc_hits.cend()) {
+ // Advance child_scored_doc_hits_iter until it points to a
+ // ScoredDocumentHit with document id <= the one pointed by
+ // join_index_iter.
+ while (child_scored_doc_hits_iter !=
+ grouped_child_scored_doc_hits.cend() &&
+ child_scored_doc_hits_iter->document_id() >
+ join_index_iter->GetCurrent().document_id()) {
+ ++child_scored_doc_hits_iter;
+ }
+
+ if (child_scored_doc_hits_iter !=
+ grouped_child_scored_doc_hits.cend() &&
+ child_scored_doc_hits_iter->document_id() ==
+ join_index_iter->GetCurrent().document_id()) {
+ // We get a join data whose child document id exists in both join
+ // index and grouped_child_scored_doc_hits. Convert its join info to
+ // referenced parent document ids and bucketize ScoredDocumentHits by
+ // it (putting into parent_to_child_docs_map).
+ const NamespaceFingerprintIdentifier& ref_ns_id =
+ join_index_iter->GetCurrent().join_info();
+ libtextclassifier3::StatusOr<DocumentId> ref_parent_doc_id_or =
+ doc_store_->GetDocumentId(ref_ns_id);
+ if (ref_parent_doc_id_or.ok()) {
+ parent_to_child_docs_map[std::move(ref_parent_doc_id_or)
+ .ValueOrDie()]
+ .push_back(*child_scored_doc_hits_iter);
+ }
+ }
+ }
+ }
+
+ // Step 1d: finally, sort each parent's joined child ScoredDocumentHits by
+ // score.
+ for (auto& [parent_doc_id, bucketized_child_scored_hits] :
+ parent_to_child_docs_map) {
+ std::sort(bucketized_child_scored_hits.begin(),
+ bucketized_child_scored_hits.end(), score_comparator);
+ }
+
+ return JoinChildrenFetcher(join_spec, std::move(parent_to_child_docs_map));
+ }
+
+ // v1
+ // TODO(b/275121148): deprecate this part after rollout v2.
+ std::sort(child_scored_document_hits.begin(),
+ child_scored_document_hits.end(), score_comparator);
// Step 1: group child documents by parent documentId. Currently we only
// support QualifiedId joining, so fetch the qualified id content of
diff --git a/icing/join/join-processor_test.cc b/icing/join/join-processor_test.cc
index f503442..a40d934 100644
--- a/icing/join/join-processor_test.cc
+++ b/icing/join/join-processor_test.cc
@@ -22,9 +22,13 @@
#include "icing/text_classifier/lib3/utils/base/statusor.h"
#include "gmock/gmock.h"
#include "gtest/gtest.h"
+#include "icing/absl_ports/canonical_errors.h"
#include "icing/document-builder.h"
#include "icing/file/filesystem.h"
#include "icing/file/portable-file-backed-proto-log.h"
+#include "icing/join/join-children-fetcher.h"
+#include "icing/join/qualified-id-join-index-impl-v1.h"
+#include "icing/join/qualified-id-join-index-impl-v2.h"
#include "icing/join/qualified-id-join-index.h"
#include "icing/join/qualified-id-join-indexing-handler.h"
#include "icing/portable/platform.h"
@@ -58,6 +62,9 @@ namespace {
using ::testing::ElementsAre;
using ::testing::IsTrue;
+// TODO(b/275121148): remove template after deprecating
+// QualifiedIdJoinIndexImplV1.
+template <typename T>
class JoinProcessorTest : public ::testing::Test {
protected:
void SetUp() override {
@@ -108,6 +115,25 @@ class JoinProcessorTest : public ::testing::Test {
.SetDataTypeJoinableString(
JOINABLE_VALUE_TYPE_QUALIFIED_ID)
.SetCardinality(CARDINALITY_OPTIONAL)))
+ .AddType(
+ SchemaTypeConfigBuilder()
+ .SetType("Message")
+ .AddProperty(PropertyConfigBuilder()
+ .SetName("content")
+ .SetDataTypeString(TERM_MATCH_EXACT,
+ TOKENIZER_PLAIN)
+ .SetCardinality(CARDINALITY_OPTIONAL))
+ .AddProperty(PropertyConfigBuilder()
+ .SetName("sender")
+ .SetDataTypeJoinableString(
+ JOINABLE_VALUE_TYPE_QUALIFIED_ID)
+ .SetCardinality(CARDINALITY_OPTIONAL))
+ .AddProperty(PropertyConfigBuilder()
+ .SetName("receiver")
+ .SetDataTypeJoinableString(
+ JOINABLE_VALUE_TYPE_QUALIFIED_ID)
+ .SetCardinality(CARDINALITY_OPTIONAL)))
+
.Build();
ASSERT_THAT(schema_store_->SetSchema(
schema, /*ignore_errors_and_delete_documents=*/false,
@@ -121,18 +147,15 @@ class JoinProcessorTest : public ::testing::Test {
DocumentStore::Create(
&filesystem_, doc_store_dir_, &fake_clock_, schema_store_.get(),
/*force_recovery_and_revalidate_documents=*/false,
- /*namespace_id_fingerprint=*/false, /*pre_mapping_fbv=*/false,
+ /*namespace_id_fingerprint=*/true, /*pre_mapping_fbv=*/false,
/*use_persistent_hash_map=*/false,
PortableFileBackedProtoLog<
DocumentWrapper>::kDeflateCompressionLevel,
/*initialize_stats=*/nullptr));
doc_store_ = std::move(create_result.document_store);
- ICING_ASSERT_OK_AND_ASSIGN(
- qualified_id_join_index_,
- QualifiedIdJoinIndex::Create(filesystem_, qualified_id_join_index_dir_,
- /*pre_mapping_fbv=*/false,
- /*use_persistent_hash_map=*/false));
+ ICING_ASSERT_OK_AND_ASSIGN(qualified_id_join_index_,
+ CreateQualifiedIdJoinIndex<T>());
}
void TearDown() override {
@@ -143,6 +166,28 @@ class JoinProcessorTest : public ::testing::Test {
filesystem_.DeleteDirectoryRecursively(test_dir_.c_str());
}
+ template <typename UnknownJoinIndexType>
+ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
+ CreateQualifiedIdJoinIndex() {
+ return absl_ports::InvalidArgumentError("Unknown type");
+ }
+
+ template <>
+ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
+ CreateQualifiedIdJoinIndex<QualifiedIdJoinIndexImplV1>() {
+ return QualifiedIdJoinIndexImplV1::Create(
+ filesystem_, qualified_id_join_index_dir_, /*pre_mapping_fbv=*/false,
+ /*use_persistent_hash_map=*/false);
+ }
+
+ template <>
+ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
+ CreateQualifiedIdJoinIndex<QualifiedIdJoinIndexImplV2>() {
+ return QualifiedIdJoinIndexImplV2::Create(filesystem_,
+ qualified_id_join_index_dir_,
+ /*pre_mapping_fbv=*/false);
+ }
+
libtextclassifier3::StatusOr<DocumentId> PutAndIndexDocument(
const DocumentProto& document) {
ICING_ASSIGN_OR_RETURN(DocumentId document_id, doc_store_->Put(document));
@@ -153,7 +198,7 @@ class JoinProcessorTest : public ::testing::Test {
ICING_ASSIGN_OR_RETURN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
ICING_RETURN_IF_ERROR(handler->Handle(tokenized_document, document_id,
/*recovery_mode=*/false,
@@ -163,8 +208,8 @@ class JoinProcessorTest : public ::testing::Test {
libtextclassifier3::StatusOr<std::vector<JoinedScoredDocumentHit>> Join(
const JoinSpecProto& join_spec,
- std::vector<ScoredDocumentHit>&& parent_scored_document_hits,
- std::vector<ScoredDocumentHit>&& child_scored_document_hits) {
+ std::vector<ScoredDocumentHit> parent_scored_document_hits,
+ std::vector<ScoredDocumentHit> child_scored_document_hits) {
JoinProcessor join_processor(
doc_store_.get(), schema_store_.get(), qualified_id_join_index_.get(),
/*current_time_ms=*/fake_clock_.GetSystemTimeMilliseconds());
@@ -191,7 +236,11 @@ class JoinProcessorTest : public ::testing::Test {
FakeClock fake_clock_;
};
-TEST_F(JoinProcessorTest, JoinByQualifiedId) {
+using TestTypes =
+ ::testing::Types<QualifiedIdJoinIndexImplV1, QualifiedIdJoinIndexImplV2>;
+TYPED_TEST_SUITE(JoinProcessorTest, TestTypes);
+
+TYPED_TEST(JoinProcessorTest, JoinByQualifiedId_allDocuments) {
DocumentProto person1 = DocumentBuilder()
.SetKey("pkg$db/namespace", "person1")
.SetSchema("Person")
@@ -227,15 +276,15 @@ TEST_F(JoinProcessorTest, JoinByQualifiedId) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(person1));
+ this->PutAndIndexDocument(person1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
- PutAndIndexDocument(person2));
+ this->PutAndIndexDocument(person2));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id4,
- PutAndIndexDocument(email2));
+ this->PutAndIndexDocument(email2));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id5,
- PutAndIndexDocument(email3));
+ this->PutAndIndexDocument(email3));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -267,8 +316,8 @@ TEST_F(JoinProcessorTest, JoinByQualifiedId) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
EXPECT_THAT(
joined_result_document_hits,
ElementsAre(EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
@@ -282,7 +331,112 @@ TEST_F(JoinProcessorTest, JoinByQualifiedId) {
{scored_doc_hit5, scored_doc_hit3}))));
}
-TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithoutJoiningProperty) {
+TYPED_TEST(JoinProcessorTest, JoinByQualifiedId_partialDocuments) {
+ DocumentProto person1 = DocumentBuilder()
+ .SetKey("pkg$db/namespace", "person1")
+ .SetSchema("Person")
+ .AddStringProperty("Name", "Alice")
+ .Build();
+ DocumentProto person2 = DocumentBuilder()
+ .SetKey("pkg$db/namespace", "person2")
+ .SetSchema("Person")
+ .AddStringProperty("Name", "Bob")
+ .Build();
+ DocumentProto person3 = DocumentBuilder()
+ .SetKey("pkg$db/namespace", "person3")
+ .SetSchema("Person")
+ .AddStringProperty("Name", "Eve")
+ .Build();
+
+ DocumentProto email1 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email1")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 1")
+ .AddStringProperty("sender", "pkg$db/namespace#person1")
+ .Build();
+ DocumentProto email2 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email2")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 2")
+ .AddStringProperty("sender", "pkg$db/namespace#person2")
+ .Build();
+ DocumentProto email3 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email3")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 3")
+ .AddStringProperty("sender", "pkg$db/namespace#person3")
+ .Build();
+ DocumentProto email4 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email4")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 4")
+ .AddStringProperty("sender", "pkg$db/namespace#person1")
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ this->PutAndIndexDocument(person1));
+ ICING_ASSERT_OK(/*document_id2 unused*/
+ this->PutAndIndexDocument(person2));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
+ this->PutAndIndexDocument(person3));
+ ICING_ASSERT_OK(/*document_id4 unused*/
+ this->PutAndIndexDocument(email1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id5,
+ this->PutAndIndexDocument(email2));
+ ICING_ASSERT_OK(/*document_id6 unused*/
+ this->PutAndIndexDocument(email3));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id7,
+ this->PutAndIndexDocument(email4));
+
+ ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
+ /*score=*/0.0);
+ ScoredDocumentHit scored_doc_hit3(document_id3, kSectionIdMaskNone,
+ /*score=*/0.0);
+ ScoredDocumentHit scored_doc_hit5(document_id5, kSectionIdMaskNone,
+ /*score=*/4.0);
+ ScoredDocumentHit scored_doc_hit7(document_id7, kSectionIdMaskNone,
+ /*score=*/5.0);
+
+ // Only join person1, person3, email2 and email4.
+ // Parent ScoredDocumentHits: person1, person3
+ std::vector<ScoredDocumentHit> parent_scored_document_hits = {
+ scored_doc_hit3, scored_doc_hit1};
+
+ // Child ScoredDocumentHits: email2, email4
+ std::vector<ScoredDocumentHit> child_scored_document_hits = {scored_doc_hit7,
+ scored_doc_hit5};
+
+ JoinSpecProto join_spec;
+ join_spec.set_parent_property_expression(
+ std::string(JoinProcessor::kQualifiedIdExpr));
+ join_spec.set_child_property_expression("sender");
+ join_spec.set_aggregation_scoring_strategy(
+ JoinSpecProto::AggregationScoringStrategy::COUNT);
+ join_spec.mutable_nested_spec()->mutable_scoring_spec()->set_order_by(
+ ScoringSpecProto::Order::DESC);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
+ EXPECT_THAT(
+ joined_result_document_hits,
+ ElementsAre(EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/0.0,
+ /*parent_scored_document_hit=*/scored_doc_hit3,
+ /*child_scored_document_hits=*/{})),
+ EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/1.0,
+ /*parent_scored_document_hit=*/scored_doc_hit1,
+ /*child_scored_document_hits=*/{scored_doc_hit7}))));
+}
+
+TYPED_TEST(JoinProcessorTest,
+ ShouldIgnoreChildDocumentsWithoutJoiningProperty) {
DocumentProto person1 = DocumentBuilder()
.SetKey("pkg$db/namespace", "person1")
.SetSchema("Person")
@@ -303,11 +457,11 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithoutJoiningProperty) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(person1));
+ this->PutAndIndexDocument(person1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
- PutAndIndexDocument(email2));
+ this->PutAndIndexDocument(email2));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -335,8 +489,8 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithoutJoiningProperty) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
// Since Email2 doesn't have "sender" property, it should be ignored.
EXPECT_THAT(
joined_result_document_hits,
@@ -345,7 +499,8 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithoutJoiningProperty) {
/*child_scored_document_hits=*/{scored_doc_hit2}))));
}
-TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithInvalidQualifiedId) {
+TYPED_TEST(JoinProcessorTest,
+ ShouldIgnoreChildDocumentsWithInvalidQualifiedId) {
DocumentProto person1 = DocumentBuilder()
.SetKey("pkg$db/namespace", "person1")
.SetSchema("Person")
@@ -379,13 +534,13 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithInvalidQualifiedId) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(person1));
+ this->PutAndIndexDocument(person1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
- PutAndIndexDocument(email2));
+ this->PutAndIndexDocument(email2));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id4,
- PutAndIndexDocument(email3));
+ this->PutAndIndexDocument(email3));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -415,8 +570,8 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithInvalidQualifiedId) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
// Email 2 and email 3 (document id 3 and 4) contain invalid qualified ids.
// Join processor should ignore them.
EXPECT_THAT(joined_result_document_hits,
@@ -426,7 +581,7 @@ TEST_F(JoinProcessorTest, ShouldIgnoreChildDocumentsWithInvalidQualifiedId) {
/*child_scored_document_hits=*/{scored_doc_hit2}))));
}
-TEST_F(JoinProcessorTest, LeftJoinShouldReturnParentWithoutChildren) {
+TYPED_TEST(JoinProcessorTest, LeftJoinShouldReturnParentWithoutChildren) {
DocumentProto person1 = DocumentBuilder()
.SetKey("pkg$db/namespace", "person1")
.SetSchema("Person")
@@ -448,11 +603,11 @@ TEST_F(JoinProcessorTest, LeftJoinShouldReturnParentWithoutChildren) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(person1));
+ this->PutAndIndexDocument(person1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
- PutAndIndexDocument(person2));
+ this->PutAndIndexDocument(person2));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -479,8 +634,8 @@ TEST_F(JoinProcessorTest, LeftJoinShouldReturnParentWithoutChildren) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
// Person1 has no child documents, but left join should also include it.
EXPECT_THAT(
joined_result_document_hits,
@@ -494,7 +649,7 @@ TEST_F(JoinProcessorTest, LeftJoinShouldReturnParentWithoutChildren) {
/*child_scored_document_hits=*/{}))));
}
-TEST_F(JoinProcessorTest, ShouldSortChildDocumentsByRankingStrategy) {
+TYPED_TEST(JoinProcessorTest, ShouldSortChildDocumentsByRankingStrategy) {
DocumentProto person1 = DocumentBuilder()
.SetKey("pkg$db/namespace", "person1")
.SetSchema("Person")
@@ -524,13 +679,13 @@ TEST_F(JoinProcessorTest, ShouldSortChildDocumentsByRankingStrategy) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(person1));
+ this->PutAndIndexDocument(person1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
- PutAndIndexDocument(email2));
+ this->PutAndIndexDocument(email2));
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id4,
- PutAndIndexDocument(email3));
+ this->PutAndIndexDocument(email3));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -560,8 +715,8 @@ TEST_F(JoinProcessorTest, ShouldSortChildDocumentsByRankingStrategy) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
// Child documents should be sorted according to the (nested) ranking
// strategy.
EXPECT_THAT(
@@ -572,7 +727,7 @@ TEST_F(JoinProcessorTest, ShouldSortChildDocumentsByRankingStrategy) {
{scored_doc_hit3, scored_doc_hit4, scored_doc_hit2}))));
}
-TEST_F(JoinProcessorTest, ShouldAllowSelfJoining) {
+TYPED_TEST(JoinProcessorTest, ShouldAllowSelfJoining) {
DocumentProto email1 =
DocumentBuilder()
.SetKey("pkg$db/namespace", "email1")
@@ -582,7 +737,7 @@ TEST_F(JoinProcessorTest, ShouldAllowSelfJoining) {
.Build();
ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
- PutAndIndexDocument(email1));
+ this->PutAndIndexDocument(email1));
ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
/*score=*/0.0);
@@ -605,8 +760,8 @@ TEST_F(JoinProcessorTest, ShouldAllowSelfJoining) {
ICING_ASSERT_OK_AND_ASSIGN(
std::vector<JoinedScoredDocumentHit> joined_result_document_hits,
- Join(join_spec, std::move(parent_scored_document_hits),
- std::move(child_scored_document_hits)));
+ this->Join(join_spec, std::move(parent_scored_document_hits),
+ std::move(child_scored_document_hits)));
EXPECT_THAT(joined_result_document_hits,
ElementsAre(EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
/*final_score=*/1.0,
@@ -614,6 +769,156 @@ TEST_F(JoinProcessorTest, ShouldAllowSelfJoining) {
/*child_scored_document_hits=*/{scored_doc_hit1}))));
}
+TYPED_TEST(JoinProcessorTest, MultipleChildSchemasJoining) {
+ DocumentProto person1 = DocumentBuilder()
+ .SetKey("pkg$db/namespace", "person1")
+ .SetSchema("Person")
+ .AddStringProperty("Name", "Alice")
+ .Build();
+ DocumentProto person2 = DocumentBuilder()
+ .SetKey("pkg$db/namespace", "person2")
+ .SetSchema("Person")
+ .AddStringProperty("Name", "Bob")
+ .Build();
+
+ DocumentProto email1 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email1")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 1")
+ .AddStringProperty("sender", "pkg$db/namespace#person2")
+ .Build();
+ DocumentProto email2 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email2")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 2")
+ .AddStringProperty("sender", "pkg$db/namespace#person1")
+ .Build();
+ DocumentProto email3 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "email3")
+ .SetSchema("Email")
+ .AddStringProperty("subject", "test subject 3")
+ .AddStringProperty("sender", "pkg$db/namespace#person1")
+ .Build();
+ DocumentProto message1 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "message1")
+ .SetSchema("Message")
+ .AddStringProperty("content", "test content 1")
+ .AddStringProperty("sender", "pkg$db/namespace#person1")
+ .AddStringProperty("receiver", "pkg$db/namespace#person2")
+ .Build();
+ DocumentProto message2 =
+ DocumentBuilder()
+ .SetKey("pkg$db/namespace", "message2")
+ .SetSchema("Message")
+ .AddStringProperty("content", "test content 2")
+ .AddStringProperty("sender", "pkg$db/namespace#person2")
+ .AddStringProperty("receiver", "pkg$db/namespace#person1")
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ this->PutAndIndexDocument(person1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ this->PutAndIndexDocument(person2));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id3,
+ this->PutAndIndexDocument(email1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id4,
+ this->PutAndIndexDocument(email2));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id5,
+ this->PutAndIndexDocument(email3));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id6,
+ this->PutAndIndexDocument(message1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id7,
+ this->PutAndIndexDocument(message2));
+
+ ScoredDocumentHit scored_doc_hit1(document_id1, kSectionIdMaskNone,
+ /*score=*/0.0);
+ ScoredDocumentHit scored_doc_hit2(document_id2, kSectionIdMaskNone,
+ /*score=*/0.0);
+ ScoredDocumentHit scored_doc_hit3(document_id3, kSectionIdMaskNone,
+ /*score=*/5.0);
+ ScoredDocumentHit scored_doc_hit4(document_id4, kSectionIdMaskNone,
+ /*score=*/3.0);
+ ScoredDocumentHit scored_doc_hit5(document_id5, kSectionIdMaskNone,
+ /*score=*/2.0);
+ ScoredDocumentHit scored_doc_hit6(document_id6, kSectionIdMaskNone,
+ /*score=*/4.0);
+ ScoredDocumentHit scored_doc_hit7(document_id7, kSectionIdMaskNone,
+ /*score=*/1.0);
+
+ // Parent ScoredDocumentHits: all Person documents
+ std::vector<ScoredDocumentHit> parent_scored_document_hits = {
+ scored_doc_hit1, scored_doc_hit2};
+
+ // Child ScoredDocumentHits: all Email and Message documents
+ std::vector<ScoredDocumentHit> child_scored_document_hits = {
+ scored_doc_hit3, scored_doc_hit4, scored_doc_hit5, scored_doc_hit6,
+ scored_doc_hit7};
+
+ // Join by "sender".
+ // - Person1: [
+ // email2 (scored_doc_hit4),
+ // email3 (scored_doc_hit5),
+ // message1 (scored_doc_hit6),
+ // ]
+ // - Person2: [
+ // email1 (scored_doc_hit3),
+ // message2 (scored_doc_hit7),
+ // ]
+ JoinSpecProto join_spec;
+ join_spec.set_parent_property_expression(
+ std::string(JoinProcessor::kQualifiedIdExpr));
+ join_spec.set_child_property_expression("sender");
+ join_spec.set_aggregation_scoring_strategy(
+ JoinSpecProto::AggregationScoringStrategy::COUNT);
+ join_spec.mutable_nested_spec()->mutable_scoring_spec()->set_order_by(
+ ScoringSpecProto::Order::DESC);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<JoinedScoredDocumentHit> joined_result_document_hits1,
+ this->Join(join_spec, parent_scored_document_hits,
+ child_scored_document_hits));
+ EXPECT_THAT(
+ joined_result_document_hits1,
+ ElementsAre(EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/3.0,
+ /*parent_scored_document_hit=*/scored_doc_hit1,
+ /*child_scored_document_hits=*/
+ {scored_doc_hit6, scored_doc_hit4, scored_doc_hit5})),
+ EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/2.0,
+ /*parent_scored_document_hit=*/scored_doc_hit2,
+ /*child_scored_document_hits=*/
+ {scored_doc_hit3, scored_doc_hit7}))));
+
+ // Join by "receiver".
+ // - Person1: [
+ // message2 (scored_doc_hit7),
+ // ]
+ // - Person2: [
+ // message1 (scored_doc_hit6),
+ // ]
+ join_spec.set_child_property_expression("receiver");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::vector<JoinedScoredDocumentHit> joined_result_document_hits2,
+ this->Join(join_spec, parent_scored_document_hits,
+ child_scored_document_hits));
+ EXPECT_THAT(
+ joined_result_document_hits2,
+ ElementsAre(EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/1.0,
+ /*parent_scored_document_hit=*/scored_doc_hit1,
+ /*child_scored_document_hits=*/{scored_doc_hit7})),
+ EqualsJoinedScoredDocumentHit(JoinedScoredDocumentHit(
+ /*final_score=*/1.0,
+ /*parent_scored_document_hit=*/scored_doc_hit2,
+ /*child_scored_document_hits=*/{scored_doc_hit6}))));
+}
+
// TODO(b/256022027): add unit tests for non-joinable property. If joinable
// value type is unset, then qualifed id join should not
// include the child document even if it contains a valid
diff --git a/icing/join/posting-list-join-data-accessor.h b/icing/join/posting-list-join-data-accessor.h
new file mode 100644
index 0000000..6669f9f
--- /dev/null
+++ b/icing/join/posting-list-join-data-accessor.h
@@ -0,0 +1,211 @@
+// Copyright (C) 2023 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_JOIN_POSTING_LIST_JOIN_DATA_ACCESSOR_H_
+#define ICING_JOIN_POSTING_LIST_JOIN_DATA_ACCESSOR_H_
+
+#include <cstdint>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/posting_list/flash-index-storage.h"
+#include "icing/file/posting_list/index-block.h"
+#include "icing/file/posting_list/posting-list-accessor.h"
+#include "icing/file/posting_list/posting-list-common.h"
+#include "icing/file/posting_list/posting-list-identifier.h"
+#include "icing/file/posting_list/posting-list-used.h"
+#include "icing/join/posting-list-join-data-serializer.h"
+#include "icing/legacy/index/icing-bit-util.h"
+#include "icing/util/status-macros.h"
+
+namespace icing {
+namespace lib {
+
+// This class is used to provide a simple abstraction for adding join data to
+// posting lists. PostingListJoinDataAccessor handles:
+// 1) selection of properly-sized posting lists for the accumulated join index
+// data during Finalize()
+// 2) chaining of max-sized posting lists.
+template <typename JoinDataType>
+class PostingListJoinDataAccessor : public PostingListAccessor {
+ public:
+ // Creates an empty PostingListJoinDataAccessor.
+ //
+ // RETURNS:
+ // - On success, a valid instance of PostingListJoinDataAccessor
+ // - INVALID_ARGUMENT error if storage has an invalid block_size.
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>>
+ Create(FlashIndexStorage* storage,
+ PostingListJoinDataSerializer<JoinDataType>* serializer);
+
+ // Creates a PostingListJoinDataAccessor with an existing posting list
+ // identified by existing_posting_list_id.
+ //
+ // RETURNS:
+ // - On success, a valid instance of PostingListJoinDataAccessor
+ // - INVALID_ARGUMENT if storage has an invalid block_size.
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>>
+ CreateFromExisting(FlashIndexStorage* storage,
+ PostingListJoinDataSerializer<JoinDataType>* serializer,
+ PostingListIdentifier existing_posting_list_id);
+
+ PostingListSerializer* GetSerializer() override { return serializer_; }
+
+ // Retrieves the next batch of data in the posting list chain.
+ //
+ // RETURNS:
+ // - On success, a vector of join data in the posting list chain
+ // - FAILED_PRECONDITION_ERROR if called on an instance that was created via
+ // Create.
+ // - INTERNAL_ERROR if unable to read the next posting list in the chain or
+ // if the posting list has been corrupted somehow.
+ libtextclassifier3::StatusOr<std::vector<JoinDataType>> GetNextDataBatch();
+
+ // Prepends one data. This may result in flushing the posting list to disk (if
+ // the PostingListJoinDataAccessor holds a max-sized posting list that is
+ // full) or freeing a pre-existing posting list if it is too small to fit all
+ // data necessary.
+ //
+ // RETURNS:
+ // - OK, on success
+ // - INVALID_ARGUMENT if !data.is_valid() or if data is greater than the
+ // previously added data.
+ // - RESOURCE_EXHAUSTED error if unable to grow the index to allocate a new
+ // posting list.
+ libtextclassifier3::Status PrependData(const JoinDataType& data);
+
+ private:
+ explicit PostingListJoinDataAccessor(
+ FlashIndexStorage* storage, PostingListUsed in_memory_posting_list,
+ PostingListJoinDataSerializer<JoinDataType>* serializer)
+ : PostingListAccessor(storage, std::move(in_memory_posting_list)),
+ serializer_(serializer) {}
+
+ PostingListJoinDataSerializer<JoinDataType>* serializer_; // Does not own.
+};
+
+template <typename JoinDataType>
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>>
+PostingListJoinDataAccessor<JoinDataType>::Create(
+ FlashIndexStorage* storage,
+ PostingListJoinDataSerializer<JoinDataType>* serializer) {
+ uint32_t max_posting_list_bytes = IndexBlock::CalculateMaxPostingListBytes(
+ storage->block_size(), serializer->GetDataTypeBytes());
+ ICING_ASSIGN_OR_RETURN(PostingListUsed in_memory_posting_list,
+ PostingListUsed::CreateFromUnitializedRegion(
+ serializer, max_posting_list_bytes));
+ return std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>(
+ new PostingListJoinDataAccessor<JoinDataType>(
+ storage, std::move(in_memory_posting_list), serializer));
+}
+
+template <typename JoinDataType>
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>>
+PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ FlashIndexStorage* storage,
+ PostingListJoinDataSerializer<JoinDataType>* serializer,
+ PostingListIdentifier existing_posting_list_id) {
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ Create(storage, serializer));
+ ICING_ASSIGN_OR_RETURN(PostingListHolder holder,
+ storage->GetPostingList(existing_posting_list_id));
+ pl_accessor->preexisting_posting_list_ =
+ std::make_unique<PostingListHolder>(std::move(holder));
+ return pl_accessor;
+}
+
+// Returns the next batch of join data for the provided posting list.
+template <typename JoinDataType>
+libtextclassifier3::StatusOr<std::vector<JoinDataType>>
+PostingListJoinDataAccessor<JoinDataType>::GetNextDataBatch() {
+ if (preexisting_posting_list_ == nullptr) {
+ if (has_reached_posting_list_chain_end_) {
+ return std::vector<JoinDataType>();
+ }
+ return absl_ports::FailedPreconditionError(
+ "Cannot retrieve data from a PostingListJoinDataAccessor that was not "
+ "created from a preexisting posting list.");
+ }
+ ICING_ASSIGN_OR_RETURN(
+ std::vector<JoinDataType> batch,
+ serializer_->GetData(&preexisting_posting_list_->posting_list));
+ uint32_t next_block_index = kInvalidBlockIndex;
+ // Posting lists will only be chained when they are max-sized, in which case
+ // next_block_index will point to the next block for the next posting list.
+ // Otherwise, next_block_index can be kInvalidBlockIndex or be used to point
+ // to the next free list block, which is not relevant here.
+ if (preexisting_posting_list_->posting_list.size_in_bytes() ==
+ storage_->max_posting_list_bytes()) {
+ next_block_index = preexisting_posting_list_->next_block_index;
+ }
+
+ if (next_block_index != kInvalidBlockIndex) {
+ // Since we only have to deal with next block for max-sized posting list
+ // block, max_num_posting_lists is 1 and posting_list_index_bits is
+ // BitsToStore(1).
+ PostingListIdentifier next_posting_list_id(
+ next_block_index, /*posting_list_index=*/0,
+ /*posting_list_index_bits=*/BitsToStore(1));
+ ICING_ASSIGN_OR_RETURN(PostingListHolder holder,
+ storage_->GetPostingList(next_posting_list_id));
+ preexisting_posting_list_ =
+ std::make_unique<PostingListHolder>(std::move(holder));
+ } else {
+ has_reached_posting_list_chain_end_ = true;
+ preexisting_posting_list_.reset();
+ }
+ return batch;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataAccessor<JoinDataType>::PrependData(
+ const JoinDataType& data) {
+ PostingListUsed& active_pl = (preexisting_posting_list_ != nullptr)
+ ? preexisting_posting_list_->posting_list
+ : in_memory_posting_list_;
+ libtextclassifier3::Status status =
+ serializer_->PrependData(&active_pl, data);
+ if (!absl_ports::IsResourceExhausted(status)) {
+ return status;
+ }
+ // There is no more room to add data to this current posting list! Therefore,
+ // we need to either move those data to a larger posting list or flush this
+ // posting list and create another max-sized posting list in the chain.
+ if (preexisting_posting_list_ != nullptr) {
+ ICING_RETURN_IF_ERROR(FlushPreexistingPostingList());
+ } else {
+ ICING_RETURN_IF_ERROR(FlushInMemoryPostingList());
+ }
+
+ // Re-add data. Should always fit since we just cleared
+ // in_memory_posting_list_. It's fine to explicitly reference
+ // in_memory_posting_list_ here because there's no way of reaching this line
+ // while preexisting_posting_list_ is still in use.
+ return serializer_->PrependData(&in_memory_posting_list_, data);
+}
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_JOIN_POSTING_LIST_JOIN_DATA_ACCESSOR_H_
diff --git a/icing/join/posting-list-join-data-accessor_test.cc b/icing/join/posting-list-join-data-accessor_test.cc
new file mode 100644
index 0000000..ddc2d32
--- /dev/null
+++ b/icing/join/posting-list-join-data-accessor_test.cc
@@ -0,0 +1,435 @@
+// Copyright (C) 2023 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/join/posting-list-join-data-accessor.h"
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/posting_list/flash-index-storage.h"
+#include "icing/file/posting_list/posting-list-accessor.h"
+#include "icing/file/posting_list/posting-list-common.h"
+#include "icing/file/posting_list/posting-list-identifier.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/join/posting-list-join-data-serializer.h"
+#include "icing/store/document-id.h"
+#include "icing/store/namespace-fingerprint-identifier.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::ElementsAreArray;
+using ::testing::Eq;
+using ::testing::Lt;
+using ::testing::Ne;
+using ::testing::SizeIs;
+
+using JoinDataType = DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>;
+
+static constexpr NamespaceId kDefaultNamespaceId = 1;
+
+class PostingListJoinDataAccessorTest : public ::testing::Test {
+ protected:
+ void SetUp() override {
+ test_dir_ = GetTestTempDir() + "/test_dir";
+ file_name_ = test_dir_ + "/test_file.idx.index";
+
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(test_dir_.c_str()));
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(test_dir_.c_str()));
+
+ serializer_ =
+ std::make_unique<PostingListJoinDataSerializer<JoinDataType>>();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ FlashIndexStorage flash_index_storage,
+ FlashIndexStorage::Create(file_name_, &filesystem_, serializer_.get()));
+ flash_index_storage_ =
+ std::make_unique<FlashIndexStorage>(std::move(flash_index_storage));
+ }
+
+ void TearDown() override {
+ flash_index_storage_.reset();
+ serializer_.reset();
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(test_dir_.c_str()));
+ }
+
+ Filesystem filesystem_;
+ std::string test_dir_;
+ std::string file_name_;
+ std::unique_ptr<PostingListJoinDataSerializer<JoinDataType>> serializer_;
+ std::unique_ptr<FlashIndexStorage> flash_index_storage_;
+};
+
+std::vector<JoinDataType> CreateData(int num_data, DocumentId start_document_id,
+ NamespaceId ref_namespace_id,
+ uint64_t start_ref_hash_uri) {
+ std::vector<JoinDataType> data;
+ data.reserve(num_data);
+ for (int i = 0; i < num_data; ++i) {
+ data.push_back(JoinDataType(
+ start_document_id,
+ NamespaceFingerprintIdentifier(ref_namespace_id,
+ /*fingerprint=*/start_ref_hash_uri)));
+
+ ++start_document_id;
+ ++start_ref_hash_uri;
+ }
+ return data;
+}
+
+TEST_F(PostingListJoinDataAccessorTest, DataAddAndRetrieveProperly) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ // Add some join data
+ std::vector<JoinDataType> data_vec =
+ CreateData(/*num_data=*/5, /*start_document_id=*/0,
+ /*ref_namespace_id=*/kDefaultNamespaceId,
+ /*start_ref_hash_uri=*/819);
+ for (const JoinDataType& data : data_vec) {
+ EXPECT_THAT(pl_accessor->PrependData(data), IsOk());
+ }
+ PostingListAccessor::FinalizeResult result =
+ std::move(*pl_accessor).Finalize();
+ EXPECT_THAT(result.status, IsOk());
+ EXPECT_THAT(result.id.block_index(), Eq(1));
+ EXPECT_THAT(result.id.posting_list_index(), Eq(0));
+
+ // Retrieve some data.
+ ICING_ASSERT_OK_AND_ASSIGN(PostingListHolder pl_holder,
+ flash_index_storage_->GetPostingList(result.id));
+ EXPECT_THAT(
+ serializer_->GetData(&pl_holder.posting_list),
+ IsOkAndHolds(ElementsAreArray(data_vec.rbegin(), data_vec.rend())));
+ EXPECT_THAT(pl_holder.next_block_index, Eq(kInvalidBlockIndex));
+}
+
+TEST_F(PostingListJoinDataAccessorTest, PreexistingPLKeepOnSameBlock) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ // Add a single data. This will fit in a min-sized posting list.
+ JoinDataType data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId, /*fingerprint=*/123));
+ ICING_ASSERT_OK(pl_accessor->PrependData(data1));
+ PostingListAccessor::FinalizeResult result1 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result1.status);
+ // Should be allocated to the first block.
+ ASSERT_THAT(result1.id.block_index(), Eq(1));
+ ASSERT_THAT(result1.id.posting_list_index(), Eq(0));
+
+ // Add one more data. The minimum size for a posting list must be able to fit
+ // two data, so this should NOT cause the previous pl to be reallocated.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), serializer_.get(), result1.id));
+ JoinDataType data2(
+ /*document_id=*/2,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId, /*fingerprint=*/456));
+ ICING_ASSERT_OK(pl_accessor->PrependData(data2));
+ PostingListAccessor::FinalizeResult result2 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result2.status);
+ // Should be in the same posting list.
+ EXPECT_THAT(result2.id, Eq(result1.id));
+
+ // The posting list at result2.id should hold all of the data that have been
+ // added.
+ ICING_ASSERT_OK_AND_ASSIGN(PostingListHolder pl_holder,
+ flash_index_storage_->GetPostingList(result2.id));
+ EXPECT_THAT(serializer_->GetData(&pl_holder.posting_list),
+ IsOkAndHolds(ElementsAre(data2, data1)));
+}
+
+TEST_F(PostingListJoinDataAccessorTest, PreexistingPLReallocateToLargerPL) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ // Adding 3 data should cause Finalize allocating a 56-byte posting list,
+ // which can store at most 4 data.
+ std::vector<JoinDataType> data_vec1 =
+ CreateData(/*num_data=*/3, /*start_document_id=*/0,
+ /*ref_namespace_id=*/kDefaultNamespaceId,
+ /*start_ref_hash_uri=*/819);
+ for (const JoinDataType& data : data_vec1) {
+ ICING_ASSERT_OK(pl_accessor->PrependData(data));
+ }
+ PostingListAccessor::FinalizeResult result1 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result1.status);
+ // Should be allocated to the first block.
+ ASSERT_THAT(result1.id.block_index(), Eq(1));
+ ASSERT_THAT(result1.id.posting_list_index(), Eq(0));
+
+ // Now add more data.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), serializer_.get(), result1.id));
+ // The current posting list can fit 1 more data. Adding 12 more data should
+ // result in these data being moved to a larger posting list. Also the total
+ // size of these data won't exceed max size posting list, so there will be
+ // only one single posting list and no chain.
+ std::vector<JoinDataType> data_vec2 = CreateData(
+ /*num_data=*/12, /*start_document_id=*/data_vec1.back().document_id() + 1,
+ /*ref_namespace_id=*/kDefaultNamespaceId, /*start_ref_hash_uri=*/819);
+
+ for (const JoinDataType& data : data_vec2) {
+ ICING_ASSERT_OK(pl_accessor->PrependData(data));
+ }
+ PostingListAccessor::FinalizeResult result2 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result2.status);
+ // Should be allocated to the second (new) block because the posting list
+ // should grow beyond the size that the first block maintains.
+ EXPECT_THAT(result2.id.block_index(), Eq(2));
+ EXPECT_THAT(result2.id.posting_list_index(), Eq(0));
+
+ // The posting list at result2.id should hold all of the data that have been
+ // added.
+ std::vector<JoinDataType> all_data_vec;
+ all_data_vec.reserve(data_vec1.size() + data_vec2.size());
+ all_data_vec.insert(all_data_vec.end(), data_vec1.begin(), data_vec1.end());
+ all_data_vec.insert(all_data_vec.end(), data_vec2.begin(), data_vec2.end());
+ ICING_ASSERT_OK_AND_ASSIGN(PostingListHolder pl_holder,
+ flash_index_storage_->GetPostingList(result2.id));
+ EXPECT_THAT(serializer_->GetData(&pl_holder.posting_list),
+ IsOkAndHolds(ElementsAreArray(all_data_vec.rbegin(),
+ all_data_vec.rend())));
+}
+
+TEST_F(PostingListJoinDataAccessorTest, MultiBlockChainsBlocksProperly) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ // Block size is 4096, sizeof(BlockHeader) is 12 and sizeof(JoinDataType)
+ // is 14, so the max size posting list can store (4096 - 12) / 14 = 291 data.
+ // Adding 292 data should cause:
+ // - 2 max size posting lists being allocated to block 1 and block 2.
+ // - Chaining: block 2 -> block 1
+ std::vector<JoinDataType> data_vec = CreateData(
+ /*num_data=*/292, /*start_document_id=*/0,
+ /*ref_namespace_id=*/kDefaultNamespaceId, /*start_ref_hash_uri=*/819);
+ for (const JoinDataType& data : data_vec) {
+ ICING_ASSERT_OK(pl_accessor->PrependData(data));
+ }
+ PostingListAccessor::FinalizeResult result1 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result1.status);
+ PostingListIdentifier second_block_id = result1.id;
+ // Should be allocated to the second block.
+ EXPECT_THAT(second_block_id, Eq(PostingListIdentifier(
+ /*block_index=*/2, /*posting_list_index=*/0,
+ /*posting_list_index_bits=*/0)));
+
+ // We should be able to retrieve all data.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListHolder pl_holder,
+ flash_index_storage_->GetPostingList(second_block_id));
+ // This pl_holder will only hold a posting list with the data that didn't fit
+ // on the first block.
+ ICING_ASSERT_OK_AND_ASSIGN(std::vector<JoinDataType> second_block_data,
+ serializer_->GetData(&pl_holder.posting_list));
+ ASSERT_THAT(second_block_data, SizeIs(Lt(data_vec.size())));
+ auto first_block_data_start = data_vec.rbegin() + second_block_data.size();
+ EXPECT_THAT(second_block_data,
+ ElementsAreArray(data_vec.rbegin(), first_block_data_start));
+
+ // Now retrieve all of the data that were on the first block.
+ uint32_t first_block_id = pl_holder.next_block_index;
+ EXPECT_THAT(first_block_id, Eq(1));
+
+ PostingListIdentifier pl_id(first_block_id, /*posting_list_index=*/0,
+ /*posting_list_index_bits=*/0);
+ ICING_ASSERT_OK_AND_ASSIGN(pl_holder,
+ flash_index_storage_->GetPostingList(pl_id));
+ EXPECT_THAT(
+ serializer_->GetData(&pl_holder.posting_list),
+ IsOkAndHolds(ElementsAreArray(first_block_data_start, data_vec.rend())));
+}
+
+TEST_F(PostingListJoinDataAccessorTest,
+ PreexistingMultiBlockReusesBlocksProperly) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ // Block size is 4096, sizeof(BlockHeader) is 12 and sizeof(JoinDataType)
+ // is 14, so the max size posting list can store (4096 - 12) / 14 = 291 data.
+ // Adding 292 data will cause:
+ // - 2 max size posting lists being allocated to block 1 and block 2.
+ // - Chaining: block 2 -> block 1
+ std::vector<JoinDataType> data_vec1 = CreateData(
+ /*num_data=*/292, /*start_document_id=*/0,
+ /*ref_namespace_id=*/kDefaultNamespaceId, /*start_ref_hash_uri=*/819);
+ for (const JoinDataType& data : data_vec1) {
+ ICING_ASSERT_OK(pl_accessor->PrependData(data));
+ }
+ PostingListAccessor::FinalizeResult result1 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result1.status);
+ PostingListIdentifier first_add_id = result1.id;
+ EXPECT_THAT(first_add_id, Eq(PostingListIdentifier(
+ /*block_index=*/2, /*posting_list_index=*/0,
+ /*posting_list_index_bits=*/0)));
+
+ // Now add more data. These should fit on the existing second block and not
+ // fill it up.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), serializer_.get(), first_add_id));
+ std::vector<JoinDataType> data_vec2 = CreateData(
+ /*num_data=*/10, /*start_document_id=*/data_vec1.back().document_id() + 1,
+ /*ref_namespace_id=*/kDefaultNamespaceId, /*start_ref_hash_uri=*/819);
+ for (const JoinDataType& data : data_vec2) {
+ ICING_ASSERT_OK(pl_accessor->PrependData(data));
+ }
+ PostingListAccessor::FinalizeResult result2 =
+ std::move(*pl_accessor).Finalize();
+ ICING_ASSERT_OK(result2.status);
+ PostingListIdentifier second_add_id = result2.id;
+ EXPECT_THAT(second_add_id, Eq(first_add_id));
+
+ // We should be able to retrieve all data.
+ std::vector<JoinDataType> all_data_vec;
+ all_data_vec.reserve(data_vec1.size() + data_vec2.size());
+ all_data_vec.insert(all_data_vec.end(), data_vec1.begin(), data_vec1.end());
+ all_data_vec.insert(all_data_vec.end(), data_vec2.begin(), data_vec2.end());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListHolder pl_holder,
+ flash_index_storage_->GetPostingList(second_add_id));
+ // This pl_holder will only hold a posting list with the data that didn't fit
+ // on the first block.
+ ICING_ASSERT_OK_AND_ASSIGN(std::vector<JoinDataType> second_block_data,
+ serializer_->GetData(&pl_holder.posting_list));
+ ASSERT_THAT(second_block_data, SizeIs(Lt(all_data_vec.size())));
+ auto first_block_data_start =
+ all_data_vec.rbegin() + second_block_data.size();
+ EXPECT_THAT(second_block_data,
+ ElementsAreArray(all_data_vec.rbegin(), first_block_data_start));
+
+ // Now retrieve all of the data that were on the first block.
+ uint32_t first_block_id = pl_holder.next_block_index;
+ EXPECT_THAT(first_block_id, Eq(1));
+
+ PostingListIdentifier pl_id(first_block_id, /*posting_list_index=*/0,
+ /*posting_list_index_bits=*/0);
+ ICING_ASSERT_OK_AND_ASSIGN(pl_holder,
+ flash_index_storage_->GetPostingList(pl_id));
+ EXPECT_THAT(serializer_->GetData(&pl_holder.posting_list),
+ IsOkAndHolds(ElementsAreArray(first_block_data_start,
+ all_data_vec.rend())));
+}
+
+TEST_F(PostingListJoinDataAccessorTest,
+ InvalidDataShouldReturnInvalidArgument) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ JoinDataType invalid_data = JoinDataType::GetInvalid();
+ EXPECT_THAT(pl_accessor->PrependData(invalid_data),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(PostingListJoinDataAccessorTest,
+ JoinDataNonIncreasingShouldReturnInvalidArgument) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ JoinDataType data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId, /*fingerprint=*/819));
+ ICING_ASSERT_OK(pl_accessor->PrependData(data1));
+
+ JoinDataType data2(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId, /*fingerprint=*/818));
+ EXPECT_THAT(pl_accessor->PrependData(data2),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ JoinDataType data3(/*document_id=*/1,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId - 1,
+ /*fingerprint=*/820));
+ EXPECT_THAT(pl_accessor->PrependData(data3),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ JoinDataType data4(/*document_id=*/0,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId + 1,
+ /*fingerprint=*/820));
+ EXPECT_THAT(pl_accessor->PrependData(data4),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(PostingListJoinDataAccessorTest,
+ NewPostingListNoDataAddedShouldReturnInvalidArgument) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ PostingListAccessor::FinalizeResult result =
+ std::move(*pl_accessor).Finalize();
+ EXPECT_THAT(result.status,
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(PostingListJoinDataAccessorTest,
+ PreexistingPostingListNoDataAddedShouldSucceed) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor1,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), serializer_.get()));
+ JoinDataType data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(kDefaultNamespaceId, /*fingerprint=*/819));
+ ICING_ASSERT_OK(pl_accessor1->PrependData(data1));
+ PostingListAccessor::FinalizeResult result1 =
+ std::move(*pl_accessor1).Finalize();
+ ICING_ASSERT_OK(result1.status);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor2,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), serializer_.get(), result1.id));
+ PostingListAccessor::FinalizeResult result2 =
+ std::move(*pl_accessor2).Finalize();
+ EXPECT_THAT(result2.status, IsOk());
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/join/posting-list-join-data-serializer.h b/icing/join/posting-list-join-data-serializer.h
new file mode 100644
index 0000000..9f39dca
--- /dev/null
+++ b/icing/join/posting-list-join-data-serializer.h
@@ -0,0 +1,803 @@
+// Copyright (C) 2023 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_JOIN_POSTING_LIST_JOIN_DATA_SERIALIZER_H_
+#define ICING_JOIN_POSTING_LIST_JOIN_DATA_SERIALIZER_H_
+
+#include <cstdint>
+#include <cstring>
+#include <limits>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/posting_list/posting-list-common.h"
+#include "icing/file/posting_list/posting-list-used.h"
+#include "icing/legacy/core/icing-string-util.h"
+#include "icing/util/logging.h"
+#include "icing/util/status-macros.h"
+
+namespace icing {
+namespace lib {
+
+// A serializer class to serialize JoinDataType to PostingListUsed. Usually
+// JoinDataType is DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>,
+// DocumentIdToJoinInfo<TermId>, or DocumentIdToJoinInfo<int64_t>.
+//
+// REQUIRES:
+// - JoinDataType is comparable by operator <.
+// - JoinDataType implements is_valid() method.
+// - JoinDataType has static method GetInvalid() that returns a JoinDataType
+// instance containing invalid data.
+template <typename JoinDataType>
+class PostingListJoinDataSerializer : public PostingListSerializer {
+ public:
+ using SpecialDataType = SpecialData<JoinDataType>;
+ static_assert(sizeof(SpecialDataType) == sizeof(JoinDataType), "");
+
+ static constexpr uint32_t kSpecialDataSize =
+ kNumSpecialData * sizeof(SpecialDataType);
+
+ uint32_t GetDataTypeBytes() const override { return sizeof(JoinDataType); }
+
+ uint32_t GetMinPostingListSize() const override {
+ static constexpr uint32_t kMinPostingListSize = kSpecialDataSize;
+ static_assert(sizeof(PostingListIndex) <= kMinPostingListSize,
+ "PostingListIndex must be small enough to fit in a "
+ "minimum-sized Posting List.");
+
+ return kMinPostingListSize;
+ }
+
+ uint32_t GetMinPostingListSizeToFit(
+ const PostingListUsed* posting_list_used) const override;
+
+ uint32_t GetBytesUsed(
+ const PostingListUsed* posting_list_used) const override;
+
+ void Clear(PostingListUsed* posting_list_used) const override;
+
+ libtextclassifier3::Status MoveFrom(PostingListUsed* dst,
+ PostingListUsed* src) const override;
+
+ // Prepend a JoinData to the posting list.
+ //
+ // RETURNS:
+ // - INVALID_ARGUMENT if !data.is_valid() or if data is not greater than the
+ // previously added data.
+ // - RESOURCE_EXHAUSTED if there is no more room to add data to the posting
+ // list.
+ libtextclassifier3::Status PrependData(PostingListUsed* posting_list_used,
+ const JoinDataType& data) const;
+
+ // Prepend multiple JoinData to the posting list.
+ // Data should be sorted in ascending order (as defined by the less than
+ // operator for JoinData)
+ // If keep_prepended is true, whatever could be prepended is kept, otherwise
+ // the posting list is reverted and left in its original state.
+ //
+ // RETURNS:
+ // The number of data that have been prepended to the posting list. If
+ // keep_prepended is false and reverted, then it returns 0.
+ libtextclassifier3::StatusOr<uint32_t> PrependDataArray(
+ PostingListUsed* posting_list_used, const JoinDataType* array,
+ uint32_t num_data, bool keep_prepended) const;
+
+ // Retrieves all data stored in the posting list.
+ //
+ // RETURNS:
+ // - On success, a vector of JoinDataType sorted by the reverse order of
+ // prepending.
+ // - INTERNAL_ERROR if the posting list has been corrupted somehow.
+ libtextclassifier3::StatusOr<std::vector<JoinDataType>> GetData(
+ const PostingListUsed* posting_list_used) const;
+
+ // Same as GetData but appends data to data_arr_out.
+ //
+ // RETURNS:
+ // - OK on success, and data_arr_out will be appended JoinDataType sorted by
+ // the reverse order of prepending.
+ // - INTERNAL_ERROR if the posting list has been corrupted somehow.
+ libtextclassifier3::Status GetData(
+ const PostingListUsed* posting_list_used,
+ std::vector<JoinDataType>* data_arr_out) const;
+
+ // Undo the last num_data data prepended. If num_data > number of data, then
+ // we clear all data.
+ //
+ // RETURNS:
+ // - OK on success
+ // - INTERNAL_ERROR if the posting list has been corrupted somehow.
+ libtextclassifier3::Status PopFrontData(PostingListUsed* posting_list_used,
+ uint32_t num_data) const;
+
+ // Helper function to determine if posting list is full.
+ bool IsFull(const PostingListUsed* posting_list_used) const {
+ return GetSpecialData(posting_list_used, /*index=*/0).data().is_valid() &&
+ GetSpecialData(posting_list_used, /*index=*/1).data().is_valid();
+ }
+
+ private:
+ // In PostingListJoinDataSerializer, there is no compression, but we still use
+ // the traditional posting list implementation.
+ //
+ // Posting list layout formats:
+ //
+ // NOT_FULL
+ // +-special-data-0--+-special-data-1--+------------+-----------------------+
+ // | | | | |
+ // |data-start-offset| Data::Invalid | 0x00000000 | (compressed) data |
+ // | | | | |
+ // +-----------------+-----------------+------------+-----------------------+
+ //
+ // ALMOST_FULL
+ // +-special-data-0--+-special-data-1--+-----+------------------------------+
+ // | | | | |
+ // | Data::Invalid | 1st data |(pad)| (compressed) data |
+ // | | | | |
+ // +-----------------+-----------------+-----+------------------------------+
+ //
+ // FULL
+ // +-special-data-0--+-special-data-1--+-----+------------------------------+
+ // | | | | |
+ // | 1st data | 2nd data |(pad)| (compressed) data |
+ // | | | | |
+ // +-----------------+-----------------+-----+------------------------------+
+ //
+ // The first two uncompressed (special) data also implicitly encode
+ // information about the size of the compressed data region.
+ //
+ // 1. If the posting list is NOT_FULL, then special_data_0 contains the byte
+ // offset of the start of the compressed data. Thus, the size of the
+ // compressed data is
+ // posting_list_used->size_in_bytes() - special_data_0.data_start_offset().
+ //
+ // 2. If posting list is ALMOST_FULL or FULL, then the compressed data region
+ // starts somewhere between
+ // [kSpecialDataSize, kSpecialDataSize + sizeof(JoinDataType) - 1] and ends
+ // at posting_list_used->size_in_bytes() - 1.
+ //
+ // EXAMPLE
+ // JoinDataType = DocumentIdToJoinInfo<int64_t>. Posting list size: 48 bytes
+ //
+ // EMPTY!
+ // +-- byte 0-11 --+---- 12-23 ----+------------ 24-47 -------------+
+ // | | | |
+ // | 48 | Data::Invalid | 0x00000000 |
+ // | | | |
+ // +---------------+---------------+--------------------------------+
+ //
+ // Add DocumentIdToJoinInfo<int64_t>(DocumentId = 12, JoinInteger = 5)
+ // NOT FULL!
+ // +-- byte 0-11 --+---- 12-23 ----+---- 24-35 ----+---- 36-47 ----+
+ // | | | | 12 |
+ // | 36 | Data::Invalid | 0x00000000 | 5 |
+ // | | | | |
+ // +---------------+---------------+---------------+---------------+
+ //
+ // Add DocumentIdToJoinInfo<int64_t>(DocumentId = 18, JoinInteger = -2)
+ // +-- byte 0-11 --+---- 12-23 ----+---- 24-35 ----+---- 36-47 ----+
+ // | | | 18 | 12 |
+ // | 24 | Data::Invalid | -2 | 5 |
+ // | | | | |
+ // +---------------+---------------+---------------+---------------+
+ //
+ // Add DocumentIdToJoinInfo<int64_t>(DocumentId = 22, JoinInteger = 3)
+ // ALMOST_FULL!
+ // +-- byte 0-11 --+---- 12-23 ----+---- 24-35 ----+---- 36-47 ----+
+ // | | 22 | 18 | 12 |
+ // | Data::Invalid | 3 | -2 | 5 |
+ // | | | | |
+ // +---------------+---------------+---------------+---------------+
+ //
+ // Add DocumentIdToJoinInfo<int64_t>(DocumentId = 27, JoinInteger = 0)
+ // FULL!
+ // +-- byte 0-11 --+---- 12-23 ----+---- 24-35 ----+---- 36-47 ----+
+ // | 27 | 22 | 18 | 12 |
+ // | 0 | 3 | -2 | 5 |
+ // | | | | |
+ // +---------------+---------------+---------------+---------------+
+
+ // Helpers to determine what state the posting list is in.
+ bool IsAlmostFull(const PostingListUsed* posting_list_used) const {
+ return !GetSpecialData(posting_list_used, /*index=*/0).data().is_valid() &&
+ GetSpecialData(posting_list_used, /*index=*/1).data().is_valid();
+ }
+
+ bool IsEmpty(const PostingListUsed* posting_list_used) const {
+ return GetSpecialData(posting_list_used, /*index=*/0).data_start_offset() ==
+ posting_list_used->size_in_bytes() &&
+ !GetSpecialData(posting_list_used, /*index=*/1).data().is_valid();
+ }
+
+ // Returns false if both special data are invalid or if data start offset
+ // stored in the special data is less than kSpecialDataSize or greater than
+ // posting_list_used->size_in_bytes(). Returns true, otherwise.
+ bool IsPostingListValid(const PostingListUsed* posting_list_used) const;
+
+ // Prepend data to a posting list that is in the ALMOST_FULL state.
+ //
+ // RETURNS:
+ // - OK, if successful
+ // - INVALID_ARGUMENT if data is not less than the previously added data.
+ libtextclassifier3::Status PrependDataToAlmostFull(
+ PostingListUsed* posting_list_used, const JoinDataType& data) const;
+
+ // Prepend data to a posting list that is in the EMPTY state. This will always
+ // succeed because there are no pre-existing data and no validly constructed
+ // posting list could fail to fit one data.
+ void PrependDataToEmpty(PostingListUsed* posting_list_used,
+ const JoinDataType& data) const;
+
+ // Prepend data to a posting list that is in the NOT_FULL state.
+ //
+ // RETURNS:
+ // - OK, if successful
+ // - INVALID_ARGUMENT if data is not less than the previously added data.
+ libtextclassifier3::Status PrependDataToNotFull(
+ PostingListUsed* posting_list_used, const JoinDataType& data,
+ uint32_t offset) const;
+
+ // Returns either 0 (FULL state), sizeof(JoinDataType) (ALMOST_FULL state) or
+ // a byte offset between kSpecialDataSize and
+ // posting_list_used->size_in_bytes() (inclusive) (NOT_FULL state).
+ uint32_t GetStartByteOffset(const PostingListUsed* posting_list_used) const;
+
+ // Sets special data 0 to properly reflect what start byte offset is (see
+ // layout comment for further details).
+ //
+ // Returns false if offset > posting_list_used->size_in_bytes() or offset is
+ // in range (kSpecialDataSize, sizeof(JoinDataType)) or
+ // (sizeof(JoinDataType), 0). True, otherwise.
+ bool SetStartByteOffset(PostingListUsed* posting_list_used,
+ uint32_t offset) const;
+
+ // Helper for MoveFrom/GetData/PopFrontData. Adds limit number of data to out
+ // or all data in the posting list if the posting list contains less than
+ // limit number of data. out can be NULL.
+ //
+ // NOTE: If called with limit=1, pop=true on a posting list that transitioned
+ // from NOT_FULL directly to FULL, GetDataInternal will not return the posting
+ // list to NOT_FULL. Instead it will leave it in a valid state, but it will be
+ // ALMOST_FULL.
+ //
+ // RETURNS:
+ // - OK on success
+ // - INTERNAL_ERROR if the posting list has been corrupted somehow.
+ libtextclassifier3::Status GetDataInternal(
+ const PostingListUsed* posting_list_used, uint32_t limit, bool pop,
+ std::vector<JoinDataType>* out) const;
+
+ // Retrieves the value stored in the index-th special data.
+ //
+ // REQUIRES:
+ // 0 <= index < kNumSpecialData.
+ //
+ // RETURNS:
+ // - A valid SpecialData<JoinDataType>.
+ SpecialDataType GetSpecialData(const PostingListUsed* posting_list_used,
+ uint32_t index) const;
+
+ // Sets the value stored in the index-th special data to special_data.
+ //
+ // REQUIRES:
+ // 0 <= index < kNumSpecialData.
+ void SetSpecialData(PostingListUsed* posting_list_used, uint32_t index,
+ const SpecialDataType& special_data) const;
+
+ // Prepends data to the memory region
+ // [offset - sizeof(JoinDataType), offset - 1] and
+ // returns the new beginning of the region.
+ //
+ // RETURNS:
+ // - The new beginning of the padded region, if successful.
+ // - INVALID_ARGUMENT if data will not fit (uncompressed) between
+ // [kSpecialDataSize, offset - 1]
+ libtextclassifier3::StatusOr<uint32_t> PrependDataUncompressed(
+ PostingListUsed* posting_list_used, const JoinDataType& data,
+ uint32_t offset) const;
+};
+
+template <typename JoinDataType>
+uint32_t PostingListJoinDataSerializer<JoinDataType>::GetBytesUsed(
+ const PostingListUsed* posting_list_used) const {
+ // The special data will be included if they represent actual data. If they
+ // represent the data start offset or the invalid data sentinel, they are not
+ // included.
+ return posting_list_used->size_in_bytes() -
+ GetStartByteOffset(posting_list_used);
+}
+
+template <typename JoinDataType>
+uint32_t
+PostingListJoinDataSerializer<JoinDataType>::GetMinPostingListSizeToFit(
+ const PostingListUsed* posting_list_used) const {
+ if (IsFull(posting_list_used) || IsAlmostFull(posting_list_used)) {
+ // If in either the FULL state or ALMOST_FULL state, this posting list *is*
+ // the minimum size posting list that can fit these data. So just return the
+ // size of the posting list.
+ return posting_list_used->size_in_bytes();
+ }
+
+ // In NOT_FULL state, BytesUsed contains no special data. The minimum sized
+ // posting list that would be guaranteed to fit these data would be
+ // ALMOST_FULL, with kInvalidData in special data 0, the uncompressed data in
+ // special data 1 and the n compressed data in the compressed region.
+ // BytesUsed contains one uncompressed data and n compressed data. Therefore,
+ // fitting these data into a posting list would require BytesUsed plus one
+ // extra data.
+ return GetBytesUsed(posting_list_used) + GetDataTypeBytes();
+}
+
+template <typename JoinDataType>
+void PostingListJoinDataSerializer<JoinDataType>::Clear(
+ PostingListUsed* posting_list_used) const {
+ // Safe to ignore return value because posting_list_used->size_in_bytes() is
+ // a valid argument.
+ SetStartByteOffset(posting_list_used,
+ /*offset=*/posting_list_used->size_in_bytes());
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::MoveFrom(
+ PostingListUsed* dst, PostingListUsed* src) const {
+ ICING_RETURN_ERROR_IF_NULL(dst);
+ ICING_RETURN_ERROR_IF_NULL(src);
+ if (GetMinPostingListSizeToFit(src) > dst->size_in_bytes()) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "src MinPostingListSizeToFit %d must be larger than size %d.",
+ GetMinPostingListSizeToFit(src), dst->size_in_bytes()));
+ }
+
+ if (!IsPostingListValid(dst)) {
+ return absl_ports::FailedPreconditionError(
+ "Dst posting list is in an invalid state and can't be used!");
+ }
+ if (!IsPostingListValid(src)) {
+ return absl_ports::InvalidArgumentError(
+ "Cannot MoveFrom an invalid src posting list!");
+ }
+
+ // Pop just enough data that all of src's compressed data fit in
+ // dst posting_list's compressed area. Then we can memcpy that area.
+ std::vector<JoinDataType> data_arr;
+ while (IsFull(src) || IsAlmostFull(src) ||
+ (dst->size_in_bytes() - kSpecialDataSize < GetBytesUsed(src))) {
+ if (!GetDataInternal(src, /*limit=*/1, /*pop=*/true, &data_arr).ok()) {
+ return absl_ports::AbortedError(
+ "Unable to retrieve data from src posting list.");
+ }
+ }
+
+ // memcpy the area and set up start byte offset.
+ Clear(dst);
+ memcpy(dst->posting_list_buffer() + dst->size_in_bytes() - GetBytesUsed(src),
+ src->posting_list_buffer() + GetStartByteOffset(src),
+ GetBytesUsed(src));
+ // Because we popped all data from src outside of the compressed area and we
+ // guaranteed that GetBytesUsed(src) is less than dst->size_in_bytes() -
+ // kSpecialDataSize. This is guaranteed to be a valid byte offset for the
+ // NOT_FULL state, so ignoring the value is safe.
+ SetStartByteOffset(dst, dst->size_in_bytes() - GetBytesUsed(src));
+
+ // Put back remaining data.
+ for (auto riter = data_arr.rbegin(); riter != data_arr.rend(); ++riter) {
+ // PrependData may return:
+ // - INVALID_ARGUMENT: if data is invalid or not less than the previous data
+ // - RESOURCE_EXHAUSTED
+ // RESOURCE_EXHAUSTED should be impossible because we've already assured
+ // that there is enough room above.
+ ICING_RETURN_IF_ERROR(PrependData(dst, *riter));
+ }
+
+ Clear(src);
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::PrependDataToAlmostFull(
+ PostingListUsed* posting_list_used, const JoinDataType& data) const {
+ SpecialDataType special_data = GetSpecialData(posting_list_used, /*index=*/1);
+ if (data < special_data.data()) {
+ return absl_ports::InvalidArgumentError(
+ "JoinData being prepended must not be smaller than the most recent "
+ "JoinData");
+ }
+
+ // Without compression, prepend a new data into ALMOST_FULL posting list will
+ // change the posting list to FULL state. Therefore, set special data 0
+ // directly.
+ SetSpecialData(posting_list_used, /*index=*/0, SpecialDataType(data));
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename JoinDataType>
+void PostingListJoinDataSerializer<JoinDataType>::PrependDataToEmpty(
+ PostingListUsed* posting_list_used, const JoinDataType& data) const {
+ // First data to be added. Just add verbatim, no compression.
+ if (posting_list_used->size_in_bytes() == kSpecialDataSize) {
+ // First data will be stored at special data 1.
+ // Safe to ignore the return value because 1 < kNumSpecialData
+ SetSpecialData(posting_list_used, /*index=*/1, SpecialDataType(data));
+ // Safe to ignore the return value because sizeof(JoinDataType) is a valid
+ // argument.
+ SetStartByteOffset(posting_list_used, /*offset=*/sizeof(JoinDataType));
+ } else {
+ // Since this is the first data, size != kSpecialDataSize and
+ // size % sizeof(JoinDataType) == 0, we know that there is room to fit
+ // 'data' into the compressed region, so ValueOrDie is safe.
+ uint32_t offset =
+ PrependDataUncompressed(posting_list_used, data,
+ /*offset=*/posting_list_used->size_in_bytes())
+ .ValueOrDie();
+ // Safe to ignore the return value because PrependDataUncompressed is
+ // guaranteed to return a valid offset.
+ SetStartByteOffset(posting_list_used, offset);
+ }
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::PrependDataToNotFull(
+ PostingListUsed* posting_list_used, const JoinDataType& data,
+ uint32_t offset) const {
+ JoinDataType curr = JoinDataType::GetInvalid();
+ memcpy(&curr, posting_list_used->posting_list_buffer() + offset,
+ sizeof(JoinDataType));
+ if (data < curr) {
+ return absl_ports::InvalidArgumentError(
+ "JoinData being prepended must not be smaller than the most recent "
+ "JoinData");
+ }
+
+ if (offset >= kSpecialDataSize + sizeof(JoinDataType)) {
+ offset =
+ PrependDataUncompressed(posting_list_used, data, offset).ValueOrDie();
+ SetStartByteOffset(posting_list_used, offset);
+ } else {
+ // The new data must be put in special data 1.
+ SetSpecialData(posting_list_used, /*index=*/1, SpecialDataType(data));
+ // State ALMOST_FULL. Safe to ignore the return value because
+ // sizeof(JoinDataType) is a valid argument.
+ SetStartByteOffset(posting_list_used, /*offset=*/sizeof(JoinDataType));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::PrependData(
+ PostingListUsed* posting_list_used, const JoinDataType& data) const {
+ if (!data.is_valid()) {
+ return absl_ports::InvalidArgumentError("Cannot prepend an invalid data!");
+ }
+ if (!IsPostingListValid(posting_list_used)) {
+ return absl_ports::FailedPreconditionError(
+ "This PostingListUsed is in an invalid state and can't add any data!");
+ }
+
+ if (IsFull(posting_list_used)) {
+ // State FULL: no space left.
+ return absl_ports::ResourceExhaustedError("No more room for data");
+ } else if (IsAlmostFull(posting_list_used)) {
+ return PrependDataToAlmostFull(posting_list_used, data);
+ } else if (IsEmpty(posting_list_used)) {
+ PrependDataToEmpty(posting_list_used, data);
+ return libtextclassifier3::Status::OK;
+ } else {
+ uint32_t offset = GetStartByteOffset(posting_list_used);
+ return PrependDataToNotFull(posting_list_used, data, offset);
+ }
+}
+
+template <typename JoinDataType>
+libtextclassifier3::StatusOr<uint32_t>
+PostingListJoinDataSerializer<JoinDataType>::PrependDataArray(
+ PostingListUsed* posting_list_used, const JoinDataType* array,
+ uint32_t num_data, bool keep_prepended) const {
+ if (!IsPostingListValid(posting_list_used)) {
+ return 0;
+ }
+
+ uint32_t i;
+ for (i = 0; i < num_data; ++i) {
+ if (!PrependData(posting_list_used, array[i]).ok()) {
+ break;
+ }
+ }
+ if (i != num_data && !keep_prepended) {
+ // Didn't fit. Undo everything and check that we have the same offset as
+ // before. PopFrontData guarantees that it will remove all 'i' data so long
+ // as there are at least 'i' data in the posting list, which we know there
+ // are.
+ ICING_RETURN_IF_ERROR(PopFrontData(posting_list_used, /*num_data=*/i));
+ return 0;
+ }
+ return i;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::StatusOr<std::vector<JoinDataType>>
+PostingListJoinDataSerializer<JoinDataType>::GetData(
+ const PostingListUsed* posting_list_used) const {
+ std::vector<JoinDataType> data_arr_out;
+ ICING_RETURN_IF_ERROR(GetData(posting_list_used, &data_arr_out));
+ return data_arr_out;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status PostingListJoinDataSerializer<JoinDataType>::GetData(
+ const PostingListUsed* posting_list_used,
+ std::vector<JoinDataType>* data_arr_out) const {
+ return GetDataInternal(posting_list_used,
+ /*limit=*/std::numeric_limits<uint32_t>::max(),
+ /*pop=*/false, data_arr_out);
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::PopFrontData(
+ PostingListUsed* posting_list_used, uint32_t num_data) const {
+ if (num_data == 1 && IsFull(posting_list_used)) {
+ // The PL is in FULL state which means that we save 2 uncompressed data in
+ // the 2 special postions. But FULL state may be reached by 2 different
+ // states.
+ // (1) In ALMOST_FULL state
+ // +------------------+-----------------+-----+---------------------------+
+ // |Data::Invalid |1st data |(pad)|(compressed) data |
+ // | | | | |
+ // +------------------+-----------------+-----+---------------------------+
+ // When we prepend another data, we can only put it at special data 0, and
+ // thus get a FULL PL
+ // +------------------+-----------------+-----+---------------------------+
+ // |new 1st data |original 1st data|(pad)|(compressed) data |
+ // | | | | |
+ // +------------------+-----------------+-----+---------------------------+
+ //
+ // (2) In NOT_FULL state
+ // +------------------+-----------------+-------+---------+---------------+
+ // |data-start-offset |Data::Invalid |(pad) |1st data |(compressed) |
+ // | | | | |data |
+ // +------------------+-----------------+-------+---------+---------------+
+ // When we prepend another data, we can reach any of the 3 following
+ // scenarios:
+ // (2.1) NOT_FULL
+ // if the space of pad and original 1st data can accommodate the new 1st
+ // data and the encoded delta value.
+ // +------------------+-----------------+-----+--------+------------------+
+ // |data-start-offset |Data::Invalid |(pad)|new |(compressed) data |
+ // | | | |1st data| |
+ // +------------------+-----------------+-----+--------+------------------+
+ // (2.2) ALMOST_FULL
+ // If the space of pad and original 1st data cannot accommodate the new 1st
+ // data and the encoded delta value but can accommodate the encoded delta
+ // value only. We can put the new 1st data at special position 1.
+ // +------------------+-----------------+---------+-----------------------+
+ // |Data::Invalid |new 1st data |(pad) |(compressed) data |
+ // | | | | |
+ // +------------------+-----------------+---------+-----------------------+
+ // (2.3) FULL
+ // In very rare case, it cannot even accommodate only the encoded delta
+ // value. we can move the original 1st data into special position 1 and the
+ // new 1st data into special position 0. This may happen because we use
+ // VarInt encoding method which may make the encoded value longer (about
+ // 4/3 times of original)
+ // +------------------+-----------------+--------------+------------------+
+ // |new 1st data |original 1st data|(pad) |(compressed) data |
+ // | | | | |
+ // +------------------+-----------------+--------------+------------------+
+ //
+ // Suppose now the PL is in FULL state. But we don't know whether it arrived
+ // this state from NOT_FULL (like (2.3)) or from ALMOST_FULL (like (1)).
+ // We'll return to ALMOST_FULL state like (1) if we simply pop the new 1st
+ // data, but we want to make the prepending operation "reversible". So
+ // there should be some way to return to NOT_FULL if possible. A simple way
+ // to do is:
+ // - Pop 2 data out of the PL to state ALMOST_FULL or NOT_FULL.
+ // - Add the second data ("original 1st data") back.
+ //
+ // Then we can return to the correct original states of (2.1) or (1). This
+ // makes our prepending operation reversible.
+ std::vector<JoinDataType> out;
+
+ // Popping 2 data should never fail because we've just ensured that the
+ // posting list is in the FULL state.
+ ICING_RETURN_IF_ERROR(
+ GetDataInternal(posting_list_used, /*limit=*/2, /*pop=*/true, &out));
+
+ // PrependData should never fail because:
+ // - out[1] is a valid data less than all previous data in the posting list.
+ // - There's no way that the posting list could run out of room because it
+ // previously stored these 2 data.
+ ICING_RETURN_IF_ERROR(PrependData(posting_list_used, out[1]));
+ } else if (num_data > 0) {
+ return GetDataInternal(posting_list_used, /*limit=*/num_data, /*pop=*/true,
+ /*out=*/nullptr);
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::Status
+PostingListJoinDataSerializer<JoinDataType>::GetDataInternal(
+ const PostingListUsed* posting_list_used, uint32_t limit, bool pop,
+ std::vector<JoinDataType>* out) const {
+ uint32_t offset = GetStartByteOffset(posting_list_used);
+ uint32_t count = 0;
+
+ // First traverse the first two special positions.
+ while (count < limit && offset < kSpecialDataSize) {
+ // offset / sizeof(JoinDataType) < kNumSpecialData
+ // because of the check above.
+ SpecialDataType special_data = GetSpecialData(
+ posting_list_used, /*index=*/offset / sizeof(JoinDataType));
+ if (out != nullptr) {
+ out->push_back(special_data.data());
+ }
+ offset += sizeof(JoinDataType);
+ ++count;
+ }
+
+ // - We don't compress the data.
+ // - The posting list size is a multiple of data type bytes.
+ // So offset of the first non-special data is guaranteed to be at
+ // kSpecialDataSize if in ALMOST_FULL or FULL state. In fact, we must not
+ // apply padding skipping logic here when still storing uncompressed data,
+ // because in this case 0 bytes are meanful (e.g. inverted doc id byte = 0).
+ while (count < limit && offset < posting_list_used->size_in_bytes()) {
+ JoinDataType data = JoinDataType::GetInvalid();
+ memcpy(&data, posting_list_used->posting_list_buffer() + offset,
+ sizeof(JoinDataType));
+ offset += sizeof(JoinDataType);
+ if (out != nullptr) {
+ out->push_back(data);
+ }
+ ++count;
+ }
+
+ if (pop) {
+ PostingListUsed* mutable_posting_list_used =
+ const_cast<PostingListUsed*>(posting_list_used);
+ // Modify the posting list so that we pop all data actually traversed.
+ if (offset >= kSpecialDataSize &&
+ offset < posting_list_used->size_in_bytes()) {
+ memset(
+ mutable_posting_list_used->posting_list_buffer() + kSpecialDataSize,
+ 0, offset - kSpecialDataSize);
+ }
+ SetStartByteOffset(mutable_posting_list_used, offset);
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename JoinDataType>
+typename PostingListJoinDataSerializer<JoinDataType>::SpecialDataType
+PostingListJoinDataSerializer<JoinDataType>::GetSpecialData(
+ const PostingListUsed* posting_list_used, uint32_t index) const {
+ // It is ok to temporarily construct a SpecialData with offset = 0 since we're
+ // going to overwrite it by memcpy.
+ SpecialDataType special_data(0);
+ memcpy(&special_data,
+ posting_list_used->posting_list_buffer() +
+ index * sizeof(SpecialDataType),
+ sizeof(SpecialDataType));
+ return special_data;
+}
+
+template <typename JoinDataType>
+void PostingListJoinDataSerializer<JoinDataType>::SetSpecialData(
+ PostingListUsed* posting_list_used, uint32_t index,
+ const SpecialDataType& special_data) const {
+ memcpy(posting_list_used->posting_list_buffer() +
+ index * sizeof(SpecialDataType),
+ &special_data, sizeof(SpecialDataType));
+}
+
+template <typename JoinDataType>
+bool PostingListJoinDataSerializer<JoinDataType>::IsPostingListValid(
+ const PostingListUsed* posting_list_used) const {
+ if (IsAlmostFull(posting_list_used)) {
+ // Special data 1 should hold a valid data.
+ if (!GetSpecialData(posting_list_used, /*index=*/1).data().is_valid()) {
+ ICING_LOG(ERROR)
+ << "Both special data cannot be invalid at the same time.";
+ return false;
+ }
+ } else if (!IsFull(posting_list_used)) {
+ // NOT_FULL. Special data 0 should hold a valid offset.
+ SpecialDataType special_data =
+ GetSpecialData(posting_list_used, /*index=*/0);
+ if (special_data.data_start_offset() > posting_list_used->size_in_bytes() ||
+ special_data.data_start_offset() < kSpecialDataSize) {
+ ICING_LOG(ERROR) << "Offset: " << special_data.data_start_offset()
+ << " size: " << posting_list_used->size_in_bytes()
+ << " sp size: " << kSpecialDataSize;
+ return false;
+ }
+ }
+ return true;
+}
+
+template <typename JoinDataType>
+uint32_t PostingListJoinDataSerializer<JoinDataType>::GetStartByteOffset(
+ const PostingListUsed* posting_list_used) const {
+ if (IsFull(posting_list_used)) {
+ return 0;
+ } else if (IsAlmostFull(posting_list_used)) {
+ return sizeof(JoinDataType);
+ } else {
+ return GetSpecialData(posting_list_used, /*index=*/0).data_start_offset();
+ }
+}
+
+template <typename JoinDataType>
+bool PostingListJoinDataSerializer<JoinDataType>::SetStartByteOffset(
+ PostingListUsed* posting_list_used, uint32_t offset) const {
+ if (offset > posting_list_used->size_in_bytes()) {
+ ICING_LOG(ERROR) << "offset cannot be a value greater than size "
+ << posting_list_used->size_in_bytes() << ". offset is "
+ << offset << ".";
+ return false;
+ }
+ if (offset < kSpecialDataSize && offset > sizeof(JoinDataType)) {
+ ICING_LOG(ERROR) << "offset cannot be a value between ("
+ << sizeof(JoinDataType) << ", " << kSpecialDataSize
+ << "). offset is " << offset << ".";
+ return false;
+ }
+ if (offset < sizeof(JoinDataType) && offset != 0) {
+ ICING_LOG(ERROR) << "offset cannot be a value between (0, "
+ << sizeof(JoinDataType) << "). offset is " << offset
+ << ".";
+ return false;
+ }
+
+ if (offset >= kSpecialDataSize) {
+ // NOT_FULL state.
+ SetSpecialData(posting_list_used, /*index=*/0, SpecialDataType(offset));
+ SetSpecialData(posting_list_used, /*index=*/1,
+ SpecialDataType(JoinDataType::GetInvalid()));
+ } else if (offset == sizeof(JoinDataType)) {
+ // ALMOST_FULL state.
+ SetSpecialData(posting_list_used, /*index=*/0,
+ SpecialDataType(JoinDataType::GetInvalid()));
+ }
+ // Nothing to do for the FULL state - the offset isn't actually stored
+ // anywhere and both 2 special data hold valid data.
+ return true;
+}
+
+template <typename JoinDataType>
+libtextclassifier3::StatusOr<uint32_t>
+PostingListJoinDataSerializer<JoinDataType>::PrependDataUncompressed(
+ PostingListUsed* posting_list_used, const JoinDataType& data,
+ uint32_t offset) const {
+ if (offset < kSpecialDataSize + sizeof(JoinDataType)) {
+ return absl_ports::InvalidArgumentError(IcingStringUtil::StringPrintf(
+ "Not enough room to prepend JoinData at offset %d.", offset));
+ }
+ offset -= sizeof(JoinDataType);
+ memcpy(posting_list_used->posting_list_buffer() + offset, &data,
+ sizeof(JoinDataType));
+ return offset;
+}
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_JOIN_POSTING_LIST_JOIN_DATA_SERIALIZER_H_
diff --git a/icing/join/posting-list-join-data-serializer_test.cc b/icing/join/posting-list-join-data-serializer_test.cc
new file mode 100644
index 0000000..20137b6
--- /dev/null
+++ b/icing/join/posting-list-join-data-serializer_test.cc
@@ -0,0 +1,653 @@
+// Copyright (C) 2022 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/join/posting-list-join-data-serializer.h"
+
+#include <algorithm>
+#include <iterator>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/file/posting_list/posting-list-used.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/testing/common-matchers.h"
+
+using testing::ElementsAre;
+using testing::ElementsAreArray;
+using testing::Eq;
+using testing::IsEmpty;
+using testing::SizeIs;
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+TEST(PostingListJoinDataSerializerTest, GetMinPostingListSizeToFitNotNull) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size =
+ 2551 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2))),
+ IsOk());
+ EXPECT_THAT(
+ serializer.GetMinPostingListSizeToFit(&pl_used),
+ Eq(2 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5))),
+ IsOk());
+ EXPECT_THAT(
+ serializer.GetMinPostingListSizeToFit(&pl_used),
+ Eq(3 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+}
+
+TEST(PostingListJoinDataSerializerTest, GetMinPostingListSizeToFitAlmostFull) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 3 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2))),
+ IsOk());
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5))),
+ IsOk());
+ EXPECT_THAT(serializer.GetMinPostingListSizeToFit(&pl_used), Eq(size));
+}
+
+TEST(PostingListJoinDataSerializerTest, GetMinPostingListSizeToFitFull) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 3 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2))),
+ IsOk());
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5))),
+ IsOk());
+ ASSERT_THAT(
+ serializer.PrependData(
+ &pl_used,
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10))),
+ IsOk());
+ EXPECT_THAT(serializer.GetMinPostingListSizeToFit(&pl_used), Eq(size));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependDataNotFull) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size =
+ 2551 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ // Make used.
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data0(
+ /*document_id=*/0,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data0), IsOk());
+ // Size = sizeof(uncompressed data0)
+ int expected_size =
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used), IsOkAndHolds(ElementsAre(data0)));
+
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/5));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data1), IsOk());
+ // Size = sizeof(uncompressed data1)
+ // + sizeof(uncompressed data0)
+ expected_size += sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data1, data0)));
+
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data2(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data2), IsOk());
+ // Size = sizeof(uncompressed data2)
+ // + sizeof(uncompressed data1)
+ // + sizeof(uncompressed data0)
+ expected_size += sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data2, data1, data0)));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependDataAlmostFull) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 4 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ // Fill up the compressed region.
+ // Transitions:
+ // Adding data0: EMPTY -> NOT_FULL
+ // Adding data1: NOT_FULL -> NOT_FULL
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data0(
+ /*document_id=*/0,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2));
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/5));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data0), IsOk());
+ EXPECT_THAT(serializer.PrependData(&pl_used, data1), IsOk());
+ int expected_size =
+ 2 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data1, data0)));
+
+ // Add one more data to transition NOT_FULL -> ALMOST_FULL
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data2(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data2), IsOk());
+ expected_size =
+ 3 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data2, data1, data0)));
+
+ // Add one more data to transition ALMOST_FULL -> FULL
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data3(
+ /*document_id=*/3, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/0));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data3), IsOk());
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data3, data2, data1, data0)));
+
+ // The posting list is FULL. Adding another data should fail.
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data4(
+ /*document_id=*/4, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/1234));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data4),
+ StatusIs(libtextclassifier3::StatusCode::RESOURCE_EXHAUSTED));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependSmallerDataShouldFail) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 4 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data(
+ /*document_id=*/100,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2));
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> smaller_data(
+ /*document_id=*/99,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2));
+
+ // NOT_FULL -> NOT_FULL
+ ASSERT_THAT(serializer.PrependData(&pl_used, data), IsOk());
+ EXPECT_THAT(serializer.PrependData(&pl_used, smaller_data),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ // NOT_FULL -> ALMOST_FULL
+ ASSERT_THAT(serializer.PrependData(&pl_used, data), IsOk());
+ EXPECT_THAT(serializer.PrependData(&pl_used, smaller_data),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ // ALMOST_FULL -> FULL
+ ASSERT_THAT(serializer.PrependData(&pl_used, data), IsOk());
+ EXPECT_THAT(serializer.PrependData(&pl_used, smaller_data),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependDataPostingListUsedMinSize) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ // PL State: EMPTY
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(0));
+ EXPECT_THAT(serializer.GetData(&pl_used), IsOkAndHolds(IsEmpty()));
+
+ // Add a data. PL should shift to ALMOST_FULL state
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data0(
+ /*document_id=*/0,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data0), IsOk());
+ // Size = sizeof(uncompressed data0)
+ int expected_size =
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used), IsOkAndHolds(ElementsAre(data0)));
+
+ // Add another data. PL should shift to FULL state.
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data1(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/5));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data1), IsOk());
+ // Size = sizeof(uncompressed data1) + sizeof(uncompressed data0)
+ expected_size += sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used), Eq(expected_size));
+ EXPECT_THAT(serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAre(data1, data0)));
+
+ // The posting list is FULL. Adding another data should fail.
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier> data2(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10));
+ EXPECT_THAT(serializer.PrependData(&pl_used, data2),
+ StatusIs(libtextclassifier3::StatusCode::RESOURCE_EXHAUSTED));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependDataArrayDoNotKeepPrepended) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 6 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_in;
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_pushed;
+
+ // Add 3 data. The PL is in the empty state and should be able to fit all 3
+ // data without issue, transitioning the PL from EMPTY -> NOT_FULL.
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/5)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/10)));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_in.size()));
+ std::move(data_in.begin(), data_in.end(), std::back_inserter(data_pushed));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+
+ // Add 2 data. The PL should transition from NOT_FULL to ALMOST_FULL.
+ data_in.clear();
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/0)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/4, NamespaceFingerprintIdentifier(/*namespace_id=*/0,
+ /*fingerprint=*/1234)));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_in.size()));
+ std::move(data_in.begin(), data_in.end(), std::back_inserter(data_pushed));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+
+ // Add 2 data. The PL should remain ALMOST_FULL since the remaining space can
+ // only fit 1 data.
+ data_in.clear();
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, NamespaceFingerprintIdentifier(/*namespace_id=*/2,
+ /*fingerprint=*/99)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/63)));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(0));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+
+ // Add 1 data. The PL should transition from ALMOST_FULL to FULL.
+ data_in.pop_back();
+ ASSERT_THAT(data_in, SizeIs(1));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_in.size()));
+ std::move(data_in.begin(), data_in.end(), std::back_inserter(data_pushed));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+}
+
+TEST(PostingListJoinDataSerializerTest, PrependDataArrayKeepPrepended) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 6 * sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_in;
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_pushed;
+
+ // Add 3 data. The PL is in the empty state and should be able to fit all 3
+ // data without issue, transitioning the PL from EMPTY -> NOT_FULL.
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/2)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/5)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/10)));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/true),
+ IsOkAndHolds(data_in.size()));
+ std::move(data_in.begin(), data_in.end(), std::back_inserter(data_pushed));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+
+ // Add 4 data. The PL should prepend 3 data and transition from NOT_FULL to
+ // FULL.
+ data_in.clear();
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/1, /*fingerprint=*/0)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/4, NamespaceFingerprintIdentifier(/*namespace_id=*/0,
+ /*fingerprint=*/1234)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, NamespaceFingerprintIdentifier(/*namespace_id=*/2,
+ /*fingerprint=*/99)));
+ data_in.push_back(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/63)));
+ EXPECT_THAT(
+ serializer.PrependDataArray(&pl_used, data_in.data(), data_in.size(),
+ /*keep_prepended=*/true),
+ IsOkAndHolds(3));
+ data_in.pop_back();
+ ASSERT_THAT(data_in, SizeIs(3));
+ std::move(data_in.begin(), data_in.end(), std::back_inserter(data_pushed));
+ EXPECT_THAT(serializer.GetBytesUsed(&pl_used),
+ Eq(data_pushed.size() *
+ sizeof(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>)));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_pushed.rbegin(), data_pushed.rend())));
+}
+
+TEST(PostingListJoinDataSerializerTest, MoveFrom) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 3 * serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used1,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr1 =
+ {DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used1, data_arr1.data(), data_arr1.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr1.size()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used2,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr2 =
+ {DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/0)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/4,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/0,
+ /*fingerprint=*/1234)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/2,
+ /*fingerprint=*/99))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used2, data_arr2.data(), data_arr2.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr2.size()));
+
+ EXPECT_THAT(serializer.MoveFrom(/*dst=*/&pl_used2, /*src=*/&pl_used1),
+ IsOk());
+ EXPECT_THAT(
+ serializer.GetData(&pl_used2),
+ IsOkAndHolds(ElementsAreArray(data_arr1.rbegin(), data_arr1.rend())));
+ EXPECT_THAT(serializer.GetData(&pl_used1), IsOkAndHolds(IsEmpty()));
+}
+
+TEST(PostingListJoinDataSerializerTest, MoveToNullReturnsFailedPrecondition) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 3 * serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr = {
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used, data_arr.data(), data_arr.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr.size()));
+
+ EXPECT_THAT(serializer.MoveFrom(/*dst=*/&pl_used, /*src=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_arr.rbegin(), data_arr.rend())));
+
+ EXPECT_THAT(serializer.MoveFrom(/*dst=*/nullptr, /*src=*/&pl_used),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_arr.rbegin(), data_arr.rend())));
+}
+
+TEST(PostingListJoinDataSerializerTest, MoveToPostingListTooSmall) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size1 = 3 * serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used1,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size1));
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr1 =
+ {DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/0)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/4,
+ NamespaceFingerprintIdentifier(/*namespace_id=*/0,
+ /*fingerprint=*/1234))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used1, data_arr1.data(), data_arr1.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr1.size()));
+
+ int size2 = serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used2,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size2));
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr2 =
+ {DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/99))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used2, data_arr2.data(), data_arr2.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr2.size()));
+
+ EXPECT_THAT(serializer.MoveFrom(/*dst=*/&pl_used2, /*src=*/&pl_used1),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used1),
+ IsOkAndHolds(ElementsAreArray(data_arr1.rbegin(), data_arr1.rend())));
+ EXPECT_THAT(
+ serializer.GetData(&pl_used2),
+ IsOkAndHolds(ElementsAreArray(data_arr2.rbegin(), data_arr2.rend())));
+}
+
+TEST(PostingListJoinDataSerializerTest, PopFrontData) {
+ PostingListJoinDataSerializer<
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>>
+ serializer;
+
+ int size = 2 * serializer.GetMinPostingListSize();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ PostingListUsed pl_used,
+ PostingListUsed::CreateFromUnitializedRegion(&serializer, size));
+
+ std::vector<DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>> data_arr = {
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/2)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/5)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/10))};
+ ASSERT_THAT(
+ serializer.PrependDataArray(&pl_used, data_arr.data(), data_arr.size(),
+ /*keep_prepended=*/false),
+ IsOkAndHolds(data_arr.size()));
+ ASSERT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_arr.rbegin(), data_arr.rend())));
+
+ // Now, pop the last data. The posting list should contain the first three
+ // data.
+ EXPECT_THAT(serializer.PopFrontData(&pl_used, /*num_data=*/1), IsOk());
+ data_arr.pop_back();
+ EXPECT_THAT(
+ serializer.GetData(&pl_used),
+ IsOkAndHolds(ElementsAreArray(data_arr.rbegin(), data_arr.rend())));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/join/qualified-id-join-index.cc b/icing/join/qualified-id-join-index-impl-v1.cc
index 07b5627..cdcb5a9 100644
--- a/icing/join/qualified-id-join-index.cc
+++ b/icing/join/qualified-id-join-index-impl-v1.cc
@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-#include "icing/join/qualified-id-join-index.h"
+#include "icing/join/qualified-id-join-index-impl-v1.h"
#include <cstring>
#include <memory>
@@ -29,9 +29,11 @@
#include "icing/file/filesystem.h"
#include "icing/file/memory-mapped-file.h"
#include "icing/join/doc-join-info.h"
+#include "icing/join/qualified-id-join-index.h"
#include "icing/store/document-id.h"
#include "icing/store/dynamic-trie-key-mapper.h"
#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-id.h"
#include "icing/store/persistent-hash-map-key-mapper.h"
#include "icing/util/crc32.h"
#include "icing/util/encode-util.h"
@@ -74,17 +76,20 @@ std::string GetQualifiedIdStoragePath(std::string_view working_path) {
} // namespace
-/* static */ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
-QualifiedIdJoinIndex::Create(const Filesystem& filesystem,
- std::string working_path, bool pre_mapping_fbv,
- bool use_persistent_hash_map) {
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+QualifiedIdJoinIndexImplV1::Create(const Filesystem& filesystem,
+ std::string working_path,
+ bool pre_mapping_fbv,
+ bool use_persistent_hash_map) {
if (!filesystem.FileExists(GetMetadataFilePath(working_path).c_str()) ||
!filesystem.DirectoryExists(
GetDocJoinInfoMapperPath(working_path).c_str()) ||
!filesystem.FileExists(GetQualifiedIdStoragePath(working_path).c_str())) {
// Discard working_path if any file/directory is missing, and reinitialize.
if (filesystem.DirectoryExists(working_path.c_str())) {
- ICING_RETURN_IF_ERROR(Discard(filesystem, working_path));
+ ICING_RETURN_IF_ERROR(
+ QualifiedIdJoinIndex::Discard(filesystem, working_path));
}
return InitializeNewFiles(filesystem, std::move(working_path),
pre_mapping_fbv, use_persistent_hash_map);
@@ -93,7 +98,7 @@ QualifiedIdJoinIndex::Create(const Filesystem& filesystem,
pre_mapping_fbv, use_persistent_hash_map);
}
-QualifiedIdJoinIndex::~QualifiedIdJoinIndex() {
+QualifiedIdJoinIndexImplV1::~QualifiedIdJoinIndexImplV1() {
if (!PersistToDisk().ok()) {
ICING_LOG(WARNING) << "Failed to persist qualified id type joinable index "
"to disk while destructing "
@@ -101,7 +106,7 @@ QualifiedIdJoinIndex::~QualifiedIdJoinIndex() {
}
}
-libtextclassifier3::Status QualifiedIdJoinIndex::Put(
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::Put(
const DocJoinInfo& doc_join_info, std::string_view ref_qualified_id_str) {
SetDirty();
@@ -128,7 +133,7 @@ libtextclassifier3::Status QualifiedIdJoinIndex::Put(
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::StatusOr<std::string_view> QualifiedIdJoinIndex::Get(
+libtextclassifier3::StatusOr<std::string_view> QualifiedIdJoinIndexImplV1::Get(
const DocJoinInfo& doc_join_info) const {
if (!doc_join_info.is_valid()) {
return absl_ports::InvalidArgumentError(
@@ -144,11 +149,13 @@ libtextclassifier3::StatusOr<std::string_view> QualifiedIdJoinIndex::Get(
return std::string_view(data, strlen(data));
}
-libtextclassifier3::Status QualifiedIdJoinIndex::Optimize(
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::Optimize(
const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
DocumentId new_last_added_document_id) {
std::string temp_working_path = working_path_ + "_temp";
- ICING_RETURN_IF_ERROR(Discard(filesystem_, temp_working_path));
+ ICING_RETURN_IF_ERROR(
+ QualifiedIdJoinIndex::Discard(filesystem_, temp_working_path));
DestructibleDirectory temp_working_path_ddir(&filesystem_,
std::move(temp_working_path));
@@ -162,9 +169,10 @@ libtextclassifier3::Status QualifiedIdJoinIndex::Optimize(
// Transfer all data from the current to new qualified id type joinable
// index. Also PersistToDisk and destruct the instance after finishing, so
// we can safely swap directories later.
- ICING_ASSIGN_OR_RETURN(std::unique_ptr<QualifiedIdJoinIndex> new_index,
- Create(filesystem_, temp_working_path_ddir.dir(),
- pre_mapping_fbv_, use_persistent_hash_map_));
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> new_index,
+ Create(filesystem_, temp_working_path_ddir.dir(), pre_mapping_fbv_,
+ use_persistent_hash_map_));
ICING_RETURN_IF_ERROR(
TransferIndex(document_id_old_to_new, new_index.get()));
new_index->set_last_added_document_id(new_last_added_document_id);
@@ -216,7 +224,7 @@ libtextclassifier3::Status QualifiedIdJoinIndex::Optimize(
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::Status QualifiedIdJoinIndex::Clear() {
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::Clear() {
SetDirty();
doc_join_info_mapper_.reset();
@@ -252,11 +260,12 @@ libtextclassifier3::Status QualifiedIdJoinIndex::Clear() {
return libtextclassifier3::Status::OK;
}
-/* static */ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
-QualifiedIdJoinIndex::InitializeNewFiles(const Filesystem& filesystem,
- std::string&& working_path,
- bool pre_mapping_fbv,
- bool use_persistent_hash_map) {
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+QualifiedIdJoinIndexImplV1::InitializeNewFiles(const Filesystem& filesystem,
+ std::string&& working_path,
+ bool pre_mapping_fbv,
+ bool use_persistent_hash_map) {
// Create working directory.
if (!filesystem.CreateDirectoryRecursively(working_path.c_str())) {
return absl_ports::InternalError(
@@ -291,8 +300,8 @@ QualifiedIdJoinIndex::InitializeNewFiles(const Filesystem& filesystem,
/*pre_mapping_mmap_size=*/pre_mapping_fbv ? 1024 * 1024 : 0));
// Create instance.
- auto new_index =
- std::unique_ptr<QualifiedIdJoinIndex>(new QualifiedIdJoinIndex(
+ auto new_index = std::unique_ptr<QualifiedIdJoinIndexImplV1>(
+ new QualifiedIdJoinIndexImplV1(
filesystem, std::move(working_path),
/*metadata_buffer=*/std::make_unique<uint8_t[]>(kMetadataFileSize),
std::move(doc_join_info_mapper), std::move(qualified_id_storage),
@@ -307,11 +316,11 @@ QualifiedIdJoinIndex::InitializeNewFiles(const Filesystem& filesystem,
return new_index;
}
-/* static */ libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
-QualifiedIdJoinIndex::InitializeExistingFiles(const Filesystem& filesystem,
- std::string&& working_path,
- bool pre_mapping_fbv,
- bool use_persistent_hash_map) {
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+QualifiedIdJoinIndexImplV1::InitializeExistingFiles(
+ const Filesystem& filesystem, std::string&& working_path,
+ bool pre_mapping_fbv, bool use_persistent_hash_map) {
// PRead metadata file.
auto metadata_buffer = std::make_unique<uint8_t[]>(kMetadataFileSize);
if (!filesystem.PRead(GetMetadataFilePath(working_path).c_str(),
@@ -358,8 +367,8 @@ QualifiedIdJoinIndex::InitializeExistingFiles(const Filesystem& filesystem,
/*pre_mapping_mmap_size=*/pre_mapping_fbv ? 1024 * 1024 : 0));
// Create instance.
- auto type_joinable_index =
- std::unique_ptr<QualifiedIdJoinIndex>(new QualifiedIdJoinIndex(
+ auto type_joinable_index = std::unique_ptr<QualifiedIdJoinIndexImplV1>(
+ new QualifiedIdJoinIndexImplV1(
filesystem, std::move(working_path), std::move(metadata_buffer),
std::move(doc_join_info_mapper), std::move(qualified_id_storage),
pre_mapping_fbv, use_persistent_hash_map));
@@ -374,9 +383,9 @@ QualifiedIdJoinIndex::InitializeExistingFiles(const Filesystem& filesystem,
return type_joinable_index;
}
-libtextclassifier3::Status QualifiedIdJoinIndex::TransferIndex(
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::TransferIndex(
const std::vector<DocumentId>& document_id_old_to_new,
- QualifiedIdJoinIndex* new_index) const {
+ QualifiedIdJoinIndexImplV1* new_index) const {
std::unique_ptr<KeyMapper<int32_t>::Iterator> iter =
doc_join_info_mapper_->GetIterator();
while (iter->Advance()) {
@@ -404,7 +413,7 @@ libtextclassifier3::Status QualifiedIdJoinIndex::TransferIndex(
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::Status QualifiedIdJoinIndex::PersistMetadataToDisk(
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::PersistMetadataToDisk(
bool force) {
if (!force && !is_info_dirty() && !is_storage_dirty()) {
return libtextclassifier3::Status::OK;
@@ -429,7 +438,7 @@ libtextclassifier3::Status QualifiedIdJoinIndex::PersistMetadataToDisk(
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::Status QualifiedIdJoinIndex::PersistStoragesToDisk(
+libtextclassifier3::Status QualifiedIdJoinIndexImplV1::PersistStoragesToDisk(
bool force) {
if (!force && !is_storage_dirty()) {
return libtextclassifier3::Status::OK;
@@ -440,8 +449,8 @@ libtextclassifier3::Status QualifiedIdJoinIndex::PersistStoragesToDisk(
return libtextclassifier3::Status::OK;
}
-libtextclassifier3::StatusOr<Crc32> QualifiedIdJoinIndex::ComputeInfoChecksum(
- bool force) {
+libtextclassifier3::StatusOr<Crc32>
+QualifiedIdJoinIndexImplV1::ComputeInfoChecksum(bool force) {
if (!force && !is_info_dirty()) {
return Crc32(crcs().component_crcs.info_crc);
}
@@ -450,7 +459,7 @@ libtextclassifier3::StatusOr<Crc32> QualifiedIdJoinIndex::ComputeInfoChecksum(
}
libtextclassifier3::StatusOr<Crc32>
-QualifiedIdJoinIndex::ComputeStoragesChecksum(bool force) {
+QualifiedIdJoinIndexImplV1::ComputeStoragesChecksum(bool force) {
if (!force && !is_storage_dirty()) {
return Crc32(crcs().component_crcs.storages_crc);
}
diff --git a/icing/join/qualified-id-join-index-impl-v1.h b/icing/join/qualified-id-join-index-impl-v1.h
new file mode 100644
index 0000000..9314602
--- /dev/null
+++ b/icing/join/qualified-id-join-index-impl-v1.h
@@ -0,0 +1,327 @@
+// Copyright (C) 2023 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_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V1_H_
+#define ICING_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V1_H_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/file-backed-vector.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/persistent-storage.h"
+#include "icing/join/doc-join-info.h"
+#include "icing/join/qualified-id-join-index.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
+#include "icing/util/crc32.h"
+
+namespace icing {
+namespace lib {
+
+// QualifiedIdJoinIndexImplV1: a class to maintain data mapping DocJoinInfo to
+// joinable qualified ids and delete propagation info.
+class QualifiedIdJoinIndexImplV1 : public QualifiedIdJoinIndex {
+ public:
+ struct Info {
+ static constexpr int32_t kMagic = 0x48cabdc6;
+
+ int32_t magic;
+ DocumentId last_added_document_id;
+
+ Crc32 ComputeChecksum() const {
+ return Crc32(
+ std::string_view(reinterpret_cast<const char*>(this), sizeof(Info)));
+ }
+ } __attribute__((packed));
+ static_assert(sizeof(Info) == 8, "");
+
+ // Metadata file layout: <Crcs><Info>
+ static constexpr int32_t kCrcsMetadataBufferOffset = 0;
+ static constexpr int32_t kInfoMetadataBufferOffset =
+ static_cast<int32_t>(sizeof(Crcs));
+ static constexpr int32_t kMetadataFileSize = sizeof(Crcs) + sizeof(Info);
+ static_assert(kMetadataFileSize == 20, "");
+
+ // Creates a QualifiedIdJoinIndexImplV1 instance to store qualified ids for
+ // future joining search. If any of the underlying file is missing, then
+ // delete the whole working_path and (re)initialize with new ones. Otherwise
+ // initialize and create the instance by existing files.
+ //
+ // filesystem: Object to make system level calls
+ // working_path: Specifies the working path for PersistentStorage.
+ // QualifiedIdJoinIndexImplV1 uses working path as working
+ // directory and all related files will be stored under this
+ // directory. It takes full ownership and of working_path_,
+ // including creation/deletion. It is the caller's
+ // responsibility to specify correct working path and avoid
+ // mixing different persistent storages together under the same
+ // path. Also the caller has the ownership for the parent
+ // directory of working_path_, and it is responsible for parent
+ // directory creation/deletion. See PersistentStorage for more
+ // details about the concept of working_path.
+ // pre_mapping_fbv: flag indicating whether memory map max possible file size
+ // for underlying FileBackedVector before growing the actual
+ // file size.
+ // use_persistent_hash_map: flag indicating whether use persistent hash map as
+ // the key mapper (if false, then fall back to
+ // dynamic trie key mapper).
+ //
+ // Returns:
+ // - FAILED_PRECONDITION_ERROR if the file checksum doesn't match the stored
+ // checksum
+ // - INTERNAL_ERROR on I/O errors
+ // - Any KeyMapper errors
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+ Create(const Filesystem& filesystem, std::string working_path,
+ bool pre_mapping_fbv, bool use_persistent_hash_map);
+
+ // Delete copy and move constructor/assignment operator.
+ QualifiedIdJoinIndexImplV1(const QualifiedIdJoinIndexImplV1&) = delete;
+ QualifiedIdJoinIndexImplV1& operator=(const QualifiedIdJoinIndexImplV1&) =
+ delete;
+
+ QualifiedIdJoinIndexImplV1(QualifiedIdJoinIndexImplV1&&) = delete;
+ QualifiedIdJoinIndexImplV1& operator=(QualifiedIdJoinIndexImplV1&&) = delete;
+
+ ~QualifiedIdJoinIndexImplV1() override;
+
+ // v2 only API. Returns UNIMPLEMENTED_ERROR.
+ libtextclassifier3::Status Put(SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id,
+ DocumentId document_id,
+ std::vector<NamespaceFingerprintIdentifier>&&
+ ref_namespace_fingerprint_ids) override {
+ return absl_ports::UnimplementedError("This API is not supported in V2");
+ }
+
+ // v2 only API. Returns UNIMPLEMENTED_ERROR.
+ libtextclassifier3::StatusOr<std::unique_ptr<JoinDataIteratorBase>>
+ GetIterator(SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) const override {
+ return absl_ports::UnimplementedError("This API is not supported in V2");
+ }
+
+ // Puts a new data into index: DocJoinInfo (DocumentId, JoinablePropertyId)
+ // references to ref_qualified_id_str (the identifier of another document).
+ //
+ // REQUIRES: ref_qualified_id_str contains no '\0'.
+ //
+ // Returns:
+ // - OK on success
+ // - INVALID_ARGUMENT_ERROR if doc_join_info is invalid
+ // - Any KeyMapper errors
+ libtextclassifier3::Status Put(
+ const DocJoinInfo& doc_join_info,
+ std::string_view ref_qualified_id_str) override;
+
+ // Gets the referenced document's qualified id string by DocJoinInfo.
+ //
+ // Returns:
+ // - A qualified id string referenced by the given DocJoinInfo (DocumentId,
+ // JoinablePropertyId) on success
+ // - INVALID_ARGUMENT_ERROR if doc_join_info is invalid
+ // - NOT_FOUND_ERROR if doc_join_info doesn't exist
+ // - Any KeyMapper errors
+ libtextclassifier3::StatusOr<std::string_view> Get(
+ const DocJoinInfo& doc_join_info) const override;
+
+ // Reduces internal file sizes by reclaiming space and ids of deleted
+ // documents. Qualified id type joinable index will convert all entries to the
+ // new document ids.
+ //
+ // - document_id_old_to_new: a map for converting old document id to new
+ // document id.
+ // - namespace_id_old_to_new: a map for converting old namespace id to new
+ // namespace id. It is unused in this implementation since we store raw
+ // qualified id string (which contains raw namespace string).
+ // - new_last_added_document_id: will be used to update the last added
+ // document id in the qualified id type joinable
+ // index.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error. This could potentially leave the index in
+ // an invalid state and the caller should handle it properly (e.g. discard
+ // and rebuild)
+ libtextclassifier3::Status Optimize(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ DocumentId new_last_added_document_id) override;
+
+ // Clears all data and set last_added_document_id to kInvalidDocumentId.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status Clear() override;
+
+ bool is_v2() const override { return false; }
+
+ int32_t size() const override { return doc_join_info_mapper_->num_keys(); }
+
+ bool empty() const override { return size() == 0; }
+
+ DocumentId last_added_document_id() const override {
+ return info().last_added_document_id;
+ }
+
+ void set_last_added_document_id(DocumentId document_id) override {
+ SetInfoDirty();
+
+ Info& info_ref = info();
+ if (info_ref.last_added_document_id == kInvalidDocumentId ||
+ document_id > info_ref.last_added_document_id) {
+ info_ref.last_added_document_id = document_id;
+ }
+ }
+
+ private:
+ explicit QualifiedIdJoinIndexImplV1(
+ const Filesystem& filesystem, std::string&& working_path,
+ std::unique_ptr<uint8_t[]> metadata_buffer,
+ std::unique_ptr<KeyMapper<int32_t>> doc_join_info_mapper,
+ std::unique_ptr<FileBackedVector<char>> qualified_id_storage,
+ bool pre_mapping_fbv, bool use_persistent_hash_map)
+ : QualifiedIdJoinIndex(filesystem, std::move(working_path)),
+ metadata_buffer_(std::move(metadata_buffer)),
+ doc_join_info_mapper_(std::move(doc_join_info_mapper)),
+ qualified_id_storage_(std::move(qualified_id_storage)),
+ pre_mapping_fbv_(pre_mapping_fbv),
+ use_persistent_hash_map_(use_persistent_hash_map),
+ is_info_dirty_(false),
+ is_storage_dirty_(false) {}
+
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+ InitializeNewFiles(const Filesystem& filesystem, std::string&& working_path,
+ bool pre_mapping_fbv, bool use_persistent_hash_map);
+
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV1>>
+ InitializeExistingFiles(const Filesystem& filesystem,
+ std::string&& working_path, bool pre_mapping_fbv,
+ bool use_persistent_hash_map);
+
+ // Transfers qualified id join index data from the current to new_index and
+ // convert to new document id according to document_id_old_to_new. It is a
+ // helper function for Optimize.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status TransferIndex(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ QualifiedIdJoinIndexImplV1* new_index) const;
+
+ // Flushes contents of metadata file.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status PersistMetadataToDisk(bool force) override;
+
+ // Flushes contents of all storages to underlying files.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status PersistStoragesToDisk(bool force) override;
+
+ // Computes and returns Info checksum.
+ //
+ // Returns:
+ // - Crc of the Info on success
+ libtextclassifier3::StatusOr<Crc32> ComputeInfoChecksum(bool force) override;
+
+ // Computes and returns all storages checksum.
+ //
+ // Returns:
+ // - Crc of all storages on success
+ // - INTERNAL_ERROR if any data inconsistency
+ libtextclassifier3::StatusOr<Crc32> ComputeStoragesChecksum(
+ bool force) override;
+
+ Crcs& crcs() override {
+ return *reinterpret_cast<Crcs*>(metadata_buffer_.get() +
+ kCrcsMetadataBufferOffset);
+ }
+
+ const Crcs& crcs() const override {
+ return *reinterpret_cast<const Crcs*>(metadata_buffer_.get() +
+ kCrcsMetadataBufferOffset);
+ }
+
+ Info& info() {
+ return *reinterpret_cast<Info*>(metadata_buffer_.get() +
+ kInfoMetadataBufferOffset);
+ }
+
+ const Info& info() const {
+ return *reinterpret_cast<const Info*>(metadata_buffer_.get() +
+ kInfoMetadataBufferOffset);
+ }
+
+ void SetInfoDirty() { is_info_dirty_ = true; }
+ // When storage is dirty, we have to set info dirty as well. So just expose
+ // SetDirty to set both.
+ void SetDirty() {
+ is_info_dirty_ = true;
+ is_storage_dirty_ = true;
+ }
+
+ bool is_info_dirty() const { return is_info_dirty_; }
+ bool is_storage_dirty() const { return is_storage_dirty_; }
+
+ // Metadata buffer
+ std::unique_ptr<uint8_t[]> metadata_buffer_;
+
+ // Persistent KeyMapper for mapping (encoded) DocJoinInfo (DocumentId,
+ // JoinablePropertyId) to another referenced document's qualified id string
+ // index in qualified_id_storage_.
+ std::unique_ptr<KeyMapper<int32_t>> doc_join_info_mapper_;
+
+ // Storage for qualified id strings.
+ std::unique_ptr<FileBackedVector<char>> qualified_id_storage_;
+
+ // TODO(b/268521214): add delete propagation storage
+
+ // Flag indicating whether memory map max possible file size for underlying
+ // FileBackedVector before growing the actual file size.
+ bool pre_mapping_fbv_;
+
+ // Flag indicating whether use persistent hash map as the key mapper (if
+ // false, then fall back to dynamic trie key mapper).
+ bool use_persistent_hash_map_;
+
+ bool is_info_dirty_;
+ bool is_storage_dirty_;
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V1_H_
diff --git a/icing/join/qualified-id-join-index_test.cc b/icing/join/qualified-id-join-index-impl-v1_test.cc
index 3d59f4b..a6e19bb 100644
--- a/icing/join/qualified-id-join-index_test.cc
+++ b/icing/join/qualified-id-join-index-impl-v1_test.cc
@@ -12,8 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-#include "icing/join/qualified-id-join-index.h"
+#include "icing/join/qualified-id-join-index-impl-v1.h"
+#include <cstdint>
#include <memory>
#include <string>
#include <string_view>
@@ -49,22 +50,22 @@ using ::testing::Pointee;
using ::testing::SizeIs;
using Crcs = PersistentStorage::Crcs;
-using Info = QualifiedIdJoinIndex::Info;
+using Info = QualifiedIdJoinIndexImplV1::Info;
static constexpr int32_t kCorruptedValueOffset = 3;
-struct QualifiedIdJoinIndexTestParam {
+struct QualifiedIdJoinIndexImplV1TestParam {
bool pre_mapping_fbv;
bool use_persistent_hash_map;
- explicit QualifiedIdJoinIndexTestParam(bool pre_mapping_fbv_in,
- bool use_persistent_hash_map_in)
+ explicit QualifiedIdJoinIndexImplV1TestParam(bool pre_mapping_fbv_in,
+ bool use_persistent_hash_map_in)
: pre_mapping_fbv(pre_mapping_fbv_in),
use_persistent_hash_map(use_persistent_hash_map_in) {}
};
-class QualifiedIdJoinIndexTest
- : public ::testing::TestWithParam<QualifiedIdJoinIndexTestParam> {
+class QualifiedIdJoinIndexImplV1Test
+ : public ::testing::TestWithParam<QualifiedIdJoinIndexImplV1TestParam> {
protected:
void SetUp() override {
base_dir_ = GetTestTempDir() + "/icing";
@@ -83,26 +84,26 @@ class QualifiedIdJoinIndexTest
std::string working_path_;
};
-TEST_P(QualifiedIdJoinIndexTest, InvalidWorkingPath) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, InvalidWorkingPath) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
- EXPECT_THAT(QualifiedIdJoinIndex::Create(
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(
filesystem_, "/dev/null/qualified_id_join_index_test",
param.pre_mapping_fbv, param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::INTERNAL));
}
-TEST_P(QualifiedIdJoinIndexTest, InitializeNewFiles) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, InitializeNewFiles) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ASSERT_FALSE(filesystem_.DirectoryExists(working_path_.c_str()));
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index, Pointee(IsEmpty()));
ICING_ASSERT_OK(index->PersistToDisk());
@@ -112,23 +113,25 @@ TEST_P(QualifiedIdJoinIndexTest, InitializeNewFiles) {
// sections.
const std::string metadata_file_path =
absl_ports::StrCat(working_path_, "/metadata");
- auto metadata_buffer =
- std::make_unique<uint8_t[]>(QualifiedIdJoinIndex::kMetadataFileSize);
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize);
ASSERT_THAT(
filesystem_.PRead(metadata_file_path.c_str(), metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize,
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize,
/*offset=*/0),
IsTrue());
// Check info section
const Info* info = reinterpret_cast<const Info*>(
- metadata_buffer.get() + QualifiedIdJoinIndex::kInfoMetadataBufferOffset);
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV1::kInfoMetadataBufferOffset);
EXPECT_THAT(info->magic, Eq(Info::kMagic));
EXPECT_THAT(info->last_added_document_id, Eq(kInvalidDocumentId));
// Check crcs section
const Crcs* crcs = reinterpret_cast<const Crcs*>(
- metadata_buffer.get() + QualifiedIdJoinIndex::kCrcsMetadataBufferOffset);
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV1::kCrcsMetadataBufferOffset);
// There are some initial info in KeyMapper, so storages_crc should be
// non-zero.
EXPECT_THAT(crcs->component_crcs.storages_crc, Ne(0));
@@ -143,16 +146,16 @@ TEST_P(QualifiedIdJoinIndexTest, InitializeNewFiles) {
.Get()));
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializationShouldFailWithoutPersistToDiskOrDestruction) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
// Insert some data.
ICING_ASSERT_OK(
@@ -168,23 +171,24 @@ TEST_P(QualifiedIdJoinIndexTest,
// Without calling PersistToDisk, checksums will not be recomputed or synced
// to disk, so initializing another instance on the same files should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(param.use_persistent_hash_map
? libtextclassifier3::StatusCode::FAILED_PRECONDITION
: libtextclassifier3::StatusCode::INTERNAL));
}
-TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedWithPersistToDisk) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test,
+ InitializationShouldSucceedWithPersistToDisk) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index1,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index1,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
// Insert some data.
ICING_ASSERT_OK(
@@ -204,10 +208,10 @@ TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedWithPersistToDisk) {
ICING_EXPECT_OK(index1->PersistToDisk());
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index2,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index2,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index2, Pointee(SizeIs(3)));
EXPECT_THAT(
index2->Get(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20)),
@@ -220,16 +224,17 @@ TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedWithPersistToDisk) {
IsOkAndHolds(/*ref_qualified_id_str=*/"namespace#uriC"));
}
-TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedAfterDestruction) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test,
+ InitializationShouldSucceedAfterDestruction) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
// Insert some data.
ICING_ASSERT_OK(
@@ -250,10 +255,10 @@ TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedAfterDestruction) {
// thus initializing another instance on the same files should succeed, and
// we should be able to get the same contents.
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index, Pointee(SizeIs(3)));
EXPECT_THAT(index->Get(DocJoinInfo(/*document_id=*/1,
/*joinable_property_id=*/20)),
@@ -267,17 +272,17 @@ TEST_P(QualifiedIdJoinIndexTest, InitializationShouldSucceedAfterDestruction) {
}
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializeExistingFilesWithDifferentMagicShouldFail) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -292,49 +297,49 @@ TEST_P(QualifiedIdJoinIndexTest,
ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
- auto metadata_buffer =
- std::make_unique<uint8_t[]>(QualifiedIdJoinIndex::kMetadataFileSize);
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize);
ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize,
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize,
/*offset=*/0),
IsTrue());
// Manually change magic and update checksums.
Crcs* crcs = reinterpret_cast<Crcs*>(
metadata_buffer.get() +
- QualifiedIdJoinIndex::kCrcsMetadataBufferOffset);
+ QualifiedIdJoinIndexImplV1::kCrcsMetadataBufferOffset);
Info* info = reinterpret_cast<Info*>(
metadata_buffer.get() +
- QualifiedIdJoinIndex::kInfoMetadataBufferOffset);
+ QualifiedIdJoinIndexImplV1::kInfoMetadataBufferOffset);
info->magic += kCorruptedValueOffset;
crcs->component_crcs.info_crc = info->ComputeChecksum().Get();
crcs->all_crc = crcs->component_crcs.ComputeChecksum().Get();
- ASSERT_THAT(filesystem_.PWrite(metadata_sfd.get(), /*offset=*/0,
- metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize),
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize),
IsTrue());
}
// Attempt to create the qualified id join index with different magic. This
// should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
HasSubstr("Incorrect magic value")));
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializeExistingFilesWithWrongAllCrcShouldFail) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -348,45 +353,45 @@ TEST_P(QualifiedIdJoinIndexTest,
ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
- auto metadata_buffer =
- std::make_unique<uint8_t[]>(QualifiedIdJoinIndex::kMetadataFileSize);
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize);
ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize,
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize,
/*offset=*/0),
IsTrue());
// Manually corrupt all_crc
Crcs* crcs = reinterpret_cast<Crcs*>(
metadata_buffer.get() +
- QualifiedIdJoinIndex::kCrcsMetadataBufferOffset);
+ QualifiedIdJoinIndexImplV1::kCrcsMetadataBufferOffset);
crcs->all_crc += kCorruptedValueOffset;
- ASSERT_THAT(filesystem_.PWrite(metadata_sfd.get(), /*offset=*/0,
- metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize),
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize),
IsTrue());
}
// Attempt to create the qualified id join index with metadata containing
// corrupted all_crc. This should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
HasSubstr("Invalid all crc")));
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializeExistingFilesWithCorruptedInfoShouldFail) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -400,10 +405,10 @@ TEST_P(QualifiedIdJoinIndexTest,
ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
- auto metadata_buffer =
- std::make_unique<uint8_t[]>(QualifiedIdJoinIndex::kMetadataFileSize);
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize);
ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize,
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize,
/*offset=*/0),
IsTrue());
@@ -411,35 +416,35 @@ TEST_P(QualifiedIdJoinIndexTest,
// corruption of info.
Info* info = reinterpret_cast<Info*>(
metadata_buffer.get() +
- QualifiedIdJoinIndex::kInfoMetadataBufferOffset);
+ QualifiedIdJoinIndexImplV1::kInfoMetadataBufferOffset);
info->last_added_document_id += kCorruptedValueOffset;
- ASSERT_THAT(filesystem_.PWrite(metadata_sfd.get(), /*offset=*/0,
- metadata_buffer.get(),
- QualifiedIdJoinIndex::kMetadataFileSize),
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV1::kMetadataFileSize),
IsTrue());
}
// Attempt to create the qualified id join index with info that doesn't match
// its checksum. This should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
HasSubstr("Invalid info crc")));
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializeExistingFilesWithCorruptedDocJoinInfoMapperShouldFail) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -472,24 +477,24 @@ TEST_P(QualifiedIdJoinIndexTest,
// Attempt to create the qualified id join index with corrupted
// doc_join_info_mapper. This should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
HasSubstr("Invalid storages crc")));
}
-TEST_P(QualifiedIdJoinIndexTest,
+TEST_P(QualifiedIdJoinIndexImplV1Test,
InitializeExistingFilesWithCorruptedQualifiedIdStorageShouldFail) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -518,22 +523,22 @@ TEST_P(QualifiedIdJoinIndexTest,
// Attempt to create the qualified id join index with corrupted
// qualified_id_storage. This should fail.
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
HasSubstr("Invalid storages crc")));
}
-TEST_P(QualifiedIdJoinIndexTest, InvalidPut) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, InvalidPut) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
DocJoinInfo default_invalid;
EXPECT_THAT(
@@ -541,23 +546,23 @@ TEST_P(QualifiedIdJoinIndexTest, InvalidPut) {
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
}
-TEST_P(QualifiedIdJoinIndexTest, InvalidGet) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, InvalidGet) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
DocJoinInfo default_invalid;
EXPECT_THAT(index->Get(default_invalid),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
}
-TEST_P(QualifiedIdJoinIndexTest, PutAndGet) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, PutAndGet) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
DocJoinInfo target_info1(/*document_id=*/1, /*joinable_property_id=*/20);
std::string_view ref_qualified_id_str_a = "namespace#uriA";
@@ -571,10 +576,10 @@ TEST_P(QualifiedIdJoinIndexTest, PutAndGet) {
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index->Put(target_info1, ref_qualified_id_str_a), IsOk());
EXPECT_THAT(index->Put(target_info2, ref_qualified_id_str_b), IsOk());
@@ -590,28 +595,28 @@ TEST_P(QualifiedIdJoinIndexTest, PutAndGet) {
// Verify we can get all of them after destructing and re-initializing.
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index, Pointee(SizeIs(3)));
EXPECT_THAT(index->Get(target_info1), IsOkAndHolds(ref_qualified_id_str_a));
EXPECT_THAT(index->Get(target_info2), IsOkAndHolds(ref_qualified_id_str_b));
EXPECT_THAT(index->Get(target_info3), IsOkAndHolds(ref_qualified_id_str_c));
}
-TEST_P(QualifiedIdJoinIndexTest, GetShouldReturnNotFoundErrorIfNotExist) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, GetShouldReturnNotFoundErrorIfNotExist) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
DocJoinInfo target_info(/*document_id=*/1, /*joinable_property_id=*/20);
std::string_view ref_qualified_id_str = "namespace#uriA";
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
// Verify entry is not found in the beginning.
EXPECT_THAT(index->Get(target_info),
@@ -627,14 +632,14 @@ TEST_P(QualifiedIdJoinIndexTest, GetShouldReturnNotFoundErrorIfNotExist) {
StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
}
-TEST_P(QualifiedIdJoinIndexTest, SetLastAddedDocumentId) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, SetLastAddedDocumentId) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
@@ -648,15 +653,15 @@ TEST_P(QualifiedIdJoinIndexTest, SetLastAddedDocumentId) {
}
TEST_P(
- QualifiedIdJoinIndexTest,
+ QualifiedIdJoinIndexImplV1Test,
SetLastAddedDocumentIdShouldIgnoreNewDocumentIdNotGreaterThanTheCurrent) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
constexpr DocumentId kDocumentId = 123;
index->set_last_added_document_id(kDocumentId);
@@ -669,14 +674,14 @@ TEST_P(
EXPECT_THAT(index->last_added_document_id(), Eq(kDocumentId));
}
-TEST_P(QualifiedIdJoinIndexTest, Optimize) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, Optimize) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/3, /*joinable_property_id=*/10),
@@ -705,7 +710,8 @@ TEST_P(QualifiedIdJoinIndexTest, Optimize) {
DocumentId new_last_added_document_id = 2;
EXPECT_THAT(
- index->Optimize(document_id_old_to_new, new_last_added_document_id),
+ index->Optimize(document_id_old_to_new, /*namespace_id_old_to_new=*/{},
+ new_last_added_document_id),
IsOk());
EXPECT_THAT(index, Pointee(SizeIs(3)));
EXPECT_THAT(index->last_added_document_id(), Eq(new_last_added_document_id));
@@ -750,14 +756,14 @@ TEST_P(QualifiedIdJoinIndexTest, Optimize) {
IsOkAndHolds("namespace#uriD"));
}
-TEST_P(QualifiedIdJoinIndexTest, OptimizeOutOfRangeDocumentId) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, OptimizeOutOfRangeDocumentId) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/99, /*joinable_property_id=*/10),
@@ -770,7 +776,7 @@ TEST_P(QualifiedIdJoinIndexTest, OptimizeOutOfRangeDocumentId) {
// There shouldn't be any error due to vector index.
EXPECT_THAT(
- index->Optimize(document_id_old_to_new,
+ index->Optimize(document_id_old_to_new, /*namespace_id_old_to_new=*/{},
/*new_last_added_document_id=*/kInvalidDocumentId),
IsOk());
EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
@@ -779,14 +785,14 @@ TEST_P(QualifiedIdJoinIndexTest, OptimizeOutOfRangeDocumentId) {
EXPECT_THAT(index, Pointee(IsEmpty()));
}
-TEST_P(QualifiedIdJoinIndexTest, OptimizeDeleteAll) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, OptimizeDeleteAll) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/3, /*joinable_property_id=*/10),
@@ -809,7 +815,7 @@ TEST_P(QualifiedIdJoinIndexTest, OptimizeDeleteAll) {
std::vector<DocumentId> document_id_old_to_new(22, kInvalidDocumentId);
EXPECT_THAT(
- index->Optimize(document_id_old_to_new,
+ index->Optimize(document_id_old_to_new, /*namespace_id_old_to_new=*/{},
/*new_last_added_document_id=*/kInvalidDocumentId),
IsOk());
EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
@@ -818,8 +824,8 @@ TEST_P(QualifiedIdJoinIndexTest, OptimizeDeleteAll) {
EXPECT_THAT(index, Pointee(IsEmpty()));
}
-TEST_P(QualifiedIdJoinIndexTest, Clear) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, Clear) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
DocJoinInfo target_info1(/*document_id=*/1, /*joinable_property_id=*/20);
DocJoinInfo target_info2(/*document_id=*/3, /*joinable_property_id=*/5);
@@ -827,10 +833,10 @@ TEST_P(QualifiedIdJoinIndexTest, Clear) {
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(target_info1, /*ref_qualified_id_str=*/"namespace#uriA"));
ICING_ASSERT_OK(
@@ -867,9 +873,9 @@ TEST_P(QualifiedIdJoinIndexTest, Clear) {
// Verify index after reconstructing.
ICING_ASSERT_OK_AND_ASSIGN(
- index, QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ index, QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
EXPECT_THAT(index->last_added_document_id(), Eq(2));
EXPECT_THAT(index->Get(target_info1),
StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
@@ -880,16 +886,16 @@ TEST_P(QualifiedIdJoinIndexTest, Clear) {
EXPECT_THAT(index->Get(target_info4), IsOkAndHolds("namespace#uriD"));
}
-TEST_P(QualifiedIdJoinIndexTest, SwitchKeyMapperTypeShouldReturnError) {
- const QualifiedIdJoinIndexTestParam& param = GetParam();
+TEST_P(QualifiedIdJoinIndexImplV1Test, SwitchKeyMapperTypeShouldReturnError) {
+ const QualifiedIdJoinIndexImplV1TestParam& param = GetParam();
{
// Create new qualified id join index
ICING_ASSERT_OK_AND_ASSIGN(
- std::unique_ptr<QualifiedIdJoinIndex> index,
- QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- param.use_persistent_hash_map));
+ std::unique_ptr<QualifiedIdJoinIndexImplV1> index,
+ QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ param.use_persistent_hash_map));
ICING_ASSERT_OK(
index->Put(DocJoinInfo(/*document_id=*/1, /*joinable_property_id=*/20),
/*ref_qualified_id_str=*/"namespace#uriA"));
@@ -898,23 +904,26 @@ TEST_P(QualifiedIdJoinIndexTest, SwitchKeyMapperTypeShouldReturnError) {
}
bool switch_key_mapper_flag = !param.use_persistent_hash_map;
- EXPECT_THAT(QualifiedIdJoinIndex::Create(filesystem_, working_path_,
- param.pre_mapping_fbv,
- switch_key_mapper_flag),
+ EXPECT_THAT(QualifiedIdJoinIndexImplV1::Create(filesystem_, working_path_,
+ param.pre_mapping_fbv,
+ switch_key_mapper_flag),
StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
}
INSTANTIATE_TEST_SUITE_P(
- QualifiedIdJoinIndexTest, QualifiedIdJoinIndexTest,
- testing::Values(
- QualifiedIdJoinIndexTestParam(/*pre_mapping_fbv_in=*/true,
- /*use_persistent_hash_map_in=*/true),
- QualifiedIdJoinIndexTestParam(/*pre_mapping_fbv_in=*/true,
- /*use_persistent_hash_map_in=*/false),
- QualifiedIdJoinIndexTestParam(/*pre_mapping_fbv_in=*/false,
- /*use_persistent_hash_map_in=*/true),
- QualifiedIdJoinIndexTestParam(/*pre_mapping_fbv_in=*/false,
- /*use_persistent_hash_map_in=*/false)));
+ QualifiedIdJoinIndexImplV1Test, QualifiedIdJoinIndexImplV1Test,
+ testing::Values(QualifiedIdJoinIndexImplV1TestParam(
+ /*pre_mapping_fbv_in=*/true,
+ /*use_persistent_hash_map_in=*/true),
+ QualifiedIdJoinIndexImplV1TestParam(
+ /*pre_mapping_fbv_in=*/true,
+ /*use_persistent_hash_map_in=*/false),
+ QualifiedIdJoinIndexImplV1TestParam(
+ /*pre_mapping_fbv_in=*/false,
+ /*use_persistent_hash_map_in=*/true),
+ QualifiedIdJoinIndexImplV1TestParam(
+ /*pre_mapping_fbv_in=*/false,
+ /*use_persistent_hash_map_in=*/false)));
} // namespace
diff --git a/icing/join/qualified-id-join-index-impl-v2.cc b/icing/join/qualified-id-join-index-impl-v2.cc
new file mode 100644
index 0000000..70fd13c
--- /dev/null
+++ b/icing/join/qualified-id-join-index-impl-v2.cc
@@ -0,0 +1,681 @@
+// Copyright (C) 2023 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/join/qualified-id-join-index-impl-v2.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/absl_ports/str_cat.h"
+#include "icing/file/destructible-directory.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/posting_list/flash-index-storage.h"
+#include "icing/file/posting_list/posting-list-accessor.h"
+#include "icing/file/posting_list/posting-list-identifier.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/join/posting-list-join-data-accessor.h"
+#include "icing/join/posting-list-join-data-serializer.h"
+#include "icing/join/qualified-id-join-index.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
+#include "icing/store/persistent-hash-map-key-mapper.h"
+#include "icing/util/crc32.h"
+#include "icing/util/encode-util.h"
+#include "icing/util/logging.h"
+#include "icing/util/status-macros.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+// Set 1M for max # of qualified id entries and 10 bytes for key-value bytes.
+// This will take at most 23 MiB disk space and mmap for persistent hash map.
+static constexpr int32_t kSchemaJoinableIdToPostingListMapperMaxNumEntries =
+ 1 << 20;
+static constexpr int32_t kSchemaJoinableIdToPostingListMapperAverageKVByteSize =
+ 10;
+
+inline DocumentId GetNewDocumentId(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ DocumentId old_document_id) {
+ if (old_document_id >= document_id_old_to_new.size()) {
+ return kInvalidDocumentId;
+ }
+ return document_id_old_to_new[old_document_id];
+}
+
+inline NamespaceId GetNewNamespaceId(
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ NamespaceId namespace_id) {
+ if (namespace_id >= namespace_id_old_to_new.size()) {
+ return kInvalidNamespaceId;
+ }
+ return namespace_id_old_to_new[namespace_id];
+}
+
+libtextclassifier3::StatusOr<PostingListIdentifier> GetPostingListIdentifier(
+ const KeyMapper<PostingListIdentifier>&
+ schema_joinable_id_to_posting_list_mapper,
+ const std::string& encoded_schema_type_joinable_property_id_str) {
+ auto posting_list_identifier_or =
+ schema_joinable_id_to_posting_list_mapper.Get(
+ encoded_schema_type_joinable_property_id_str);
+ if (!posting_list_identifier_or.ok()) {
+ if (absl_ports::IsNotFound(posting_list_identifier_or.status())) {
+ // Not found. Return invalid posting list id.
+ return PostingListIdentifier::kInvalid;
+ }
+ // Real error.
+ return posting_list_identifier_or;
+ }
+ return std::move(posting_list_identifier_or).ValueOrDie();
+}
+
+libtextclassifier3::StatusOr<std::string> EncodeSchemaTypeJoinablePropertyId(
+ SchemaTypeId schema_type_id, JoinablePropertyId joinable_property_id) {
+ if (schema_type_id < 0) {
+ return absl_ports::InvalidArgumentError("Invalid schema type id");
+ }
+
+ if (!IsJoinablePropertyIdValid(joinable_property_id)) {
+ return absl_ports::InvalidArgumentError("Invalid joinable property id");
+ }
+
+ static constexpr int kEncodedSchemaTypeIdLength = 3;
+
+ // encoded_schema_type_id_str should be 1 to 3 bytes based on the value of
+ // schema_type_id.
+ std::string encoded_schema_type_id_str =
+ encode_util::EncodeIntToCString(schema_type_id);
+ // Make encoded_schema_type_id_str to fixed kEncodedSchemaTypeIdLength bytes.
+ while (encoded_schema_type_id_str.size() < kEncodedSchemaTypeIdLength) {
+ // C string cannot contain 0 bytes, so we append it using 1, just like what
+ // we do in encode_util::EncodeIntToCString.
+ //
+ // The reason that this works is because DecodeIntToString decodes a byte
+ // value of 0x01 as 0x00. When EncodeIntToCString returns an encoded
+ // schema type id that is less than 3 bytes, it means that the id contains
+ // unencoded leading 0x00. So here we're explicitly encoding those bytes as
+ // 0x01.
+ encoded_schema_type_id_str.push_back(1);
+ }
+
+ return absl_ports::StrCat(
+ encoded_schema_type_id_str,
+ encode_util::EncodeIntToCString(joinable_property_id));
+}
+
+std::string GetMetadataFilePath(std::string_view working_path) {
+ return absl_ports::StrCat(working_path, "/metadata");
+}
+
+std::string GetSchemaJoinableIdToPostingListMapperPath(
+ std::string_view working_path) {
+ return absl_ports::StrCat(working_path,
+ "/schema_joinable_id_to_posting_list_mapper");
+}
+
+std::string GetFlashIndexStorageFilePath(std::string_view working_path) {
+ return absl_ports::StrCat(working_path, "/flash_index_storage");
+}
+
+} // namespace
+
+libtextclassifier3::Status
+QualifiedIdJoinIndexImplV2::JoinDataIterator::Advance() {
+ if (pl_accessor_ == nullptr) {
+ return absl_ports::ResourceExhaustedError("End of iterator");
+ }
+
+ if (!should_retrieve_next_batch_) {
+ // In this case, cached_batch_join_data_ is not empty (contains some data
+ // fetched in the previous round), so move curr_ to the next position and
+ // check if we have to fetch the next batch.
+ //
+ // Note: in the 1st round, should_retrieve_next_batch_ is true, so this part
+ // will never be executed.
+ ++curr_;
+ should_retrieve_next_batch_ = curr_ >= cached_batch_join_data_.cend();
+ }
+
+ if (should_retrieve_next_batch_) {
+ // Fetch next batch if needed.
+ ICING_RETURN_IF_ERROR(GetNextDataBatch());
+ should_retrieve_next_batch_ = false;
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status
+QualifiedIdJoinIndexImplV2::JoinDataIterator::GetNextDataBatch() {
+ auto cached_batch_join_data_or = pl_accessor_->GetNextDataBatch();
+ if (!cached_batch_join_data_or.ok()) {
+ ICING_LOG(WARNING)
+ << "Fail to get next batch data from posting list due to: "
+ << cached_batch_join_data_or.status().error_message();
+ return std::move(cached_batch_join_data_or).status();
+ }
+
+ cached_batch_join_data_ = std::move(cached_batch_join_data_or).ValueOrDie();
+ curr_ = cached_batch_join_data_.cbegin();
+
+ if (cached_batch_join_data_.empty()) {
+ return absl_ports::ResourceExhaustedError("End of iterator");
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+QualifiedIdJoinIndexImplV2::Create(const Filesystem& filesystem,
+ std::string working_path,
+ bool pre_mapping_fbv) {
+ if (!filesystem.FileExists(GetMetadataFilePath(working_path).c_str()) ||
+ !filesystem.DirectoryExists(
+ GetSchemaJoinableIdToPostingListMapperPath(working_path).c_str()) ||
+ !filesystem.FileExists(
+ GetFlashIndexStorageFilePath(working_path).c_str())) {
+ // Discard working_path if any file/directory is missing, and reinitialize.
+ if (filesystem.DirectoryExists(working_path.c_str())) {
+ ICING_RETURN_IF_ERROR(
+ QualifiedIdJoinIndex::Discard(filesystem, working_path));
+ }
+ return InitializeNewFiles(filesystem, std::move(working_path),
+ pre_mapping_fbv);
+ }
+ return InitializeExistingFiles(filesystem, std::move(working_path),
+ pre_mapping_fbv);
+}
+
+QualifiedIdJoinIndexImplV2::~QualifiedIdJoinIndexImplV2() {
+ if (!PersistToDisk().ok()) {
+ ICING_LOG(WARNING) << "Failed to persist qualified id join index (v2) to "
+ "disk while destructing "
+ << working_path_;
+ }
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::Put(
+ SchemaTypeId schema_type_id, JoinablePropertyId joinable_property_id,
+ DocumentId document_id,
+ std::vector<NamespaceFingerprintIdentifier>&&
+ ref_namespace_fingerprint_ids) {
+ std::sort(ref_namespace_fingerprint_ids.begin(),
+ ref_namespace_fingerprint_ids.end());
+
+ // Dedupe.
+ auto last = std::unique(ref_namespace_fingerprint_ids.begin(),
+ ref_namespace_fingerprint_ids.end());
+ ref_namespace_fingerprint_ids.erase(last,
+ ref_namespace_fingerprint_ids.end());
+ if (ref_namespace_fingerprint_ids.empty()) {
+ return libtextclassifier3::Status::OK;
+ }
+
+ SetDirty();
+ ICING_ASSIGN_OR_RETURN(
+ std::string encoded_schema_type_joinable_property_id_str,
+ EncodeSchemaTypeJoinablePropertyId(schema_type_id, joinable_property_id));
+
+ ICING_ASSIGN_OR_RETURN(
+ PostingListIdentifier posting_list_identifier,
+ GetPostingListIdentifier(*schema_joinable_id_to_posting_list_mapper_,
+ encoded_schema_type_joinable_property_id_str));
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor;
+ if (posting_list_identifier.is_valid()) {
+ ICING_ASSIGN_OR_RETURN(
+ pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), posting_list_serializer_.get(),
+ posting_list_identifier));
+ } else {
+ ICING_ASSIGN_OR_RETURN(
+ pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ flash_index_storage_.get(), posting_list_serializer_.get()));
+ }
+
+ // Prepend join data into posting list.
+ for (const NamespaceFingerprintIdentifier& ref_namespace_fingerprint_id :
+ ref_namespace_fingerprint_ids) {
+ ICING_RETURN_IF_ERROR(pl_accessor->PrependData(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ document_id, ref_namespace_fingerprint_id)));
+ }
+
+ // Finalize the posting list and update mapper.
+ PostingListAccessor::FinalizeResult result =
+ std::move(*pl_accessor).Finalize();
+ if (!result.status.ok()) {
+ return result.status;
+ }
+ if (!result.id.is_valid()) {
+ return absl_ports::InternalError("Fail to flush data into posting list(s)");
+ }
+ ICING_RETURN_IF_ERROR(schema_joinable_id_to_posting_list_mapper_->Put(
+ encoded_schema_type_joinable_property_id_str, result.id));
+
+ // Update info.
+ info().num_data += ref_namespace_fingerprint_ids.size();
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndex::JoinDataIteratorBase>>
+QualifiedIdJoinIndexImplV2::GetIterator(
+ SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) const {
+ ICING_ASSIGN_OR_RETURN(
+ std::string encoded_schema_type_joinable_property_id_str,
+ EncodeSchemaTypeJoinablePropertyId(schema_type_id, joinable_property_id));
+
+ ICING_ASSIGN_OR_RETURN(
+ PostingListIdentifier posting_list_identifier,
+ GetPostingListIdentifier(*schema_joinable_id_to_posting_list_mapper_,
+ encoded_schema_type_joinable_property_id_str));
+
+ if (!posting_list_identifier.is_valid()) {
+ return std::make_unique<JoinDataIterator>(nullptr);
+ }
+
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), posting_list_serializer_.get(),
+ posting_list_identifier));
+
+ return std::make_unique<JoinDataIterator>(std::move(pl_accessor));
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::Optimize(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ DocumentId new_last_added_document_id) {
+ std::string temp_working_path = working_path_ + "_temp";
+ ICING_RETURN_IF_ERROR(
+ QualifiedIdJoinIndex::Discard(filesystem_, temp_working_path));
+
+ DestructibleDirectory temp_working_path_ddir(&filesystem_,
+ std::move(temp_working_path));
+ if (!temp_working_path_ddir.is_valid()) {
+ return absl_ports::InternalError(
+ "Unable to create temp directory to build new qualified id join index "
+ "(v2)");
+ }
+
+ {
+ // Transfer all data from the current to new qualified id join index. Also
+ // PersistToDisk and destruct the instance after finishing, so we can safely
+ // swap directories later.
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> new_index,
+ Create(filesystem_, temp_working_path_ddir.dir(), pre_mapping_fbv_));
+ ICING_RETURN_IF_ERROR(TransferIndex(
+ document_id_old_to_new, namespace_id_old_to_new, new_index.get()));
+ new_index->set_last_added_document_id(new_last_added_document_id);
+ ICING_RETURN_IF_ERROR(new_index->PersistToDisk());
+ }
+
+ // Destruct current index's storage instances to safely swap directories.
+ // TODO(b/268521214): handle delete propagation storage
+ schema_joinable_id_to_posting_list_mapper_.reset();
+ flash_index_storage_.reset();
+
+ if (!filesystem_.SwapFiles(temp_working_path_ddir.dir().c_str(),
+ working_path_.c_str())) {
+ return absl_ports::InternalError(
+ "Unable to apply new qualified id join index (v2) due to failed swap");
+ }
+
+ // Reinitialize qualified id join index.
+ if (!filesystem_.PRead(GetMetadataFilePath(working_path_).c_str(),
+ metadata_buffer_.get(), kMetadataFileSize,
+ /*offset=*/0)) {
+ return absl_ports::InternalError("Fail to read metadata file");
+ }
+ ICING_ASSIGN_OR_RETURN(
+ schema_joinable_id_to_posting_list_mapper_,
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Create(
+ filesystem_,
+ GetSchemaJoinableIdToPostingListMapperPath(working_path_),
+ pre_mapping_fbv_,
+ /*max_num_entries=*/
+ kSchemaJoinableIdToPostingListMapperMaxNumEntries,
+ /*average_kv_byte_size=*/
+ kSchemaJoinableIdToPostingListMapperAverageKVByteSize));
+ ICING_ASSIGN_OR_RETURN(
+ FlashIndexStorage flash_index_storage,
+ FlashIndexStorage::Create(GetFlashIndexStorageFilePath(working_path_),
+ &filesystem_, posting_list_serializer_.get()));
+ flash_index_storage_ =
+ std::make_unique<FlashIndexStorage>(std::move(flash_index_storage));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::Clear() {
+ SetDirty();
+
+ schema_joinable_id_to_posting_list_mapper_.reset();
+ // Discard and reinitialize schema_joinable_id_to_posting_list_mapper.
+ std::string schema_joinable_id_to_posting_list_mapper_path =
+ GetSchemaJoinableIdToPostingListMapperPath(working_path_);
+ ICING_RETURN_IF_ERROR(
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Delete(
+ filesystem_, schema_joinable_id_to_posting_list_mapper_path));
+ ICING_ASSIGN_OR_RETURN(
+ schema_joinable_id_to_posting_list_mapper_,
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Create(
+ filesystem_,
+ std::move(schema_joinable_id_to_posting_list_mapper_path),
+ pre_mapping_fbv_,
+ /*max_num_entries=*/
+ kSchemaJoinableIdToPostingListMapperMaxNumEntries,
+ /*average_kv_byte_size=*/
+ kSchemaJoinableIdToPostingListMapperAverageKVByteSize));
+
+ // Discard and reinitialize flash_index_storage.
+ flash_index_storage_.reset();
+ if (!filesystem_.DeleteFile(
+ GetFlashIndexStorageFilePath(working_path_).c_str())) {
+ return absl_ports::InternalError("Fail to delete flash index storage file");
+ }
+ ICING_ASSIGN_OR_RETURN(
+ FlashIndexStorage flash_index_storage,
+ FlashIndexStorage::Create(GetFlashIndexStorageFilePath(working_path_),
+ &filesystem_, posting_list_serializer_.get()));
+ flash_index_storage_ =
+ std::make_unique<FlashIndexStorage>(std::move(flash_index_storage));
+
+ // TODO(b/268521214): clear delete propagation storage
+
+ info().num_data = 0;
+ info().last_added_document_id = kInvalidDocumentId;
+ return libtextclassifier3::Status::OK;
+}
+
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+QualifiedIdJoinIndexImplV2::InitializeNewFiles(const Filesystem& filesystem,
+ std::string&& working_path,
+ bool pre_mapping_fbv) {
+ // Create working directory.
+ if (!filesystem.CreateDirectoryRecursively(working_path.c_str())) {
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Failed to create directory: ", working_path));
+ }
+
+ // Initialize schema_joinable_id_to_posting_list_mapper
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<KeyMapper<PostingListIdentifier>>
+ schema_joinable_id_to_posting_list_mapper,
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Create(
+ filesystem, GetSchemaJoinableIdToPostingListMapperPath(working_path),
+ pre_mapping_fbv,
+ /*max_num_entries=*/
+ kSchemaJoinableIdToPostingListMapperMaxNumEntries,
+ /*average_kv_byte_size=*/
+ kSchemaJoinableIdToPostingListMapperAverageKVByteSize));
+
+ // Initialize flash_index_storage
+ auto posting_list_serializer =
+ std::make_unique<PostingListJoinDataSerializer<JoinDataType>>();
+ ICING_ASSIGN_OR_RETURN(
+ FlashIndexStorage flash_index_storage,
+ FlashIndexStorage::Create(GetFlashIndexStorageFilePath(working_path),
+ &filesystem, posting_list_serializer.get()));
+
+ // Create instance.
+ auto new_join_index = std::unique_ptr<QualifiedIdJoinIndexImplV2>(
+ new QualifiedIdJoinIndexImplV2(
+ filesystem, std::move(working_path),
+ /*metadata_buffer=*/std::make_unique<uint8_t[]>(kMetadataFileSize),
+ std::move(schema_joinable_id_to_posting_list_mapper),
+ std::move(posting_list_serializer),
+ std::make_unique<FlashIndexStorage>(std::move(flash_index_storage)),
+ pre_mapping_fbv));
+ // Initialize info content.
+ new_join_index->info().magic = Info::kMagic;
+ new_join_index->info().num_data = 0;
+ new_join_index->info().last_added_document_id = kInvalidDocumentId;
+ // Initialize new PersistentStorage. The initial checksums will be computed
+ // and set via InitializeNewStorage.
+ ICING_RETURN_IF_ERROR(new_join_index->InitializeNewStorage());
+
+ return new_join_index;
+}
+
+/* static */ libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+QualifiedIdJoinIndexImplV2::InitializeExistingFiles(
+ const Filesystem& filesystem, std::string&& working_path,
+ bool pre_mapping_fbv) {
+ // PRead metadata file.
+ auto metadata_buffer = std::make_unique<uint8_t[]>(kMetadataFileSize);
+ if (!filesystem.PRead(GetMetadataFilePath(working_path).c_str(),
+ metadata_buffer.get(), kMetadataFileSize,
+ /*offset=*/0)) {
+ return absl_ports::InternalError("Fail to read metadata file");
+ }
+
+ // Initialize schema_joinable_id_to_posting_list_mapper
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<KeyMapper<PostingListIdentifier>>
+ schema_joinable_id_to_posting_list_mapper,
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Create(
+ filesystem, GetSchemaJoinableIdToPostingListMapperPath(working_path),
+ pre_mapping_fbv,
+ /*max_num_entries=*/
+ kSchemaJoinableIdToPostingListMapperMaxNumEntries,
+ /*average_kv_byte_size=*/
+ kSchemaJoinableIdToPostingListMapperAverageKVByteSize));
+
+ // Initialize flash_index_storage
+ auto posting_list_serializer =
+ std::make_unique<PostingListJoinDataSerializer<JoinDataType>>();
+ ICING_ASSIGN_OR_RETURN(
+ FlashIndexStorage flash_index_storage,
+ FlashIndexStorage::Create(GetFlashIndexStorageFilePath(working_path),
+ &filesystem, posting_list_serializer.get()));
+
+ // Create instance.
+ auto join_index = std::unique_ptr<QualifiedIdJoinIndexImplV2>(
+ new QualifiedIdJoinIndexImplV2(
+ filesystem, std::move(working_path), std::move(metadata_buffer),
+ std::move(schema_joinable_id_to_posting_list_mapper),
+ std::move(posting_list_serializer),
+ std::make_unique<FlashIndexStorage>(std::move(flash_index_storage)),
+ pre_mapping_fbv));
+ // Initialize existing PersistentStorage. Checksums will be validated.
+ ICING_RETURN_IF_ERROR(join_index->InitializeExistingStorage());
+
+ // Validate magic.
+ if (join_index->info().magic != Info::kMagic) {
+ return absl_ports::FailedPreconditionError("Incorrect magic value");
+ }
+
+ return join_index;
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::TransferIndex(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ QualifiedIdJoinIndexImplV2* new_index) const {
+ std::unique_ptr<KeyMapper<PostingListIdentifier>::Iterator> iter =
+ schema_joinable_id_to_posting_list_mapper_->GetIterator();
+
+ // Iterate through all (schema_type_id, joinable_property_id).
+ while (iter->Advance()) {
+ PostingListIdentifier old_pl_id = iter->GetValue();
+ if (!old_pl_id.is_valid()) {
+ // Skip invalid posting list id.
+ continue;
+ }
+
+ // Read all join data from old posting lists and convert to new join data
+ // with new document id, namespace id.
+ std::vector<JoinDataType> new_join_data_vec;
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>
+ old_pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::CreateFromExisting(
+ flash_index_storage_.get(), posting_list_serializer_.get(),
+ old_pl_id));
+ ICING_ASSIGN_OR_RETURN(std::vector<JoinDataType> batch_old_join_data,
+ old_pl_accessor->GetNextDataBatch());
+ while (!batch_old_join_data.empty()) {
+ for (const JoinDataType& old_join_data : batch_old_join_data) {
+ DocumentId new_document_id = GetNewDocumentId(
+ document_id_old_to_new, old_join_data.document_id());
+ NamespaceId new_ref_namespace_id = GetNewNamespaceId(
+ namespace_id_old_to_new, old_join_data.join_info().namespace_id());
+
+ // Transfer if the document and namespace are not deleted or outdated.
+ if (new_document_id != kInvalidDocumentId &&
+ new_ref_namespace_id != kInvalidNamespaceId) {
+ // We can reuse the fingerprint from old_join_data, since document uri
+ // (and its fingerprint) will never change.
+ new_join_data_vec.push_back(JoinDataType(
+ new_document_id, NamespaceFingerprintIdentifier(
+ new_ref_namespace_id,
+ old_join_data.join_info().fingerprint())));
+ }
+ }
+ ICING_ASSIGN_OR_RETURN(batch_old_join_data,
+ old_pl_accessor->GetNextDataBatch());
+ }
+
+ if (new_join_data_vec.empty()) {
+ continue;
+ }
+
+ // NamespaceId order may change, so we have to sort the vector.
+ std::sort(new_join_data_vec.begin(), new_join_data_vec.end());
+
+ // Create new posting list in new_index and prepend all new join data into
+ // it.
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>>
+ new_pl_accessor,
+ PostingListJoinDataAccessor<JoinDataType>::Create(
+ new_index->flash_index_storage_.get(),
+ new_index->posting_list_serializer_.get()));
+ for (const JoinDataType& new_join_data : new_join_data_vec) {
+ ICING_RETURN_IF_ERROR(new_pl_accessor->PrependData(new_join_data));
+ }
+
+ // Finalize the posting list and update mapper of new_index.
+ PostingListAccessor::FinalizeResult result =
+ std::move(*new_pl_accessor).Finalize();
+ if (!result.status.ok()) {
+ return result.status;
+ }
+ if (!result.id.is_valid()) {
+ return absl_ports::InternalError(
+ "Fail to flush data into posting list(s)");
+ }
+ ICING_RETURN_IF_ERROR(
+ new_index->schema_joinable_id_to_posting_list_mapper_->Put(
+ iter->GetKey(), result.id));
+
+ // Update info.
+ new_index->info().num_data += new_join_data_vec.size();
+ }
+
+ // TODO(b/268521214): transfer delete propagation storage
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::PersistMetadataToDisk(
+ bool force) {
+ if (!force && !is_info_dirty() && !is_storage_dirty()) {
+ return libtextclassifier3::Status::OK;
+ }
+
+ std::string metadata_file_path = GetMetadataFilePath(working_path_);
+
+ ScopedFd sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
+ if (!sfd.is_valid()) {
+ return absl_ports::InternalError("Fail to open metadata file for write");
+ }
+
+ if (!filesystem_.PWrite(sfd.get(), /*offset=*/0, metadata_buffer_.get(),
+ kMetadataFileSize)) {
+ return absl_ports::InternalError("Fail to write metadata file");
+ }
+
+ if (!filesystem_.DataSync(sfd.get())) {
+ return absl_ports::InternalError("Fail to sync metadata to disk");
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status QualifiedIdJoinIndexImplV2::PersistStoragesToDisk(
+ bool force) {
+ if (!force && !is_storage_dirty()) {
+ return libtextclassifier3::Status::OK;
+ }
+
+ ICING_RETURN_IF_ERROR(
+ schema_joinable_id_to_posting_list_mapper_->PersistToDisk());
+ if (!flash_index_storage_->PersistToDisk()) {
+ return absl_ports::InternalError(
+ "Fail to persist FlashIndexStorage to disk");
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<Crc32>
+QualifiedIdJoinIndexImplV2::ComputeInfoChecksum(bool force) {
+ if (!force && !is_info_dirty()) {
+ return Crc32(crcs().component_crcs.info_crc);
+ }
+
+ return info().ComputeChecksum();
+}
+
+libtextclassifier3::StatusOr<Crc32>
+QualifiedIdJoinIndexImplV2::ComputeStoragesChecksum(bool force) {
+ if (!force && !is_storage_dirty()) {
+ return Crc32(crcs().component_crcs.storages_crc);
+ }
+
+ ICING_ASSIGN_OR_RETURN(
+ Crc32 schema_joinable_id_to_posting_list_mapper_crc,
+ schema_joinable_id_to_posting_list_mapper_->ComputeChecksum());
+
+ return Crc32(schema_joinable_id_to_posting_list_mapper_crc.Get());
+}
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/join/qualified-id-join-index-impl-v2.h b/icing/join/qualified-id-join-index-impl-v2.h
new file mode 100644
index 0000000..2b0bf3f
--- /dev/null
+++ b/icing/join/qualified-id-join-index-impl-v2.h
@@ -0,0 +1,369 @@
+// Copyright (C) 2023 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_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V2_H_
+#define ICING_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V2_H_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/persistent-storage.h"
+#include "icing/file/posting_list/flash-index-storage.h"
+#include "icing/file/posting_list/posting-list-identifier.h"
+#include "icing/join/doc-join-info.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/join/posting-list-join-data-accessor.h"
+#include "icing/join/posting-list-join-data-serializer.h"
+#include "icing/join/qualified-id-join-index.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
+#include "icing/util/crc32.h"
+
+namespace icing {
+namespace lib {
+
+// QualifiedIdJoinIndexImplV2: a class to maintain join data (DocumentId to
+// referenced NamespaceFingerprintIdentifier). It stores join data in posting
+// lists and bucketizes them by (schema_type_id, joinable_property_id).
+class QualifiedIdJoinIndexImplV2 : public QualifiedIdJoinIndex {
+ public:
+ using JoinDataType = DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>;
+
+ class JoinDataIterator : public JoinDataIteratorBase {
+ public:
+ explicit JoinDataIterator(
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor)
+ : pl_accessor_(std::move(pl_accessor)),
+ should_retrieve_next_batch_(true) {}
+
+ ~JoinDataIterator() override = default;
+
+ // Advances to the next data.
+ //
+ // Returns:
+ // - OK on success
+ // - RESOURCE_EXHAUSTED_ERROR if reaching the end (i.e. no more relevant
+ // data)
+ // - Any other PostingListJoinDataAccessor errors
+ libtextclassifier3::Status Advance() override;
+
+ const JoinDataType& GetCurrent() const override { return *curr_; }
+
+ private:
+ // Gets next batch of data from the posting list chain, caches in
+ // cached_batch_integer_index_data_, and sets curr_ to the begin of the
+ // cache.
+ libtextclassifier3::Status GetNextDataBatch();
+
+ std::unique_ptr<PostingListJoinDataAccessor<JoinDataType>> pl_accessor_;
+ std::vector<JoinDataType> cached_batch_join_data_;
+ std::vector<JoinDataType>::const_iterator curr_;
+ bool should_retrieve_next_batch_;
+ };
+
+ struct Info {
+ static constexpr int32_t kMagic = 0x12d1c074;
+
+ int32_t magic;
+ int32_t num_data;
+ DocumentId last_added_document_id;
+
+ Crc32 ComputeChecksum() const {
+ return Crc32(
+ std::string_view(reinterpret_cast<const char*>(this), sizeof(Info)));
+ }
+ } __attribute__((packed));
+ static_assert(sizeof(Info) == 12, "");
+
+ // Metadata file layout: <Crcs><Info>
+ static constexpr int32_t kCrcsMetadataBufferOffset = 0;
+ static constexpr int32_t kInfoMetadataBufferOffset =
+ static_cast<int32_t>(sizeof(Crcs));
+ static constexpr int32_t kMetadataFileSize = sizeof(Crcs) + sizeof(Info);
+ static_assert(kMetadataFileSize == 24, "");
+
+ static constexpr WorkingPathType kWorkingPathType =
+ WorkingPathType::kDirectory;
+
+ // Creates a QualifiedIdJoinIndexImplV2 instance to store join data
+ // (DocumentId to referenced NamespaceFingerPrintIdentifier) for future
+ // joining search. If any of the underlying file is missing, then delete the
+ // whole working_path and (re)initialize with new ones. Otherwise initialize
+ // and create the instance by existing files.
+ //
+ // filesystem: Object to make system level calls
+ // working_path: Specifies the working path for PersistentStorage.
+ // QualifiedIdJoinIndexImplV2 uses working path as working
+ // directory and all related files will be stored under this
+ // directory. It takes full ownership and of working_path_,
+ // including creation/deletion. It is the caller's
+ // responsibility to specify correct working path and avoid
+ // mixing different persistent storages together under the same
+ // path. Also the caller has the ownership for the parent
+ // directory of working_path_, and it is responsible for parent
+ // directory creation/deletion. See PersistentStorage for more
+ // details about the concept of working_path.
+ // pre_mapping_fbv: flag indicating whether memory map max possible file size
+ // for underlying FileBackedVector before growing the actual
+ // file size.
+ //
+ // Returns:
+ // - FAILED_PRECONDITION_ERROR if the file checksum doesn't match the stored
+ // checksum
+ // - INTERNAL_ERROR on I/O errors
+ // - Any KeyMapper errors
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+ Create(const Filesystem& filesystem, std::string working_path,
+ bool pre_mapping_fbv);
+
+ // Delete copy and move constructor/assignment operator.
+ QualifiedIdJoinIndexImplV2(const QualifiedIdJoinIndexImplV2&) = delete;
+ QualifiedIdJoinIndexImplV2& operator=(const QualifiedIdJoinIndexImplV2&) =
+ delete;
+
+ QualifiedIdJoinIndexImplV2(QualifiedIdJoinIndexImplV2&&) = delete;
+ QualifiedIdJoinIndexImplV2& operator=(QualifiedIdJoinIndexImplV2&&) = delete;
+
+ ~QualifiedIdJoinIndexImplV2() override;
+
+ // v1 only API. Returns UNIMPLEMENTED_ERROR.
+ libtextclassifier3::Status Put(
+ const DocJoinInfo& doc_join_info,
+ std::string_view ref_qualified_id_str) override {
+ return absl_ports::UnimplementedError("This API is not supported in V2");
+ }
+
+ // v1 only API. Returns UNIMPLEMENTED_ERROR.
+ libtextclassifier3::StatusOr<std::string_view> Get(
+ const DocJoinInfo& doc_join_info) const override {
+ return absl_ports::UnimplementedError("This API is not supported in V2");
+ }
+
+ // Puts a list of referenced (parent) NamespaceFingerprintIdentifiers into
+ // the join index, given the (child) DocumentId, SchemaTypeId and
+ // JoinablePropertyId.
+ //
+ // Returns:
+ // - OK on success
+ // - INVALID_ARGUMENT_ERROR if schema_type_id, joinable_property_id, or
+ // document_id is invalid
+ // - Any KeyMapper/FlashIndexStorage errors
+ libtextclassifier3::Status Put(SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id,
+ DocumentId document_id,
+ std::vector<NamespaceFingerprintIdentifier>&&
+ ref_namespace_fingerprint_ids) override;
+
+ // Returns a JoinDataIterator for iterating through all join data of the
+ // specified (schema_type_id, joinable_property_id).
+ //
+ // Returns:
+ // - On success: a JoinDataIterator
+ // - INVALID_ARGUMENT_ERROR if schema_type_id or joinable_property_id is
+ // invalid
+ // - Any KeyMapper/FlashIndexStorage errors
+ libtextclassifier3::StatusOr<std::unique_ptr<JoinDataIteratorBase>>
+ GetIterator(SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) const override;
+
+ // Reduces internal file sizes by reclaiming space and ids of deleted
+ // documents. Qualified id join index will convert all entries to the new
+ // document ids and namespace ids.
+ //
+ // - document_id_old_to_new: a map for converting old document id to new
+ // document id.
+ // - namespace_id_old_to_new: a map for converting old namespace id to new
+ // namespace id.
+ // - new_last_added_document_id: will be used to update the last added
+ // document id in the qualified id join index.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error. This could potentially leave the index in
+ // an invalid state and the caller should handle it properly (e.g. discard
+ // and rebuild)
+ libtextclassifier3::Status Optimize(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ DocumentId new_last_added_document_id) override;
+
+ // Clears all data and set last_added_document_id to kInvalidDocumentId.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status Clear() override;
+
+ bool is_v2() const override { return true; }
+
+ int32_t size() const override { return info().num_data; }
+
+ bool empty() const override { return size() == 0; }
+
+ DocumentId last_added_document_id() const override {
+ return info().last_added_document_id;
+ }
+
+ void set_last_added_document_id(DocumentId document_id) override {
+ SetInfoDirty();
+
+ Info& info_ref = info();
+ if (info_ref.last_added_document_id == kInvalidDocumentId ||
+ document_id > info_ref.last_added_document_id) {
+ info_ref.last_added_document_id = document_id;
+ }
+ }
+
+ private:
+ explicit QualifiedIdJoinIndexImplV2(
+ const Filesystem& filesystem, std::string&& working_path,
+ std::unique_ptr<uint8_t[]> metadata_buffer,
+ std::unique_ptr<KeyMapper<PostingListIdentifier>>
+ schema_joinable_id_to_posting_list_mapper,
+ std::unique_ptr<PostingListJoinDataSerializer<JoinDataType>>
+ posting_list_serializer,
+ std::unique_ptr<FlashIndexStorage> flash_index_storage,
+ bool pre_mapping_fbv)
+ : QualifiedIdJoinIndex(filesystem, std::move(working_path)),
+ metadata_buffer_(std::move(metadata_buffer)),
+ schema_joinable_id_to_posting_list_mapper_(
+ std::move(schema_joinable_id_to_posting_list_mapper)),
+ posting_list_serializer_(std::move(posting_list_serializer)),
+ flash_index_storage_(std::move(flash_index_storage)),
+ pre_mapping_fbv_(pre_mapping_fbv),
+ is_info_dirty_(false),
+ is_storage_dirty_(false) {}
+
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+ InitializeNewFiles(const Filesystem& filesystem, std::string&& working_path,
+ bool pre_mapping_fbv);
+
+ static libtextclassifier3::StatusOr<
+ std::unique_ptr<QualifiedIdJoinIndexImplV2>>
+ InitializeExistingFiles(const Filesystem& filesystem,
+ std::string&& working_path, bool pre_mapping_fbv);
+
+ // Transfers qualified id join index data from the current to new_index and
+ // convert to new document id according to document_id_old_to_new and
+ // namespace_id_old_to_new. It is a helper function for Optimize.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status TransferIndex(
+ const std::vector<DocumentId>& document_id_old_to_new,
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ QualifiedIdJoinIndexImplV2* new_index) const;
+
+ // Flushes contents of metadata file.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status PersistMetadataToDisk(bool force) override;
+
+ // Flushes contents of all storages to underlying files.
+ //
+ // Returns:
+ // - OK on success
+ // - INTERNAL_ERROR on I/O error
+ libtextclassifier3::Status PersistStoragesToDisk(bool force) override;
+
+ // Computes and returns Info checksum.
+ //
+ // Returns:
+ // - Crc of the Info on success
+ libtextclassifier3::StatusOr<Crc32> ComputeInfoChecksum(bool force) override;
+
+ // Computes and returns all storages checksum.
+ //
+ // Returns:
+ // - Crc of all storages on success
+ // - INTERNAL_ERROR if any data inconsistency
+ libtextclassifier3::StatusOr<Crc32> ComputeStoragesChecksum(
+ bool force) override;
+
+ Crcs& crcs() override {
+ return *reinterpret_cast<Crcs*>(metadata_buffer_.get() +
+ kCrcsMetadataBufferOffset);
+ }
+
+ const Crcs& crcs() const override {
+ return *reinterpret_cast<const Crcs*>(metadata_buffer_.get() +
+ kCrcsMetadataBufferOffset);
+ }
+
+ Info& info() {
+ return *reinterpret_cast<Info*>(metadata_buffer_.get() +
+ kInfoMetadataBufferOffset);
+ }
+
+ const Info& info() const {
+ return *reinterpret_cast<const Info*>(metadata_buffer_.get() +
+ kInfoMetadataBufferOffset);
+ }
+
+ void SetInfoDirty() { is_info_dirty_ = true; }
+ // When storage is dirty, we have to set info dirty as well. So just expose
+ // SetDirty to set both.
+ void SetDirty() {
+ is_info_dirty_ = true;
+ is_storage_dirty_ = true;
+ }
+
+ bool is_info_dirty() const { return is_info_dirty_; }
+ bool is_storage_dirty() const { return is_storage_dirty_; }
+
+ // Metadata buffer
+ std::unique_ptr<uint8_t[]> metadata_buffer_;
+
+ // Persistent KeyMapper for mapping (schema_type_id, joinable_property_id) to
+ // PostingListIdentifier.
+ std::unique_ptr<KeyMapper<PostingListIdentifier>>
+ schema_joinable_id_to_posting_list_mapper_;
+
+ // Posting list related members. Use posting list to store join data
+ // (document id to referenced NamespaceFingerprintIdentifier).
+ std::unique_ptr<PostingListJoinDataSerializer<JoinDataType>>
+ posting_list_serializer_;
+ std::unique_ptr<FlashIndexStorage> flash_index_storage_;
+
+ // TODO(b/268521214): add delete propagation storage
+
+ // Flag indicating whether memory map max possible file size for underlying
+ // FileBackedVector before growing the actual file size.
+ bool pre_mapping_fbv_;
+
+ bool is_info_dirty_;
+ bool is_storage_dirty_;
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_JOIN_QUALIFIED_ID_JOIN_INDEX_IMPL_V2_H_
diff --git a/icing/join/qualified-id-join-index-impl-v2_test.cc b/icing/join/qualified-id-join-index-impl-v2_test.cc
new file mode 100644
index 0000000..d73d6c2
--- /dev/null
+++ b/icing/join/qualified-id-join-index-impl-v2_test.cc
@@ -0,0 +1,1414 @@
+// Copyright (C) 2023 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/join/qualified-id-join-index-impl-v2.h"
+
+#include <cstdint>
+#include <memory>
+#include <numeric>
+#include <string>
+#include <string_view>
+#include <utility>
+#include <vector>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/absl_ports/str_cat.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/persistent-storage.h"
+#include "icing/file/posting_list/posting-list-identifier.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/join/qualified-id-join-index.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
+#include "icing/store/persistent-hash-map-key-mapper.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/tmp-directory.h"
+#include "icing/util/crc32.h"
+#include "icing/util/status-macros.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+using ::testing::ElementsAre;
+using ::testing::Eq;
+using ::testing::HasSubstr;
+using ::testing::IsEmpty;
+using ::testing::IsTrue;
+using ::testing::Lt;
+using ::testing::Ne;
+using ::testing::Not;
+using ::testing::Pointee;
+using ::testing::SizeIs;
+
+using Crcs = PersistentStorage::Crcs;
+using Info = QualifiedIdJoinIndexImplV2::Info;
+
+static constexpr int32_t kCorruptedValueOffset = 3;
+
+class QualifiedIdJoinIndexImplV2Test : public ::testing::Test {
+ protected:
+ void SetUp() override {
+ base_dir_ = GetTestTempDir() + "/icing";
+ ASSERT_THAT(filesystem_.CreateDirectoryRecursively(base_dir_.c_str()),
+ IsTrue());
+
+ working_path_ = base_dir_ + "/qualified_id_join_index_impl_v2_test";
+ }
+
+ void TearDown() override {
+ filesystem_.DeleteDirectoryRecursively(base_dir_.c_str());
+ }
+
+ Filesystem filesystem_;
+ std::string base_dir_;
+ std::string working_path_;
+};
+
+libtextclassifier3::StatusOr<
+ std::vector<QualifiedIdJoinIndexImplV2::JoinDataType>>
+GetJoinData(const QualifiedIdJoinIndexImplV2& index,
+ SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) {
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<QualifiedIdJoinIndex::JoinDataIteratorBase> iter,
+ index.GetIterator(schema_type_id, joinable_property_id));
+
+ std::vector<QualifiedIdJoinIndexImplV2::JoinDataType> result;
+ while (iter->Advance().ok()) {
+ result.push_back(iter->GetCurrent());
+ }
+
+ return result;
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, InvalidWorkingPath) {
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(
+ filesystem_, "/dev/null/qualified_id_join_index_impl_v2_test",
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, InitializeNewFiles) {
+ {
+ // Create new qualified id join index
+ ASSERT_FALSE(filesystem_.DirectoryExists(working_path_.c_str()));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ // Metadata file should be initialized correctly for both info and crcs
+ // sections.
+ const std::string metadata_file_path =
+ absl_ports::StrCat(working_path_, "/metadata");
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize);
+ ASSERT_THAT(
+ filesystem_.PRead(metadata_file_path.c_str(), metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize,
+ /*offset=*/0),
+ IsTrue());
+
+ // Check info section
+ const Info* info = reinterpret_cast<const Info*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kInfoMetadataBufferOffset);
+ EXPECT_THAT(info->magic, Eq(Info::kMagic));
+ EXPECT_THAT(info->num_data, Eq(0));
+ EXPECT_THAT(info->last_added_document_id, Eq(kInvalidDocumentId));
+
+ // Check crcs section
+ const Crcs* crcs = reinterpret_cast<const Crcs*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kCrcsMetadataBufferOffset);
+ // There are some initial info in KeyMapper, so storages_crc should be
+ // non-zero.
+ EXPECT_THAT(crcs->component_crcs.storages_crc, Ne(0));
+ EXPECT_THAT(crcs->component_crcs.info_crc,
+ Eq(Crc32(std::string_view(reinterpret_cast<const char*>(info),
+ sizeof(Info)))
+ .Get()));
+ EXPECT_THAT(crcs->all_crc,
+ Eq(Crc32(std::string_view(
+ reinterpret_cast<const char*>(&crcs->component_crcs),
+ sizeof(Crcs::ComponentCrcs)))
+ .Get()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializationShouldFailWithoutPersistToDiskOrDestruction) {
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/78);
+
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ // Insert some data.
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2, id1}));
+ ICING_ASSERT_OK(index->PersistToDisk());
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/3, /*joinable_property_id=*/10, /*document_id=*/6,
+ /*ref_namespace_fingerprint_ids=*/{id3}));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}));
+
+ // Without calling PersistToDisk, checksums will not be recomputed or synced
+ // to disk, so initializing another instance on the same files should fail.
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializationShouldSucceedWithPersistToDisk) {
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/78);
+
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index1,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ // Insert some data.
+ ICING_ASSERT_OK(index1->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2, id1}));
+ ICING_ASSERT_OK(index1->Put(
+ /*schema_type_id=*/3, /*joinable_property_id=*/10, /*document_id=*/6,
+ /*ref_namespace_fingerprint_ids=*/{id3}));
+ ICING_ASSERT_OK(index1->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}));
+ ASSERT_THAT(index1, Pointee(SizeIs(4)));
+
+ // After calling PersistToDisk, all checksums should be recomputed and synced
+ // correctly to disk, so initializing another instance on the same files
+ // should succeed, and we should be able to get the same contents.
+ ICING_EXPECT_OK(index1->PersistToDisk());
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index2,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index2, Pointee(SizeIs(4)));
+ EXPECT_THAT(
+ GetJoinData(*index2, /*schema_type_id=*/2, /*joinable_property_id=*/1),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1))));
+ EXPECT_THAT(
+ GetJoinData(*index2, /*schema_type_id=*/3, /*joinable_property_id=*/10),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, /*join_info=*/id3))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializationShouldSucceedAfterDestruction) {
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/78);
+
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ // Insert some data.
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2, id1}));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/3, /*joinable_property_id=*/10, /*document_id=*/6,
+ /*ref_namespace_fingerprint_ids=*/{id3}));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}));
+ ASSERT_THAT(index, Pointee(SizeIs(4)));
+ }
+
+ {
+ // The previous instance went out of scope and was destructed. Although we
+ // didn't call PersistToDisk explicitly, the destructor should invoke it and
+ // thus initializing another instance on the same files should succeed, and
+ // we should be able to get the same contents.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/2, /*joinable_property_id=*/1),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1))));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/3, /*joinable_property_id=*/10),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, /*join_info=*/id3))));
+ }
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializeExistingFilesWithDifferentMagicShouldFail) {
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/
+ {NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/12)}));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ {
+ const std::string metadata_file_path =
+ absl_ports::StrCat(working_path_, "/metadata");
+ ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
+ ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
+
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize);
+ ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize,
+ /*offset=*/0),
+ IsTrue());
+
+ // Manually change magic and update checksum
+ Crcs* crcs = reinterpret_cast<Crcs*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kCrcsMetadataBufferOffset);
+ Info* info = reinterpret_cast<Info*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kInfoMetadataBufferOffset);
+ info->magic += kCorruptedValueOffset;
+ crcs->component_crcs.info_crc = info->ComputeChecksum().Get();
+ crcs->all_crc = crcs->component_crcs.ComputeChecksum().Get();
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize),
+ IsTrue());
+ }
+
+ // Attempt to create the qualified id join index with different magic. This
+ // should fail.
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
+ HasSubstr("Incorrect magic value")));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializeExistingFilesWithWrongAllCrcShouldFail) {
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/
+ {NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/12)}));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ {
+ const std::string metadata_file_path =
+ absl_ports::StrCat(working_path_, "/metadata");
+ ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
+ ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
+
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize);
+ ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize,
+ /*offset=*/0),
+ IsTrue());
+
+ // Manually corrupt all_crc
+ Crcs* crcs = reinterpret_cast<Crcs*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kCrcsMetadataBufferOffset);
+ crcs->all_crc += kCorruptedValueOffset;
+
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize),
+ IsTrue());
+ }
+
+ // Attempt to create the qualified id join index with metadata containing
+ // corrupted all_crc. This should fail.
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
+ HasSubstr("Invalid all crc")));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ InitializeExistingFilesWithCorruptedInfoShouldFail) {
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/
+ {NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/12)}));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ {
+ const std::string metadata_file_path =
+ absl_ports::StrCat(working_path_, "/metadata");
+ ScopedFd metadata_sfd(filesystem_.OpenForWrite(metadata_file_path.c_str()));
+ ASSERT_THAT(metadata_sfd.is_valid(), IsTrue());
+
+ auto metadata_buffer = std::make_unique<uint8_t[]>(
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize);
+ ASSERT_THAT(filesystem_.PRead(metadata_sfd.get(), metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize,
+ /*offset=*/0),
+ IsTrue());
+
+ // Modify info, but don't update the checksum. This would be similar to
+ // corruption of info.
+ Info* info = reinterpret_cast<Info*>(
+ metadata_buffer.get() +
+ QualifiedIdJoinIndexImplV2::kInfoMetadataBufferOffset);
+ info->last_added_document_id += kCorruptedValueOffset;
+
+ ASSERT_THAT(filesystem_.PWrite(
+ metadata_sfd.get(), /*offset=*/0, metadata_buffer.get(),
+ QualifiedIdJoinIndexImplV2::kMetadataFileSize),
+ IsTrue());
+ }
+
+ // Attempt to create the qualified id join index with info that doesn't match
+ // its checksum. This should fail.
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
+ HasSubstr("Invalid info crc")));
+}
+
+TEST_F(
+ QualifiedIdJoinIndexImplV2Test,
+ InitializeExistingFilesWithCorruptedSchemaJoinableIdToPostingListMapperShouldFail) {
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/
+ {NamespaceFingerprintIdentifier(/*namespace_id=*/1,
+ /*fingerprint=*/12)}));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ // Corrupt schema_joinable_id_to_posting_list_mapper manually.
+ {
+ std::string mapper_working_path = absl_ports::StrCat(
+ working_path_, "/schema_joinable_id_to_posting_list_mapper");
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<PostingListIdentifier>> mapper,
+ PersistentHashMapKeyMapper<PostingListIdentifier>::Create(
+ filesystem_, std::move(mapper_working_path),
+ /*pre_mapping_fbv=*/false));
+ ICING_ASSERT_OK_AND_ASSIGN(Crc32 old_crc, mapper->ComputeChecksum());
+ ICING_ASSERT_OK(mapper->Put("foo", PostingListIdentifier::kInvalid));
+ ICING_ASSERT_OK(mapper->PersistToDisk());
+ ICING_ASSERT_OK_AND_ASSIGN(Crc32 new_crc, mapper->ComputeChecksum());
+ ASSERT_THAT(old_crc, Not(Eq(new_crc)));
+ }
+
+ // Attempt to create the qualified id join index with corrupted
+ // doc_join_info_mapper. This should fail.
+ EXPECT_THAT(QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION,
+ HasSubstr("Invalid storages crc")));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, InvalidPut) {
+ NamespaceFingerprintIdentifier id(/*namespace_id=*/1, /*fingerprint=*/12);
+
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(
+ index->Put(/*schema_type_id=*/-1, /*joinable_property_id=*/1,
+ /*document_id=*/5, /*ref_namespace_fingerprint_ids=*/{id}),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(
+ index->Put(/*schema_type_id=*/2, /*joinable_property_id=*/-1,
+ /*document_id=*/5, /*ref_namespace_fingerprint_ids=*/{id}),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(index->Put(/*schema_type_id=*/2, /*joinable_property_id=*/1,
+ /*document_id=*/kInvalidDocumentId,
+ /*ref_namespace_fingerprint_ids=*/{id}),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, InvalidGetIterator) {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(
+ index->GetIterator(/*schema_type_id=*/-1, /*joinable_property_id=*/1),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(
+ index->GetIterator(/*schema_type_id=*/2, /*joinable_property_id=*/-1),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ PutEmptyRefNamespaceFingerprintIdsShouldReturnOk) {
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{}),
+ IsOk());
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id + 1, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id + 1),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ PutAndGetSingleSchemaTypeAndJoinableProperty) {
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/3, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/2, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/0, /*fingerprint=*/78);
+
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2, id1}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/6,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, /*join_info=*/id3),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id + 1, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id + 1),
+ IsOkAndHolds(IsEmpty()));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ // Verify we can get all of them after destructing and re-initializing.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/6, /*join_info=*/id3),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id + 1, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id + 1),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test,
+ PutAndGetMultipleSchemaTypesAndJoinableProperties) {
+ SchemaTypeId schema_type_id1 = 2;
+ SchemaTypeId schema_type_id2 = 4;
+
+ JoinablePropertyId joinable_property_id1 = 1;
+ JoinablePropertyId joinable_property_id2 = 10;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/3, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/2, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/0, /*fingerprint=*/78);
+
+ {
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id1}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id2, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id2, joinable_property_id1, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id2, joinable_property_id2, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+
+ EXPECT_THAT(GetJoinData(*index, schema_type_id1, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id1, joinable_property_id2),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id2, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id3))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id2, joinable_property_id2),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4))));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ }
+
+ // Verify we can get all of them after destructing and re-initializing.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id1, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id1))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id1, joinable_property_id2),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/5, /*join_info=*/id2))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id2, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id3))));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id2, joinable_property_id2),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/12, /*join_info=*/id4))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, SetLastAddedDocumentId) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ constexpr DocumentId kDocumentId = 100;
+ index->set_last_added_document_id(kDocumentId);
+ EXPECT_THAT(index->last_added_document_id(), Eq(kDocumentId));
+
+ constexpr DocumentId kNextDocumentId = 123;
+ index->set_last_added_document_id(kNextDocumentId);
+ EXPECT_THAT(index->last_added_document_id(), Eq(kNextDocumentId));
+}
+
+TEST_F(
+ QualifiedIdJoinIndexImplV2Test,
+ SetLastAddedDocumentIdShouldIgnoreNewDocumentIdNotGreaterThanTheCurrent) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ constexpr DocumentId kDocumentId = 123;
+ index->set_last_added_document_id(kDocumentId);
+ ASSERT_THAT(index->last_added_document_id(), Eq(kDocumentId));
+
+ constexpr DocumentId kNextDocumentId = 100;
+ ASSERT_THAT(kNextDocumentId, Lt(kDocumentId));
+ index->set_last_added_document_id(kNextDocumentId);
+ // last_added_document_id() should remain unchanged.
+ EXPECT_THAT(index->last_added_document_id(), Eq(kDocumentId));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, Optimize) {
+ // General test for Optimize().
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id1 = 2;
+ SchemaTypeId schema_type_id2 = 5;
+
+ JoinablePropertyId joinable_property_id1 = 11;
+ JoinablePropertyId joinable_property_id2 = 15;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/2, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/3, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/4, /*fingerprint=*/103);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/0, /*fingerprint=*/104);
+ NamespaceFingerprintIdentifier id5(/*namespace_id=*/0, /*fingerprint=*/105);
+ NamespaceFingerprintIdentifier id6(/*namespace_id=*/1, /*fingerprint=*/106);
+ NamespaceFingerprintIdentifier id7(/*namespace_id=*/3, /*fingerprint=*/107);
+ NamespaceFingerprintIdentifier id8(/*namespace_id=*/2, /*fingerprint=*/108);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id1, /*document_id=*/3,
+ /*ref_namespace_fingerprint_ids=*/{id1, id2, id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id2, joinable_property_id2, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id2, joinable_property_id2, /*document_id=*/8,
+ /*ref_namespace_fingerprint_ids=*/{id5, id6}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id1, /*document_id=*/13,
+ /*ref_namespace_fingerprint_ids=*/{id7}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id1, /*document_id=*/21,
+ /*ref_namespace_fingerprint_ids=*/{id8}),
+ IsOk());
+ index->set_last_added_document_id(21);
+
+ ASSERT_THAT(index, Pointee(SizeIs(8)));
+
+ // Delete doc id = 5, 13, compress and keep the rest.
+ std::vector<DocumentId> document_id_old_to_new(22, kInvalidDocumentId);
+ document_id_old_to_new[3] = 0;
+ document_id_old_to_new[8] = 1;
+ document_id_old_to_new[21] = 2;
+
+ // Delete namespace id 1, 2 (and invalidate id1, id6, id8). Reorder namespace
+ // ids [0, 3, 4] to [1, 2, 0].
+ std::vector<NamespaceId> namespace_id_old_to_new(5, kInvalidNamespaceId);
+ namespace_id_old_to_new[0] = 1;
+ namespace_id_old_to_new[3] = 2;
+ namespace_id_old_to_new[4] = 0;
+
+ DocumentId new_last_added_document_id = 2;
+ EXPECT_THAT(index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ new_last_added_document_id),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(3)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(new_last_added_document_id));
+
+ // Verify GetIterator API should work normally after Optimize().
+ // 1) schema_type_id1, joinable_property_id1:
+ // - old_doc_id=21, old_ref_namespace_id=2: NOT FOUND
+ // - old_doc_id=13, old_ref_namespace_id=3: NOT FOUND
+ // - old_doc_id=3, old_ref_namespace_id=4:
+ // become new_doc_id=0, new_ref_namespace_id=0
+ // - old_doc_id=3, old_ref_namespace_id=3:
+ // become new_doc_id=0, new_ref_namespace_id=2
+ // - old_doc_id=3, old_ref_namespace_id=2: NOT FOUND
+ //
+ // For new_doc_id=0, it should reorder due to posting list restriction.
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id1, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/102)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/103)))));
+
+ // 2) schema_type_id2, joinable_property_id2:
+ // - old_doc_id=8, old_ref_namespace_id=1: NOT FOUND
+ // - old_doc_id=8, old_ref_namespace_id=0:
+ // become new_doc_id=1, new_ref_namespace_id=1
+ // - old_doc_id=5, old_ref_namespace_id=0: NOT FOUND
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id2, joinable_property_id2),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/105)))));
+
+ // Verify Put API should work normally after Optimize().
+ NamespaceFingerprintIdentifier id9(/*namespace_id=*/1, /*fingerprint=*/109);
+ EXPECT_THAT(
+ index->Put(schema_type_id1, joinable_property_id1, /*document_id=*/99,
+ /*ref_namespace_fingerprint_ids=*/{id9}),
+ IsOk());
+ index->set_last_added_document_id(99);
+
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(99));
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id1, joinable_property_id1),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/99, /*join_info=*/id9),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/102)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/103)))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeDocumentIdChange) {
+ // Specific test for Optimize(): document id compaction.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/103);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/104);
+ NamespaceFingerprintIdentifier id5(/*namespace_id=*/1, /*fingerprint=*/105);
+ NamespaceFingerprintIdentifier id6(/*namespace_id=*/1, /*fingerprint=*/106);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/3,
+ /*ref_namespace_fingerprint_ids=*/{id1, id2}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/8,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/13,
+ /*ref_namespace_fingerprint_ids=*/{id5}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/21,
+ /*ref_namespace_fingerprint_ids=*/{id6}),
+ IsOk());
+ index->set_last_added_document_id(21);
+
+ ASSERT_THAT(index, Pointee(SizeIs(6)));
+
+ // Delete doc id = 5, 8, compress and keep the rest.
+ std::vector<DocumentId> document_id_old_to_new(22, kInvalidDocumentId);
+ document_id_old_to_new[3] = 0;
+ document_id_old_to_new[13] = 1;
+ document_id_old_to_new[21] = 2;
+
+ // No change for namespace id.
+ std::vector<NamespaceId> namespace_id_old_to_new = {0, 1};
+
+ DocumentId new_last_added_document_id = 2;
+ EXPECT_THAT(index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ new_last_added_document_id),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(new_last_added_document_id));
+
+ // Verify GetIterator API should work normally after Optimize().
+ // - old_doc_id=21, join_info=id6: become doc_id=2, join_info=id6
+ // - old_doc_id=13, join_info=id5: become doc_id=1, join_info=id5
+ // - old_doc_id=8, join_info=id4: NOT FOUND
+ // - old_doc_id=5, join_info=id3: NOT FOUND
+ // - old_doc_id=3, join_info=id2: become doc_id=0, join_info=id2
+ // - old_doc_id=3, join_info=id1: become doc_id=0, join_info=id1
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, /*join_info=*/id6),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, /*join_info=*/id5),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/id1))));
+
+ // Verify Put API should work normally after Optimize().
+ NamespaceFingerprintIdentifier id7(/*namespace_id=*/1, /*fingerprint=*/107);
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/99,
+ /*ref_namespace_fingerprint_ids=*/{id7}),
+ IsOk());
+ index->set_last_added_document_id(99);
+
+ EXPECT_THAT(index, Pointee(SizeIs(5)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(99));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/99, /*join_info=*/id7),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, /*join_info=*/id6),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, /*join_info=*/id5),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/id1))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeOutOfRangeDocumentId) {
+ // Specific test for Optimize() for out of range document id.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+ NamespaceFingerprintIdentifier id(/*namespace_id=*/1, /*fingerprint=*/101);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/99,
+ /*ref_namespace_fingerprint_ids=*/{id}),
+ IsOk());
+ index->set_last_added_document_id(99);
+
+ // Create document_id_old_to_new with size = 1. Optimize should handle out of
+ // range DocumentId properly.
+ std::vector<DocumentId> document_id_old_to_new = {kInvalidDocumentId};
+ std::vector<NamespaceId> namespace_id_old_to_new = {0, 1};
+
+ // There shouldn't be any error due to vector index.
+ EXPECT_THAT(
+ index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ /*new_last_added_document_id=*/kInvalidDocumentId),
+ IsOk());
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ // Verify all data are discarded after Optimize().
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeDeleteAllDocuments) {
+ // Specific test for Optimize(): delete all document ids.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/103);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/104);
+ NamespaceFingerprintIdentifier id5(/*namespace_id=*/1, /*fingerprint=*/105);
+ NamespaceFingerprintIdentifier id6(/*namespace_id=*/1, /*fingerprint=*/106);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/3,
+ /*ref_namespace_fingerprint_ids=*/{id1, id2}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/8,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/13,
+ /*ref_namespace_fingerprint_ids=*/{id5}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/21,
+ /*ref_namespace_fingerprint_ids=*/{id6}),
+ IsOk());
+ index->set_last_added_document_id(21);
+
+ ASSERT_THAT(index, Pointee(SizeIs(6)));
+
+ // Delete all documents.
+ std::vector<DocumentId> document_id_old_to_new(22, kInvalidDocumentId);
+
+ // No change for namespace id.
+ std::vector<NamespaceId> namespace_id_old_to_new = {0, 1};
+
+ EXPECT_THAT(
+ index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ /*new_last_added_document_id=*/kInvalidDocumentId),
+ IsOk());
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ // Verify all data are discarded after Optimize().
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeNamespaceIdChange) {
+ // Specific test for Optimize(): referenced namespace id compaction.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/3, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/5, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/4, /*fingerprint=*/103);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/0, /*fingerprint=*/104);
+ NamespaceFingerprintIdentifier id5(/*namespace_id=*/2, /*fingerprint=*/105);
+ NamespaceFingerprintIdentifier id6(/*namespace_id=*/1, /*fingerprint=*/106);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/2,
+ /*ref_namespace_fingerprint_ids=*/{id1}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/3,
+ /*ref_namespace_fingerprint_ids=*/{id2}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/8,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/13,
+ /*ref_namespace_fingerprint_ids=*/{id5}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/21,
+ /*ref_namespace_fingerprint_ids=*/{id6}),
+ IsOk());
+ index->set_last_added_document_id(21);
+
+ ASSERT_THAT(index, Pointee(SizeIs(6)));
+
+ // No change for document id.
+ std::vector<DocumentId> document_id_old_to_new(22);
+ std::iota(document_id_old_to_new.begin(), document_id_old_to_new.end(), 0);
+
+ // Delete namespace id 2, 4. Reorder namespace id [0, 1, 3, 5] to [2, 3, 1,
+ // 0].
+ std::vector<NamespaceId> namespace_id_old_to_new(6, kInvalidNamespaceId);
+ namespace_id_old_to_new[0] = 2;
+ namespace_id_old_to_new[1] = 3;
+ namespace_id_old_to_new[3] = 1;
+ namespace_id_old_to_new[5] = 0;
+
+ DocumentId new_last_added_document_id = 21;
+ EXPECT_THAT(index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ new_last_added_document_id),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(new_last_added_document_id));
+
+ // Verify GetIterator API should work normally after Optimize().
+ // - id6 (old_namespace_id=1): new_namespace_id=3 (document_id = 21)
+ // - id5 (old_namespace_id=2): NOT FOUND
+ // - id4 (old_namespace_id=0): new_namespace_id=2 (document_id = 8)
+ // - id3 (old_namespace_id=4): NOT FOUND
+ // - id2 (old_namespace_id=5): new_namespace_id=0 (document_id = 3)
+ // - id1 (old_namespace_id=3): new_namespace_id=1 (document_id = 2)
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/21, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/3, /*fingerprint=*/106)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/8, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/104)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/102)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/101)))));
+
+ // Verify Put API should work normally after Optimize().
+ NamespaceFingerprintIdentifier id7(/*namespace_id=*/1, /*fingerprint=*/107);
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/99,
+ /*ref_namespace_fingerprint_ids=*/{id7}),
+ IsOk());
+ index->set_last_added_document_id(99);
+
+ EXPECT_THAT(index, Pointee(SizeIs(5)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(99));
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/99, /*join_info=*/id7),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/21, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/3, /*fingerprint=*/106)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/8, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/104)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/3, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/102)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/2, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/101)))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeNamespaceIdChangeShouldReorder) {
+ // Specific test for Optimize(): referenced namespace id reorder.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/0, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/2, /*fingerprint=*/103);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/104);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/0,
+ /*ref_namespace_fingerprint_ids=*/{id1, id2, id3}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/1,
+ /*ref_namespace_fingerprint_ids=*/{id4}),
+ IsOk());
+ index->set_last_added_document_id(1);
+
+ ASSERT_THAT(index, Pointee(SizeIs(4)));
+
+ // No change for document id.
+ std::vector<DocumentId> document_id_old_to_new = {0, 1};
+
+ // Reorder namespace id [0, 1, 2] to [2, 0, 1].
+ std::vector<NamespaceId> namespace_id_old_to_new = {2, 0, 1};
+
+ DocumentId new_last_added_document_id = 1;
+ EXPECT_THAT(index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ new_last_added_document_id),
+ IsOk());
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(new_last_added_document_id));
+
+ // Verify GetIterator API should work normally after Optimize().
+ // - id4 (old_namespace_id=1): new_namespace_id=0 (document_id = 1)
+ // - id3 (old_namespace_id=2): new_namespace_id=1 (document_id = 0)
+ // - id2 (old_namespace_id=1): new_namespace_id=0 (document_id = 0)
+ // - id1 (old_namespace_id=0): new_namespace_id=2 (document_id = 0)
+ //
+ // Should reorder to [id4, id1, id3, id2] due to posting list restriction.
+ EXPECT_THAT(
+ GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(ElementsAre(
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/1, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/104)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/2, /*fingerprint=*/101)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/1, /*fingerprint=*/103)),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/0, /*join_info=*/NamespaceFingerprintIdentifier(
+ /*namespace_id=*/0, /*fingerprint=*/102)))));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeOutOfRangeNamespaceId) {
+ // Specific test for Optimize(): out of range referenced namespace id.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+ NamespaceFingerprintIdentifier id(/*namespace_id=*/99, /*fingerprint=*/101);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/0,
+ /*ref_namespace_fingerprint_ids=*/{id}),
+ IsOk());
+ index->set_last_added_document_id(0);
+
+ // Create namespace_id_old_to_new with size = 1. Optimize should handle out of
+ // range NamespaceId properly.
+ std::vector<DocumentId> document_id_old_to_new = {0};
+ std::vector<NamespaceId> namespace_id_old_to_new = {kInvalidNamespaceId};
+
+ // There shouldn't be any error due to vector index.
+ EXPECT_THAT(
+ index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ /*new_last_added_document_id=*/kInvalidDocumentId),
+ IsOk());
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ // Verify all data are discarded after Optimize().
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, OptimizeDeleteAllNamespaces) {
+ // Specific test for Optimize(): delete all referenced namespace ids.
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+
+ SchemaTypeId schema_type_id = 2;
+ JoinablePropertyId joinable_property_id = 1;
+
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/0, /*fingerprint=*/101);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/102);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/2, /*fingerprint=*/103);
+
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/0,
+ /*ref_namespace_fingerprint_ids=*/{id1}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/1,
+ /*ref_namespace_fingerprint_ids=*/{id2}),
+ IsOk());
+ EXPECT_THAT(
+ index->Put(schema_type_id, joinable_property_id, /*document_id=*/2,
+ /*ref_namespace_fingerprint_ids=*/{id3}),
+ IsOk());
+ index->set_last_added_document_id(3);
+
+ ASSERT_THAT(index, Pointee(SizeIs(3)));
+
+ // No change for document id.
+ std::vector<DocumentId> document_id_old_to_new = {0, 1, 2};
+
+ // Delete all namespaces.
+ std::vector<NamespaceId> namespace_id_old_to_new(3, kInvalidNamespaceId);
+
+ EXPECT_THAT(
+ index->Optimize(document_id_old_to_new, namespace_id_old_to_new,
+ /*new_last_added_document_id=*/kInvalidDocumentId),
+ IsOk());
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ // Verify all data are discarded after Optimize().
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+ EXPECT_THAT(GetJoinData(*index, schema_type_id, joinable_property_id),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexImplV2Test, Clear) {
+ NamespaceFingerprintIdentifier id1(/*namespace_id=*/1, /*fingerprint=*/12);
+ NamespaceFingerprintIdentifier id2(/*namespace_id=*/1, /*fingerprint=*/34);
+ NamespaceFingerprintIdentifier id3(/*namespace_id=*/1, /*fingerprint=*/56);
+ NamespaceFingerprintIdentifier id4(/*namespace_id=*/1, /*fingerprint=*/78);
+
+ // Create new qualified id join index
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> index,
+ QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ // Insert some data.
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/5,
+ /*ref_namespace_fingerprint_ids=*/{id2, id1}));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/3, /*joinable_property_id=*/10, /*document_id=*/6,
+ /*ref_namespace_fingerprint_ids=*/{id3}));
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/12,
+ /*ref_namespace_fingerprint_ids=*/{id4}));
+ ASSERT_THAT(index, Pointee(SizeIs(4)));
+ index->set_last_added_document_id(12);
+ ASSERT_THAT(index->last_added_document_id(), Eq(12));
+
+ // After Clear(), last_added_document_id should be set to kInvalidDocumentId,
+ // and the previous added data should be deleted.
+ EXPECT_THAT(index->Clear(), IsOk());
+ EXPECT_THAT(index, Pointee(IsEmpty()));
+ EXPECT_THAT(index->last_added_document_id(), Eq(kInvalidDocumentId));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/2, /*joinable_property_id=*/1),
+ IsOkAndHolds(IsEmpty()));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/3, /*joinable_property_id=*/10),
+ IsOkAndHolds(IsEmpty()));
+
+ // Join index should be able to work normally after Clear().
+ ICING_ASSERT_OK(index->Put(
+ /*schema_type_id=*/2, /*joinable_property_id=*/1, /*document_id=*/20,
+ /*ref_namespace_fingerprint_ids=*/{id4, id2, id1, id3}));
+ index->set_last_added_document_id(20);
+
+ EXPECT_THAT(index, Pointee(SizeIs(4)));
+ EXPECT_THAT(index->last_added_document_id(), Eq(20));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/2, /*joinable_property_id=*/1),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id3),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id1))));
+
+ ICING_ASSERT_OK(index->PersistToDisk());
+ index.reset();
+
+ // Verify index after reconstructing.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ index, QualifiedIdJoinIndexImplV2::Create(filesystem_, working_path_,
+ /*pre_mapping_fbv=*/false));
+ EXPECT_THAT(index->last_added_document_id(), Eq(20));
+ EXPECT_THAT(
+ GetJoinData(*index, /*schema_type_id=*/2, /*joinable_property_id=*/1),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id4),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id3),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id2),
+ DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/20, /*join_info=*/id1))));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/join/qualified-id-join-index.h b/icing/join/qualified-id-join-index.h
index 86297cd..4e487f9 100644
--- a/icing/join/qualified-id-join-index.h
+++ b/icing/join/qualified-id-join-index.h
@@ -19,81 +19,42 @@
#include <memory>
#include <string>
#include <string_view>
+#include <utility>
#include <vector>
#include "icing/text_classifier/lib3/utils/base/status.h"
#include "icing/text_classifier/lib3/utils/base/statusor.h"
-#include "icing/file/file-backed-vector.h"
#include "icing/file/filesystem.h"
#include "icing/file/persistent-storage.h"
#include "icing/join/doc-join-info.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
#include "icing/store/document-id.h"
-#include "icing/store/key-mapper.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
#include "icing/util/crc32.h"
namespace icing {
namespace lib {
-// QualifiedIdJoinIndex: a class to maintain data mapping DocJoinInfo to
-// joinable qualified ids and delete propagation info.
+// QualifiedIdJoinIndex: an abstract class to maintain data for qualified id
+// joining.
class QualifiedIdJoinIndex : public PersistentStorage {
public:
- struct Info {
- static constexpr int32_t kMagic = 0x48cabdc6;
+ class JoinDataIteratorBase {
+ public:
+ virtual ~JoinDataIteratorBase() = default;
- int32_t magic;
- DocumentId last_added_document_id;
+ virtual libtextclassifier3::Status Advance() = 0;
- Crc32 ComputeChecksum() const {
- return Crc32(
- std::string_view(reinterpret_cast<const char*>(this), sizeof(Info)));
- }
- } __attribute__((packed));
- static_assert(sizeof(Info) == 8, "");
-
- // Metadata file layout: <Crcs><Info>
- static constexpr int32_t kCrcsMetadataBufferOffset = 0;
- static constexpr int32_t kInfoMetadataBufferOffset =
- static_cast<int32_t>(sizeof(Crcs));
- static constexpr int32_t kMetadataFileSize = sizeof(Crcs) + sizeof(Info);
- static_assert(kMetadataFileSize == 20, "");
+ virtual const DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>&
+ GetCurrent() const = 0;
+ };
static constexpr WorkingPathType kWorkingPathType =
WorkingPathType::kDirectory;
- // Creates a QualifiedIdJoinIndex instance to store qualified ids for future
- // joining search. If any of the underlying file is missing, then delete the
- // whole working_path and (re)initialize with new ones. Otherwise initialize
- // and create the instance by existing files.
- //
- // filesystem: Object to make system level calls
- // working_path: Specifies the working path for PersistentStorage.
- // QualifiedIdJoinIndex uses working path as working directory
- // and all related files will be stored under this directory. It
- // takes full ownership and of working_path_, including
- // creation/deletion. It is the caller's responsibility to
- // specify correct working path and avoid mixing different
- // persistent storages together under the same path. Also the
- // caller has the ownership for the parent directory of
- // working_path_, and it is responsible for parent directory
- // creation/deletion. See PersistentStorage for more details
- // about the concept of working_path.
- // pre_mapping_fbv: flag indicating whether memory map max possible file size
- // for underlying FileBackedVector before growing the actual
- // file size.
- // use_persistent_hash_map: flag indicating whether use persistent hash map as
- // the key mapper (if false, then fall back to
- // dynamic trie key mapper).
- //
- // Returns:
- // - FAILED_PRECONDITION_ERROR if the file checksum doesn't match the stored
- // checksum
- // - INTERNAL_ERROR on I/O errors
- // - Any KeyMapper errors
- static libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
- Create(const Filesystem& filesystem, std::string working_path,
- bool pre_mapping_fbv, bool use_persistent_hash_map);
-
// Deletes QualifiedIdJoinIndex under working_path.
//
// Returns:
@@ -105,17 +66,11 @@ class QualifiedIdJoinIndex : public PersistentStorage {
kWorkingPathType);
}
- // Delete copy and move constructor/assignment operator.
- QualifiedIdJoinIndex(const QualifiedIdJoinIndex&) = delete;
- QualifiedIdJoinIndex& operator=(const QualifiedIdJoinIndex&) = delete;
-
- QualifiedIdJoinIndex(QualifiedIdJoinIndex&&) = delete;
- QualifiedIdJoinIndex& operator=(QualifiedIdJoinIndex&&) = delete;
+ virtual ~QualifiedIdJoinIndex() override = default;
- ~QualifiedIdJoinIndex() override;
-
- // Puts a new data into index: DocJoinInfo (DocumentId, JoinablePropertyId)
- // references to ref_qualified_id_str (the identifier of another document).
+ // (v1 only) Puts a new data into index: DocJoinInfo (DocumentId,
+ // JoinablePropertyId) references to ref_qualified_id_str (the identifier of
+ // another document).
//
// REQUIRES: ref_qualified_id_str contains no '\0'.
//
@@ -123,10 +78,26 @@ class QualifiedIdJoinIndex : public PersistentStorage {
// - OK on success
// - INVALID_ARGUMENT_ERROR if doc_join_info is invalid
// - Any KeyMapper errors
- libtextclassifier3::Status Put(const DocJoinInfo& doc_join_info,
- std::string_view ref_qualified_id_str);
+ virtual libtextclassifier3::Status Put(
+ const DocJoinInfo& doc_join_info,
+ std::string_view ref_qualified_id_str) = 0;
- // Gets the referenced document's qualified id string by DocJoinInfo.
+ // (v2 only) Puts a list of referenced NamespaceFingerprintIdentifier into
+ // index, given the DocumentId, SchemaTypeId and JoinablePropertyId.
+ //
+ // Returns:
+ // - OK on success
+ // - INVALID_ARGUMENT_ERROR if schema_type_id, joinable_property_id, or
+ // document_id is invalid
+ // - Any KeyMapper/FlashIndexStorage errors
+ virtual libtextclassifier3::Status Put(
+ SchemaTypeId schema_type_id, JoinablePropertyId joinable_property_id,
+ DocumentId document_id,
+ std::vector<NamespaceFingerprintIdentifier>&&
+ ref_namespace_fingerprint_ids) = 0;
+
+ // (v1 only) Gets the referenced document's qualified id string by
+ // DocJoinInfo.
//
// Returns:
// - A qualified id string referenced by the given DocJoinInfo (DocumentId,
@@ -134,8 +105,20 @@ class QualifiedIdJoinIndex : public PersistentStorage {
// - INVALID_ARGUMENT_ERROR if doc_join_info is invalid
// - NOT_FOUND_ERROR if doc_join_info doesn't exist
// - Any KeyMapper errors
- libtextclassifier3::StatusOr<std::string_view> Get(
- const DocJoinInfo& doc_join_info) const;
+ virtual libtextclassifier3::StatusOr<std::string_view> Get(
+ const DocJoinInfo& doc_join_info) const = 0;
+
+ // (v2 only) Returns a JoinDataIterator for iterating through all join data of
+ // the specified (schema_type_id, joinable_property_id).
+ //
+ // Returns:
+ // - On success: a JoinDataIterator
+ // - INVALID_ARGUMENT_ERROR if schema_type_id or joinable_property_id is
+ // invalid
+ // - Any KeyMapper/FlashIndexStorage errors
+ virtual libtextclassifier3::StatusOr<std::unique_ptr<JoinDataIteratorBase>>
+ GetIterator(SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) const = 0;
// Reduces internal file sizes by reclaiming space and ids of deleted
// documents. Qualified id type joinable index will convert all entries to the
@@ -143,6 +126,8 @@ class QualifiedIdJoinIndex : public PersistentStorage {
//
// - document_id_old_to_new: a map for converting old document id to new
// document id.
+ // - namespace_id_old_to_new: a map for converting old namespace id to new
+ // namespace id.
// - new_last_added_document_id: will be used to update the last added
// document id in the qualified id type joinable
// index.
@@ -152,154 +137,48 @@ class QualifiedIdJoinIndex : public PersistentStorage {
// - INTERNAL_ERROR on I/O error. This could potentially leave the index in
// an invalid state and the caller should handle it properly (e.g. discard
// and rebuild)
- libtextclassifier3::Status Optimize(
+ virtual libtextclassifier3::Status Optimize(
const std::vector<DocumentId>& document_id_old_to_new,
- DocumentId new_last_added_document_id);
+ const std::vector<NamespaceId>& namespace_id_old_to_new,
+ DocumentId new_last_added_document_id) = 0;
// Clears all data and set last_added_document_id to kInvalidDocumentId.
//
// Returns:
// - OK on success
// - INTERNAL_ERROR on I/O error
- libtextclassifier3::Status Clear();
+ virtual libtextclassifier3::Status Clear() = 0;
- int32_t size() const { return doc_join_info_mapper_->num_keys(); }
+ virtual bool is_v2() const = 0;
- bool empty() const { return size() == 0; }
+ virtual int32_t size() const = 0;
- DocumentId last_added_document_id() const {
- return info().last_added_document_id;
- }
+ virtual bool empty() const = 0;
- void set_last_added_document_id(DocumentId document_id) {
- SetInfoDirty();
+ virtual DocumentId last_added_document_id() const = 0;
- Info& info_ref = info();
- if (info_ref.last_added_document_id == kInvalidDocumentId ||
- document_id > info_ref.last_added_document_id) {
- info_ref.last_added_document_id = document_id;
- }
- }
+ virtual void set_last_added_document_id(DocumentId document_id) = 0;
- private:
- explicit QualifiedIdJoinIndex(
- const Filesystem& filesystem, std::string&& working_path,
- std::unique_ptr<uint8_t[]> metadata_buffer,
- std::unique_ptr<KeyMapper<int32_t>> doc_join_info_mapper,
- std::unique_ptr<FileBackedVector<char>> qualified_id_storage,
- bool pre_mapping_fbv, bool use_persistent_hash_map)
+ protected:
+ explicit QualifiedIdJoinIndex(const Filesystem& filesystem,
+ std::string&& working_path)
: PersistentStorage(filesystem, std::move(working_path),
- kWorkingPathType),
- metadata_buffer_(std::move(metadata_buffer)),
- doc_join_info_mapper_(std::move(doc_join_info_mapper)),
- qualified_id_storage_(std::move(qualified_id_storage)),
- pre_mapping_fbv_(pre_mapping_fbv),
- use_persistent_hash_map_(use_persistent_hash_map),
- is_info_dirty_(false),
- is_storage_dirty_(false) {}
-
- static libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
- InitializeNewFiles(const Filesystem& filesystem, std::string&& working_path,
- bool pre_mapping_fbv, bool use_persistent_hash_map);
-
- static libtextclassifier3::StatusOr<std::unique_ptr<QualifiedIdJoinIndex>>
- InitializeExistingFiles(const Filesystem& filesystem,
- std::string&& working_path, bool pre_mapping_fbv,
- bool use_persistent_hash_map);
-
- // Transfers qualified id type joinable index data from the current to
- // new_index and convert to new document id according to
- // document_id_old_to_new. It is a helper function for Optimize.
- //
- // Returns:
- // - OK on success
- // - INTERNAL_ERROR on I/O error
- libtextclassifier3::Status TransferIndex(
- const std::vector<DocumentId>& document_id_old_to_new,
- QualifiedIdJoinIndex* new_index) const;
-
- // Flushes contents of metadata file.
- //
- // Returns:
- // - OK on success
- // - INTERNAL_ERROR on I/O error
- libtextclassifier3::Status PersistMetadataToDisk(bool force) override;
-
- // Flushes contents of all storages to underlying files.
- //
- // Returns:
- // - OK on success
- // - INTERNAL_ERROR on I/O error
- libtextclassifier3::Status PersistStoragesToDisk(bool force) override;
-
- // Computes and returns Info checksum.
- //
- // Returns:
- // - Crc of the Info on success
- libtextclassifier3::StatusOr<Crc32> ComputeInfoChecksum(bool force) override;
-
- // Computes and returns all storages checksum.
- //
- // Returns:
- // - Crc of all storages on success
- // - INTERNAL_ERROR if any data inconsistency
- libtextclassifier3::StatusOr<Crc32> ComputeStoragesChecksum(
- bool force) override;
-
- Crcs& crcs() override {
- return *reinterpret_cast<Crcs*>(metadata_buffer_.get() +
- kCrcsMetadataBufferOffset);
- }
-
- const Crcs& crcs() const override {
- return *reinterpret_cast<const Crcs*>(metadata_buffer_.get() +
- kCrcsMetadataBufferOffset);
- }
-
- Info& info() {
- return *reinterpret_cast<Info*>(metadata_buffer_.get() +
- kInfoMetadataBufferOffset);
- }
-
- const Info& info() const {
- return *reinterpret_cast<const Info*>(metadata_buffer_.get() +
- kInfoMetadataBufferOffset);
- }
-
- void SetInfoDirty() { is_info_dirty_ = true; }
- // When storage is dirty, we have to set info dirty as well. So just expose
- // SetDirty to set both.
- void SetDirty() {
- is_info_dirty_ = true;
- is_storage_dirty_ = true;
- }
-
- bool is_info_dirty() const { return is_info_dirty_; }
- bool is_storage_dirty() const { return is_storage_dirty_; }
-
- // Metadata buffer
- std::unique_ptr<uint8_t[]> metadata_buffer_;
-
- // Persistent KeyMapper for mapping (encoded) DocJoinInfo (DocumentId,
- // JoinablePropertyId) to another referenced document's qualified id string
- // index in qualified_id_storage_.
- std::unique_ptr<KeyMapper<int32_t>> doc_join_info_mapper_;
+ kWorkingPathType) {}
- // Storage for qualified id strings.
- std::unique_ptr<FileBackedVector<char>> qualified_id_storage_;
+ virtual libtextclassifier3::Status PersistStoragesToDisk(
+ bool force) override = 0;
- // TODO(b/268521214): add delete propagation storage
+ virtual libtextclassifier3::Status PersistMetadataToDisk(
+ bool force) override = 0;
- // Flag indicating whether memory map max possible file size for underlying
- // FileBackedVector before growing the actual file size.
- bool pre_mapping_fbv_;
+ virtual libtextclassifier3::StatusOr<Crc32> ComputeInfoChecksum(
+ bool force) override = 0;
- // Flag indicating whether use persistent hash map as the key mapper (if
- // false, then fall back to dynamic trie key mapper).
- bool use_persistent_hash_map_;
+ virtual libtextclassifier3::StatusOr<Crc32> ComputeStoragesChecksum(
+ bool force) override = 0;
- bool is_info_dirty_;
- bool is_storage_dirty_;
+ virtual Crcs& crcs() override = 0;
+ virtual const Crcs& crcs() const override = 0;
};
} // namespace lib
diff --git a/icing/join/qualified-id-join-indexing-handler-v1_test.cc b/icing/join/qualified-id-join-indexing-handler-v1_test.cc
new file mode 100644
index 0000000..9700132
--- /dev/null
+++ b/icing/join/qualified-id-join-indexing-handler-v1_test.cc
@@ -0,0 +1,558 @@
+// Copyright (C) 2023 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 <memory>
+#include <string>
+#include <string_view>
+#include <utility>
+
+#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/document-builder.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/portable-file-backed-proto-log.h"
+#include "icing/join/qualified-id-join-index-impl-v1.h"
+#include "icing/join/qualified-id-join-index.h"
+#include "icing/join/qualified-id-join-indexing-handler.h"
+#include "icing/join/qualified-id.h"
+#include "icing/portable/platform.h"
+#include "icing/proto/document.pb.h"
+#include "icing/proto/schema.pb.h"
+#include "icing/schema-builder.h"
+#include "icing/schema/joinable-property.h"
+#include "icing/schema/schema-store.h"
+#include "icing/store/document-id.h"
+#include "icing/store/document-store.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/fake-clock.h"
+#include "icing/testing/icu-data-file-helper.h"
+#include "icing/testing/test-data.h"
+#include "icing/testing/tmp-directory.h"
+#include "icing/tokenization/language-segmenter-factory.h"
+#include "icing/tokenization/language-segmenter.h"
+#include "icing/util/tokenized-document.h"
+#include "unicode/uloc.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+using ::testing::Eq;
+using ::testing::IsEmpty;
+using ::testing::IsTrue;
+
+// Schema type for referenced documents: ReferencedType
+static constexpr std::string_view kReferencedType = "ReferencedType";
+static constexpr std::string_view kPropertyName = "name";
+
+// Joinable properties and joinable property id. Joinable property id is
+// determined by the lexicographical order of joinable property path.
+// Schema type with joinable property: FakeType
+static constexpr std::string_view kFakeType = "FakeType";
+static constexpr std::string_view kPropertyQualifiedId = "qualifiedId";
+
+static constexpr JoinablePropertyId kQualifiedIdJoinablePropertyId = 0;
+
+// Schema type with nested joinable properties: NestedType
+static constexpr std::string_view kNestedType = "NestedType";
+static constexpr std::string_view kPropertyNestedDoc = "nested";
+static constexpr std::string_view kPropertyQualifiedId2 = "qualifiedId2";
+
+static constexpr JoinablePropertyId kNestedQualifiedIdJoinablePropertyId = 0;
+static constexpr JoinablePropertyId kQualifiedId2JoinablePropertyId = 1;
+
+static constexpr DocumentId kDefaultDocumentId = 3;
+
+// TODO(b/275121148): remove this test after deprecating
+// QualifiedIdJoinIndexImplV1.
+class QualifiedIdJoinIndexingHandlerV1Test : public ::testing::Test {
+ protected:
+ void SetUp() override {
+ if (!IsCfStringTokenization() && !IsReverseJniTokenization()) {
+ ICING_ASSERT_OK(
+ // File generated via icu_data_file rule in //icing/BUILD.
+ icu_data_file_helper::SetUpICUDataFile(
+ GetTestFilePath("icing/icu.dat")));
+ }
+
+ base_dir_ = GetTestTempDir() + "/icing_test";
+ ASSERT_THAT(filesystem_.CreateDirectoryRecursively(base_dir_.c_str()),
+ IsTrue());
+
+ qualified_id_join_index_dir_ = base_dir_ + "/qualified_id_join_index";
+ schema_store_dir_ = base_dir_ + "/schema_store";
+ doc_store_dir_ = base_dir_ + "/doc_store";
+
+ ICING_ASSERT_OK_AND_ASSIGN(qualified_id_join_index_,
+ QualifiedIdJoinIndexImplV1::Create(
+ filesystem_, qualified_id_join_index_dir_,
+ /*pre_mapping_fbv=*/false,
+ /*use_persistent_hash_map=*/false));
+
+ language_segmenter_factory::SegmenterOptions segmenter_options(ULOC_US);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ lang_segmenter_,
+ language_segmenter_factory::Create(std::move(segmenter_options)));
+
+ ASSERT_THAT(
+ filesystem_.CreateDirectoryRecursively(schema_store_dir_.c_str()),
+ IsTrue());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ schema_store_,
+ SchemaStore::Create(&filesystem_, schema_store_dir_, &fake_clock_));
+ SchemaProto schema =
+ SchemaBuilder()
+ .AddType(
+ SchemaTypeConfigBuilder()
+ .SetType(kReferencedType)
+ .AddProperty(PropertyConfigBuilder()
+ .SetName(kPropertyName)
+ .SetDataTypeString(TERM_MATCH_EXACT,
+ TOKENIZER_PLAIN)
+ .SetCardinality(CARDINALITY_OPTIONAL)))
+ .AddType(SchemaTypeConfigBuilder().SetType(kFakeType).AddProperty(
+ PropertyConfigBuilder()
+ .SetName(kPropertyQualifiedId)
+ .SetDataTypeJoinableString(JOINABLE_VALUE_TYPE_QUALIFIED_ID)
+ .SetCardinality(CARDINALITY_OPTIONAL)))
+ .AddType(
+ SchemaTypeConfigBuilder()
+ .SetType(kNestedType)
+ .AddProperty(
+ PropertyConfigBuilder()
+ .SetName(kPropertyNestedDoc)
+ .SetDataTypeDocument(
+ kFakeType, /*index_nested_properties=*/true)
+ .SetCardinality(CARDINALITY_OPTIONAL))
+ .AddProperty(PropertyConfigBuilder()
+ .SetName(kPropertyQualifiedId2)
+ .SetDataTypeJoinableString(
+ JOINABLE_VALUE_TYPE_QUALIFIED_ID)
+ .SetCardinality(CARDINALITY_OPTIONAL)))
+ .Build();
+ ICING_ASSERT_OK(schema_store_->SetSchema(
+ schema, /*ignore_errors_and_delete_documents=*/false,
+ /*allow_circular_schema_definitions=*/false));
+
+ ASSERT_THAT(filesystem_.CreateDirectoryRecursively(doc_store_dir_.c_str()),
+ IsTrue());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentStore::CreateResult create_result,
+ DocumentStore::Create(&filesystem_, doc_store_dir_, &fake_clock_,
+ schema_store_.get(),
+ /*force_recovery_and_revalidate_documents=*/false,
+ /*namespace_id_fingerprint=*/false,
+ /*pre_mapping_fbv=*/false,
+ /*use_persistent_hash_map=*/false,
+ PortableFileBackedProtoLog<
+ DocumentWrapper>::kDeflateCompressionLevel,
+ /*initialize_stats=*/nullptr));
+ doc_store_ = std::move(create_result.document_store);
+ }
+
+ void TearDown() override {
+ doc_store_.reset();
+ schema_store_.reset();
+ lang_segmenter_.reset();
+ qualified_id_join_index_.reset();
+
+ filesystem_.DeleteDirectoryRecursively(base_dir_.c_str());
+ }
+
+ Filesystem filesystem_;
+ FakeClock fake_clock_;
+ std::string base_dir_;
+ std::string qualified_id_join_index_dir_;
+ std::string schema_store_dir_;
+ std::string doc_store_dir_;
+
+ std::unique_ptr<QualifiedIdJoinIndex> qualified_id_join_index_;
+ std::unique_ptr<LanguageSegmenter> lang_segmenter_;
+ std::unique_ptr<SchemaStore> schema_store_;
+ std::unique_ptr<DocumentStore> doc_store_;
+};
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test,
+ CreationWithNullPointerShouldFail) {
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ /*clock=*/nullptr, doc_store_.get(), qualified_id_join_index_.get()),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ &fake_clock_, /*doc_store=*/nullptr, qualified_id_join_index_.get()),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ &fake_clock_, doc_store_.get(), /*qualified_id_join_index=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test, HandleJoinableProperty) {
+ DocumentProto referenced_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kInvalidDocumentId));
+ // Handle document.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ IsOk());
+
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ IsOkAndHolds("pkg$db/ns#ref_type/1"));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test, HandleNestedJoinableProperty) {
+ DocumentProto referenced_document1 =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+ DocumentProto referenced_document2 =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/2")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "two")
+ .Build();
+
+ DocumentProto nested_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "nested_type/1")
+ .SetSchema(std::string(kNestedType))
+ .AddDocumentProperty(
+ std::string(kPropertyNestedDoc),
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "nested_fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/2")
+ .Build())
+ .AddStringProperty(std::string(kPropertyQualifiedId2),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ nested_document));
+
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kInvalidDocumentId));
+ // Handle nested_document.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+ EXPECT_THAT(handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
+ IsOk());
+
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kNestedQualifiedIdJoinablePropertyId)),
+ IsOkAndHolds("pkg$db/ns#ref_type/2"));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedId2JoinablePropertyId)),
+ IsOkAndHolds("pkg$db/ns#ref_type/1"));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test,
+ HandleShouldSkipInvalidFormatQualifiedId) {
+ static constexpr std::string_view kInvalidFormatQualifiedId =
+ "invalid_format_qualified_id";
+ ASSERT_THAT(QualifiedId::Parse(kInvalidFormatQualifiedId),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ std::string(kInvalidFormatQualifiedId))
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kInvalidDocumentId));
+ // Handle document. Should ignore invalid format qualified id.
+ // Index data should remain unchanged since there is no valid qualified id,
+ // but last_added_document_id should be updated.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ IsOk());
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test, HandleShouldSkipEmptyQualifiedId) {
+ // Create a document without any qualified id.
+ DocumentProto document = DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+ ASSERT_THAT(tokenized_document.qualified_id_join_properties(), IsEmpty());
+
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kInvalidDocumentId));
+ // Handle document. Index data should remain unchanged since there is no
+ // qualified id, but last_added_document_id should be updated.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ IsOk());
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test,
+ HandleInvalidDocumentIdShouldReturnInvalidArgumentError) {
+ DocumentProto referenced_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+
+ qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+
+ // Handling document with kInvalidDocumentId should cause a failure, and both
+ // index data and last_added_document_id should remain unchanged.
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kInvalidDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kInvalidDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Recovery mode should get the same result.
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kInvalidDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kInvalidDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test,
+ HandleOutOfOrderDocumentIdShouldReturnInvalidArgumentError) {
+ DocumentProto referenced_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+
+ qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+
+ // Handling document with document_id < last_added_document_id should cause a
+ // failure, and both index data and last_added_document_id should remain
+ // unchanged.
+ ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId - 1), IsTrue());
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId - 1,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Handling document with document_id == last_added_document_id should cause a
+ // failure, and both index data and last_added_document_id should remain
+ // unchanged.
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerV1Test,
+ HandleRecoveryModeShouldIgnoreDocsLELastAddedDocId) {
+ DocumentProto referenced_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ document));
+
+ qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+
+ // Handle document with document_id < last_added_document_id in recovery mode.
+ // We should not get any error, but the handler should ignore the document, so
+ // both index data and last_added_document_id should remain unchanged.
+ ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId - 1), IsTrue());
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId - 1,
+ /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ IsOk());
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Handle document with document_id == last_added_document_id in recovery
+ // mode. We should not get any error, but the handler should ignore the
+ // document, so both index data and last_added_document_id should remain
+ // unchanged.
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId,
+ /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ IsOk());
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Handle document with document_id > last_added_document_id in recovery mode.
+ // The handler should index this document and update last_added_document_id.
+ ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId + 1), IsTrue());
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, kDefaultDocumentId + 1,
+ /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ IsOk());
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kDefaultDocumentId + 1));
+ EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
+ kDefaultDocumentId + 1, kQualifiedIdJoinablePropertyId)),
+ IsOkAndHolds("pkg$db/ns#ref_type/1"));
+}
+
+} // namespace
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/join/qualified-id-join-indexing-handler.cc b/icing/join/qualified-id-join-indexing-handler.cc
index 344cf41..df86cba 100644
--- a/icing/join/qualified-id-join-indexing-handler.cc
+++ b/icing/join/qualified-id-join-indexing-handler.cc
@@ -14,8 +14,13 @@
#include "icing/join/qualified-id-join-indexing-handler.h"
+#include <cstdint>
+#include <limits>
#include <memory>
+#include <optional>
#include <string_view>
+#include <utility>
+#include <vector>
#include "icing/text_classifier/lib3/utils/base/status.h"
#include "icing/text_classifier/lib3/utils/base/statusor.h"
@@ -26,7 +31,11 @@
#include "icing/legacy/core/icing-string-util.h"
#include "icing/proto/logging.pb.h"
#include "icing/schema/joinable-property.h"
+#include "icing/store/document-filter-data.h"
#include "icing/store/document-id.h"
+#include "icing/store/document-store.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
#include "icing/util/clock.h"
#include "icing/util/logging.h"
#include "icing/util/status-macros.h"
@@ -38,12 +47,15 @@ namespace lib {
/* static */ libtextclassifier3::StatusOr<
std::unique_ptr<QualifiedIdJoinIndexingHandler>>
QualifiedIdJoinIndexingHandler::Create(
- const Clock* clock, QualifiedIdJoinIndex* qualified_id_join_index) {
+ const Clock* clock, const DocumentStore* doc_store,
+ QualifiedIdJoinIndex* qualified_id_join_index) {
ICING_RETURN_ERROR_IF_NULL(clock);
+ ICING_RETURN_ERROR_IF_NULL(doc_store);
ICING_RETURN_ERROR_IF_NULL(qualified_id_join_index);
return std::unique_ptr<QualifiedIdJoinIndexingHandler>(
- new QualifiedIdJoinIndexingHandler(clock, qualified_id_join_index));
+ new QualifiedIdJoinIndexingHandler(clock, doc_store,
+ qualified_id_join_index));
}
libtextclassifier3::Status QualifiedIdJoinIndexingHandler::Handle(
@@ -69,30 +81,89 @@ libtextclassifier3::Status QualifiedIdJoinIndexingHandler::Handle(
}
qualified_id_join_index_.set_last_added_document_id(document_id);
- for (const JoinableProperty<std::string_view>& qualified_id_property :
- tokenized_document.qualified_id_join_properties()) {
- if (qualified_id_property.values.empty()) {
- continue;
+ if (qualified_id_join_index_.is_v2()) {
+ // v2
+ std::optional<DocumentFilterData> filter_data =
+ doc_store_.GetAliveDocumentFilterData(
+ document_id,
+ /*current_time_ms=*/std::numeric_limits<int64_t>::min());
+ if (!filter_data) {
+ // This should not happen.
+ return absl_ports::InternalError(
+ "Failed to get alive document filter data when indexing");
}
- DocJoinInfo info(document_id, qualified_id_property.metadata.id);
- // Currently we only support single (non-repeated) joinable value under a
- // property.
- std::string_view ref_qualified_id_str = qualified_id_property.values[0];
-
- // Attempt to parse qualified id string to make sure the format is correct.
- if (!QualifiedId::Parse(ref_qualified_id_str).ok()) {
- // Skip incorrect format of qualified id string to save disk space.
- continue;
+ for (const JoinableProperty<std::string_view>& qualified_id_property :
+ tokenized_document.qualified_id_join_properties()) {
+ // Parse all qualified id strings and convert them to
+ // NamespaceFingerprintIdentifier.
+ std::vector<NamespaceFingerprintIdentifier> ref_doc_ns_fingerprint_ids;
+ for (std::string_view ref_qualified_id_str :
+ qualified_id_property.values) {
+ // Attempt to parse qualified id string to make sure the format is
+ // correct.
+ auto ref_qualified_id_or = QualifiedId::Parse(ref_qualified_id_str);
+ if (!ref_qualified_id_or.ok()) {
+ // Skip incorrect format of qualified id string.
+ continue;
+ }
+
+ QualifiedId ref_qualified_id =
+ std::move(ref_qualified_id_or).ValueOrDie();
+ auto ref_namespace_id_or =
+ doc_store_.GetNamespaceId(ref_qualified_id.name_space());
+ if (!ref_namespace_id_or.ok()) {
+ // Skip invalid namespace id.
+ continue;
+ }
+ NamespaceId ref_namespace_id =
+ std::move(ref_namespace_id_or).ValueOrDie();
+
+ ref_doc_ns_fingerprint_ids.push_back(NamespaceFingerprintIdentifier(
+ ref_namespace_id, ref_qualified_id.uri()));
+ }
+
+ // Batch add all join data of this (schema_type_id, joinable_property_id)
+ // into to the index.
+ libtextclassifier3::Status status = qualified_id_join_index_.Put(
+ filter_data->schema_type_id(), qualified_id_property.metadata.id,
+ document_id, std::move(ref_doc_ns_fingerprint_ids));
+ if (!status.ok()) {
+ ICING_LOG(WARNING)
+ << "Failed to add data into qualified id join index v2 due to: "
+ << status.error_message();
+ return status;
+ }
}
-
- libtextclassifier3::Status status =
- qualified_id_join_index_.Put(info, ref_qualified_id_str);
- if (!status.ok()) {
- ICING_LOG(WARNING)
- << "Failed to add data into qualified id join index due to: "
- << status.error_message();
- return status;
+ } else {
+ // v1
+ // TODO(b/275121148): deprecate this part after rollout v2.
+ for (const JoinableProperty<std::string_view>& qualified_id_property :
+ tokenized_document.qualified_id_join_properties()) {
+ if (qualified_id_property.values.empty()) {
+ continue;
+ }
+
+ DocJoinInfo info(document_id, qualified_id_property.metadata.id);
+ // Currently we only support single (non-repeated) joinable value under a
+ // property.
+ std::string_view ref_qualified_id_str = qualified_id_property.values[0];
+
+ // Attempt to parse qualified id string to make sure the format is
+ // correct.
+ if (!QualifiedId::Parse(ref_qualified_id_str).ok()) {
+ // Skip incorrect format of qualified id string to save disk space.
+ continue;
+ }
+
+ libtextclassifier3::Status status =
+ qualified_id_join_index_.Put(info, ref_qualified_id_str);
+ if (!status.ok()) {
+ ICING_LOG(WARNING)
+ << "Failed to add data into qualified id join index due to: "
+ << status.error_message();
+ return status;
+ }
}
}
diff --git a/icing/join/qualified-id-join-indexing-handler.h b/icing/join/qualified-id-join-indexing-handler.h
index f44e45d..8a11bf9 100644
--- a/icing/join/qualified-id-join-indexing-handler.h
+++ b/icing/join/qualified-id-join-indexing-handler.h
@@ -15,11 +15,15 @@
#ifndef ICING_JOIN_QUALIFIED_ID_JOIN_INDEXING_HANDLER_H_
#define ICING_JOIN_QUALIFIED_ID_JOIN_INDEXING_HANDLER_H_
+#include <memory>
+
#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
#include "icing/index/data-indexing-handler.h"
#include "icing/join/qualified-id-join-index.h"
#include "icing/proto/logging.pb.h"
#include "icing/store/document-id.h"
+#include "icing/store/document-store.h"
#include "icing/util/clock.h"
#include "icing/util/tokenized-document.h"
@@ -37,7 +41,8 @@ class QualifiedIdJoinIndexingHandler : public DataIndexingHandler {
// - FAILED_PRECONDITION_ERROR if any of the input pointer is null
static libtextclassifier3::StatusOr<
std::unique_ptr<QualifiedIdJoinIndexingHandler>>
- Create(const Clock* clock, QualifiedIdJoinIndex* qualified_id_join_index);
+ Create(const Clock* clock, const DocumentStore* doc_store,
+ QualifiedIdJoinIndex* qualified_id_join_index);
~QualifiedIdJoinIndexingHandler() override = default;
@@ -57,10 +62,13 @@ class QualifiedIdJoinIndexingHandler : public DataIndexingHandler {
private:
explicit QualifiedIdJoinIndexingHandler(
- const Clock* clock, QualifiedIdJoinIndex* qualified_id_join_index)
+ const Clock* clock, const DocumentStore* doc_store,
+ QualifiedIdJoinIndex* qualified_id_join_index)
: DataIndexingHandler(clock),
+ doc_store_(*doc_store),
qualified_id_join_index_(*qualified_id_join_index) {}
+ const DocumentStore& doc_store_; // Does not own.
QualifiedIdJoinIndex& qualified_id_join_index_; // Does not own.
};
diff --git a/icing/join/qualified-id-join-indexing-handler_test.cc b/icing/join/qualified-id-join-indexing-handler_test.cc
index 7e89dfa..53d35c7 100644
--- a/icing/join/qualified-id-join-indexing-handler_test.cc
+++ b/icing/join/qualified-id-join-indexing-handler_test.cc
@@ -17,12 +17,19 @@
#include <memory>
#include <string>
#include <string_view>
+#include <utility>
+#include <vector>
#include "icing/text_classifier/lib3/utils/base/status.h"
+#include "icing/text_classifier/lib3/utils/base/statusor.h"
#include "gmock/gmock.h"
#include "gtest/gtest.h"
+#include "icing/absl_ports/str_cat.h"
#include "icing/document-builder.h"
#include "icing/file/filesystem.h"
+#include "icing/file/portable-file-backed-proto-log.h"
+#include "icing/join/document-id-to-join-info.h"
+#include "icing/join/qualified-id-join-index-impl-v2.h"
#include "icing/join/qualified-id-join-index.h"
#include "icing/join/qualified-id.h"
#include "icing/portable/platform.h"
@@ -31,7 +38,11 @@
#include "icing/schema-builder.h"
#include "icing/schema/joinable-property.h"
#include "icing/schema/schema-store.h"
+#include "icing/store/document-filter-data.h"
#include "icing/store/document-id.h"
+#include "icing/store/document-store.h"
+#include "icing/store/namespace-fingerprint-identifier.h"
+#include "icing/store/namespace-id.h"
#include "icing/testing/common-matchers.h"
#include "icing/testing/fake-clock.h"
#include "icing/testing/icu-data-file-helper.h"
@@ -39,6 +50,7 @@
#include "icing/testing/tmp-directory.h"
#include "icing/tokenization/language-segmenter-factory.h"
#include "icing/tokenization/language-segmenter.h"
+#include "icing/util/status-macros.h"
#include "icing/util/tokenized-document.h"
#include "unicode/uloc.h"
@@ -47,9 +59,11 @@ namespace lib {
namespace {
+using ::testing::ElementsAre;
using ::testing::Eq;
using ::testing::IsEmpty;
using ::testing::IsTrue;
+using ::testing::NotNull;
// Schema type for referenced documents: ReferencedType
static constexpr std::string_view kReferencedType = "ReferencedType";
@@ -61,18 +75,11 @@ static constexpr std::string_view kPropertyName = "name";
static constexpr std::string_view kFakeType = "FakeType";
static constexpr std::string_view kPropertyQualifiedId = "qualifiedId";
-static constexpr JoinablePropertyId kQualifiedIdJoinablePropertyId = 0;
-
// Schema type with nested joinable properties: NestedType
static constexpr std::string_view kNestedType = "NestedType";
static constexpr std::string_view kPropertyNestedDoc = "nested";
static constexpr std::string_view kPropertyQualifiedId2 = "qualifiedId2";
-static constexpr JoinablePropertyId kNestedQualifiedIdJoinablePropertyId = 0;
-static constexpr JoinablePropertyId kQualifiedId2JoinablePropertyId = 1;
-
-static constexpr DocumentId kDefaultDocumentId = 3;
-
class QualifiedIdJoinIndexingHandlerTest : public ::testing::Test {
protected:
void SetUp() override {
@@ -89,12 +96,12 @@ class QualifiedIdJoinIndexingHandlerTest : public ::testing::Test {
qualified_id_join_index_dir_ = base_dir_ + "/qualified_id_join_index";
schema_store_dir_ = base_dir_ + "/schema_store";
+ doc_store_dir_ = base_dir_ + "/doc_store";
- ICING_ASSERT_OK_AND_ASSIGN(
- qualified_id_join_index_,
- QualifiedIdJoinIndex::Create(filesystem_, qualified_id_join_index_dir_,
- /*pre_mapping_fbv=*/false,
- /*use_persistent_hash_map=*/false));
+ ICING_ASSERT_OK_AND_ASSIGN(qualified_id_join_index_,
+ QualifiedIdJoinIndexImplV2::Create(
+ filesystem_, qualified_id_join_index_dir_,
+ /*pre_mapping_fbv=*/false));
language_segmenter_factory::SegmenterOptions segmenter_options(ULOC_US);
ICING_ASSERT_OK_AND_ASSIGN(
@@ -140,9 +147,52 @@ class QualifiedIdJoinIndexingHandlerTest : public ::testing::Test {
ICING_ASSERT_OK(schema_store_->SetSchema(
schema, /*ignore_errors_and_delete_documents=*/false,
/*allow_circular_schema_definitions=*/false));
+
+ ASSERT_THAT(filesystem_.CreateDirectoryRecursively(doc_store_dir_.c_str()),
+ IsTrue());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentStore::CreateResult create_result,
+ DocumentStore::Create(&filesystem_, doc_store_dir_, &fake_clock_,
+ schema_store_.get(),
+ /*force_recovery_and_revalidate_documents=*/false,
+ /*namespace_id_fingerprint=*/true,
+ /*pre_mapping_fbv=*/false,
+ /*use_persistent_hash_map=*/false,
+ PortableFileBackedProtoLog<
+ DocumentWrapper>::kDeflateCompressionLevel,
+ /*initialize_stats=*/nullptr));
+ doc_store_ = std::move(create_result.document_store);
+
+ // Get FakeType related ids.
+ ICING_ASSERT_OK_AND_ASSIGN(fake_type_id_,
+ schema_store_->GetSchemaTypeId(kFakeType));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ const JoinablePropertyMetadata* metadata1,
+ schema_store_->GetJoinablePropertyMetadata(
+ fake_type_id_, std::string(kPropertyQualifiedId)));
+ ASSERT_THAT(metadata1, NotNull());
+ fake_type_joinable_property_id_ = metadata1->id;
+
+ // Get NestedType related ids.
+ ICING_ASSERT_OK_AND_ASSIGN(nested_type_id_,
+ schema_store_->GetSchemaTypeId(kNestedType));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ const JoinablePropertyMetadata* metadata2,
+ schema_store_->GetJoinablePropertyMetadata(
+ nested_type_id_,
+ absl_ports::StrCat(kPropertyNestedDoc, ".", kPropertyQualifiedId)));
+ ASSERT_THAT(metadata2, NotNull());
+ nested_type_nested_joinable_property_id_ = metadata2->id;
+ ICING_ASSERT_OK_AND_ASSIGN(
+ const JoinablePropertyMetadata* metadata3,
+ schema_store_->GetJoinablePropertyMetadata(
+ nested_type_id_, std::string(kPropertyQualifiedId2)));
+ ASSERT_THAT(metadata3, NotNull());
+ nested_type_joinable_property_id_ = metadata3->id;
}
void TearDown() override {
+ doc_store_.reset();
schema_store_.reset();
lang_segmenter_.reset();
qualified_id_join_index_.reset();
@@ -155,30 +205,77 @@ class QualifiedIdJoinIndexingHandlerTest : public ::testing::Test {
std::string base_dir_;
std::string qualified_id_join_index_dir_;
std::string schema_store_dir_;
+ std::string doc_store_dir_;
- std::unique_ptr<QualifiedIdJoinIndex> qualified_id_join_index_;
+ std::unique_ptr<QualifiedIdJoinIndexImplV2> qualified_id_join_index_;
std::unique_ptr<LanguageSegmenter> lang_segmenter_;
std::unique_ptr<SchemaStore> schema_store_;
+ std::unique_ptr<DocumentStore> doc_store_;
+
+ // FakeType related ids.
+ SchemaTypeId fake_type_id_;
+ JoinablePropertyId fake_type_joinable_property_id_;
+
+ // NestedType related ids.
+ SchemaTypeId nested_type_id_;
+ JoinablePropertyId nested_type_nested_joinable_property_id_;
+ JoinablePropertyId nested_type_joinable_property_id_;
};
+libtextclassifier3::StatusOr<
+ std::vector<QualifiedIdJoinIndexImplV2::JoinDataType>>
+GetJoinData(const QualifiedIdJoinIndexImplV2& index,
+ SchemaTypeId schema_type_id,
+ JoinablePropertyId joinable_property_id) {
+ ICING_ASSIGN_OR_RETURN(
+ std::unique_ptr<QualifiedIdJoinIndex::JoinDataIteratorBase> iter,
+ index.GetIterator(schema_type_id, joinable_property_id));
+
+ std::vector<QualifiedIdJoinIndexImplV2::JoinDataType> result;
+ while (iter->Advance().ok()) {
+ result.push_back(iter->GetCurrent());
+ }
+
+ return result;
+}
+
TEST_F(QualifiedIdJoinIndexingHandlerTest, CreationWithNullPointerShouldFail) {
- EXPECT_THAT(QualifiedIdJoinIndexingHandler::Create(
- /*clock=*/nullptr, qualified_id_join_index_.get()),
- StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ /*clock=*/nullptr, doc_store_.get(), qualified_id_join_index_.get()),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
- EXPECT_THAT(QualifiedIdJoinIndexingHandler::Create(
- &fake_clock_, /*qualified_id_join_index=*/nullptr),
- StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ &fake_clock_, /*doc_store=*/nullptr, qualified_id_join_index_.get()),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
+
+ EXPECT_THAT(
+ QualifiedIdJoinIndexingHandler::Create(
+ &fake_clock_, doc_store_.get(), /*qualified_id_join_index=*/nullptr),
+ StatusIs(libtextclassifier3::StatusCode::FAILED_PRECONDITION));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest, HandleJoinableProperty) {
+ // Create and put referenced (parent) document. Get its document id and
+ // namespace id.
DocumentProto referenced_document =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/1")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "one")
.Build();
-
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id,
+ doc_store_->Put(referenced_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id,
+ doc_store_->GetNamespaceId(referenced_document.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id(
+ /*namespace_id=*/ref_doc_ns_id, /*target_str=*/referenced_document.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id),
+ IsOkAndHolds(ref_doc_id));
+
+ // Create and put (child) document. Also tokenize it.
DocumentProto document =
DocumentBuilder()
.SetKey("icing", "fake_type/1")
@@ -186,44 +283,81 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest, HandleJoinableProperty) {
.AddStringProperty(std::string(kPropertyQualifiedId),
"pkg$db/ns#ref_type/1")
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
- document));
+ std::move(document)));
+ // Handle document.
ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
Eq(kInvalidDocumentId));
- // Handle document.
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- IsOkAndHolds("pkg$db/ns#ref_type/1"));
+ // Verify the state of qualified_id_join_index_ after Handle().
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain
+ // [(doc_id, ref_doc_ns_fingerprint_id)].
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/doc_id,
+ /*join_info=*/ref_doc_ns_fingerprint_id))));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest, HandleNestedJoinableProperty) {
+ // Create and put referenced (parent) document1. Get its document id and
+ // namespace id.
DocumentProto referenced_document1 =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/1")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "one")
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id1,
+ doc_store_->Put(referenced_document1));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id1,
+ doc_store_->GetNamespaceId(referenced_document1.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id1(
+ /*namespace_id=*/ref_doc_ns_id1,
+ /*target_str=*/referenced_document1.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id1),
+ IsOkAndHolds(ref_doc_id1));
+
+ // Create and put referenced (parent) document2. Get its document id and
+ // namespace id.
DocumentProto referenced_document2 =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/2")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "two")
.Build();
-
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id2,
+ doc_store_->Put(referenced_document2));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id2,
+ doc_store_->GetNamespaceId(referenced_document2.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id2(
+ /*namespace_id=*/ref_doc_ns_id2,
+ /*target_str=*/referenced_document2.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id2),
+ IsOkAndHolds(ref_doc_id2));
+
+ // Create and put (child) document:
+ // - kPropertyNestedDoc.kPropertyQualifiedId refers to referenced_document2.
+ // - kPropertyQualifiedId2 refers to referenced_document1.
+ //
+ // Also tokenize it.
DocumentProto nested_document =
DocumentBuilder()
.SetKey("pkg$db/ns", "nested_type/1")
@@ -239,31 +373,51 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest, HandleNestedJoinableProperty) {
.AddStringProperty(std::string(kPropertyQualifiedId2),
"pkg$db/ns#ref_type/1")
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id,
+ doc_store_->Put(nested_document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
nested_document));
+ // Handle nested_document.
ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
Eq(kInvalidDocumentId));
- // Handle nested_document.
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
- EXPECT_THAT(handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/false,
- /*put_document_stats=*/nullptr),
- IsOk());
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
+ IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kNestedQualifiedIdJoinablePropertyId)),
- IsOkAndHolds("pkg$db/ns#ref_type/2"));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedId2JoinablePropertyId)),
- IsOkAndHolds("pkg$db/ns#ref_type/1"));
+ // Verify the state of qualified_id_join_index_ after Handle().
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
+ // (kNestedType, kPropertyNestedDoc.kPropertyQualifiedId) should contain
+ // [(doc_id, ref_doc_ns_fingerprint_id2)].
+ EXPECT_THAT(
+ GetJoinData(
+ *qualified_id_join_index_, /*schema_type_id=*/nested_type_id_,
+ /*joinable_property_id=*/nested_type_nested_joinable_property_id_),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/doc_id,
+ /*join_info=*/ref_doc_ns_fingerprint_id2))));
+ // (kNestedType, kPropertyQualifiedId2) should contain
+ // [(doc_id, ref_doc_ns_fingerprint_id1)].
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/nested_type_id_,
+ /*joinable_property_id=*/nested_type_joinable_property_id_),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/doc_id,
+ /*join_info=*/ref_doc_ns_fingerprint_id1))));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest,
@@ -273,6 +427,8 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
ASSERT_THAT(QualifiedId::Parse(kInvalidFormatQualifiedId),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ // Create and put (child) document with an invalid format referenced qualified
+ // id. Also tokenize it.
DocumentProto document =
DocumentBuilder()
.SetKey("icing", "fake_type/1")
@@ -280,71 +436,133 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
.AddStringProperty(std::string(kPropertyQualifiedId),
std::string(kInvalidFormatQualifiedId))
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
document));
+ // Handle document. Should ignore invalid format qualified id.
ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
Eq(kInvalidDocumentId));
- // Handle document. Should ignore invalid format qualified id.
- // Index data should remain unchanged since there is no valid qualified id,
- // but last_added_document_id should be updated.
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Verify the state of qualified_id_join_index_ after Handle(). Index data
+ // should remain unchanged since there is no valid qualified id, but
+ // last_added_document_id should be updated.
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerTest,
+ HandleShouldSkipNonExistingNamespace) {
+ static constexpr std::string_view kUnknownNamespace = "UnknownNamespace";
+ // Create and put (child) document which references to a parent qualified id
+ // with an unknown namespace. Also tokenize it.
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(
+ std::string(kPropertyQualifiedId),
+ absl_ports::StrCat(kUnknownNamespace, "#", "ref_type/1"))
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ std::move(document)));
+
+ // Handle document.
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(kInvalidDocumentId));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
+ EXPECT_THAT(
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
+ IsOk());
+
+ // Verify the state of qualified_id_join_index_ after Handle().
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should be empty since
+ // "UnknownNamespace#ref_type/1" should be skipped.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest, HandleShouldSkipEmptyQualifiedId) {
- // Create a document without any qualified id.
+ // Create and put (child) document without any qualified id. Also tokenize it.
DocumentProto document = DocumentBuilder()
.SetKey("icing", "fake_type/1")
.SetSchema(std::string(kFakeType))
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
document));
ASSERT_THAT(tokenized_document.qualified_id_join_properties(), IsEmpty());
+ // Handle document.
ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
Eq(kInvalidDocumentId));
- // Handle document. Index data should remain unchanged since there is no
- // qualified id, but last_added_document_id should be updated.
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Verify the state of qualified_id_join_index_ after Handle(). Index data
+ // should remain unchanged since there is no qualified id, but
+ // last_added_document_id should be updated.
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest,
HandleInvalidDocumentIdShouldReturnInvalidArgumentError) {
+ // Create and put referenced (parent) document. Get its document id and
+ // namespace id.
DocumentProto referenced_document =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/1")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "one")
.Build();
-
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id,
+ doc_store_->Put(referenced_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id,
+ doc_store_->GetNamespaceId(referenced_document.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id(
+ /*namespace_id=*/ref_doc_ns_id, /*target_str=*/referenced_document.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id),
+ IsOkAndHolds(ref_doc_id));
+
+ // Create and put (child) document. Also tokenize it.
DocumentProto document =
DocumentBuilder()
.SetKey("icing", "fake_type/1")
@@ -352,31 +570,35 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
.AddStringProperty(std::string(kPropertyQualifiedId),
"pkg$db/ns#ref_type/1")
.Build();
+ ICING_ASSERT_OK(doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
- document));
+ std::move(document)));
- qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
+ qualified_id_join_index_->set_last_added_document_id(ref_doc_id);
ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
+ Eq(ref_doc_id));
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
- // Handling document with kInvalidDocumentId should cause a failure, and both
- // index data and last_added_document_id should remain unchanged.
+ // Handling document with kInvalidDocumentId should cause a failure.
EXPECT_THAT(
handler->Handle(tokenized_document, kInvalidDocumentId,
/*recovery_mode=*/false, /*put_document_stats=*/nullptr),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ // Verify the state of qualified_id_join_index_ after Handle(). Both index
+ // data and last_added_document_id should remain unchanged.
EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kInvalidDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ Eq(ref_doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
// Recovery mode should get the same result.
EXPECT_THAT(
@@ -384,21 +606,35 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
/*recovery_mode=*/false, /*put_document_stats=*/nullptr),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kInvalidDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ Eq(ref_doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest,
HandleOutOfOrderDocumentIdShouldReturnInvalidArgumentError) {
+ // Create and put referenced (parent) document. Get its document id and
+ // namespace id.
DocumentProto referenced_document =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/1")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "one")
.Build();
-
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id,
+ doc_store_->Put(referenced_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id,
+ doc_store_->GetNamespaceId(referenced_document.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id(
+ /*namespace_id=*/ref_doc_ns_id, /*target_str=*/referenced_document.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id),
+ IsOkAndHolds(ref_doc_id));
+
+ // Create and put (child) document. Also tokenize it.
DocumentProto document =
DocumentBuilder()
.SetKey("icing", "fake_type/1")
@@ -406,57 +642,75 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
.AddStringProperty(std::string(kPropertyQualifiedId),
"pkg$db/ns#ref_type/1")
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
- document));
-
- qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
- ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
+ std::move(document)));
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
- // Handling document with document_id < last_added_document_id should cause a
- // failure, and both index data and last_added_document_id should remain
- // unchanged.
- ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId - 1), IsTrue());
+ // Handling document with document_id == last_added_document_id should cause a
+ // failure.
+ qualified_id_join_index_->set_last_added_document_id(doc_id);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId - 1,
- /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ // Verify the state of qualified_id_join_index_ after Handle(). Both index
+ // data and last_added_document_id should remain unchanged.
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
- // Handling document with document_id == last_added_document_id should cause a
- // failure, and both index data and last_added_document_id should remain
- // unchanged.
+ // Handling document with document_id < last_added_document_id should cause a
+ // failure.
+ qualified_id_join_index_->set_last_added_document_id(doc_id + 1);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(doc_id + 1));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/false, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/false,
+ /*put_document_stats=*/nullptr),
StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+ // Verify the state of qualified_id_join_index_ after Handle(). Both index
+ // data and last_added_document_id should remain unchanged.
EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ Eq(doc_id + 1));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
}
TEST_F(QualifiedIdJoinIndexingHandlerTest,
- HandleRecoveryModeShouldIgnoreDocsLELastAddedDocId) {
+ HandleRecoveryModeShouldIndexDocsGtLastAddedDocId) {
+ // Create and put referenced (parent) document. Get its document id and
+ // namespace id.
DocumentProto referenced_document =
DocumentBuilder()
.SetKey("pkg$db/ns", "ref_type/1")
.SetSchema(std::string(kReferencedType))
.AddStringProperty(std::string(kPropertyName), "one")
.Build();
-
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id,
+ doc_store_->Put(referenced_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id,
+ doc_store_->GetNamespaceId(referenced_document.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id(
+ /*namespace_id=*/ref_doc_ns_id, /*target_str=*/referenced_document.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id),
+ IsOkAndHolds(ref_doc_id));
+
+ // Create and put (child) document. Also tokenize it.
DocumentProto document =
DocumentBuilder()
.SetKey("icing", "fake_type/1")
@@ -464,60 +718,109 @@ TEST_F(QualifiedIdJoinIndexingHandlerTest,
.AddStringProperty(std::string(kPropertyQualifiedId),
"pkg$db/ns#ref_type/1")
.Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
ICING_ASSERT_OK_AND_ASSIGN(
TokenizedDocument tokenized_document,
TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
- document));
-
- qualified_id_join_index_->set_last_added_document_id(kDefaultDocumentId);
- ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
+ std::move(document)));
ICING_ASSERT_OK_AND_ASSIGN(
std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
- QualifiedIdJoinIndexingHandler::Create(&fake_clock_,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
qualified_id_join_index_.get()));
- // Handle document with document_id < last_added_document_id in recovery mode.
- // We should not get any error, but the handler should ignore the document, so
- // both index data and last_added_document_id should remain unchanged.
- ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId - 1), IsTrue());
+ // Handle document with document_id > last_added_document_id in recovery mode.
+ // The handler should index this document and update last_added_document_id.
+ qualified_id_join_index_->set_last_added_document_id(doc_id - 1);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(doc_id - 1));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId - 1,
- /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/true,
+ /*put_document_stats=*/nullptr),
IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(
+ ElementsAre(DocumentIdToJoinInfo<NamespaceFingerprintIdentifier>(
+ /*document_id=*/doc_id,
+ /*join_info=*/ref_doc_ns_fingerprint_id))));
+}
+
+TEST_F(QualifiedIdJoinIndexingHandlerTest,
+ HandleRecoveryModeShouldIgnoreDocsLeLastAddedDocId) {
+ // Create and put referenced (parent) document. Get its document id and
+ // namespace id.
+ DocumentProto referenced_document =
+ DocumentBuilder()
+ .SetKey("pkg$db/ns", "ref_type/1")
+ .SetSchema(std::string(kReferencedType))
+ .AddStringProperty(std::string(kPropertyName), "one")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId ref_doc_id,
+ doc_store_->Put(referenced_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ NamespaceId ref_doc_ns_id,
+ doc_store_->GetNamespaceId(referenced_document.namespace_()));
+ NamespaceFingerprintIdentifier ref_doc_ns_fingerprint_id(
+ /*namespace_id=*/ref_doc_ns_id, /*target_str=*/referenced_document.uri());
+ ASSERT_THAT(doc_store_->GetDocumentId(ref_doc_ns_fingerprint_id),
+ IsOkAndHolds(ref_doc_id));
+
+ // Create and put (child) document. Also tokenize it.
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("icing", "fake_type/1")
+ .SetSchema(std::string(kFakeType))
+ .AddStringProperty(std::string(kPropertyQualifiedId),
+ "pkg$db/ns#ref_type/1")
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId doc_id, doc_store_->Put(document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ TokenizedDocument tokenized_document,
+ TokenizedDocument::Create(schema_store_.get(), lang_segmenter_.get(),
+ std::move(document)));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<QualifiedIdJoinIndexingHandler> handler,
+ QualifiedIdJoinIndexingHandler::Create(&fake_clock_, doc_store_.get(),
+ qualified_id_join_index_.get()));
// Handle document with document_id == last_added_document_id in recovery
// mode. We should not get any error, but the handler should ignore the
// document, so both index data and last_added_document_id should remain
// unchanged.
+ qualified_id_join_index_->set_last_added_document_id(doc_id);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId,
- /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/true,
+ /*put_document_stats=*/nullptr),
IsOk());
- EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId, kQualifiedIdJoinablePropertyId)),
- StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(qualified_id_join_index_->last_added_document_id(), Eq(doc_id));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
- // Handle document with document_id > last_added_document_id in recovery mode.
- // The handler should index this document and update last_added_document_id.
- ASSERT_THAT(IsDocumentIdValid(kDefaultDocumentId + 1), IsTrue());
+ // Handle document with document_id < last_added_document_id in recovery mode.
+ // We should not get any error, but the handler should ignore the document, so
+ // both index data and last_added_document_id should remain unchanged.
+ qualified_id_join_index_->set_last_added_document_id(doc_id + 1);
+ ASSERT_THAT(qualified_id_join_index_->last_added_document_id(),
+ Eq(doc_id + 1));
EXPECT_THAT(
- handler->Handle(tokenized_document, kDefaultDocumentId + 1,
- /*recovery_mode=*/true, /*put_document_stats=*/nullptr),
+ handler->Handle(tokenized_document, doc_id, /*recovery_mode=*/true,
+ /*put_document_stats=*/nullptr),
IsOk());
EXPECT_THAT(qualified_id_join_index_->last_added_document_id(),
- Eq(kDefaultDocumentId + 1));
- EXPECT_THAT(qualified_id_join_index_->Get(DocJoinInfo(
- kDefaultDocumentId + 1, kQualifiedIdJoinablePropertyId)),
- IsOkAndHolds("pkg$db/ns#ref_type/1"));
+ Eq(doc_id + 1));
+ // (kFakeType, kPropertyQualifiedId) should contain nothing.
+ EXPECT_THAT(
+ GetJoinData(*qualified_id_join_index_, /*schema_type_id=*/fake_type_id_,
+ /*joinable_property_id=*/fake_type_joinable_property_id_),
+ IsOkAndHolds(IsEmpty()));
}
} // namespace