aboutsummaryrefslogtreecommitdiff
path: root/icing/store
diff options
context:
space:
mode:
authorCassie Wang <cassiewang@google.com>2019-12-20 15:11:45 -0800
committerCassie Wang <cassiewang@google.com>2019-12-20 16:18:05 -0800
commit128c9db88925c8425f2ad81e1d8985461d7ba21a (patch)
treef97ee47cc99d2c162eb30a5e051c606823dfd1ec /icing/store
parent1897505cb34f3d53e848da13fafe7691c17417ea (diff)
downloadicing-128c9db88925c8425f2ad81e1d8985461d7ba21a.tar.gz
Port over Icing c++ code from upstream
Change-Id: Ia3981fed7e0e70589efc027d4123f306cdfbe990
Diffstat (limited to 'icing/store')
-rw-r--r--icing/store/document-associated-score-data.h62
-rw-r--r--icing/store/document-filter-data.h67
-rw-r--r--icing/store/document-id.h40
-rw-r--r--icing/store/document-store.cc1214
-rw-r--r--icing/store/document-store.h450
-rw-r--r--icing/store/document-store_test.cc1886
-rw-r--r--icing/store/key-mapper.h267
-rw-r--r--icing/store/key-mapper_test.cc168
8 files changed, 4154 insertions, 0 deletions
diff --git a/icing/store/document-associated-score-data.h b/icing/store/document-associated-score-data.h
new file mode 100644
index 0000000..65b35e1
--- /dev/null
+++ b/icing/store/document-associated-score-data.h
@@ -0,0 +1,62 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_STORE_DOCUMENT_ASSOCIATED_SCORE_DATA_H_
+#define ICING_STORE_DOCUMENT_ASSOCIATED_SCORE_DATA_H_
+
+#include <cstdint>
+#include <type_traits>
+
+#include "icing/legacy/core/icing-packed-pod.h"
+
+namespace icing {
+namespace lib {
+
+// This is the cache entity of document-associated scores. It contains scores
+// that are related to the document itself. The ground-truth data is stored
+// somewhere else. The cache includes:
+// 1. Document score. It's defined in and passed from DocumentProto.score.
+// Positive values are required.
+// 2. Document creation timestamp. Unix timestamp of when the document is
+// created and inserted into Icing.
+class DocumentAssociatedScoreData {
+ public:
+ explicit DocumentAssociatedScoreData(int document_score,
+ int64_t creation_timestamp_secs)
+ : document_score_(document_score),
+ creation_timestamp_secs_(creation_timestamp_secs) {}
+
+ bool operator==(const DocumentAssociatedScoreData& other) const {
+ return document_score_ == other.document_score() &&
+ creation_timestamp_secs_ == other.creation_timestamp_secs();
+ }
+
+ int document_score() const { return document_score_; }
+
+ int64_t creation_timestamp_secs() const { return creation_timestamp_secs_; }
+
+ private:
+ int document_score_;
+ int64_t creation_timestamp_secs_;
+} __attribute__((packed));
+
+static_assert(sizeof(DocumentAssociatedScoreData) == 12,
+ "Size of DocumentAssociatedScoreData should be 12");
+static_assert(icing_is_packed_pod<DocumentAssociatedScoreData>::value,
+ "go/icing-ubsan");
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_DOCUMENT_ASSOCIATED_SCORE_DATA_H_
diff --git a/icing/store/document-filter-data.h b/icing/store/document-filter-data.h
new file mode 100644
index 0000000..86d0efd
--- /dev/null
+++ b/icing/store/document-filter-data.h
@@ -0,0 +1,67 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_STORE_DOCUMENT_FILTER_DATA_H_
+#define ICING_STORE_DOCUMENT_FILTER_DATA_H_
+
+#include <cstdint>
+#include <type_traits>
+
+#include "icing/legacy/core/icing-packed-pod.h"
+
+namespace icing {
+namespace lib {
+
+using NamespaceId = int16_t;
+using SchemaTypeId = int16_t;
+
+class DocumentFilterData {
+ public:
+ explicit DocumentFilterData(NamespaceId namespace_id,
+ SchemaTypeId schema_type_id,
+ int64_t expiration_timestamp_secs)
+ : expiration_timestamp_secs_(expiration_timestamp_secs),
+ namespace_id_(namespace_id),
+ schema_type_id_(schema_type_id) {}
+
+ bool operator==(const DocumentFilterData& other) const {
+ return namespace_id_ == other.namespace_id() &&
+ schema_type_id_ == other.schema_type_id() &&
+ expiration_timestamp_secs_ == other.expiration_timestamp_secs();
+ }
+
+ NamespaceId namespace_id() const { return namespace_id_; }
+
+ SchemaTypeId schema_type_id() const { return schema_type_id_; }
+ void set_schema_type_id(SchemaTypeId schema_type_id) {
+ schema_type_id_ = schema_type_id;
+ }
+
+ int64_t expiration_timestamp_secs() const {
+ return expiration_timestamp_secs_;
+ }
+
+ private:
+ int64_t expiration_timestamp_secs_;
+ NamespaceId namespace_id_;
+ SchemaTypeId schema_type_id_;
+} __attribute__((packed));
+
+static_assert(sizeof(DocumentFilterData) == 12, "");
+static_assert(icing_is_packed_pod<DocumentFilterData>::value, "go/icing-ubsan");
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_DOCUMENT_FILTER_DATA_H_
diff --git a/icing/store/document-id.h b/icing/store/document-id.h
new file mode 100644
index 0000000..cbe9959
--- /dev/null
+++ b/icing/store/document-id.h
@@ -0,0 +1,40 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_STORE_DOCUMENT_ID_H_
+#define ICING_STORE_DOCUMENT_ID_H_
+
+#include <cstdint>
+
+namespace icing {
+namespace lib {
+
+// Id of a document
+using DocumentId = int32_t;
+
+// We use 20 bits to encode document_ids and use the largest value (1M - 1) to
+// represent an invalid document_id.
+inline constexpr int kDocumentIdBits = 20;
+inline constexpr DocumentId kInvalidDocumentId = (1u << kDocumentIdBits) - 1;
+inline constexpr DocumentId kMinDocumentId = 0;
+inline constexpr DocumentId kMaxDocumentId = kInvalidDocumentId - 1;
+
+constexpr bool IsDocumentIdValid(DocumentId document_id) {
+ return document_id >= kMinDocumentId && document_id <= kMaxDocumentId;
+}
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_DOCUMENT_ID_H_
diff --git a/icing/store/document-store.cc b/icing/store/document-store.cc
new file mode 100644
index 0000000..b9b6738
--- /dev/null
+++ b/icing/store/document-store.cc
@@ -0,0 +1,1214 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/store/document-store.h"
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <utility>
+#include <vector>
+
+#include "utils/base/status.h"
+#include "utils/base/statusor.h"
+#include "utils/hash/farmhash.h"
+#include "icing/absl_ports/annotate.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/absl_ports/status_macros.h"
+#include "icing/absl_ports/str_cat.h"
+#include "icing/file/file-backed-proto-log.h"
+#include "icing/file/file-backed-vector.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/memory-mapped-file.h"
+#include "icing/legacy/core/icing-string-util.h"
+#include "icing/proto/document.pb.h"
+#include "icing/proto/document_wrapper.pb.h"
+#include "icing/schema/schema-store.h"
+#include "icing/store/document-associated-score-data.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/util/clock.h"
+#include "icing/util/crc32.h"
+#include "icing/util/logging.h"
+
+namespace icing {
+namespace lib {
+
+namespace {
+
+// Used in DocumentId mapper to mark a document as deleted
+constexpr int64_t kDocDeletedFlag = -1;
+constexpr char kDocumentLogFilename[] = "document_log";
+constexpr char kDocumentIdMapperFilename[] = "document_id_mapper";
+constexpr char kDocumentStoreHeaderFilename[] = "document_store_header";
+constexpr char kScoreCacheFilename[] = "score_cache";
+constexpr char kFilterCacheFilename[] = "filter_cache";
+constexpr char kNamespaceMapperFilename[] = "namespace_mapper";
+
+constexpr int32_t kUriMapperMaxSize = 12 * 1024 * 1024; // 12 MiB
+
+// 384 KiB for a KeyMapper would allow each internal array to have a max of
+// 128 KiB for storage.
+constexpr int32_t kNamespaceMapperMaxSize = 3 * 128 * 1024; // 384 KiB
+
+DocumentWrapper CreateDocumentWrapper(DocumentProto&& document) {
+ DocumentWrapper document_wrapper;
+ *document_wrapper.mutable_document() = std::move(document);
+ return document_wrapper;
+}
+
+DocumentWrapper CreateDocumentTombstone(std::string_view document_namespace,
+ std::string_view document_uri) {
+ DocumentWrapper document_wrapper;
+ document_wrapper.set_deleted(true);
+ DocumentProto* document = document_wrapper.mutable_document();
+ document->set_namespace_(std::string(document_namespace));
+ document->set_uri(std::string(document_uri));
+ return document_wrapper;
+}
+
+DocumentWrapper CreateNamespaceTombstone(std::string_view document_namespace) {
+ DocumentWrapper document_wrapper;
+ document_wrapper.set_deleted(true);
+ DocumentProto* document = document_wrapper.mutable_document();
+ document->set_namespace_(std::string(document_namespace));
+ return document_wrapper;
+}
+
+DocumentWrapper CreateSchemaTypeTombstone(
+ std::string_view document_schema_type) {
+ DocumentWrapper document_wrapper;
+ document_wrapper.set_deleted(true);
+ DocumentProto* document = document_wrapper.mutable_document();
+ document->set_schema(std::string(document_schema_type));
+ return document_wrapper;
+}
+
+std::string MakeHeaderFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kDocumentStoreHeaderFilename);
+}
+
+std::string MakeDocumentIdMapperFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kDocumentIdMapperFilename);
+}
+
+std::string MakeDocumentLogFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kDocumentLogFilename);
+}
+
+std::string MakeScoreCacheFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kScoreCacheFilename);
+}
+
+std::string MakeFilterCacheFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kFilterCacheFilename);
+}
+
+std::string MakeNamespaceMapperFilename(const std::string& base_dir) {
+ return absl_ports::StrCat(base_dir, "/", kNamespaceMapperFilename);
+}
+
+// TODO(adorokhine): This class internally uses an 8-byte fingerprint of the
+// Key and stores the key/value in a file-backed-trie that adds an ~80 byte
+// overhead per key. As we know that these fingerprints are always 8-bytes in
+// length and that they're random, we might be able to store them more
+// compactly.
+std::string MakeFingerprint(std::string_view name_space, std::string_view uri) {
+ // Using a 64-bit fingerprint to represent the key could lead to collisions.
+ // But, even with 200K unique keys, the probability of collision is about
+ // one-in-a-billion (https://en.wikipedia.org/wiki/Birthday_attack).
+ uint64_t fprint =
+ tc3farmhash::Fingerprint64(absl_ports::StrCat(name_space, uri));
+
+ std::string encoded_fprint;
+ // DynamicTrie cannot handle keys with '0' as bytes. So, we encode it in
+ // base128 and add 1 to make sure that no byte is '0'. This increases the
+ // size of the encoded_fprint from 8-bytes to 10-bytes.
+ while (fprint) {
+ encoded_fprint.push_back((fprint & 0x7F) + 1);
+ fprint >>= 7;
+ }
+ return encoded_fprint;
+}
+
+int64_t CalculateExpirationTimestampSecs(int64_t creation_timestamp_secs,
+ int64_t ttl_secs) {
+ if (ttl_secs == 0) {
+ // Special case where a TTL of 0 indicates the document should never
+ // expire. int64_t max, interpreted as seconds since epoch, represents
+ // some point in the year 292,277,026,596. So we're probably ok to use
+ // this as "never reaching this point".
+ return std::numeric_limits<int64_t>::max();
+ }
+
+ int64_t expiration_timestamp_secs;
+ if (__builtin_add_overflow(creation_timestamp_secs, ttl_secs,
+ &expiration_timestamp_secs)) {
+ // Overflow detected. Treat overflow as the same behavior of just int64_t
+ // max
+ return std::numeric_limits<int64_t>::max();
+ }
+
+ return expiration_timestamp_secs;
+}
+
+} // namespace
+
+DocumentStore::DocumentStore(const Filesystem* filesystem,
+ const std::string_view base_dir,
+ const Clock* clock,
+ const SchemaStore* schema_store)
+ : filesystem_(filesystem),
+ base_dir_(base_dir),
+ clock_(*clock),
+ schema_store_(schema_store),
+ document_validator_(schema_store) {}
+
+libtextclassifier3::StatusOr<DocumentId> DocumentStore::Put(
+ const DocumentProto& document) {
+ return Put(DocumentProto(document));
+}
+
+DocumentStore::~DocumentStore() {
+ if (initialized_) {
+ if (!PersistToDisk().ok()) {
+ ICING_LOG(ERROR)
+ << "Error persisting to disk in DocumentStore destructor";
+ }
+ }
+}
+
+libtextclassifier3::StatusOr<std::unique_ptr<DocumentStore>>
+DocumentStore::Create(const Filesystem* filesystem, const std::string& base_dir,
+ const Clock* clock, const SchemaStore* schema_store) {
+ auto document_store = std::unique_ptr<DocumentStore>(
+ new DocumentStore(filesystem, base_dir, clock, schema_store));
+ ICING_RETURN_IF_ERROR(document_store->Initialize());
+ return document_store;
+}
+
+libtextclassifier3::Status DocumentStore::Initialize() {
+ auto create_result_or = FileBackedProtoLog<DocumentWrapper>::Create(
+ filesystem_, MakeDocumentLogFilename(base_dir_),
+ FileBackedProtoLog<DocumentWrapper>::Options(
+ /*compress_in=*/true));
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ if (!create_result_or.ok()) {
+ ICING_LOG(ERROR) << create_result_or.status().error_message()
+ << "Failed to initialize DocumentLog";
+ return create_result_or.status();
+ }
+ FileBackedProtoLog<DocumentWrapper>::CreateResult create_result =
+ std::move(create_result_or).ValueOrDie();
+ document_log_ = std::move(create_result.proto_log);
+
+ if (create_result.data_loss) {
+ ICING_LOG(WARNING)
+ << "Data loss in document log, regenerating derived files.";
+ libtextclassifier3::Status status = RegenerateDerivedFiles();
+ if (!status.ok()) {
+ ICING_LOG(ERROR)
+ << "Failed to regenerate derived files for DocumentStore";
+ return status;
+ }
+ } else {
+ if (!InitializeDerivedFiles().ok()) {
+ ICING_VLOG(1)
+ << "Couldn't find derived files or failed to initialize them, "
+ "regenerating derived files for DocumentStore.";
+ libtextclassifier3::Status status = RegenerateDerivedFiles();
+ if (!status.ok()) {
+ ICING_LOG(ERROR)
+ << "Failed to regenerate derived files for DocumentStore";
+ return status;
+ }
+ }
+ }
+
+ initialized_ = true;
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::InitializeDerivedFiles() {
+ if (!HeaderExists()) {
+ // Without a header, we don't know if things are consistent between each
+ // other so the caller should just regenerate everything from ground
+ // truth.
+ return absl_ports::InternalError("DocumentStore header doesn't exist");
+ }
+
+ DocumentStore::Header header;
+ if (!filesystem_->Read(MakeHeaderFilename(base_dir_).c_str(), &header,
+ sizeof(header))) {
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Couldn't read: ", MakeHeaderFilename(base_dir_)));
+ }
+
+ if (header.magic != DocumentStore::Header::kMagic) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Invalid header kMagic for file: ", MakeHeaderFilename(base_dir_)));
+ }
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto document_key_mapper_or =
+ KeyMapper<DocumentId>::Create(*filesystem_, base_dir_, kUriMapperMaxSize);
+ if (!document_key_mapper_or.ok()) {
+ ICING_LOG(ERROR) << document_key_mapper_or.status().error_message()
+ << "Failed to initialize KeyMapper";
+ return document_key_mapper_or.status();
+ }
+ document_key_mapper_ = std::move(document_key_mapper_or).ValueOrDie();
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto document_id_mapper_or = FileBackedVector<int64_t>::Create(
+ *filesystem_, MakeDocumentIdMapperFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC);
+ if (!document_id_mapper_or.ok()) {
+ ICING_LOG(ERROR) << document_id_mapper_or.status().error_message()
+ << "Failed to initialize DocumentIdMapper";
+ return document_id_mapper_or.status();
+ }
+ document_id_mapper_ = std::move(document_id_mapper_or).ValueOrDie();
+
+ ICING_ASSIGN_OR_RETURN(score_cache_,
+ FileBackedVector<DocumentAssociatedScoreData>::Create(
+ *filesystem_, MakeScoreCacheFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC));
+
+ ICING_ASSIGN_OR_RETURN(filter_cache_,
+ FileBackedVector<DocumentFilterData>::Create(
+ *filesystem_, MakeFilterCacheFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC));
+
+ ICING_ASSIGN_OR_RETURN(
+ namespace_mapper_,
+ KeyMapper<NamespaceId>::Create(*filesystem_,
+ MakeNamespaceMapperFilename(base_dir_),
+ kNamespaceMapperMaxSize));
+
+ ICING_ASSIGN_OR_RETURN(Crc32 checksum, ComputeChecksum());
+ if (checksum.Get() != header.checksum) {
+ return absl_ports::InternalError(
+ "Combined checksum of DocStore was inconsistent");
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::RegenerateDerivedFiles() {
+ ICING_RETURN_IF_ERROR(ResetDocumentKeyMapper());
+ ICING_RETURN_IF_ERROR(ResetDocumentIdMapper());
+ ICING_RETURN_IF_ERROR(ResetDocumentAssociatedScoreCache());
+ ICING_RETURN_IF_ERROR(ResetFilterCache());
+ ICING_RETURN_IF_ERROR(ResetNamespaceMapper());
+
+ // Iterates through document log
+ auto iterator = document_log_->GetIterator();
+ auto iterator_status = iterator.Advance();
+ while (iterator_status.ok()) {
+ ICING_ASSIGN_OR_RETURN(DocumentWrapper document_wrapper,
+ document_log_->ReadProto(iterator.GetOffset()));
+ if (document_wrapper.deleted()) {
+ if (!document_wrapper.document().uri().empty()) {
+ // Individual document deletion.
+ auto document_id_or =
+ GetDocumentId(document_wrapper.document().namespace_(),
+ document_wrapper.document().uri());
+ // Updates document_id mapper with deletion
+ if (document_id_or.ok()) {
+ ICING_RETURN_IF_ERROR(document_id_mapper_->Set(
+ document_id_or.ValueOrDie(), kDocDeletedFlag));
+ } else if (!absl_ports::IsNotFound(document_id_or.status())) {
+ // Real error
+ return absl_ports::Annotate(
+ document_id_or.status(),
+ absl_ports::StrCat("Failed to find document id. namespace: ",
+ document_wrapper.document().namespace_(),
+ ", uri: ", document_wrapper.document().uri()));
+ }
+ } else if (!document_wrapper.document().namespace_().empty()) {
+ // Namespace deletion.
+ ICING_RETURN_IF_ERROR(UpdateDerivedFilesNamespaceDeleted(
+ document_wrapper.document().namespace_()));
+
+ } else if (!document_wrapper.document().schema().empty()) {
+ // SchemaType deletion.
+ auto schema_type_id_or = schema_store_->GetSchemaTypeId(
+ document_wrapper.document().schema());
+
+ if (schema_type_id_or.ok()) {
+ ICING_RETURN_IF_ERROR(UpdateDerivedFilesSchemaTypeDeleted(
+ schema_type_id_or.ValueOrDie()));
+ } else {
+ // The deleted schema type doesn't have a SchemaTypeId we can refer
+ // to in the FilterCache.
+ //
+ // TODO(cassiewang): We could avoid reading out all the documents.
+ // When we see a schema type doesn't have a SchemaTypeId, assign the
+ // unknown schema type a unique, temporary SchemaTypeId and store
+ // that in the FilterCache. Then, when we see the schema type
+ // tombstone here, we can look up its temporary SchemaTypeId and
+ // just iterate through the FilterCache to mark those documents as
+ // deleted.
+ int size = document_id_mapper_->num_elements();
+ for (DocumentId document_id = 0; document_id < size; document_id++) {
+ auto document_or = Get(document_id);
+ if (absl_ports::IsNotFound(document_or.status())) {
+ // Skip nonexistent documents
+ continue;
+ } else if (!document_or.ok()) {
+ // Real error, pass up
+ return absl_ports::Annotate(
+ document_or.status(),
+ IcingStringUtil::StringPrintf(
+ "Failed to retrieve Document for DocumentId %d",
+ document_id));
+ }
+
+ // Guaranteed to have a document now.
+ DocumentProto document = document_or.ValueOrDie();
+
+ if (document.schema() == document_wrapper.document().schema()) {
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(document_id, kDocDeletedFlag));
+ }
+ }
+ }
+ } else {
+ return absl_ports::InternalError(
+ "Encountered an invalid tombstone during recovery!");
+ }
+ } else {
+ // Updates key mapper and document_id mapper with the new document
+ DocumentId new_document_id = document_id_mapper_->num_elements();
+ ICING_RETURN_IF_ERROR(document_key_mapper_->Put(
+ MakeFingerprint(document_wrapper.document().namespace_(),
+ document_wrapper.document().uri()),
+ new_document_id));
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(new_document_id, iterator.GetOffset()));
+
+ ICING_RETURN_IF_ERROR(UpdateDocumentAssociatedScoreCache(
+ new_document_id,
+ DocumentAssociatedScoreData(
+ document_wrapper.document().score(),
+ document_wrapper.document().creation_timestamp_secs())));
+
+ SchemaTypeId schema_type_id;
+ auto schema_type_id_or =
+ schema_store_->GetSchemaTypeId(document_wrapper.document().schema());
+ if (absl_ports::IsNotFound(schema_type_id_or.status())) {
+ // Didn't find a SchemaTypeId. This means that the DocumentStore and
+ // the SchemaStore are out of sync. But DocumentStore can't do
+ // anything about it so just ignore this for now. This should be
+ // detected/handled by the owner of DocumentStore. Set it to some
+ // arbitrary invalid value for now, it'll get updated to the correct
+ // ID later.
+ schema_type_id = -1;
+ } else if (!schema_type_id_or.ok()) {
+ // Real error. Pass it up
+ return schema_type_id_or.status();
+ } else {
+ // We're guaranteed that SchemaTypeId is valid now
+ schema_type_id = schema_type_id_or.ValueOrDie();
+ }
+
+ ICING_ASSIGN_OR_RETURN(
+ NamespaceId namespace_id,
+ namespace_mapper_->GetOrPut(document_wrapper.document().namespace_(),
+ namespace_mapper_->num_keys()));
+
+ int64_t expiration_timestamp_secs = CalculateExpirationTimestampSecs(
+ document_wrapper.document().creation_timestamp_secs(),
+ document_wrapper.document().ttl_secs());
+
+ ICING_RETURN_IF_ERROR(UpdateFilterCache(
+ new_document_id, DocumentFilterData(namespace_id, schema_type_id,
+ expiration_timestamp_secs)));
+ }
+ iterator_status = iterator.Advance();
+ }
+
+ if (!absl_ports::IsOutOfRange(iterator_status)) {
+ ICING_LOG(WARNING)
+ << "Failed to iterate through proto log while regenerating "
+ "derived files";
+ return absl_ports::Annotate(iterator_status,
+ "Failed to iterate through proto log.");
+ }
+
+ // Write the header
+ ICING_ASSIGN_OR_RETURN(Crc32 checksum, ComputeChecksum());
+ ICING_RETURN_IF_ERROR(UpdateHeader(checksum));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::ResetDocumentKeyMapper() {
+ // TODO(b/139734457): Replace ptr.reset()->Delete->Create flow with Reset().
+ document_key_mapper_.reset();
+ // TODO(b/144458732): Implement a more robust version of TC_RETURN_IF_ERROR
+ // that can support error logging.
+ libtextclassifier3::Status status =
+ KeyMapper<DocumentId>::Delete(*filesystem_, base_dir_);
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete old key mapper";
+ return status;
+ }
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto document_key_mapper_or =
+ KeyMapper<DocumentId>::Create(*filesystem_, base_dir_, kUriMapperMaxSize);
+ if (!document_key_mapper_or.ok()) {
+ ICING_LOG(ERROR) << document_key_mapper_or.status().error_message()
+ << "Failed to re-init key mapper";
+ return document_key_mapper_or.status();
+ }
+ document_key_mapper_ = std::move(document_key_mapper_or).ValueOrDie();
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::ResetDocumentIdMapper() {
+ // TODO(b/139734457): Replace ptr.reset()->Delete->Create flow with Reset().
+ document_id_mapper_.reset();
+ // TODO(b/144458732): Implement a more robust version of TC_RETURN_IF_ERROR
+ // that can support error logging.
+ libtextclassifier3::Status status = FileBackedVector<int64_t>::Delete(
+ *filesystem_, MakeDocumentIdMapperFilename(base_dir_));
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete old document_id mapper";
+ return status;
+ }
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto document_id_mapper_or = FileBackedVector<int64_t>::Create(
+ *filesystem_, MakeDocumentIdMapperFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC);
+ if (!document_id_mapper_or.ok()) {
+ ICING_LOG(ERROR) << document_id_mapper_or.status().error_message()
+ << "Failed to re-init document_id mapper";
+ return document_id_mapper_or.status();
+ }
+ document_id_mapper_ = std::move(document_id_mapper_or).ValueOrDie();
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::ResetDocumentAssociatedScoreCache() {
+ // TODO(b/139734457): Replace ptr.reset()->Delete->Create flow with Reset().
+ score_cache_.reset();
+ ICING_RETURN_IF_ERROR(FileBackedVector<DocumentAssociatedScoreData>::Delete(
+ *filesystem_, MakeScoreCacheFilename(base_dir_)));
+ ICING_ASSIGN_OR_RETURN(score_cache_,
+ FileBackedVector<DocumentAssociatedScoreData>::Create(
+ *filesystem_, MakeScoreCacheFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC));
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::ResetFilterCache() {
+ // TODO(b/139734457): Replace ptr.reset()->Delete->Create flow with Reset().
+ filter_cache_.reset();
+ ICING_RETURN_IF_ERROR(FileBackedVector<DocumentFilterData>::Delete(
+ *filesystem_, MakeFilterCacheFilename(base_dir_)));
+ ICING_ASSIGN_OR_RETURN(filter_cache_,
+ FileBackedVector<DocumentFilterData>::Create(
+ *filesystem_, MakeFilterCacheFilename(base_dir_),
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC));
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::ResetNamespaceMapper() {
+ // TODO(b/139734457): Replace ptr.reset()->Delete->Create flow with Reset().
+ namespace_mapper_.reset();
+ // TODO(b/144458732): Implement a more robust version of TC_RETURN_IF_ERROR
+ // that can support error logging.
+ libtextclassifier3::Status status = KeyMapper<NamespaceId>::Delete(
+ *filesystem_, MakeNamespaceMapperFilename(base_dir_));
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete old namespace_id mapper";
+ return status;
+ }
+ ICING_ASSIGN_OR_RETURN(
+ namespace_mapper_,
+ KeyMapper<NamespaceId>::Create(*filesystem_,
+ MakeNamespaceMapperFilename(base_dir_),
+ kNamespaceMapperMaxSize));
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<Crc32> DocumentStore::ComputeChecksum() const {
+ Crc32 total_checksum;
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto checksum_or = document_log_->ComputeChecksum();
+ if (!checksum_or.ok()) {
+ ICING_LOG(ERROR) << checksum_or.status().error_message()
+ << "Failed to compute checksum of DocumentLog";
+ return checksum_or.status();
+ }
+ Crc32 document_log_checksum = std::move(checksum_or).ValueOrDie();
+
+ Crc32 document_key_mapper_checksum = document_key_mapper_->ComputeChecksum();
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ checksum_or = document_id_mapper_->ComputeChecksum();
+ if (!checksum_or.ok()) {
+ ICING_LOG(ERROR) << checksum_or.status().error_message()
+ << "Failed to compute checksum of DocumentIdMapper";
+ return checksum_or.status();
+ }
+ Crc32 document_id_mapper_checksum = std::move(checksum_or).ValueOrDie();
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ checksum_or = score_cache_->ComputeChecksum();
+ if (!checksum_or.ok()) {
+ ICING_LOG(ERROR) << checksum_or.status().error_message()
+ << "Failed to compute checksum of score cache";
+ return checksum_or.status();
+ }
+ Crc32 score_cache_checksum = std::move(checksum_or).ValueOrDie();
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ checksum_or = filter_cache_->ComputeChecksum();
+ if (!checksum_or.ok()) {
+ ICING_LOG(ERROR) << checksum_or.status().error_message()
+ << "Failed to compute checksum of filter cache";
+ return checksum_or.status();
+ }
+ Crc32 filter_cache_checksum = std::move(checksum_or).ValueOrDie();
+
+ Crc32 namespace_mapper_checksum = namespace_mapper_->ComputeChecksum();
+
+ total_checksum.Append(std::to_string(document_log_checksum.Get()));
+ total_checksum.Append(std::to_string(document_key_mapper_checksum.Get()));
+ total_checksum.Append(std::to_string(document_id_mapper_checksum.Get()));
+ total_checksum.Append(std::to_string(score_cache_checksum.Get()));
+ total_checksum.Append(std::to_string(filter_cache_checksum.Get()));
+ total_checksum.Append(std::to_string(namespace_mapper_checksum.Get()));
+
+ return total_checksum;
+}
+
+bool DocumentStore::HeaderExists() {
+ if (!filesystem_->FileExists(MakeHeaderFilename(base_dir_).c_str())) {
+ return false;
+ }
+
+ int64_t file_size =
+ filesystem_->GetFileSize(MakeHeaderFilename(base_dir_).c_str());
+
+ // If it's been truncated to size 0 before, we consider it to be a new file
+ return file_size != 0 && file_size != Filesystem::kBadFileSize;
+}
+
+libtextclassifier3::Status DocumentStore::UpdateHeader(const Crc32& checksum) {
+ // Write the header
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = checksum.Get();
+
+ // This should overwrite the header.
+ if (!filesystem_->Write(MakeHeaderFilename(base_dir_).c_str(), &header,
+ sizeof(header))) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Failed to write DocStore header: ", MakeHeaderFilename(base_dir_)));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<DocumentId> DocumentStore::Put(
+ DocumentProto&& document) {
+ ICING_RETURN_IF_ERROR(document_validator_.Validate(document));
+
+ // Copy fields needed before they are moved
+ std::string name_space = document.namespace_();
+ std::string uri = document.uri();
+ std::string schema = document.schema();
+ int document_score = document.score();
+ int64_t creation_timestamp_secs = document.creation_timestamp_secs();
+
+ // Sets the creation timestamp if caller hasn't specified.
+ if (document.creation_timestamp_secs() == 0) {
+ creation_timestamp_secs = clock_.GetCurrentSeconds();
+ document.set_creation_timestamp_secs(creation_timestamp_secs);
+ }
+
+ int64_t expiration_timestamp_secs = CalculateExpirationTimestampSecs(
+ creation_timestamp_secs, document.ttl_secs());
+
+ // Update ground truth first
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto offset_or =
+ document_log_->WriteProto(CreateDocumentWrapper(std::move(document)));
+ if (!offset_or.ok()) {
+ ICING_LOG(ERROR) << offset_or.status().error_message()
+ << "Failed to write document";
+ return offset_or.status();
+ }
+ int64_t file_offset = std::move(offset_or).ValueOrDie();
+
+ // Get existing document id
+ auto old_document_id_or = GetDocumentId(name_space, uri);
+ if (!old_document_id_or.ok() &&
+ !absl_ports::IsNotFound(old_document_id_or.status())) {
+ return absl_ports::InternalError("Failed to read from key mapper");
+ }
+
+ // Creates a new document id, updates key mapper and document_id mapper
+ DocumentId new_document_id = document_id_mapper_->num_elements();
+ ICING_RETURN_IF_ERROR(document_key_mapper_->Put(
+ MakeFingerprint(name_space, uri), new_document_id));
+ ICING_RETURN_IF_ERROR(document_id_mapper_->Set(new_document_id, file_offset));
+
+ ICING_RETURN_IF_ERROR(UpdateDocumentAssociatedScoreCache(
+ new_document_id,
+ DocumentAssociatedScoreData(document_score, creation_timestamp_secs)));
+
+ // Update namespace maps
+ ICING_ASSIGN_OR_RETURN(
+ NamespaceId namespace_id,
+ namespace_mapper_->GetOrPut(name_space, namespace_mapper_->num_keys()));
+
+ ICING_ASSIGN_OR_RETURN(SchemaTypeId schema_type_id,
+ schema_store_->GetSchemaTypeId(schema));
+
+ ICING_RETURN_IF_ERROR(UpdateFilterCache(
+ new_document_id, DocumentFilterData(namespace_id, schema_type_id,
+ expiration_timestamp_secs)));
+
+ if (old_document_id_or.ok()) {
+ // Mark the old document id as deleted.
+ ICING_RETURN_IF_ERROR(document_id_mapper_->Set(
+ old_document_id_or.ValueOrDie(), kDocDeletedFlag));
+ }
+
+ return new_document_id;
+}
+
+libtextclassifier3::StatusOr<DocumentProto> DocumentStore::Get(
+ const std::string_view name_space, const std::string_view uri) const {
+ ICING_ASSIGN_OR_RETURN(DocumentId document_id,
+ GetDocumentId(name_space, uri));
+ return Get(document_id);
+}
+
+libtextclassifier3::StatusOr<DocumentProto> DocumentStore::Get(
+ DocumentId document_id) const {
+ ICING_ASSIGN_OR_RETURN(int64_t document_log_offset,
+ DoesDocumentExistAndGetFileOffset(document_id));
+
+ // TODO(b/144458732): Implement a more robust version of TC_ASSIGN_OR_RETURN
+ // that can support error logging.
+ auto document_wrapper_or = document_log_->ReadProto(document_log_offset);
+ if (!document_wrapper_or.ok()) {
+ ICING_LOG(ERROR) << document_wrapper_or.status().error_message()
+ << "Failed to read from document log";
+ return document_wrapper_or.status();
+ }
+ DocumentWrapper document_wrapper =
+ std::move(document_wrapper_or).ValueOrDie();
+
+ return std::move(*document_wrapper.mutable_document());
+}
+
+libtextclassifier3::StatusOr<DocumentId> DocumentStore::GetDocumentId(
+ const std::string_view name_space, const std::string_view uri) const {
+ auto document_id_or =
+ document_key_mapper_->Get(MakeFingerprint(name_space, uri));
+ if (!document_id_or.ok()) {
+ return absl_ports::Annotate(
+ document_id_or.status(),
+ absl_ports::StrCat("Failed to find DocumentId by key: ", name_space,
+ ", ", uri));
+ }
+
+ // Guaranteed to have a DocumentId now
+ return document_id_or.ValueOrDie();
+}
+
+libtextclassifier3::StatusOr<int64_t>
+DocumentStore::DoesDocumentExistAndGetFileOffset(DocumentId document_id) const {
+ if (!IsDocumentIdValid(document_id)) {
+ return absl_ports::InvalidArgumentError(
+ IcingStringUtil::StringPrintf("DocumentId %d is invalid", document_id));
+ }
+
+ auto file_offset_or = document_id_mapper_->Get(document_id);
+
+ bool deleted =
+ file_offset_or.ok() && *file_offset_or.ValueOrDie() == kDocDeletedFlag;
+ if (deleted || absl_ports::IsOutOfRange(file_offset_or.status())) {
+ // Document has been deleted or doesn't exist
+ return absl_ports::NotFoundError(
+ IcingStringUtil::StringPrintf("Document %d not found", document_id));
+ }
+
+ ICING_ASSIGN_OR_RETURN(const DocumentFilterData* filter_data,
+ filter_cache_->Get(document_id));
+ if (clock_.GetCurrentSeconds() >= filter_data->expiration_timestamp_secs()) {
+ // Past the expiration time, so also return NOT FOUND since it *shouldn't*
+ // exist anymore.
+ return absl_ports::NotFoundError(
+ IcingStringUtil::StringPrintf("Document %d not found", document_id));
+ }
+
+ ICING_RETURN_IF_ERROR(file_offset_or.status());
+ return *file_offset_or.ValueOrDie();
+}
+
+bool DocumentStore::DoesDocumentExist(DocumentId document_id) const {
+ // If we can successfully get the document log offset, the document exists.
+ return DoesDocumentExistAndGetFileOffset(document_id).ok();
+}
+
+libtextclassifier3::Status DocumentStore::Delete(
+ const std::string_view name_space, const std::string_view uri) {
+ // Try to get the DocumentId first
+ auto document_id_or = GetDocumentId(name_space, uri);
+ if (absl_ports::IsNotFound(document_id_or.status())) {
+ // No need to delete nonexistent (name_space, uri)
+ return libtextclassifier3::Status::OK;
+ } else if (!document_id_or.ok()) {
+ // Real error
+ return absl_ports::Annotate(
+ document_id_or.status(),
+ absl_ports::StrCat("Failed to delete Document. namespace: ", name_space,
+ ", uri: ", uri));
+ }
+
+ // Check if the DocumentId's Document still exists.
+ DocumentId document_id = document_id_or.ValueOrDie();
+ auto file_offset_or = DoesDocumentExistAndGetFileOffset(document_id);
+ if (absl_ports::IsNotFound(file_offset_or.status())) {
+ // No need to delete nonexistent documents
+ return libtextclassifier3::Status::OK;
+ } else if (!file_offset_or.ok()) {
+ // Real error, pass it up
+ return absl_ports::Annotate(
+ file_offset_or.status(),
+ IcingStringUtil::StringPrintf(
+ "Failed to retrieve file offset for DocumentId %d", document_id));
+ }
+
+ // Update ground truth first.
+ // To delete a proto we don't directly remove it. Instead, we mark it as
+ // deleted first by appending a tombstone of it and actually remove it from
+ // file later in Optimize()
+ // TODO(b/144458732): Implement a more robust version of ICING_RETURN_IF_ERROR
+ // that can support error logging.
+ libtextclassifier3::Status status =
+ document_log_->WriteProto(CreateDocumentTombstone(name_space, uri))
+ .status();
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete Document. namespace: " << name_space
+ << ", uri: " << uri;
+ return status;
+ }
+
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(document_id_or.ValueOrDie(), kDocDeletedFlag));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<NamespaceId> DocumentStore::GetNamespaceId(
+ std::string_view name_space) const {
+ return namespace_mapper_->Get(name_space);
+}
+
+libtextclassifier3::StatusOr<DocumentAssociatedScoreData>
+DocumentStore::GetDocumentAssociatedScoreData(DocumentId document_id) const {
+ auto score_data_or = score_cache_->Get(document_id);
+ if (!score_data_or.ok()) {
+ ICING_LOG(ERROR) << " while trying to access DocumentId " << document_id
+ << " from score_cache_";
+ return score_data_or.status();
+ }
+ return *std::move(score_data_or).ValueOrDie();
+}
+
+libtextclassifier3::StatusOr<DocumentFilterData>
+DocumentStore::GetDocumentFilterData(DocumentId document_id) const {
+ auto filter_data_or = filter_cache_->Get(document_id);
+ if (!filter_data_or.ok()) {
+ ICING_LOG(ERROR) << " while trying to access DocumentId " << document_id
+ << " from filter_cache_";
+ return filter_data_or.status();
+ }
+ return *std::move(filter_data_or).ValueOrDie();
+}
+
+libtextclassifier3::Status DocumentStore::DeleteByNamespace(
+ std::string_view name_space) {
+ auto namespace_id_or = namespace_mapper_->Get(name_space);
+ if (absl_ports::IsNotFound(namespace_id_or.status())) {
+ // Namespace doesn't exist. Don't need to delete anything.
+ return libtextclassifier3::Status::OK;
+ } else if (!namespace_id_or.ok()) {
+ // Real error, pass it up.
+ return namespace_id_or.status();
+ }
+
+ // Update ground truth first.
+ // To delete an entire namespace, we append a tombstone that only contains
+ // the deleted bit and the name of the deleted namespace.
+ // TODO(b/144458732): Implement a more robust version of
+ // ICING_RETURN_IF_ERROR that can support error logging.
+ libtextclassifier3::Status status =
+ document_log_->WriteProto(CreateNamespaceTombstone(name_space)).status();
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete namespace. namespace = "
+ << name_space;
+ return status;
+ }
+
+ return UpdateDerivedFilesNamespaceDeleted(name_space);
+}
+
+libtextclassifier3::Status DocumentStore::UpdateDerivedFilesNamespaceDeleted(
+ std::string_view name_space) {
+ auto namespace_id_or = namespace_mapper_->Get(name_space);
+ if (absl_ports::IsNotFound(namespace_id_or.status())) {
+ // Namespace doesn't exist. Don't need to delete anything.
+ return libtextclassifier3::Status::OK;
+ } else if (!namespace_id_or.ok()) {
+ // Real error, pass it up.
+ return namespace_id_or.status();
+ }
+
+ // Guaranteed to have a NamespaceId now.
+ NamespaceId namespace_id = namespace_id_or.ValueOrDie();
+
+ // Traverse FilterCache and delete all docs that match namespace_id
+ for (DocumentId document_id = 0; document_id < filter_cache_->num_elements();
+ ++document_id) {
+ // filter_cache_->Get can only fail if document_id is < 0
+ // or >= filter_cache_->num_elements. So, this error SHOULD NEVER HAPPEN.
+ ICING_ASSIGN_OR_RETURN(const DocumentFilterData* data,
+ filter_cache_->Get(document_id));
+ if (data->namespace_id() == namespace_id) {
+ // docid_mapper_->Set can only fail if document_id is < 0
+ // or >= docid_mapper_->num_elements. So the only possible way to get an
+ // error here would be if filter_cache_->num_elements >
+ // docid_mapper_->num_elements, which SHOULD NEVER HAPPEN.
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(document_id, kDocDeletedFlag));
+ }
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::DeleteBySchemaType(
+ std::string_view schema_type) {
+ auto schema_type_id_or = schema_store_->GetSchemaTypeId(schema_type);
+ if (absl_ports::IsNotFound(schema_type_id_or.status())) {
+ // SchemaType doesn't exist. Don't need to delete anything.
+ return libtextclassifier3::Status::OK;
+ } else if (!schema_type_id_or.ok()) {
+ // Real error, pass it up.
+ return schema_type_id_or.status();
+ }
+
+ // Update ground truth first.
+ // To delete an entire schema type, we append a tombstone that only contains
+ // the deleted bit and the name of the deleted schema type.
+ // TODO(b/144458732): Implement a more robust version of
+ // ICING_RETURN_IF_ERROR that can support error logging.
+ libtextclassifier3::Status status =
+ document_log_->WriteProto(CreateSchemaTypeTombstone(schema_type))
+ .status();
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete schema_type. schema_type = "
+ << schema_type;
+ return status;
+ }
+
+ // Guaranteed to have a SchemaTypeId now
+ SchemaTypeId schema_type_id = schema_type_id_or.ValueOrDie();
+
+ ICING_RETURN_IF_ERROR(UpdateDerivedFilesSchemaTypeDeleted(schema_type_id));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::UpdateDerivedFilesSchemaTypeDeleted(
+ SchemaTypeId schema_type_id) {
+ // Traverse FilterCache and delete all docs that match schema_type_id.
+ for (DocumentId document_id = 0; document_id < filter_cache_->num_elements();
+ ++document_id) {
+ // filter_cache_->Get can only fail if document_id is < 0
+ // or >= filter_cache_->num_elements. So, this error SHOULD NEVER HAPPEN.
+ ICING_ASSIGN_OR_RETURN(const DocumentFilterData* data,
+ filter_cache_->Get(document_id));
+ if (data->schema_type_id() == schema_type_id) {
+ // docid_mapper_->Set can only fail if document_id is < 0
+ // or >= docid_mapper_->num_elements. So the only possible way to get an
+ // error here would be if filter_cache_->num_elements >
+ // docid_mapper_->num_elements, which SHOULD NEVER HAPPEN.
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(document_id, kDocDeletedFlag));
+ }
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::PersistToDisk() {
+ ICING_RETURN_IF_ERROR(document_log_->PersistToDisk());
+ ICING_RETURN_IF_ERROR(document_key_mapper_->PersistToDisk());
+ ICING_RETURN_IF_ERROR(document_id_mapper_->PersistToDisk());
+ ICING_RETURN_IF_ERROR(score_cache_->PersistToDisk());
+ ICING_RETURN_IF_ERROR(filter_cache_->PersistToDisk());
+ ICING_RETURN_IF_ERROR(namespace_mapper_->PersistToDisk());
+
+ // Update the combined checksum and write to header file.
+ ICING_ASSIGN_OR_RETURN(Crc32 checksum, ComputeChecksum());
+ ICING_RETURN_IF_ERROR(UpdateHeader(checksum));
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::StatusOr<int64_t> DocumentStore::GetDiskUsage() const {
+ ICING_ASSIGN_OR_RETURN(const int64_t document_log_disk_usage,
+ document_log_->GetDiskUsage());
+ ICING_ASSIGN_OR_RETURN(const int64_t document_key_mapper_disk_usage,
+ document_key_mapper_->GetDiskUsage());
+ ICING_ASSIGN_OR_RETURN(const int64_t document_id_mapper_disk_usage,
+ document_id_mapper_->GetDiskUsage());
+ ICING_ASSIGN_OR_RETURN(const int64_t score_cache_disk_usage,
+ score_cache_->GetDiskUsage());
+ ICING_ASSIGN_OR_RETURN(const int64_t filter_cache_disk_usage,
+ filter_cache_->GetDiskUsage());
+ ICING_ASSIGN_OR_RETURN(const int64_t namespace_mapper_disk_usage,
+ namespace_mapper_->GetDiskUsage());
+
+ return document_log_disk_usage + document_key_mapper_disk_usage +
+ document_id_mapper_disk_usage + score_cache_disk_usage +
+ filter_cache_disk_usage + namespace_mapper_disk_usage;
+}
+
+libtextclassifier3::Status DocumentStore::UpdateSchemaStore(
+ const SchemaStore* schema_store) {
+ // Update all references to the SchemaStore
+ schema_store_ = schema_store;
+ document_validator_.UpdateSchemaStore(schema_store);
+
+ int size = document_id_mapper_->num_elements();
+ for (DocumentId document_id = 0; document_id < size; document_id++) {
+ auto document_or = Get(document_id);
+ if (absl_ports::IsNotFound(document_or.status())) {
+ // Skip nonexistent documents
+ continue;
+ } else if (!document_or.ok()) {
+ // Real error, pass up
+ return absl_ports::Annotate(
+ document_or.status(),
+ IcingStringUtil::StringPrintf(
+ "Failed to retrieve Document for DocumentId %d", document_id));
+ }
+
+ // Guaranteed to have a document now.
+ DocumentProto document = document_or.ValueOrDie();
+
+ // Revalidate that this document is still compatible
+ if (document_validator_.Validate(document).ok()) {
+ // Update the SchemaTypeId for this entry
+ ICING_ASSIGN_OR_RETURN(SchemaTypeId schema_type_id,
+ schema_store_->GetSchemaTypeId(document.schema()));
+ filter_cache_->mutable_array()[document_id].set_schema_type_id(
+ schema_type_id);
+ } else {
+ // Document is no longer valid with the new SchemaStore. Mark as
+ // deleted
+ ICING_RETURN_IF_ERROR(Delete(document.namespace_(), document.uri()));
+ }
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::OptimizedUpdateSchemaStore(
+ const SchemaStore* schema_store,
+ const SchemaStore::SetSchemaResult& set_schema_result) {
+ if (!set_schema_result.success) {
+ // No new schema was set, no work to be done
+ return libtextclassifier3::Status::OK;
+ }
+
+ // Update all references to the SchemaStore
+ schema_store_ = schema_store;
+ document_validator_.UpdateSchemaStore(schema_store);
+
+ // Append a tombstone for each deleted schema type. This way, we don't have
+ // to read out each document, check if the schema type has been deleted, and
+ // append a tombstone per-document.
+ for (const auto& schema_type :
+ set_schema_result.schema_types_deleted_by_name) {
+ // TODO(b/144458732): Implement a more robust version of
+ // ICING_RETURN_IF_ERROR that can support error logging.
+ libtextclassifier3::Status status =
+ document_log_->WriteProto(CreateSchemaTypeTombstone(schema_type))
+ .status();
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to delete schema_type. schema_type = "
+ << schema_type;
+ return status;
+ }
+ }
+
+ int size = document_id_mapper_->num_elements();
+ for (DocumentId document_id = 0; document_id < size; document_id++) {
+ auto exists_or = DoesDocumentExistAndGetFileOffset(document_id);
+ if (absl_ports::IsNotFound(exists_or.status())) {
+ // Skip nonexistent documents
+ continue;
+ } else if (!exists_or.ok()) {
+ // Real error, pass up
+ return absl_ports::Annotate(
+ exists_or.status(),
+ IcingStringUtil::StringPrintf("Failed to retrieve DocumentId %d",
+ document_id));
+ }
+
+ // Guaranteed that the document exists now.
+ ICING_ASSIGN_OR_RETURN(const DocumentFilterData* filter_data,
+ filter_cache_->Get(document_id));
+
+ if (set_schema_result.schema_types_deleted_by_id.count(
+ filter_data->schema_type_id()) != 0) {
+ // We already created a tombstone for this deleted type. Just update the
+ // derived files now.
+ ICING_RETURN_IF_ERROR(
+ document_id_mapper_->Set(document_id, kDocDeletedFlag));
+ continue;
+ }
+
+ // Check if we need to update the FilterCache entry for this document. It
+ // may have been assigned a different SchemaTypeId in the new SchemaStore.
+ bool update_filter_cache =
+ set_schema_result.old_schema_type_ids_changed.count(
+ filter_data->schema_type_id()) != 0;
+
+ // Check if we need to revalidate this document if the type is now
+ // incompatible
+ bool revalidate_document =
+ set_schema_result.schema_types_incompatible_by_id.count(
+ filter_data->schema_type_id()) != 0;
+
+ if (update_filter_cache || revalidate_document) {
+ ICING_ASSIGN_OR_RETURN(DocumentProto document, Get(document_id));
+
+ if (update_filter_cache) {
+ ICING_ASSIGN_OR_RETURN(
+ SchemaTypeId schema_type_id,
+ schema_store_->GetSchemaTypeId(document.schema()));
+ filter_cache_->mutable_array()[document_id].set_schema_type_id(
+ schema_type_id);
+ }
+
+ if (revalidate_document) {
+ if (!document_validator_.Validate(document).ok()) {
+ // Document is no longer valid with the new SchemaStore. Mark as
+ // deleted
+ ICING_RETURN_IF_ERROR(Delete(document.namespace_(), document.uri()));
+ }
+ }
+ }
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+// TODO(b/121227117): Implement Optimize()
+libtextclassifier3::Status DocumentStore::Optimize() {
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::OptimizeInto(
+ const std::string& new_directory) {
+ // Validates directory
+ if (new_directory == base_dir_) {
+ return absl_ports::InvalidArgumentError(
+ "New directory is the same as the current one.");
+ }
+
+ ICING_ASSIGN_OR_RETURN(auto new_doc_store,
+ DocumentStore::Create(filesystem_, new_directory,
+ &clock_, schema_store_));
+
+ // Writes all valid docs into new document store (new directory)
+ int size = document_id_mapper_->num_elements();
+ for (DocumentId document_id = 0; document_id < size; document_id++) {
+ auto document_or = Get(document_id);
+ if (absl_ports::IsNotFound(document_or.status())) {
+ // Skip nonexistent documents
+ continue;
+ } else if (!document_or.ok()) {
+ // Real error, pass up
+ return absl_ports::Annotate(
+ document_or.status(),
+ IcingStringUtil::StringPrintf(
+ "Failed to retrieve Document for DocumentId %d", document_id));
+ }
+
+ // Guaranteed to have a document now.
+ DocumentProto document_to_keep = document_or.ValueOrDie();
+ // TODO(b/144458732): Implement a more robust version of
+ // ICING_RETURN_IF_ERROR that can support error logging.
+ libtextclassifier3::Status status =
+ new_doc_store->Put(std::move(document_to_keep)).status();
+ if (!status.ok()) {
+ ICING_LOG(ERROR) << status.error_message()
+ << "Failed to write into new document store";
+ return status;
+ }
+ }
+
+ ICING_RETURN_IF_ERROR(new_doc_store->PersistToDisk());
+ return libtextclassifier3::Status::OK;
+}
+
+libtextclassifier3::Status DocumentStore::UpdateDocumentAssociatedScoreCache(
+ DocumentId document_id, const DocumentAssociatedScoreData& score_data) {
+ return score_cache_->Set(document_id, score_data);
+}
+
+libtextclassifier3::Status DocumentStore::UpdateFilterCache(
+ DocumentId document_id, const DocumentFilterData& filter_data) {
+ return filter_cache_->Set(document_id, filter_data);
+}
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/store/document-store.h b/icing/store/document-store.h
new file mode 100644
index 0000000..018e19e
--- /dev/null
+++ b/icing/store/document-store.h
@@ -0,0 +1,450 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_STORE_DOCUMENT_STORE_H_
+#define ICING_STORE_DOCUMENT_STORE_H_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "utils/base/status.h"
+#include "utils/base/statusor.h"
+#include "icing/file/file-backed-proto-log.h"
+#include "icing/file/file-backed-vector.h"
+#include "icing/file/filesystem.h"
+#include "icing/proto/document.pb.h"
+#include "icing/proto/document_wrapper.pb.h"
+#include "icing/schema/schema-store.h"
+#include "icing/store/document-associated-score-data.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/store/key-mapper.h"
+#include "icing/util/clock.h"
+#include "icing/util/crc32.h"
+#include "icing/util/document-validator.h"
+
+namespace icing {
+namespace lib {
+
+// Provides storage interfaces for documents.
+class DocumentStore {
+ public:
+ struct Header {
+ static constexpr int32_t kMagic = 0x746f7265;
+
+ // Holds the magic as a quick sanity check against file corruption.
+ int32_t magic;
+
+ // Checksum of the DocumentStore's sub-component's checksums.
+ uint32_t checksum;
+ };
+
+ // Not copyable
+ DocumentStore(const DocumentStore&) = delete;
+ DocumentStore& operator=(const DocumentStore&) = delete;
+
+ // Persists and updates checksum of subcomponents.
+ ~DocumentStore();
+
+ // Factory method to create, initialize, and return a DocumentStore. The base
+ // directory is used to persist document store files. If document store was
+ // previously initialized with this directory, it will reload the files saved
+ // by the last instance.
+ //
+ // Does not take any ownership, and all pointers must refer to valid objects
+ // that outlive the one constructed.
+ //
+ // TODO(cassiewang): Consider returning a status indicating that derived files
+ // were regenerated. This may be helpful in logs.
+ //
+ // Returns:
+ // A valid document store on success
+ // INTERNAL_ERROR on IO error
+ static libtextclassifier3::StatusOr<std::unique_ptr<DocumentStore>> Create(
+ const Filesystem* filesystem, const std::string& base_dir,
+ const Clock* clock, const SchemaStore* schema_store);
+
+ // Returns the maximum DocumentId that the DocumentStore has assigned. If
+ // there has not been any DocumentIds assigned, i.e. the DocumentStore is
+ // empty, then kInvalidDocumentId is returned. This does not filter out
+ // DocumentIds of deleted documents.
+ const DocumentId last_added_document_id() const {
+ if (document_id_mapper_->num_elements() == 0) {
+ return kInvalidDocumentId;
+ }
+ return document_id_mapper_->num_elements() - 1;
+ }
+
+ // Puts the document into document store.
+ //
+ // Returns:
+ // A newly generated document id on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<DocumentId> Put(const DocumentProto& document);
+ libtextclassifier3::StatusOr<DocumentId> Put(DocumentProto&& document);
+
+ // Finds and returns the document identified by the given key (namespace +
+ // uri)
+ //
+ // Returns:
+ // The document found on success
+ // NOT_FOUND if the key doesn't exist or document has been deleted
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<DocumentProto> Get(std::string_view name_space,
+ std::string_view uri) const;
+
+ // Finds and returns the document identified by the given document id
+ //
+ // Returns:
+ // The document found on success
+ // INVALID_ARGUMENT if document_id is less than 0 or greater than the
+ // maximum value
+ // NOT_FOUND if the document doesn't exist or has been deleted
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<DocumentProto> Get(DocumentId document_id) const;
+
+ // Returns true if there's an existing document associated with the given
+ // document id.
+ bool DoesDocumentExist(DocumentId document_id) const;
+
+ // Deletes the document identified by the given namespace and uri
+ //
+ // NOTE: Space is not reclaimed for deleted documents until Optimize() is
+ // called.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status Delete(std::string_view name_space,
+ std::string_view uri);
+
+ // Returns the NamespaceId of the string namespace
+ //
+ // Returns:
+ // NamespaceId on success
+ // NOT_FOUND if the namespace doesn't exist
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<NamespaceId> GetNamespaceId(
+ std::string_view name_space) const;
+
+ // Returns the DocumentAssociatedScoreData of the document specified by the
+ // DocumentId.
+ //
+ // NOTE: This does not check if the document exists and will return the
+ // DocumentFilterData of the document even if it has been deleted. Users
+ // should check DoesDocumentExist(document_id) if they only want existing
+ // documents' DocumentFilterData.
+ //
+ // Returns:
+ // DocumentAssociatedScoreData on success
+ // OUT_OF_RANGE if document_id is negative or exceeds previously seen
+ // DocumentIds
+ libtextclassifier3::StatusOr<DocumentAssociatedScoreData>
+ GetDocumentAssociatedScoreData(DocumentId document_id) const;
+
+ // Returns the DocumentFilterData of the document specified by the DocumentId.
+ //
+ // NOTE: This does not check if the document exists and will return the
+ // DocumentFilterData of the document even if it has been deleted. Users
+ // should check DoesDocumentExist(document_id) if they only want existing
+ // documents' DocumentFilterData.
+ //
+ // Returns:
+ // DocumentFilterData on success
+ // OUT_OF_RANGE if document_id is negative or exceeds previously seen
+ // DocumentIds
+ libtextclassifier3::StatusOr<DocumentFilterData> GetDocumentFilterData(
+ DocumentId document_id) const;
+
+ // Deletes all documents belonging to the given namespace.
+ //
+ // NOTE: Space is not reclaimed for deleted documents until Optimize() is
+ // called.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status DeleteByNamespace(std::string_view name_space);
+
+ // Deletes all documents belonging to the given schema type
+ //
+ // NOTE: Space is not reclaimed for deleted documents until Optimize() is
+ // called.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status DeleteBySchemaType(std::string_view schema_type);
+
+ // Syncs all the data and metadata changes to disk.
+ // Returns any encountered IO errors.
+ libtextclassifier3::Status PersistToDisk();
+
+ // Calculates and returns the disk usage in bytes.
+ //
+ // Returns:
+ // Disk usage on success
+ // INTERNAL_ERROR on IO error
+ //
+ // TODO(samzheng): consider returning a struct which has the breakdown of each
+ // component.
+ libtextclassifier3::StatusOr<int64_t> GetDiskUsage() const;
+
+ // Update any derived data off of the SchemaStore with the new SchemaStore.
+ // This may include pointers, SchemaTypeIds, etc.
+ //
+ // NOTE: This function may delete documents. A document may be invalidated by
+ // the new SchemaStore, such as failing validation or having its schema type
+ // deleted from the schema.
+ //
+ // This is best used if the caller is unsure about what's changed in the
+ // SchemaStore, and wants to update all information no matter what. If the
+ // caller does know what has changed, then it's recommended to call
+ // OptimizedUpdateSchemaStore.
+ //
+ // Returns;
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status UpdateSchemaStore(const SchemaStore* schema_store);
+
+ // Performs the same funtionality as UpdateSchemaStore, but this can be more
+ // optimized in terms of less disk reads and less work if we know exactly
+ // what's changed between the old and new SchemaStore.
+ //
+ // Returns;
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status OptimizedUpdateSchemaStore(
+ const SchemaStore* schema_store,
+ const SchemaStore::SetSchemaResult& set_schema_result);
+
+ // Reduces internal file sizes by reclaiming space of deleted documents and
+ // regenerating derived files.
+ //
+ // NOTE: The tasks in this method are too expensive to be executed in
+ // real-time. The caller should decide how frequently and when to call this
+ // method based on device usage.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status Optimize();
+
+ // Copy data from current base directory into a new directory. Any outdated or
+ // deleted data won't be copied. During the process, document ids will be
+ // reassigned so any files / classes that are based on old document ids may be
+ // outdated.
+ //
+ // NOTE: The tasks in this method are too expensive to be executed in
+ // real-time. The caller should decide how frequently and when to call this
+ // method based on device usage.
+ //
+ // Returns:
+ // OK on success
+ // INVALID_ARGUMENT if new_directory is same as current base directory
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status OptimizeInto(const std::string& new_directory);
+
+ // Computes the combined checksum of the document store - includes the ground
+ // truth and all derived files.
+ //
+ // Returns:
+ // Combined checksum on success
+ // INTERNAL_ERROR on compute error
+ libtextclassifier3::StatusOr<Crc32> ComputeChecksum() const;
+
+ private:
+ // Use DocumentStore::Create() to instantiate.
+ DocumentStore(const Filesystem* filesystem, std::string_view base_dir,
+ const Clock* clock, const SchemaStore* schema_store);
+
+ const Filesystem* const filesystem_;
+ const std::string base_dir_;
+ const Clock& clock_;
+
+ // Handles the ground truth schema and all of the derived data off of the
+ // schema
+ const SchemaStore* schema_store_;
+
+ // Used to validate incoming documents
+ DocumentValidator document_validator_;
+
+ // A log used to store all documents, it serves as a ground truth of doc
+ // store. key_mapper_ and document_id_mapper_ can be regenerated from it.
+ std::unique_ptr<FileBackedProtoLog<DocumentWrapper>> document_log_;
+
+ // Key (namespace + uri) to DocumentId mapping
+ std::unique_ptr<KeyMapper<DocumentId>> document_key_mapper_;
+
+ // DocumentId to file offset mapping
+ std::unique_ptr<FileBackedVector<int64_t>> document_id_mapper_;
+
+ // A cache of document associated scores. The ground truth of the scores is
+ // DocumentProto stored in document_log_. This cache contains:
+ // - Document score
+ // - Document creation timestamp in seconds
+ std::unique_ptr<FileBackedVector<DocumentAssociatedScoreData>> score_cache_;
+
+ // A cache of data, indexed by DocumentId, used to filter documents. Currently
+ // contains:
+ // - NamespaceId
+ // - SchemaTypeId
+ // - Expiration timestamp in seconds
+ std::unique_ptr<FileBackedVector<DocumentFilterData>> filter_cache_;
+
+ // Maps namespaces to a densely-assigned unique id. Namespaces are assigned an
+ // id when the first document belonging to that namespace is added to the
+ // DocumentStore. Namespaces may be removed from the mapper during compaction.
+ std::unique_ptr<KeyMapper<NamespaceId>> namespace_mapper_;
+
+ // Used internally to indicate whether the class has been initialized. This is
+ // to guard against cases where the object has been created, but Initialize
+ // fails in the constructor. If we have successfully exited the constructor,
+ // then this field can be ignored. Clients of DocumentStore should not need to
+ // worry about this field.
+ bool initialized_ = false;
+
+ libtextclassifier3::Status Initialize();
+
+ // Creates sub-components and verifies the integrity of each sub-component.
+ //
+ // Returns an error if subcomponents failed to initialize successfully.
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status InitializeDerivedFiles();
+
+ // Re-generates all files derived from the ground truth: the document log.
+ //
+ // NOTE: if this function fails, the only thing we can do is to retry it until
+ // it succeeds or prevent the initialization of a DocumentStore. The
+ // DocumentStore object wouldn't work reliably if this fails.
+ //
+ // Steps:
+ // 1. Delete all derived files.
+ // 2. Iterate through document log, put data into new key mapper and
+ // document_id
+ // mapper.
+ // 3. Create header and store the updated combined checksum
+ libtextclassifier3::Status RegenerateDerivedFiles();
+
+ // Resets the unique_ptr to the document_key_mapper, deletes the underlying
+ // file, and re-creates a new instance of the document_key_mapper .
+ //
+ // Returns OK or any IO errors.
+ libtextclassifier3::Status ResetDocumentKeyMapper();
+
+ // Resets the unique_ptr to the document_id_mapper, deletes the underlying
+ // file, and re-creates a new instance of the document_id_mapper.
+ //
+ // Returns OK or any IO errors.
+ libtextclassifier3::Status ResetDocumentIdMapper();
+
+ // Resets the unique_ptr to the score_cache, deletes the underlying file, and
+ // re-creates a new instance of the score_cache.
+ //
+ // Returns OK or any IO errors.
+ libtextclassifier3::Status ResetDocumentAssociatedScoreCache();
+
+ // Resets the unique_ptr to the filter_cache, deletes the underlying file, and
+ // re-creates a new instance of the filter_cache.
+ //
+ // Returns OK or any IO errors.
+ libtextclassifier3::Status ResetFilterCache();
+
+ // Resets the unique_ptr to the namespace_mapper, deletes the underlying file,
+ // and re-creates a new instance of the namespace_mapper.
+ //
+ // Returns OK or any IO errors.
+ libtextclassifier3::Status ResetNamespaceMapper();
+
+ // Checks if the header exists already. This does not create the header file
+ // if it doesn't exist.
+ bool HeaderExists();
+
+ // Update and replace the header file. Creates the header file if it doesn't
+ // exist.
+ libtextclassifier3::Status UpdateHeader(const Crc32& checksum);
+
+ // Update derived files that `name_space` has been deleted. This is primarily
+ // useful if we're trying to update derived files when we've already seen a
+ // namespace tombstone, and don't need to write another tombstone.
+ //
+ // NOTE: Space is not reclaimed in the derived files until Optimize() is
+ // called.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status UpdateDerivedFilesNamespaceDeleted(
+ std::string_view name_space);
+
+ // Update derived files that the schema type schema_type_id has been deleted.
+ // This is primarily useful if we're trying to update derived files when we've
+ // already seen a schema type tombstone, and don't need to write another
+ // tombstone.
+ //
+ // NOTE: Space is not reclaimed in the derived files until Optimize() is
+ // called.
+ //
+ // Returns:
+ // OK on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::Status UpdateDerivedFilesSchemaTypeDeleted(
+ SchemaTypeId schema_type_id);
+
+ // Helper method to find a DocumentId that is associated with the given
+ // namespace and uri.
+ //
+ // NOTE: The DocumentId may refer to a invalid document (deleted
+ // or expired). Callers can call DoesDocumentExist(document_id) to ensure it
+ // refers to a valid Document.
+ //
+ // Returns:
+ // A DocumentId on success
+ // NOT_FOUND if the key doesn't exist
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<DocumentId> GetDocumentId(
+ std::string_view name_space, std::string_view uri) const;
+
+ // Helper method to validate the document id and return the file offset of the
+ // associated document in document_log_.
+ //
+ // This can be a more informative call than just DoesDocumentExist because it
+ // can return more status errors on whether the Document actually doesn't
+ // exist or if there was an internal error while accessing files.
+ //
+ // Returns:
+ // The file offset on success
+ // INVALID_ARGUMENT if document_id is less than 0 or greater than the
+ // maximum value
+ // NOT_FOUND if the document doesn't exist (i.e. deleted or expired)
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<int64_t> DoesDocumentExistAndGetFileOffset(
+ DocumentId document_id) const;
+
+ // Updates the entry in the score cache for document_id.
+ libtextclassifier3::Status UpdateDocumentAssociatedScoreCache(
+ DocumentId document_id, const DocumentAssociatedScoreData& score_data);
+
+ // Updates the entry in the filter cache for document_id.
+ libtextclassifier3::Status UpdateFilterCache(
+ DocumentId document_id, const DocumentFilterData& filter_data);
+};
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_DOCUMENT_STORE_H_
diff --git a/icing/store/document-store_test.cc b/icing/store/document-store_test.cc
new file mode 100644
index 0000000..45e2b9c
--- /dev/null
+++ b/icing/store/document-store_test.cc
@@ -0,0 +1,1886 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/store/document-store.h"
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/absl_ports/str_cat.h"
+#include "icing/document-builder.h"
+#include "icing/file/file-backed-vector.h"
+#include "icing/file/filesystem.h"
+#include "icing/file/memory-mapped-file.h"
+#include "icing/file/mock-filesystem.h"
+#include "icing/portable/equals-proto.h"
+#include "icing/proto/document.pb.h"
+#include "icing/proto/schema.pb.h"
+#include "icing/schema/schema-store.h"
+#include "icing/store/document-filter-data.h"
+#include "icing/store/document-id.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/fake-clock.h"
+#include "icing/testing/tmp-directory.h"
+#include "icing/util/crc32.h"
+
+namespace icing {
+namespace lib {
+
+using ::icing::lib::portable_equals_proto::EqualsProto;
+using ::testing::_;
+using ::testing::Eq;
+using ::testing::Gt;
+using ::testing::HasSubstr;
+using ::testing::IsFalse;
+using ::testing::IsTrue;
+using ::testing::Not;
+using ::testing::Return;
+
+class DocumentStoreTest : public ::testing::Test {
+ protected:
+ DocumentStoreTest()
+ : test_dir_(GetTestTempDir() + "/icing"),
+ document_store_dir_(test_dir_ + "/document_store"),
+ schema_store_dir_(test_dir_ + "/schema_store") {
+ filesystem_.CreateDirectoryRecursively(test_dir_.c_str());
+ filesystem_.CreateDirectoryRecursively(document_store_dir_.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir_.c_str());
+ test_document1_ =
+ DocumentBuilder()
+ .SetKey("icing", "email/1")
+ .SetSchema("email")
+ .AddStringProperty("subject", "subject foo")
+ .AddStringProperty("body", "body bar")
+ .SetScore(document1_score_)
+ .SetCreationTimestampSecs(
+ document1_creation_timestamp_) // A random timestamp
+ .SetTtlSecs(document1_ttl_)
+ .Build();
+ test_document2_ =
+ DocumentBuilder()
+ .SetKey("icing", "email/2")
+ .SetSchema("email")
+ .AddStringProperty("subject", "subject foo 2")
+ .AddStringProperty("body", "body bar 2")
+ .SetScore(document2_score_)
+ .SetCreationTimestampSecs(
+ document2_creation_timestamp_) // A random timestamp
+ .SetTtlSecs(document2_ttl_)
+ .Build();
+ }
+
+ void SetUp() override {
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+
+ auto subject = type_config->add_properties();
+ subject->set_property_name("subject");
+ subject->set_data_type(PropertyConfigProto::DataType::STRING);
+ subject->set_cardinality(PropertyConfigProto::Cardinality::OPTIONAL);
+ subject->mutable_indexing_config()->set_term_match_type(
+ TermMatchType::EXACT_ONLY);
+ subject->mutable_indexing_config()->set_tokenizer_type(
+ IndexingConfig::TokenizerType::PLAIN);
+
+ auto body = type_config->add_properties();
+ body->set_property_name("body");
+ body->set_data_type(PropertyConfigProto::DataType::STRING);
+ body->set_cardinality(PropertyConfigProto::Cardinality::OPTIONAL);
+ body->mutable_indexing_config()->set_term_match_type(
+ TermMatchType::EXACT_ONLY);
+ body->mutable_indexing_config()->set_tokenizer_type(
+ IndexingConfig::TokenizerType::PLAIN);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ schema_store_, SchemaStore::Create(&filesystem_, schema_store_dir_));
+ ASSERT_THAT(schema_store_->SetSchema(schema), IsOk());
+ }
+
+ void TearDown() override {
+ filesystem_.DeleteDirectoryRecursively(test_dir_.c_str());
+ }
+
+ const Filesystem filesystem_;
+ const std::string test_dir_;
+ FakeClock fake_clock_;
+ const std::string document_store_dir_;
+ const std::string schema_store_dir_;
+ DocumentProto test_document1_;
+ DocumentProto test_document2_;
+ std::unique_ptr<SchemaStore> schema_store_;
+
+ // Document1 values
+ const int document1_score_ = 1;
+ const int64_t document1_creation_timestamp_ = 1;
+ const int64_t document1_ttl_ = 0;
+ const int64_t document1_expiration_timestamp_ =
+ std::numeric_limits<int64_t>::max(); // special_case where ttl=0
+
+ // Document2 values
+ const int document2_score_ = 2;
+ const int64_t document2_creation_timestamp_ = 2;
+ const int64_t document2_ttl_ = 1;
+ const int64_t document2_expiration_timestamp_ = 3; // creation + ttl
+};
+
+TEST_F(DocumentStoreTest, InitializationFailure) {
+ MockFilesystem mock_filesystem;
+ ON_CALL(mock_filesystem, OpenForWrite(_)).WillByDefault(Return(false));
+
+ EXPECT_THAT(DocumentStore::Create(&mock_filesystem, document_store_dir_,
+ &fake_clock_, schema_store_.get()),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(DocumentStoreTest, PutAndGetInSameNamespaceOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Both documents have namespace of "icing"
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(test_document1_));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+
+ EXPECT_THAT(doc_store->Get(document_id1),
+ IsOkAndHolds(EqualsProto(test_document1_)));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+}
+
+TEST_F(DocumentStoreTest, PutAndGetAcrossNamespacesOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Can handle different namespaces with same url
+ DocumentProto foo_document = DocumentBuilder()
+ .SetKey("foo", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+ DocumentProto bar_document = DocumentBuilder()
+ .SetKey("bar", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(foo_document));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(DocumentProto(bar_document)));
+
+ EXPECT_THAT(doc_store->Get(document_id1),
+ IsOkAndHolds(EqualsProto(foo_document)));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(bar_document)));
+}
+
+// Validates that putting an document with the same key will overwrite previous
+// document and old doc ids are not getting reused.
+TEST_F(DocumentStoreTest, PutSameKey) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Creates two documents with the same key (namespace + uri)
+ DocumentProto document1 = DocumentProto(test_document1_);
+ DocumentProto document2 = DocumentProto(test_document1_);
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(document1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(document2));
+ EXPECT_THAT(document_id1, Not(document_id2));
+ // document2 overrides document1, so document_id1 becomes invalid
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(document2)));
+
+ // Makes sure that old doc ids are not getting reused.
+ DocumentProto document3 = DocumentProto(test_document1_);
+ document3.set_uri("another/uri/1");
+ EXPECT_THAT(doc_store->Put(document3), IsOkAndHolds(Not(document_id1)));
+}
+
+TEST_F(DocumentStoreTest, IsDocumentExisting) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(DocumentProto(test_document1_)));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+
+ EXPECT_THAT(doc_store->DoesDocumentExist(document_id1), IsTrue());
+ EXPECT_THAT(doc_store->DoesDocumentExist(document_id2), IsTrue());
+
+ DocumentId invalid_document_id_negative = -1;
+ EXPECT_THAT(doc_store->DoesDocumentExist(invalid_document_id_negative),
+ IsFalse());
+
+ DocumentId invalid_document_id_greater_than_max = kMaxDocumentId + 2;
+ EXPECT_THAT(
+ doc_store->DoesDocumentExist(invalid_document_id_greater_than_max),
+ IsFalse());
+
+ EXPECT_THAT(doc_store->DoesDocumentExist(kInvalidDocumentId), IsFalse());
+
+ DocumentId invalid_document_id_out_of_range = document_id2 + 1;
+ EXPECT_THAT(doc_store->DoesDocumentExist(invalid_document_id_out_of_range),
+ IsFalse());
+}
+
+TEST_F(DocumentStoreTest, GetDeletedDocumentNotFound) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_EXPECT_OK(document_store->Put(DocumentProto(test_document1_)));
+ EXPECT_THAT(
+ document_store->Get(test_document1_.namespace_(), test_document1_.uri()),
+ IsOkAndHolds(EqualsProto(test_document1_)));
+
+ ICING_EXPECT_OK(document_store->Delete(test_document1_.namespace_(),
+ test_document1_.uri()));
+ EXPECT_THAT(
+ document_store->Get(test_document1_.namespace_(), test_document1_.uri()),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, GetExpiredDocumentNotFound) {
+ DocumentProto document = DocumentBuilder()
+ .SetKey("namespace", "uri")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(10)
+ .SetTtlSecs(100)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_EXPECT_OK(document_store->Put(document));
+ EXPECT_THAT(document_store->Get("namespace", "uri"),
+ IsOkAndHolds(EqualsProto(document)));
+
+ // Some arbitrary time before the document's creation time (10) + ttl (100)
+ fake_clock_.SetSeconds(109);
+ EXPECT_THAT(document_store->Get("namespace", "uri"),
+ IsOkAndHolds(EqualsProto(document)));
+
+ // Some arbitrary time equal to the document's creation time (10) + ttl (100)
+ fake_clock_.SetSeconds(110);
+ EXPECT_THAT(document_store->Get("namespace", "uri"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Some arbitrary time past the document's creation time (10) + ttl (100)
+ fake_clock_.SetSeconds(200);
+ EXPECT_THAT(document_store->Get("namespace", "uri"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, GetInvalidDocumentId) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id,
+ doc_store->Put(DocumentProto(test_document1_)));
+
+ DocumentId invalid_document_id_negative = -1;
+ EXPECT_THAT(doc_store->Get(invalid_document_id_negative),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ DocumentId invalid_document_id_greater_than_max = kMaxDocumentId + 2;
+ EXPECT_THAT(doc_store->Get(invalid_document_id_greater_than_max),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ EXPECT_THAT(doc_store->Get(kInvalidDocumentId),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT));
+
+ DocumentId invalid_document_id_out_of_range = document_id + 1;
+ EXPECT_THAT(doc_store->Get(invalid_document_id_out_of_range),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, Delete) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Get() after Delete() returns NOT_FOUND
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id,
+ doc_store->Put(DocumentProto(test_document1_)));
+ EXPECT_THAT(doc_store->Delete("icing", "email/1"), IsOk());
+ EXPECT_THAT(doc_store->Get(document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // Validates that deleting something non-existing won't append anything to
+ // ground truth
+ int64_t ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ // icing + email/1 has already been deleted.
+ EXPECT_THAT(doc_store->Delete("icing", "email/1"), IsOk());
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ EXPECT_THAT(ground_truth_size_before, Eq(ground_truth_size_after));
+}
+
+TEST_F(DocumentStoreTest, DeleteByNamespaceOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ DocumentProto document1 = test_document1_;
+ document1.set_namespace_("namespace.1");
+ document1.set_uri("uri1");
+ ICING_ASSERT_OK(doc_store->Put(document1));
+
+ DocumentProto document2 = test_document1_;
+ document2.set_namespace_("namespace.2");
+ document2.set_uri("uri1");
+ ICING_ASSERT_OK(doc_store->Put(document2));
+
+ DocumentProto document3 = test_document1_;
+ document3.set_namespace_("namespace.3");
+ document3.set_uri("uri1");
+ ICING_ASSERT_OK(doc_store->Put(document3));
+
+ DocumentProto document4 = test_document1_;
+ document4.set_namespace_("namespace.1");
+ document4.set_uri("uri2");
+ ICING_ASSERT_OK(doc_store->Put(document4));
+
+ // DELETE namespace.1. document1 and document 4 should be deleted. document2
+ // and document3 should still be retrievable.
+ ICING_EXPECT_OK(doc_store->DeleteByNamespace("namespace.1"));
+ EXPECT_THAT(doc_store->Get(document1.namespace_(), document1.uri()),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document2.namespace_(), document2.uri()),
+ IsOkAndHolds(EqualsProto(document2)));
+ EXPECT_THAT(doc_store->Get(document3.namespace_(), document3.uri()),
+ IsOkAndHolds(EqualsProto(document3)));
+ EXPECT_THAT(doc_store->Get(document4.namespace_(), document4.uri()),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, DeleteByNamespaceNonexistentNamespaceOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Validates that deleting something non-existing won't append anything to
+ // ground truth
+ int64_t ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+
+ ICING_EXPECT_OK(doc_store->DeleteByNamespace("nonexistent_namespace"));
+
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ EXPECT_THAT(ground_truth_size_before, Eq(ground_truth_size_after));
+}
+
+TEST_F(DocumentStoreTest, DeleteByNamespaceRecoversOk) {
+ DocumentProto document1 = test_document1_;
+ document1.set_namespace_("namespace.1");
+ document1.set_uri("uri1");
+
+ DocumentProto document2 = test_document1_;
+ document2.set_namespace_("namespace.2");
+ document2.set_uri("uri1");
+
+ DocumentProto document3 = test_document1_;
+ document3.set_namespace_("namespace.3");
+ document3.set_uri("uri1");
+
+ DocumentProto document4 = test_document1_;
+ document4.set_namespace_("namespace.1");
+ document4.set_uri("uri2");
+
+ int64_t ground_truth_size_before;
+ {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK(doc_store->Put(document1));
+ ICING_ASSERT_OK(doc_store->Put(document2));
+ ICING_ASSERT_OK(doc_store->Put(document3));
+ ICING_ASSERT_OK(doc_store->Put(document4));
+
+ // DELETE namespace.1. document1 and document 4 should be deleted. document2
+ // and document3 should still be retrievable.
+ ICING_EXPECT_OK(doc_store->DeleteByNamespace("namespace.1"));
+
+ ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ } // Destructors should update checksum and persist all data to file.
+
+ // Change the DocStore's header combined checksum so that it won't match the
+ // recalculated checksum on initialization. This will force a regeneration of
+ // derived files from ground truth.
+ const std::string header_file =
+ absl_ports::StrCat(document_store_dir_, "/document_store_header");
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = 10; // Arbitrary garbage checksum
+ filesystem_.DeleteFile(header_file.c_str());
+ filesystem_.Write(header_file.c_str(), &header, sizeof(header));
+
+ // Successfully recover from a corrupt derived file issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Make sure we didn't add anything to the ground truth after we recovered.
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ EXPECT_EQ(ground_truth_size_before, ground_truth_size_after);
+
+ EXPECT_THAT(doc_store->Get(document1.namespace_(), document1.uri()),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document2.namespace_(), document2.uri()),
+ IsOkAndHolds(EqualsProto(document2)));
+ EXPECT_THAT(doc_store->Get(document3.namespace_(), document3.uri()),
+ IsOkAndHolds(EqualsProto(document3)));
+ EXPECT_THAT(doc_store->Get(document4.namespace_(), document4.uri()),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, DeleteBySchemaTypeOk) {
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+ type_config = schema.add_types();
+ type_config->set_schema_type("person");
+
+ std::string schema_store_dir = schema_store_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+
+ ICING_ASSERT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ DocumentProto email_document_1 = DocumentBuilder()
+ .SetKey("namespace1", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_1_document_id,
+ document_store->Put(email_document_1));
+
+ DocumentProto email_document_2 = DocumentBuilder()
+ .SetKey("namespace2", "2")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_2_document_id,
+ document_store->Put(email_document_2));
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetKey("namespace", "3")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId message_document_id,
+ document_store->Put(message_document));
+
+ DocumentProto person_document = DocumentBuilder()
+ .SetKey("namespace", "4")
+ .SetSchema("person")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId person_document_id,
+ document_store->Put(person_document));
+
+ // Delete the "email" type and ensure that it works across both
+ // email_document's namespaces. And that other documents aren't affected.
+ ICING_EXPECT_OK(document_store->DeleteBySchemaType("email"));
+ EXPECT_THAT(document_store->Get(email_1_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(email_2_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+ EXPECT_THAT(document_store->Get(person_document_id),
+ IsOkAndHolds(EqualsProto(person_document)));
+
+ // Delete the "message" type and check that other documents aren't affected
+ ICING_EXPECT_OK(document_store->DeleteBySchemaType("message"));
+ EXPECT_THAT(document_store->Get(email_1_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(email_2_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(person_document_id),
+ IsOkAndHolds(EqualsProto(person_document)));
+}
+
+TEST_F(DocumentStoreTest, DeleteBySchemaTypeNonexistentSchemaTypeOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Validates that deleting something non-existing won't append anything to
+ // ground truth
+ int64_t ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+
+ ICING_EXPECT_OK(document_store->DeleteBySchemaType("nonexistent_type"));
+
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+
+ EXPECT_THAT(ground_truth_size_before, Eq(ground_truth_size_after));
+}
+
+TEST_F(DocumentStoreTest, DeleteBySchemaTypeRecoversOk) {
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ std::string schema_store_dir = schema_store_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+
+ ICING_ASSERT_OK(schema_store->SetSchema(schema));
+
+ DocumentId email_document_id;
+ DocumentId message_document_id;
+
+ DocumentProto email_document = DocumentBuilder()
+ .SetKey("namespace", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(1)
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetKey("namespace", "2")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ int64_t ground_truth_size_before;
+ {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(email_document_id,
+ document_store->Put(email_document));
+ ICING_ASSERT_OK_AND_ASSIGN(message_document_id,
+ document_store->Put(message_document));
+
+ // Delete "email". "message" documents should still be retrievable.
+ ICING_EXPECT_OK(document_store->DeleteBySchemaType("email"));
+
+ ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ } // Destructors should update checksum and persist all data to file.
+
+ // Change the DocumentStore's header combined checksum so that it won't match
+ // the recalculated checksum on initialization. This will force a regeneration
+ // of derived files from ground truth.
+ const std::string header_file =
+ absl_ports::StrCat(document_store_dir_, "/document_store_header");
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = 10; // Arbitrary garbage checksum
+ filesystem_.DeleteFile(header_file.c_str());
+ filesystem_.Write(header_file.c_str(), &header, sizeof(header));
+
+ // Successfully recover from a corrupt derived file issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ // Make sure we didn't add anything to the ground truth after we recovered.
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ EXPECT_EQ(ground_truth_size_before, ground_truth_size_after);
+
+ EXPECT_THAT(document_store->Get(email_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+}
+
+TEST_F(DocumentStoreTest, DeletedSchemaTypeFromSchemaStoreRecoversOk) {
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ std::string schema_store_dir = schema_store_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+
+ ICING_ASSERT_OK(schema_store->SetSchema(schema));
+
+ DocumentId email_document_id;
+ DocumentId message_document_id;
+
+ DocumentProto email_document = DocumentBuilder()
+ .SetKey("namespace", "email")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(1)
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetKey("namespace", "message")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(1)
+ .Build();
+ int64_t ground_truth_size_before;
+ {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(email_document_id,
+ document_store->Put(email_document));
+ ICING_ASSERT_OK_AND_ASSIGN(message_document_id,
+ document_store->Put(message_document));
+
+ // Delete "email". "message" documents should still be retrievable.
+ ICING_EXPECT_OK(document_store->DeleteBySchemaType("email"));
+
+ EXPECT_THAT(document_store->Get(email_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+
+ ground_truth_size_before = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ } // Destructors should update checksum and persist all data to file.
+
+ // Change the DocumentStore's header combined checksum so that it won't match
+ // the recalculated checksum on initialization. This will force a regeneration
+ // of derived files from ground truth.
+ const std::string header_file =
+ absl_ports::StrCat(document_store_dir_, "/document_store_header");
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = 10; // Arbitrary garbage checksum
+ filesystem_.DeleteFile(header_file.c_str());
+ filesystem_.Write(header_file.c_str(), &header, sizeof(header));
+
+ SchemaProto new_schema;
+ type_config = new_schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_EXPECT_OK(schema_store->SetSchema(
+ new_schema, /*ignore_errors_and_delete_documents=*/true));
+
+ // Successfully recover from a corrupt derived file issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ // Make sure we didn't add anything to the ground truth after we recovered.
+ int64_t ground_truth_size_after = filesystem_.GetFileSize(
+ absl_ports::StrCat(document_store_dir_, "/document_log").c_str());
+ EXPECT_EQ(ground_truth_size_before, ground_truth_size_after);
+
+ EXPECT_THAT(document_store->Get(email_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+}
+
+TEST_F(DocumentStoreTest, OptimizeInto) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ DocumentProto document1 = DocumentBuilder()
+ .SetKey("namespace", "uri1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(100)
+ .SetTtlSecs(1000)
+ .Build();
+
+ DocumentProto document2 = DocumentBuilder()
+ .SetKey("namespace", "uri2")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(100)
+ .SetTtlSecs(1000)
+ .Build();
+
+ DocumentProto document3 = DocumentBuilder()
+ .SetKey("namespace", "uri3")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(100)
+ .SetTtlSecs(100)
+ .Build();
+
+ // Nothing should have expired yet.
+ fake_clock_.SetSeconds(100);
+
+ ICING_ASSERT_OK(doc_store->Put(document1));
+ ICING_ASSERT_OK(doc_store->Put(document2));
+ ICING_ASSERT_OK(doc_store->Put(document3));
+
+ std::string original_document_log = document_store_dir_ + "/document_log";
+ int64_t original_size =
+ filesystem_.GetFileSize(original_document_log.c_str());
+
+ // Optimizing into the same directory is not allowed
+ EXPECT_THAT(doc_store->OptimizeInto(document_store_dir_),
+ StatusIs(libtextclassifier3::StatusCode::INVALID_ARGUMENT,
+ HasSubstr("directory is the same")));
+
+ std::string optimized_dir = document_store_dir_ + "_optimize";
+ std::string optimized_document_log = optimized_dir + "/document_log";
+
+ // Validates that the optimized document log has the same size if nothing is
+ // deleted
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(optimized_dir.c_str()));
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(optimized_dir.c_str()));
+ ICING_ASSERT_OK(doc_store->OptimizeInto(optimized_dir));
+ int64_t optimized_size1 =
+ filesystem_.GetFileSize(optimized_document_log.c_str());
+ EXPECT_EQ(original_size, optimized_size1);
+
+ // Validates that the optimized document log has a smaller size if something
+ // is deleted
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(optimized_dir.c_str()));
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(optimized_dir.c_str()));
+ ICING_ASSERT_OK(doc_store->Delete("namespace", "uri1"));
+ ICING_ASSERT_OK(doc_store->OptimizeInto(optimized_dir));
+ int64_t optimized_size2 =
+ filesystem_.GetFileSize(optimized_document_log.c_str());
+ EXPECT_THAT(original_size, Gt(optimized_size2));
+
+ // Document3 has expired since this is past its creation (100) + ttl (100).
+ // But document1 and document2 should be fine since their ttl's were 1000.
+ fake_clock_.SetSeconds(300);
+
+ // Validates that the optimized document log has a smaller size if something
+ // expired
+ ASSERT_TRUE(filesystem_.DeleteDirectoryRecursively(optimized_dir.c_str()));
+ ASSERT_TRUE(filesystem_.CreateDirectoryRecursively(optimized_dir.c_str()));
+ ICING_ASSERT_OK(doc_store->OptimizeInto(optimized_dir));
+ int64_t optimized_size3 =
+ filesystem_.GetFileSize(optimized_document_log.c_str());
+ EXPECT_THAT(optimized_size2, Gt(optimized_size3));
+}
+
+TEST_F(DocumentStoreTest, ShouldRecoverFromDataLoss) {
+ DocumentId document_id1, document_id2;
+ {
+ // Can put and delete fine.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id1,
+ doc_store->Put(DocumentProto(test_document1_)));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ IsOkAndHolds(EqualsProto(test_document1_)));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ EXPECT_THAT(doc_store->Delete("icing", "email/1"), IsOk());
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ }
+
+ // "Corrupt" the content written in the log by adding non-checksummed data to
+ // it. This will mess up the checksum of the proto log, forcing it to rewind
+ // to the last saved point.
+ DocumentProto document = DocumentBuilder().SetKey("namespace", "uri").Build();
+ const std::string serialized_document = document.SerializeAsString();
+
+ const std::string document_log_file =
+ absl_ports::StrCat(document_store_dir_, "/document_log");
+ int64_t file_size = filesystem_.GetFileSize(document_log_file.c_str());
+ filesystem_.PWrite(document_log_file.c_str(), file_size,
+ serialized_document.data(), serialized_document.size());
+
+ // Successfully recover from a data loss issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+
+ // Checks derived filter cache
+ EXPECT_THAT(doc_store->GetDocumentFilterData(document_id2),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0, document2_expiration_timestamp_)));
+ // Checks derived score cache
+ EXPECT_THAT(doc_store->GetDocumentAssociatedScoreData(document_id2),
+ IsOkAndHolds(DocumentAssociatedScoreData(
+ document2_score_, document2_creation_timestamp_)));
+}
+
+TEST_F(DocumentStoreTest, ShouldRecoverFromCorruptDerivedFile) {
+ DocumentId document_id1, document_id2;
+ {
+ // Can put and delete fine.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id1,
+ doc_store->Put(DocumentProto(test_document1_)));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ IsOkAndHolds(EqualsProto(test_document1_)));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ EXPECT_THAT(doc_store->Delete("icing", "email/1"), IsOk());
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ }
+
+ // "Corrupt" one of the derived files by adding non-checksummed data to
+ // it. This will mess up the checksum and throw an error on the derived file's
+ // initialization.
+ const std::string document_id_mapper_file =
+ absl_ports::StrCat(document_store_dir_, "/document_id_mapper");
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<FileBackedVector<int64_t>> document_id_mapper,
+ FileBackedVector<int64_t>::Create(
+ filesystem_, document_id_mapper_file,
+ MemoryMappedFile::READ_WRITE_AUTO_SYNC));
+ int64_t corrupt_document_id = 3;
+ int64_t corrupt_offset = 3;
+ EXPECT_THAT(document_id_mapper->Set(corrupt_document_id, corrupt_offset),
+ IsOk());
+
+ // Successfully recover from a corrupt derived file issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+
+ // Checks derived filter cache
+ EXPECT_THAT(doc_store->GetDocumentFilterData(document_id2),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0, document2_expiration_timestamp_)));
+ // Checks derived score cache
+ EXPECT_THAT(doc_store->GetDocumentAssociatedScoreData(document_id2),
+ IsOkAndHolds(DocumentAssociatedScoreData(
+ document2_score_, document2_creation_timestamp_)));
+}
+
+TEST_F(DocumentStoreTest, ShouldRecoverFromBadChecksum) {
+ DocumentId document_id1, document_id2;
+ {
+ // Can put and delete fine.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id1,
+ doc_store->Put(DocumentProto(test_document1_)));
+ ICING_ASSERT_OK_AND_ASSIGN(document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ IsOkAndHolds(EqualsProto(test_document1_)));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ EXPECT_THAT(doc_store->Delete("icing", "email/1"), IsOk());
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+ }
+
+ // Change the DocStore's header combined checksum so that it won't match the
+ // recalculated checksum on initialization. This will force a regeneration of
+ // derived files from ground truth.
+ const std::string header_file =
+ absl_ports::StrCat(document_store_dir_, "/document_store_header");
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = 10; // Arbitrary garbage checksum
+ filesystem_.DeleteFile(header_file.c_str());
+ filesystem_.Write(header_file.c_str(), &header, sizeof(header));
+
+ // Successfully recover from a corrupt derived file issue.
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ EXPECT_THAT(doc_store->Get(document_id1),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(doc_store->Get(document_id2),
+ IsOkAndHolds(EqualsProto(test_document2_)));
+
+ // Checks derived filter cache
+ EXPECT_THAT(doc_store->GetDocumentFilterData(document_id2),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0, document2_expiration_timestamp_)));
+ // Checks derived score cache
+ EXPECT_THAT(doc_store->GetDocumentAssociatedScoreData(document_id2),
+ IsOkAndHolds(DocumentAssociatedScoreData(
+ document2_score_, document2_creation_timestamp_)));
+}
+
+TEST_F(DocumentStoreTest, GetDiskUsage) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(int64_t size1, doc_store->GetDiskUsage());
+ EXPECT_THAT(size1, Gt(0));
+
+ ICING_ASSERT_OK(doc_store->Put(test_document1_));
+ ICING_ASSERT_OK_AND_ASSIGN(int64_t size2, doc_store->GetDiskUsage());
+ EXPECT_THAT(size2, Gt(size1));
+
+ ICING_ASSERT_OK(doc_store->Put(test_document2_));
+ EXPECT_THAT(doc_store->GetDiskUsage(), IsOkAndHolds(Gt(size2)));
+ doc_store.reset();
+
+ // Bad file system
+ MockFilesystem mock_filesystem;
+ ON_CALL(mock_filesystem, GetDiskUsage(A<const char *>()))
+ .WillByDefault(Return(Filesystem::kBadFileSize));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store_with_mock_filesystem,
+ DocumentStore::Create(&mock_filesystem, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+ EXPECT_THAT(doc_store_with_mock_filesystem->GetDiskUsage(),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(DocumentStoreTest, MaxDocumentId) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ // Since the DocumentStore is empty, we get an invalid DocumentId
+ EXPECT_THAT(doc_store->last_added_document_id(), Eq(kInvalidDocumentId));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(DocumentProto(test_document1_)));
+ EXPECT_THAT(doc_store->last_added_document_id(), Eq(document_id1));
+
+ // Still returns the last DocumentId even if it was deleted
+ ICING_ASSERT_OK(doc_store->Delete("icing", "email/1"));
+ EXPECT_THAT(doc_store->last_added_document_id(), Eq(document_id1));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(DocumentProto(test_document2_)));
+ EXPECT_THAT(doc_store->last_added_document_id(), Eq(document_id2));
+}
+
+TEST_F(DocumentStoreTest, GetNamespaceId) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ DocumentProto document_namespace1 =
+ DocumentBuilder().SetKey("namespace1", "1").SetSchema("email").Build();
+ DocumentProto document_namespace2 =
+ DocumentBuilder().SetKey("namespace2", "2").SetSchema("email").Build();
+
+ ICING_ASSERT_OK(doc_store->Put(DocumentProto(document_namespace1)));
+ ICING_ASSERT_OK(doc_store->Put(DocumentProto(document_namespace2)));
+
+ // NamespaceId of 0 since it was the first namespace seen by the DocumentStore
+ EXPECT_THAT(doc_store->GetNamespaceId("namespace1"), IsOkAndHolds(Eq(0)));
+
+ // NamespaceId of 1 since it was the second namespace seen by the
+ // DocumentStore
+ EXPECT_THAT(doc_store->GetNamespaceId("namespace2"), IsOkAndHolds(Eq(1)));
+
+ // NamespaceMapper doesn't care if the document has been deleted
+ EXPECT_THAT(doc_store->GetNamespaceId("namespace1"), IsOkAndHolds(Eq(0)));
+}
+
+TEST_F(DocumentStoreTest, GetDuplicateNamespaceId) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ DocumentProto document1 =
+ DocumentBuilder().SetKey("namespace", "1").SetSchema("email").Build();
+ DocumentProto document2 =
+ DocumentBuilder().SetKey("namespace", "2").SetSchema("email").Build();
+
+ ICING_ASSERT_OK(doc_store->Put(document1));
+ ICING_ASSERT_OK(doc_store->Put(document2));
+
+ // NamespaceId of 0 since it was the first namespace seen by the DocumentStore
+ EXPECT_THAT(doc_store->GetNamespaceId("namespace"), IsOkAndHolds(Eq(0)));
+}
+
+TEST_F(DocumentStoreTest, NonexistentNamespaceNotFound) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ EXPECT_THAT(doc_store->GetNamespaceId("nonexistent_namespace"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+}
+
+TEST_F(DocumentStoreTest, FilterCacheHoldsDeletedDocumentData) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id,
+ doc_store->Put(test_document1_));
+
+ EXPECT_THAT(
+ doc_store->GetDocumentFilterData(document_id),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0,
+ /*expiration_timestamp_secs=*/document1_expiration_timestamp_)));
+
+ // FilterCache doesn't care if the document has been deleted
+ ICING_ASSERT_OK(doc_store->Delete("icing", "email/1"));
+ EXPECT_THAT(
+ doc_store->GetDocumentFilterData(document_id),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0,
+ /*expiration_timestamp_secs=*/document1_expiration_timestamp_)));
+}
+
+TEST_F(DocumentStoreTest,
+ ExpirationTimestampIsSumOfNonZeroTtlAndCreationTimestamp) {
+ DocumentProto document = DocumentBuilder()
+ .SetKey("namespace1", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(100)
+ .SetTtlSecs(1000)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id, doc_store->Put(document));
+
+ EXPECT_THAT(
+ doc_store->GetDocumentFilterData(document_id),
+ IsOkAndHolds(DocumentFilterData(/*namespace_id=*/0,
+ /*schema_type_id=*/0,
+ /*expiration_timestamp_secs=*/1100)));
+}
+
+TEST_F(DocumentStoreTest, ExpirationTimestampIsInt64MaxIfTtlIsZero) {
+ DocumentProto document = DocumentBuilder()
+ .SetKey("namespace1", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(100)
+ .SetTtlSecs(0)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id, doc_store->Put(document));
+
+ EXPECT_THAT(
+ doc_store->GetDocumentFilterData(document_id),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0,
+ /*expiration_timestamp_secs=*/std::numeric_limits<int64_t>::max())));
+}
+
+TEST_F(DocumentStoreTest, ExpirationTimestampIsInt64MaxOnOverflow) {
+ DocumentProto document =
+ DocumentBuilder()
+ .SetKey("namespace1", "1")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(std::numeric_limits<int64_t>::max() - 1)
+ .SetTtlSecs(std::numeric_limits<int64_t>::max() - 1)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id, doc_store->Put(document));
+
+ EXPECT_THAT(
+ doc_store->GetDocumentFilterData(document_id),
+ IsOkAndHolds(DocumentFilterData(
+ /*namespace_id=*/0,
+ /*schema_type_id=*/0,
+ /*expiration_timestamp_secs=*/std::numeric_limits<int64_t>::max())));
+}
+
+TEST_F(DocumentStoreTest, CreationTimestampShouldBePopulated) {
+ // Creates a document without a given creation timestamp
+ DocumentProto document_without_creation_timestamp =
+ DocumentBuilder()
+ .SetKey("icing", "email/1")
+ .SetSchema("email")
+ .AddStringProperty("subject", "subject foo")
+ .AddStringProperty("body", "body bar")
+ .Build();
+
+ std::time_t fake_real_time = 100;
+ fake_clock_.SetSeconds(fake_real_time);
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentId document_id,
+ doc_store->Put(document_without_creation_timestamp));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentProto document_with_creation_timestamp,
+ doc_store->Get(document_id));
+
+ // Now the creation timestamp should be set by document store.
+ EXPECT_THAT(document_with_creation_timestamp.creation_timestamp_secs(),
+ Eq(fake_real_time));
+}
+
+TEST_F(DocumentStoreTest, ShouldWriteAndReadScoresCorrectly) {
+ DocumentProto document1 = DocumentBuilder()
+ .SetKey("icing", "email/1")
+ .SetSchema("email")
+ .AddStringProperty("subject", "subject foo")
+ // With default doc score 0
+ .Build();
+ DocumentProto document2 = DocumentBuilder()
+ .SetKey("icing", "email/1")
+ .SetSchema("email")
+ .AddStringProperty("subject", "subject foo")
+ .SetScore(5)
+ .Build();
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> doc_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id1,
+ doc_store->Put(document1));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId document_id2,
+ doc_store->Put(document2));
+
+ EXPECT_THAT(doc_store->GetDocumentAssociatedScoreData(document_id1),
+ IsOkAndHolds(DocumentAssociatedScoreData(
+ /*document_score=*/0, /*creation_timestamp_secs=*/0)));
+
+ EXPECT_THAT(doc_store->GetDocumentAssociatedScoreData(document_id2),
+ IsOkAndHolds(DocumentAssociatedScoreData(
+ /*document_score=*/5, /*creation_timestamp_secs=*/0)));
+}
+
+TEST_F(DocumentStoreTest, ComputeChecksumSameBetweenCalls) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_EXPECT_OK(document_store->Put(test_document1_));
+ ICING_ASSERT_OK_AND_ASSIGN(Crc32 checksum, document_store->ComputeChecksum());
+
+ // Calling ComputeChecksum again shouldn't change anything
+ EXPECT_THAT(document_store->ComputeChecksum(), IsOkAndHolds(checksum));
+}
+
+TEST_F(DocumentStoreTest, ComputeChecksumSameAcrossInstances) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_EXPECT_OK(document_store->Put(test_document1_));
+ ICING_ASSERT_OK_AND_ASSIGN(Crc32 checksum, document_store->ComputeChecksum());
+
+ // Destroy the previous instance and recreate DocumentStore
+ document_store.reset();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ document_store, DocumentStore::Create(&filesystem_, document_store_dir_,
+ &fake_clock_, schema_store_.get()));
+
+ EXPECT_THAT(document_store->ComputeChecksum(), IsOkAndHolds(checksum));
+}
+
+TEST_F(DocumentStoreTest, ComputeChecksumChangesOnModification) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store_.get()));
+
+ ICING_EXPECT_OK(document_store->Put(test_document1_));
+ ICING_ASSERT_OK_AND_ASSIGN(Crc32 checksum, document_store->ComputeChecksum());
+
+ ICING_EXPECT_OK(document_store->Put(test_document2_));
+ EXPECT_THAT(document_store->ComputeChecksum(),
+ IsOkAndHolds(Not(Eq(checksum))));
+}
+
+TEST_F(DocumentStoreTest, RegenerateDerivedFilesSkipsUnknownSchemaTypeIds) {
+ const std::string schema_store_dir = schema_store_dir_ + "_custom";
+
+ DocumentId email_document_id;
+ NamespaceId email_namespace_id;
+ int64_t email_expiration_timestamp;
+ DocumentProto email_document = DocumentBuilder()
+ .SetKey("namespace", "email_uri")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ DocumentId message_document_id;
+ NamespaceId message_namespace_id;
+ int64_t message_expiration_timestamp;
+ DocumentProto message_document = DocumentBuilder()
+ .SetKey("namespace", "message_uri")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ {
+ // Set a schema with "email" and "message"
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId message_schema_type_id,
+ schema_store->GetSchemaTypeId("message"));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ // Insert and verify a "email "document
+ ICING_ASSERT_OK_AND_ASSIGN(
+ email_document_id, document_store->Put(DocumentProto(email_document)));
+ EXPECT_THAT(document_store->Get(email_document_id),
+ IsOkAndHolds(EqualsProto(email_document)));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData email_data,
+ document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(email_schema_type_id));
+ email_namespace_id = email_data.namespace_id();
+ email_expiration_timestamp = email_data.expiration_timestamp_secs();
+
+ // Insert and verify a "message" document
+ ICING_ASSERT_OK_AND_ASSIGN(
+ message_document_id,
+ document_store->Put(DocumentProto(message_document)));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData message_data,
+ document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(message_schema_type_id));
+ message_namespace_id = message_data.namespace_id();
+ message_expiration_timestamp = message_data.expiration_timestamp_secs();
+ } // Everything destructs and commits changes to file
+
+ // Change the DocumentStore's header combined checksum so that it won't match
+ // the recalculated checksum on initialization. This will force a regeneration
+ // of derived files from ground truth.
+ const std::string header_file =
+ absl_ports::StrCat(document_store_dir_, "/document_store_header");
+ DocumentStore::Header header;
+ header.magic = DocumentStore::Header::kMagic;
+ header.checksum = 10; // Arbitrary garbage checksum
+ filesystem_.DeleteFile(header_file.c_str());
+ filesystem_.Write(header_file.c_str(), &header, sizeof(header));
+
+ // Change the schema so that we don't know of the Document's type anymore.
+ // Since we can't set backwards incompatible changes, we do some file-level
+ // hacks to "reset" the schema. Without a previously existing schema, the new
+ // schema isn't considered backwards incompatible
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+
+ // Successfully recover from a corrupt derived file issue. We don't fail just
+ // because the "message" schema type is missing
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+
+ // "email" document is fine
+ EXPECT_THAT(document_store->Get(email_document_id),
+ IsOkAndHolds(EqualsProto(email_document)));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData email_data,
+ document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(email_schema_type_id));
+ // Make sure that all the other fields are stll valid/the same
+ EXPECT_THAT(email_data.namespace_id(), Eq(email_namespace_id));
+ EXPECT_THAT(email_data.expiration_timestamp_secs(),
+ Eq(email_expiration_timestamp));
+
+ // "message" document has an invalid SchemaTypeId
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData message_data,
+ document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(-1));
+ // Make sure that all the other fields are stll valid/the same
+ EXPECT_THAT(message_data.namespace_id(), Eq(message_namespace_id));
+ EXPECT_THAT(message_data.expiration_timestamp_secs(),
+ Eq(message_expiration_timestamp));
+}
+
+TEST_F(DocumentStoreTest, UpdateSchemaStoreUpdatesSchemaTypeIds) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId old_email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId old_message_schema_type_id,
+ schema_store->GetSchemaTypeId("message"));
+
+ DocumentProto email_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri")
+ .SetSchema("email")
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("message_uri")
+ .SetSchema("message")
+ .Build();
+
+ // Add the documents and check SchemaTypeIds match
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_document_id,
+ document_store->Put(email_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData email_data,
+ document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(old_email_schema_type_id));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId message_document_id,
+ document_store->Put(message_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData message_data,
+ document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(old_message_schema_type_id));
+
+ // Rearrange the schema types. Since SchemaTypeId is assigned based on order,
+ // this should change the SchemaTypeIds.
+ schema.clear_types();
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+ type_config = schema.add_types();
+ type_config->set_schema_type("email");
+
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId new_email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId new_message_schema_type_id,
+ schema_store->GetSchemaTypeId("message"));
+
+ // SchemaTypeIds should have changed.
+ EXPECT_NE(old_email_schema_type_id, new_email_schema_type_id);
+ EXPECT_NE(old_message_schema_type_id, new_message_schema_type_id);
+
+ ICING_EXPECT_OK(document_store->UpdateSchemaStore(schema_store.get()));
+
+ // Check that the FilterCache holds the new SchemaTypeIds
+ ICING_ASSERT_OK_AND_ASSIGN(
+ email_data, document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(new_email_schema_type_id));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ message_data, document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(new_message_schema_type_id));
+}
+
+TEST_F(DocumentStoreTest, UpdateSchemaStoreDeletesInvalidDocuments) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+
+ auto property_config = type_config->add_properties();
+ property_config->set_property_name("subject");
+ property_config->set_data_type(PropertyConfigProto::DataType::STRING);
+ property_config->set_cardinality(PropertyConfigProto::Cardinality::OPTIONAL);
+ property_config->mutable_indexing_config()->set_term_match_type(
+ TermMatchType::EXACT_ONLY);
+ property_config->mutable_indexing_config()->set_tokenizer_type(
+ IndexingConfig::TokenizerType::PLAIN);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ // Add two documents, with and without a subject
+ DocumentProto email_without_subject = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri_without_subject")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ DocumentProto email_with_subject = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri_with_subject")
+ .SetSchema("email")
+ .AddStringProperty("subject", "foo")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ // Insert documents and check they're ok
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_without_subject_document_id,
+ document_store->Put(email_without_subject));
+ EXPECT_THAT(document_store->Get(email_without_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_without_subject)));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_with_subject_document_id,
+ document_store->Put(email_with_subject));
+ EXPECT_THAT(document_store->Get(email_with_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_with_subject)));
+
+ // Changing an OPTIONAL field to REQUIRED is backwards incompatible, and will
+ // invalidate all documents that don't have this property set
+ schema.mutable_types(0)->mutable_properties(0)->set_cardinality(
+ PropertyConfigProto::Cardinality::REQUIRED);
+
+ ICING_EXPECT_OK(schema_store->SetSchema(
+ schema, /*ignore_errors_and_delete_documents=*/true));
+
+ ICING_EXPECT_OK(document_store->UpdateSchemaStore(schema_store.get()));
+
+ // The email without a subject should be marked as deleted
+ EXPECT_THAT(document_store->Get(email_without_subject_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // The email with a subject should be unaffected
+ EXPECT_THAT(document_store->Get(email_with_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_with_subject)));
+}
+
+TEST_F(DocumentStoreTest,
+ UpdateSchemaStoreDeletesDocumentsByDeletedSchemaType) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ // Add a "email" and "message" document
+ DocumentProto email_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("message_uri")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ // Insert documents and check they're ok
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_document_id,
+ document_store->Put(email_document));
+ EXPECT_THAT(document_store->Get(email_document_id),
+ IsOkAndHolds(EqualsProto(email_document)));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId message_document_id,
+ document_store->Put(message_document));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+
+ SchemaProto new_schema;
+ type_config = new_schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_EXPECT_OK(
+ schema_store->SetSchema(new_schema,
+ /*ignore_errors_and_delete_documents=*/true));
+
+ ICING_EXPECT_OK(document_store->UpdateSchemaStore(schema_store.get()));
+
+ // The "email" type is unknown now, so the "email" document should be deleted
+ EXPECT_THAT(document_store->Get(email_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // The "message" document should be unaffected
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+}
+
+TEST_F(DocumentStoreTest, OptimizedUpdateSchemaStoreUpdatesSchemaTypeIds) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId old_email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId old_message_schema_type_id,
+ schema_store->GetSchemaTypeId("message"));
+
+ DocumentProto email_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri")
+ .SetSchema("email")
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("message_uri")
+ .SetSchema("message")
+ .Build();
+
+ // Add the documents and check SchemaTypeIds match
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_document_id,
+ document_store->Put(email_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData email_data,
+ document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(old_email_schema_type_id));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId message_document_id,
+ document_store->Put(message_document));
+ ICING_ASSERT_OK_AND_ASSIGN(
+ DocumentFilterData message_data,
+ document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(old_message_schema_type_id));
+
+ // Rearrange the schema types. Since SchemaTypeId is assigned based on order,
+ // this should change the SchemaTypeIds.
+ schema.clear_types();
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+ type_config = schema.add_types();
+ type_config->set_schema_type("email");
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaStore::SetSchemaResult set_schema_result,
+ schema_store->SetSchema(schema));
+
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId new_email_schema_type_id,
+ schema_store->GetSchemaTypeId("email"));
+ ICING_ASSERT_OK_AND_ASSIGN(SchemaTypeId new_message_schema_type_id,
+ schema_store->GetSchemaTypeId("message"));
+
+ // SchemaTypeIds should have changed.
+ EXPECT_NE(old_email_schema_type_id, new_email_schema_type_id);
+ EXPECT_NE(old_message_schema_type_id, new_message_schema_type_id);
+
+ ICING_EXPECT_OK(document_store->OptimizedUpdateSchemaStore(
+ schema_store.get(), set_schema_result));
+
+ // Check that the FilterCache holds the new SchemaTypeIds
+ ICING_ASSERT_OK_AND_ASSIGN(
+ email_data, document_store->GetDocumentFilterData(email_document_id));
+ EXPECT_THAT(email_data.schema_type_id(), Eq(new_email_schema_type_id));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ message_data, document_store->GetDocumentFilterData(message_document_id));
+ EXPECT_THAT(message_data.schema_type_id(), Eq(new_message_schema_type_id));
+}
+
+TEST_F(DocumentStoreTest, OptimizedUpdateSchemaStoreDeletesInvalidDocuments) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+
+ auto property_config = type_config->add_properties();
+ property_config->set_property_name("subject");
+ property_config->set_data_type(PropertyConfigProto::DataType::STRING);
+ property_config->set_cardinality(PropertyConfigProto::Cardinality::OPTIONAL);
+ property_config->mutable_indexing_config()->set_term_match_type(
+ TermMatchType::EXACT_ONLY);
+ property_config->mutable_indexing_config()->set_tokenizer_type(
+ IndexingConfig::TokenizerType::PLAIN);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ // Add two documents, with and without a subject
+ DocumentProto email_without_subject = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri_without_subject")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ DocumentProto email_with_subject = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri_with_subject")
+ .SetSchema("email")
+ .AddStringProperty("subject", "foo")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ // Insert documents and check they're ok
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_without_subject_document_id,
+ document_store->Put(email_without_subject));
+ EXPECT_THAT(document_store->Get(email_without_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_without_subject)));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_with_subject_document_id,
+ document_store->Put(email_with_subject));
+ EXPECT_THAT(document_store->Get(email_with_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_with_subject)));
+
+ // Changing an OPTIONAL field to REQUIRED is backwards incompatible, and will
+ // invalidate all documents that don't have this property set
+ schema.mutable_types(0)->mutable_properties(0)->set_cardinality(
+ PropertyConfigProto::Cardinality::REQUIRED);
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ SchemaStore::SetSchemaResult set_schema_result,
+ schema_store->SetSchema(schema,
+ /*ignore_errors_and_delete_documents=*/true));
+
+ ICING_EXPECT_OK(document_store->OptimizedUpdateSchemaStore(
+ schema_store.get(), set_schema_result));
+
+ // The email without a subject should be marked as deleted
+ EXPECT_THAT(document_store->Get(email_without_subject_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // The email with a subject should be unaffected
+ EXPECT_THAT(document_store->Get(email_with_subject_document_id),
+ IsOkAndHolds(EqualsProto(email_with_subject)));
+}
+
+TEST_F(DocumentStoreTest,
+ OptimizedUpdateSchemaStoreDeletesDocumentsByDeletedSchemaType) {
+ const std::string schema_store_dir = test_dir_ + "_custom";
+ filesystem_.DeleteDirectoryRecursively(schema_store_dir.c_str());
+ filesystem_.CreateDirectoryRecursively(schema_store_dir.c_str());
+
+ // Set a schema
+ SchemaProto schema;
+ auto type_config = schema.add_types();
+ type_config->set_schema_type("email");
+ type_config = schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<SchemaStore> schema_store,
+ SchemaStore::Create(&filesystem_, schema_store_dir));
+ ICING_EXPECT_OK(schema_store->SetSchema(schema));
+
+ // Add a "email" and "message" document
+ DocumentProto email_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("email_uri")
+ .SetSchema("email")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ DocumentProto message_document = DocumentBuilder()
+ .SetNamespace("namespace")
+ .SetUri("message_uri")
+ .SetSchema("message")
+ .SetCreationTimestampSecs(0)
+ .Build();
+
+ // Insert documents and check they're ok
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<DocumentStore> document_store,
+ DocumentStore::Create(&filesystem_, document_store_dir_, &fake_clock_,
+ schema_store.get()));
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId email_document_id,
+ document_store->Put(email_document));
+ EXPECT_THAT(document_store->Get(email_document_id),
+ IsOkAndHolds(EqualsProto(email_document)));
+
+ ICING_ASSERT_OK_AND_ASSIGN(DocumentId message_document_id,
+ document_store->Put(message_document));
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+
+ SchemaProto new_schema;
+ type_config = new_schema.add_types();
+ type_config->set_schema_type("message");
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ SchemaStore::SetSchemaResult set_schema_result,
+ schema_store->SetSchema(new_schema,
+ /*ignore_errors_and_delete_documents=*/true));
+
+ ICING_EXPECT_OK(document_store->OptimizedUpdateSchemaStore(
+ schema_store.get(), set_schema_result));
+
+ // The "email" type is unknown now, so the "email" document should be deleted
+ EXPECT_THAT(document_store->Get(email_document_id),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+
+ // The "message" document should be unaffected
+ EXPECT_THAT(document_store->Get(message_document_id),
+ IsOkAndHolds(EqualsProto(message_document)));
+}
+
+} // namespace lib
+} // namespace icing
diff --git a/icing/store/key-mapper.h b/icing/store/key-mapper.h
new file mode 100644
index 0000000..1d2d455
--- /dev/null
+++ b/icing/store/key-mapper.h
@@ -0,0 +1,267 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef ICING_STORE_KEY_MAPPER_H_
+#define ICING_STORE_KEY_MAPPER_H_
+
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <string>
+#include <string_view>
+#include <type_traits>
+
+#include "utils/base/status.h"
+#include "utils/base/statusor.h"
+#include "icing/absl_ports/canonical_errors.h"
+#include "icing/absl_ports/status_macros.h"
+#include "icing/absl_ports/str_cat.h"
+#include "icing/file/filesystem.h"
+#include "icing/legacy/index/icing-dynamic-trie.h"
+#include "icing/legacy/index/icing-filesystem.h"
+#include "icing/util/crc32.h"
+
+namespace icing {
+namespace lib {
+
+// File-backed mapping between the string key and a trivially copyable value
+// type.
+//
+// KeyMapper is thread-compatible
+template <typename T>
+class KeyMapper {
+ public:
+ // Returns an initialized instance of KeyMapper that can immediately handle
+ // read/write operations.
+ // Returns any encountered IO errors.
+ //
+ // base_dir : Base directory used to save all the files required to persist
+ // KeyMapper. If this base_dir was previously used to create a
+ // KeyMapper, then this existing data would be loaded. Otherwise,
+ // an empty KeyMapper would be created.
+ // maximum_size_bytes : The maximum allowable size of the key mapper storage.
+ static libtextclassifier3::StatusOr<std::unique_ptr<KeyMapper<T>>> Create(
+ const Filesystem& filesystem, std::string_view base_dir,
+ int maximum_size_bytes);
+
+ // Deletes all the files associated with the KeyMapper. Returns success or any
+ // encountered IO errors
+ //
+ // base_dir : Base directory used to save all the files required to persist
+ // KeyMapper. Should be the same as passed into Create().
+ static libtextclassifier3::Status Delete(const Filesystem& filesystem,
+ std::string_view base_dir);
+
+ ~KeyMapper() = default;
+
+ // Inserts/Updates value for key.
+ // Returns any encountered IO errors.
+ //
+ // NOTE: Put() doesn't automatically flush changes to disk and relies on
+ // either explicit calls to PersistToDisk() or a clean shutdown of the class.
+ libtextclassifier3::Status Put(std::string_view key, T value);
+
+ // Finds the current value for key and returns it. If key is not present, it
+ // is inserted with next_value and next_value is returned.
+ //
+ // Returns any IO errors that may occur during Put.
+ libtextclassifier3::StatusOr<T> GetOrPut(std::string_view key, T next_value);
+
+ // Returns the value corresponding to the key.
+ //
+ // Returns NOT_FOUND error if the key was missing.
+ // Returns any encountered IO errors.
+ libtextclassifier3::StatusOr<T> Get(std::string_view key) const;
+
+ // Count of unique keys stored in the KeyMapper.
+ int32_t num_keys() const { return trie_.size(); }
+
+ // Syncs all the changes made to the KeyMapper to disk.
+ // Returns any encountered IO errors.
+ //
+ // NOTE: To control disk-churn, Put() doesn't automatically persist every
+ // change to disk. The caller should explicitly call PersistToDisk() to make
+ // sure that the data is durable.
+ libtextclassifier3::Status PersistToDisk();
+
+ // Calculates and returns the disk usage in bytes.
+ //
+ // Returns:
+ // Disk usage on success
+ // INTERNAL_ERROR on IO error
+ libtextclassifier3::StatusOr<int64_t> GetDiskUsage() const;
+
+ // Computes and returns the checksum of the header and contents.
+ Crc32 ComputeChecksum();
+
+ private:
+ static constexpr char kKeyMapperDir[] = "key_mapper_dir";
+ static constexpr char kKeyMapperPrefix[] = "key_mapper";
+
+ // Use KeyMapper::Create() to instantiate.
+ explicit KeyMapper(std::string_view key_mapper_dir);
+
+ // Load any existing KeyMapper data from disk, or creates a new instance
+ // of KeyMapper on disk and gets ready to process read/write operations.
+ //
+ // Returns any encountered IO errors.
+ libtextclassifier3::Status Initialize(int maximum_size_bytes);
+
+ const std::string file_prefix_;
+
+ // TODO(adorokhine) Filesystem is a forked class that's available both in
+ // icing and icing namespaces. We will need icing::Filesystem in order
+ // to use IcingDynamicTrie. Filesystem class should be fully refactored
+ // to have a single definition across both namespaces. Such a class should
+ // use icing (and general google3) coding conventions and behave like
+ // a proper C++ class.
+ const IcingFilesystem icing_filesystem_;
+ IcingDynamicTrie trie_;
+
+ static_assert(std::is_trivially_copyable<T>::value,
+ "T must be trivially copyable");
+};
+
+template <typename T>
+libtextclassifier3::StatusOr<std::unique_ptr<KeyMapper<T>>>
+KeyMapper<T>::Create(const Filesystem& filesystem, std::string_view base_dir,
+ int maximum_size_bytes) {
+ // We create a subdirectory since the trie creates and stores multiple files.
+ // This makes it easier to isolate the trie files away from other files that
+ // could potentially be in the same base_dir, and makes it easier to delete.
+ const std::string key_mapper_dir =
+ absl_ports::StrCat(base_dir, "/", kKeyMapperDir);
+ if (!filesystem.CreateDirectoryRecursively(key_mapper_dir.c_str())) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Failed to create KeyMapper directory: ", key_mapper_dir));
+ }
+ auto mapper = std::unique_ptr<KeyMapper<T>>(new KeyMapper<T>(key_mapper_dir));
+ ICING_RETURN_IF_ERROR(mapper->Initialize(maximum_size_bytes));
+ return mapper;
+}
+
+template <typename T>
+libtextclassifier3::Status KeyMapper<T>::Delete(const Filesystem& filesystem,
+ std::string_view base_dir) {
+ std::string key_mapper_dir = absl_ports::StrCat(base_dir, "/", kKeyMapperDir);
+ if (!filesystem.DeleteDirectoryRecursively(key_mapper_dir.c_str())) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Failed to delete KeyMapper directory: ", key_mapper_dir));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename T>
+KeyMapper<T>::KeyMapper(std::string_view key_mapper_dir)
+ : file_prefix_(absl_ports::StrCat(key_mapper_dir, "/", kKeyMapperPrefix)),
+ trie_(file_prefix_,
+ IcingDynamicTrie::RuntimeOptions().set_storage_policy(
+ IcingDynamicTrie::RuntimeOptions::kMapSharedWithCrc),
+ &icing_filesystem_) {}
+
+template <typename T>
+libtextclassifier3::Status KeyMapper<T>::Initialize(int maximum_size_bytes) {
+ IcingDynamicTrie::Options options;
+ // Divide the max space between the three internal arrays: nodes, nexts and
+ // suffixes. MaxNodes and MaxNexts are in units of their own data structures.
+ // MaxSuffixesSize is in units of bytes.
+ options.max_nodes = maximum_size_bytes / (3 * sizeof(IcingDynamicTrie::Node));
+ options.max_nexts = options.max_nodes;
+ options.max_suffixes_size =
+ sizeof(IcingDynamicTrie::Node) * options.max_nodes;
+ options.value_size = sizeof(T);
+
+ if (!trie_.CreateIfNotExist(options)) {
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Failed to create KeyMapper file: ", file_prefix_));
+ }
+ if (!trie_.Init()) {
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Failed to init KeyMapper file: ", file_prefix_));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename T>
+libtextclassifier3::StatusOr<T> KeyMapper<T>::GetOrPut(std::string_view key,
+ T next_value) {
+ std::string string_key(key);
+ uint32_t value_index;
+ if (!trie_.Insert(string_key.c_str(), &next_value, &value_index,
+ /*replace=*/false)) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Unable to insert key ", key, " into KeyMapper ", file_prefix_, "."));
+ }
+ // This memory address could be unaligned since we're just grabbing the value
+ // from somewhere in the trie's suffix array. The suffix array is filled with
+ // chars, so the address might not be aligned to T values.
+ const T* unaligned_value =
+ static_cast<const T*>(trie_.GetValueAtIndex(value_index));
+
+ // memcpy the value to ensure that the returned value here is in a T-aligned
+ // address
+ T aligned_value;
+ memcpy(&aligned_value, unaligned_value, sizeof(T));
+ return aligned_value;
+}
+
+template <typename T>
+libtextclassifier3::Status KeyMapper<T>::Put(std::string_view key, T value) {
+ std::string string_key(key);
+ if (!trie_.Insert(string_key.c_str(), &value)) {
+ return absl_ports::InternalError(absl_ports::StrCat(
+ "Unable to insert key ", key, " into KeyMapper ", file_prefix_, "."));
+ }
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename T>
+libtextclassifier3::StatusOr<T> KeyMapper<T>::Get(std::string_view key) const {
+ std::string string_key(key);
+ T value;
+ if (!trie_.Find(string_key.c_str(), &value)) {
+ return absl_ports::NotFoundError(absl_ports::StrCat(
+ "Key not found ", key, " in KeyMapper ", file_prefix_, "."));
+ }
+ return value;
+}
+
+template <typename T>
+libtextclassifier3::Status KeyMapper<T>::PersistToDisk() {
+ if (!trie_.Sync()) {
+ return absl_ports::InternalError(
+ absl_ports::StrCat("Failed to sync KeyMapper file: ", file_prefix_));
+ }
+
+ return libtextclassifier3::Status::OK;
+}
+
+template <typename T>
+libtextclassifier3::StatusOr<int64_t> KeyMapper<T>::GetDiskUsage() const {
+ int64_t size = trie_.GetDiskUsage();
+ if (size == IcingFilesystem::kBadFileSize || size < 0) {
+ return absl_ports::InternalError("Failed to get disk usage of key mapper");
+ }
+ return size;
+}
+
+template <typename T>
+Crc32 KeyMapper<T>::ComputeChecksum() {
+ return Crc32(trie_.UpdateCrc());
+}
+
+} // namespace lib
+} // namespace icing
+
+#endif // ICING_STORE_KEY_MAPPER_H_
diff --git a/icing/store/key-mapper_test.cc b/icing/store/key-mapper_test.cc
new file mode 100644
index 0000000..c75c203
--- /dev/null
+++ b/icing/store/key-mapper_test.cc
@@ -0,0 +1,168 @@
+// Copyright (C) 2019 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "icing/store/key-mapper.h"
+
+#include "gmock/gmock.h"
+#include "gtest/gtest.h"
+#include "icing/store/document-id.h"
+#include "icing/testing/common-matchers.h"
+#include "icing/testing/tmp-directory.h"
+
+using ::testing::_;
+using ::testing::HasSubstr;
+
+namespace icing {
+namespace lib {
+namespace {
+constexpr int kMaxKeyMapperSize = 3 * 1024 * 1024; // 3 MiB
+
+class KeyMapperTest : public testing::Test {
+ protected:
+ void SetUp() override { base_dir_ = GetTestTempDir() + "/key_mapper"; }
+
+ void TearDown() override {
+ filesystem_.DeleteDirectoryRecursively(base_dir_.c_str());
+ }
+
+ std::string base_dir_;
+ Filesystem filesystem_;
+};
+
+TEST_F(KeyMapperTest, InvalidBaseDir) {
+ ASSERT_THAT(
+ KeyMapper<DocumentId>::Create(filesystem_, "/dev/null", kMaxKeyMapperSize)
+ .status()
+ .error_message(),
+ HasSubstr("Failed to create KeyMapper"));
+}
+
+TEST_F(KeyMapperTest, NegativeMaxKeyMapperSizeReturnsInternalError) {
+ ASSERT_THAT(KeyMapper<DocumentId>::Create(filesystem_, base_dir_, -1),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(KeyMapperTest, TooLargeMaxKeyMapperSizeReturnsInternalError) {
+ ASSERT_THAT(KeyMapper<DocumentId>::Create(filesystem_, base_dir_,
+ std::numeric_limits<int>::max()),
+ StatusIs(libtextclassifier3::StatusCode::INTERNAL));
+}
+
+TEST_F(KeyMapperTest, CreateNewKeyMapper) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ EXPECT_THAT(key_mapper->num_keys(), 0);
+}
+
+TEST_F(KeyMapperTest, CanUpdateSameKeyMultipleTimes) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 100));
+ ICING_EXPECT_OK(key_mapper->Put("default-youtube.com", 50));
+
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(100));
+
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 200));
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(200));
+ EXPECT_THAT(key_mapper->num_keys(), 2);
+
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 300));
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(300));
+ EXPECT_THAT(key_mapper->num_keys(), 2);
+}
+
+TEST_F(KeyMapperTest, GetOrPutOk) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+
+ EXPECT_THAT(key_mapper->Get("foo"),
+ StatusIs(libtextclassifier3::StatusCode::NOT_FOUND));
+ EXPECT_THAT(key_mapper->GetOrPut("foo", 1), IsOkAndHolds(1));
+ EXPECT_THAT(key_mapper->Get("foo"), IsOkAndHolds(1));
+}
+
+TEST_F(KeyMapperTest, CanPersistToDiskRegularly) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ // Can persist an empty KeyMapper.
+ ICING_EXPECT_OK(key_mapper->PersistToDisk());
+ EXPECT_THAT(key_mapper->num_keys(), 0);
+
+ // Can persist the smallest KeyMapper.
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 100));
+ ICING_EXPECT_OK(key_mapper->PersistToDisk());
+ EXPECT_THAT(key_mapper->num_keys(), 1);
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(100));
+
+ // Can continue to add keys after PersistToDisk().
+ ICING_EXPECT_OK(key_mapper->Put("default-youtube.com", 200));
+ EXPECT_THAT(key_mapper->num_keys(), 2);
+ EXPECT_THAT(key_mapper->Get("default-youtube.com"), IsOkAndHolds(200));
+
+ // Can continue to update the same key after PersistToDisk().
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 300));
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(300));
+ EXPECT_THAT(key_mapper->num_keys(), 2);
+}
+
+TEST_F(KeyMapperTest, CanUseAcrossMultipleInstances) {
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 100));
+ ICING_EXPECT_OK(key_mapper->PersistToDisk());
+
+ key_mapper.reset();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ EXPECT_THAT(key_mapper->num_keys(), 1);
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(100));
+
+ // Can continue to read/write to the KeyMapper.
+ ICING_EXPECT_OK(key_mapper->Put("default-youtube.com", 200));
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 300));
+ EXPECT_THAT(key_mapper->num_keys(), 2);
+ EXPECT_THAT(key_mapper->Get("default-youtube.com"), IsOkAndHolds(200));
+ EXPECT_THAT(key_mapper->Get("default-google.com"), IsOkAndHolds(300));
+}
+
+TEST_F(KeyMapperTest, CanDeleteAndRestartKeyMapping) {
+ // Can delete even if there's nothing there
+ ICING_EXPECT_OK(KeyMapper<DocumentId>::Delete(filesystem_, base_dir_));
+
+ ICING_ASSERT_OK_AND_ASSIGN(
+ std::unique_ptr<KeyMapper<DocumentId>> key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 100));
+ ICING_EXPECT_OK(key_mapper->PersistToDisk());
+ ICING_EXPECT_OK(KeyMapper<DocumentId>::Delete(filesystem_, base_dir_));
+
+ key_mapper.reset();
+ ICING_ASSERT_OK_AND_ASSIGN(
+ key_mapper,
+ KeyMapper<DocumentId>::Create(filesystem_, base_dir_, kMaxKeyMapperSize));
+ EXPECT_THAT(key_mapper->num_keys(), 0);
+ ICING_EXPECT_OK(key_mapper->Put("default-google.com", 100));
+ EXPECT_THAT(key_mapper->num_keys(), 1);
+}
+
+} // namespace
+} // namespace lib
+} // namespace icing