1 /* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
3 * Copyright 2015 Couchbase, Inc
5 * Licensed under the Apache License, Version 2.0 (the "License");
6 * you may not use this file except in compliance with the License.
7 * You may obtain a copy of the License at
9 * http://www.apache.org/licenses/LICENSE-2.0
11 * Unless required by applicable law or agreed to in writing, software
12 * distributed under the License is distributed on an "AS IS" BASIS,
13 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 * See the License for the specific language governing permissions and
15 * limitations under the License.
22 #include "bloomfilter.h"
23 #include "checkpoint.h"
24 #include "collections/vbucket_manifest.h"
25 #include "dcp/dcp-types.h"
26 #include "hash_table.h"
28 #include "item_pager.h"
30 #include "monotonic.h"
32 #include <platform/non_negative_counter.h>
33 #include <relaxed_atomic.h>
38 class ConflictResolution;
40 class PreLinkDocumentContext;
41 class EventuallyPersistentEngine;
46 * The following will be used to identify
47 * the source of an item's expiration.
49 enum class ExpireBy { Pager, Compactor, Access };
51 /* Structure that holds info needed for notification for an item being updated
54 VBNotifyCtx() : bySeqno(0), notifyReplication(false), notifyFlusher(false) {
56 Monotonic<int64_t> bySeqno;
57 bool notifyReplication;
62 * Structure that holds info needed to queue an item in chkpt or vb backfill
65 struct VBQueueItemCtx {
66 VBQueueItemCtx(GenerateBySeqno genBySeqno,
68 TrackCasDrift trackCasDrift,
70 PreLinkDocumentContext* preLinkDocumentContext_)
71 : genBySeqno(genBySeqno),
73 trackCasDrift(trackCasDrift),
74 isBackfillItem(isBackfillItem),
75 preLinkDocumentContext(preLinkDocumentContext_) {
77 /* Indicates if we should queue an item or not. If this is false other
78 members should not be used */
79 GenerateBySeqno genBySeqno;
81 TrackCasDrift trackCasDrift;
83 PreLinkDocumentContext* preLinkDocumentContext;
87 * Structure that holds seqno based or checkpoint persistence based high
88 * priority requests to a vbucket
90 struct HighPriorityVBEntry {
91 HighPriorityVBEntry(const void* c,
93 HighPriorityVBNotify reqType)
94 : cookie(c), id(idNum), reqType(reqType), start(gethrtime()) {
99 HighPriorityVBNotify reqType;
101 /* for stats (histogram) */
105 typedef std::unique_ptr<Callback<const uint16_t, const VBNotifyCtx&>>
109 * Function object that returns true if the given vbucket is acceptable.
111 class VBucketFilter {
115 * Instiatiate a VBucketFilter that always returns true.
117 explicit VBucketFilter() : acceptable() {}
120 * Instantiate a VBucketFilter that returns true for any of the
123 explicit VBucketFilter(const std::vector<uint16_t> &a) :
124 acceptable(a.begin(), a.end()) {}
126 explicit VBucketFilter(const std::set<uint16_t> &s) : acceptable(s) {}
128 void assign(const std::set<uint16_t> &a) {
132 bool operator ()(uint16_t v) const {
133 return acceptable.empty() || acceptable.find(v) != acceptable.end();
136 size_t size() const { return acceptable.size(); }
138 bool empty() const { return acceptable.empty(); }
145 * Calculate the difference between this and another filter.
146 * If "this" contains elements, [1,2,3,4] and other contains [3,4,5,6]
147 * the returned filter contains: [1,2,5,6]
148 * @param other the other filter to compare with
149 * @return a new filter with the elements present in only one of the two
152 VBucketFilter filter_diff(const VBucketFilter &other) const;
155 * Calculate the intersection between this and another filter.
156 * If "this" contains elements, [1,2,3,4] and other contains [3,4,5,6]
157 * the returned filter contains: [3,4]
158 * @param other the other filter to compare with
159 * @return a new filter with the elements present in both of the two
162 VBucketFilter filter_intersection(const VBucketFilter &other) const;
164 const std::set<uint16_t> &getVBSet() const { return acceptable; }
166 bool addVBucket(uint16_t vbucket) {
167 std::pair<std::set<uint16_t>::iterator, bool> rv = acceptable.insert(vbucket);
171 void removeVBucket(uint16_t vbucket) {
172 acceptable.erase(vbucket);
176 * Dump the filter in a human readable form ( "{ bucket, bucket, bucket }"
177 * to the specified output stream.
179 friend std::ostream& operator<< (std::ostream& out,
180 const VBucketFilter &filter);
184 std::set<uint16_t> acceptable;
187 class EventuallyPersistentEngine;
190 class VBucketMemoryDeletionTask;
193 * An individual vbucket.
195 class VBucket : public std::enable_shared_from_this<VBucket> {
198 // Identifier for a vBucket
199 typedef uint16_t id_type;
202 vbucket_state_t newState,
204 CheckpointConfig& chkConfig,
206 uint64_t lastSnapStart,
207 uint64_t lastSnapEnd,
208 std::unique_ptr<FailoverTable> table,
209 std::shared_ptr<Callback<id_type>> flusherCb,
210 std::unique_ptr<AbstractStoredValueFactory> valFact,
211 NewSeqnoCallback newSeqnoCb,
212 Configuration& config,
213 item_eviction_policy_t evictionPolicy,
214 vbucket_state_t initState = vbucket_state_dead,
215 uint64_t purgeSeqno = 0,
217 const std::string& collectionsManifest = "");
221 int64_t getHighSeqno() const {
222 return checkpointManager.getHighSeqno();
225 size_t getChkMgrMemUsage() {
226 return checkpointManager.getMemoryUsage();
229 size_t getChkMgrMemUsageOfUnrefCheckpoints() {
230 return checkpointManager.getMemoryUsageOfUnrefCheckpoints();
233 uint64_t getPurgeSeqno() const {
237 void setPurgeSeqno(uint64_t to) {
241 void setPersistedSnapshot(uint64_t start, uint64_t end) {
242 LockHolder lh(snapshotMutex);
243 persisted_snapshot_start = start;
244 persisted_snapshot_end = end;
247 snapshot_range_t getPersistedSnapshot() const {
248 LockHolder lh(snapshotMutex);
249 return {persisted_snapshot_start, persisted_snapshot_end};
252 uint64_t getMaxCas() const {
253 return hlc.getMaxHLC();
256 void setMaxCas(uint64_t cas) {
260 void setMaxCasAndTrackDrift(uint64_t cas) {
261 hlc.setMaxHLCAndTrackDrift(cas);
264 void forceMaxCas(uint64_t cas) {
265 hlc.forceMaxHLC(cas);
268 HLC::DriftStats getHLCDriftStats() const {
269 return hlc.getDriftStats();
272 HLC::DriftExceptions getHLCDriftExceptionCounters() const {
273 return hlc.getDriftExceptionCounters();
276 void setHLCDriftAheadThreshold(std::chrono::microseconds threshold) {
277 hlc.setDriftAheadThreshold(threshold);
280 void setHLCDriftBehindThreshold(std::chrono::microseconds threshold) {
281 hlc.setDriftBehindThreshold(threshold);
284 bool isTakeoverBackedUp() {
285 return takeover_backed_up.load();
288 void setTakeoverBackedUpState(bool to) {
290 takeover_backed_up.compare_exchange_strong(inverse, to);
293 // States whether the VBucket is in the process of being created
294 bool isBucketCreation() const {
295 return bucketCreation.load();
298 bool setBucketCreation(bool rv) {
300 return bucketCreation.compare_exchange_strong(inverse, rv);
304 * @return true if the vbucket deletion is to be deferred to a background
307 bool isDeletionDeferred() const {
308 return deferredDeletion.load();
312 * @param value true if the vbucket's deletion should be deferred to a
313 * background task. This is for VBucket objects created by
314 * makeVBucket and owned by a VBucketPtr. If the VBucket was manually
315 * created this will have no effect on deletion.
317 void setDeferredDeletion(bool value) {
318 deferredDeletion.store(true);
322 * @param A cookie to notify when the deferred deletion completes.
324 void setDeferredDeletionCookie(const void* cookie) {
325 deferredDeletionCookie = cookie;
329 * @return the cookie which could of been set when setupDeferredDeletion was
332 const void* getDeferredDeletionCookie() const {
333 return deferredDeletionCookie;
337 * Setup deferred deletion, this is where deletion of the vbucket is
338 * deferred and completed by an AUXIO/NONIO task. AUXIO for EPVBucket
339 * as it will hit disk for the data file unlink, NONIO is used for
340 * EphemeralVBucket as only memory resources need freeing.
341 * @param cookie A cookie to notify when the deletion task completes.
343 virtual void setupDeferredDeletion(const void* cookie) = 0;
345 // Returns the last persisted sequence number for the VBucket
346 virtual uint64_t getPersistenceSeqno() const = 0;
349 * Returns the sequence number to expose publically as the highest
350 * persisted seqno. Note this is may differ from getPersistenceSeqno,
351 * depending on the Bucket type.
353 * Historical note: This is the same as PersistenceSeqno for EP buckets,
354 * and hence before Spock wasn't a separate function; however for Ephemeral
355 * buckets we need to distinguish between what sequence number we report
356 * to external clients for Observe/persistTo, and what sequence number we
357 * report to internal DCP / ns_server for takeover:
358 * a) Clients need 0 for the Ephemeral "persisted to" seqno (as
359 * there isn't any Persistence and we can't claim something is on-disk
361 * b) ns_server / replication needs a non-zero, "logically-persisted" seqno
362 * from the replica to know that a vBucket is ready for takeover.
363 * As such, getPublicPersistenceSeqno() is used for (a), and
364 * getPersistenceSeqno() is used for (b).
366 virtual uint64_t getPublicPersistenceSeqno() const = 0;
368 void setPersistenceSeqno(uint64_t seqno) {
369 persistenceSeqno.store(seqno);
372 id_type getId() const { return id; }
373 vbucket_state_t getState(void) const { return state.load(); }
374 void setState(vbucket_state_t to);
375 cb::RWLock& getStateLock() {return stateLock;}
377 vbucket_state_t getInitialState(void) { return initialState; }
378 void setInitialState(vbucket_state_t initState) {
379 initialState = initState;
382 vbucket_state getVBucketState() const;
385 * This method performs operations on the stored value prior
386 * to expiring the item.
388 * @param v the stored value
390 void handlePreExpiry(StoredValue& v);
392 bool addPendingOp(const void *cookie) {
393 LockHolder lh(pendingOpLock);
394 if (state != vbucket_state_pending) {
395 // State transitioned while we were waiting.
398 // Start a timer when enqueuing the first client.
399 if (pendingOps.empty()) {
400 pendingOpsStart = gethrtime();
402 pendingOps.push_back(cookie);
404 ++stats.pendingOpsTotal;
408 void doStatsForQueueing(const Item& item, size_t itemBytes);
409 void doStatsForFlushing(const Item& item, size_t itemBytes);
410 void incrMetaDataDisk(const Item& qi);
411 void decrMetaDataDisk(const Item& qi);
413 /// Reset all statistics assocated with this vBucket.
414 virtual void resetStats();
416 // Get age sum in millisecond
417 uint64_t getQueueAge() {
418 uint64_t currDirtyQueueAge = dirtyQueueAge.load(
419 std::memory_order_relaxed);
420 rel_time_t currentAge = ep_current_time() * dirtyQueueSize;
421 if (currentAge < currDirtyQueueAge) {
424 return (currentAge - currDirtyQueueAge) * 1000;
427 void fireAllOps(EventuallyPersistentEngine &engine);
430 HashTableDepthStatVisitor v;
435 size_t getBackfillSize() {
436 LockHolder lh(backfill.mutex);
437 return backfill.items.size();
441 * Process an item that is got from a backfill (TAP or DCP).
442 * It puts it onto a queue for persistence and/or generates a seqno and
445 * @param qi item to be processed
446 * @param generateBySeqno indicates if a new seqno must generated or the
447 * seqno in the item must be used
451 virtual void queueBackfillItem(queued_item& qi,
452 const GenerateBySeqno generateBySeqno) = 0;
454 void getBackfillItems(std::vector<queued_item> &items) {
455 LockHolder lh(backfill.mutex);
456 size_t num_items = backfill.items.size();
457 while (!backfill.items.empty()) {
458 items.push_back(backfill.items.front());
459 backfill.items.pop();
461 stats.vbBackfillQueueSize.fetch_sub(num_items);
462 stats.memOverhead->fetch_sub(num_items * sizeof(queued_item));
465 bool isBackfillPhase() {
466 return backfill.isBackfillPhase.load();
469 void setBackfillPhase(bool backfillPhase) {
470 backfill.isBackfillPhase.store(backfillPhase);
474 * Returns the map of bgfetch items for this vbucket, clearing the
477 virtual vb_bgfetch_queue_t getBGFetchItems() = 0;
479 virtual bool hasPendingBGFetchItems() = 0;
481 static const char* toString(vbucket_state_t s) {
483 case vbucket_state_active: return "active"; break;
484 case vbucket_state_replica: return "replica"; break;
485 case vbucket_state_pending: return "pending"; break;
486 case vbucket_state_dead: return "dead"; break;
491 static vbucket_state_t fromString(const char* state) {
492 if (strcmp(state, "active") == 0) {
493 return vbucket_state_active;
494 } else if (strcmp(state, "replica") == 0) {
495 return vbucket_state_replica;
496 } else if (strcmp(state, "pending") == 0) {
497 return vbucket_state_pending;
499 return vbucket_state_dead;
504 * Checks and decides whether to add high priority request on the vbucket.
505 * This is an async request made by modules like ns-server during
506 * rebalance. The request is for a response from the vbucket when it
507 * 'sees' beyond a certain sequence number or when a certain checkpoint
509 * Depending on the vbucket type, the meaning 'seeing' a sequence number
510 * changes. That is, it could mean persisted in case of EPVBucket and
511 * added to the sequenced data structure in case of EphemeralVBucket.
513 * @param seqnoOrChkId seqno to be seen or checkpoint id to be persisted
514 * @param cookie cookie of conn to be notified
515 * @param reqType indicating request for seqno or chk persistence
517 * @return RequestScheduled if a high priority request is added and
518 * notification will be done asynchronously
519 * NotSupported if the request is not supported for the reqType
520 * RequestNotScheduled if a high priority request is NOT added (as
521 * it is not required). This implies there won't
522 * be a subsequent notification
524 virtual HighPriorityVBReqStatus checkAddHighPriorityVBEntry(
525 uint64_t seqnoOrChkId,
527 HighPriorityVBNotify reqType) = 0;
530 * Notify the high priority requests on the vbucket.
531 * This is the response to async requests made by modules like ns-server
534 * @param engine Ref to ep-engine
535 * @param id seqno or checkpoint id causing the notification(s).
536 * @param notifyType indicating notify for seqno or chk persistence
538 virtual void notifyHighPriorityRequests(
539 EventuallyPersistentEngine& engine,
541 HighPriorityVBNotify notifyType) = 0;
543 virtual void notifyAllPendingConnsFailed(EventuallyPersistentEngine& e) = 0;
546 * Get high priority notifications for a seqno or checkpoint persisted
548 * @param engine Ref to ep-engine
549 * @param id seqno or checkpoint id for which notifies are to be found
550 * @param notifyType indicating notify for seqno or chk persistence
552 * @return map of notifications with conn cookie as the key and notify
553 * status as the value
555 std::map<const void*, ENGINE_ERROR_CODE> getHighPriorityNotifications(
556 EventuallyPersistentEngine& engine,
558 HighPriorityVBNotify notifyType);
560 size_t getHighPriorityChkSize() {
561 return numHpVBReqs.load();
565 * BloomFilter operations for vbucket
567 void createFilter(size_t key_count, double probability);
568 void initTempFilter(size_t key_count, double probability);
569 void addToFilter(const DocKey& key);
570 virtual bool maybeKeyExistsInFilter(const DocKey& key);
571 bool isTempFilterAvailable();
572 void addToTempFilter(const DocKey& key);
575 void setFilterStatus(bfilter_status_t to);
576 std::string getFilterStatusString();
577 size_t getFilterSize();
578 size_t getNumOfKeysInFilter();
580 uint64_t nextHLCCas() {
581 return hlc.nextHLC();
584 // Applicable only for FULL EVICTION POLICY
585 bool isResidentRatioUnderThreshold(float threshold);
587 virtual void addStats(bool details, ADD_STAT add_stat, const void* c) = 0;
589 virtual KVShard* getShard() = 0;
592 * Returns the number of alive (non-deleted) Items the VBucket.
594 * Includes items which are not currently resident in memory (i.e. under
595 * Full eviction and have been fully evicted from memory).
596 * Does *not* include deleted items.
598 virtual size_t getNumItems() const = 0;
600 size_t getNumNonResidentItems() const;
602 size_t getNumTempItems(void) {
603 return ht.getNumTempItems();
606 void incrRollbackItemCount(uint64_t val) {
607 rollbackItemCount.fetch_add(val, std::memory_order_relaxed);
610 uint64_t getRollbackItemCount(void) {
611 return rollbackItemCount.load(std::memory_order_relaxed);
614 // Return the persistence checkpoint ID
615 uint64_t getPersistenceCheckpointId() const;
617 // Set the persistence checkpoint ID to the given value.
618 void setPersistenceCheckpointId(uint64_t checkpointId);
620 // Mark the value associated with the given key as dirty
621 void markDirty(const DocKey& key);
624 * Obtain the read handle for the collections manifest.
625 * The caller will have read-only access to manifest using the methods
626 * exposed by the ReadHandle
628 Collections::VB::Manifest::ReadHandle lockCollections() const {
629 return manifest.lock();
633 * Update the Collections::VB::Manifest and the VBucket.
634 * Adds SystemEvents for the create and delete of collections into the
637 * @param m A Collections::Manifest to apply to the VB::Manifest
639 void updateFromManifest(const Collections::Manifest& m) {
640 manifest.wlock().update(*this, m);
644 * Finalise the deletion of a collection (no items remain in the collection)
646 * @param collection "string-view" name of the collection
647 * @param revision The Manifest revision which initiated the delete.
649 void completeDeletion(cb::const_char_buffer collection, uint32_t revision) {
650 manifest.wlock().completeDeletion(*this, collection, revision);
654 * Add a collection to this vbucket with a pre-assigned seqno. I.e.
655 * this VB is a replica.
657 * @param collection collection name to add.
658 * @param revision revision of the collection to add.
659 * @param bySeqno The seqno assigned to the collection create event.
661 void replicaAddCollection(cb::const_char_buffer collection,
664 manifest.wlock().replicaAdd(*this, collection, revision, bySeqno);
668 * Delete a collection from this vbucket with a pre-assigned seqno. I.e.
669 * this VB is a replica.
671 * @param collection collection name to delete.
672 * @param revision revision of the manifest starting the delete.
673 * @param bySeqno The seqno assigned to the collection delete event.
675 void replicaBeginDeleteCollection(cb::const_char_buffer collection,
678 manifest.wlock().replicaBeginDelete(
679 *this, collection, revision, bySeqno);
683 * Delete a collection from this vbucket with a pre-assigned seqno. I.e.
684 * this VB is a replica.
686 * @param separator The new separator.
687 * @param revision The revision which changed the separator.
688 * @param bySeqno The seqno assigned to the change separator event.
690 void replicaChangeCollectionSeparator(cb::const_char_buffer separator,
693 manifest.wlock().replicaChangeSeparator(
694 *this, separator, revision, bySeqno);
698 * Get the collection manifest
700 * @return const reference to the manifest
702 const Collections::VB::Manifest& getManifest() const {
706 static const vbucket_state_t ACTIVE;
707 static const vbucket_state_t REPLICA;
708 static const vbucket_state_t PENDING;
709 static const vbucket_state_t DEAD;
712 CheckpointManager checkpointManager;
714 // Struct for managing 'backfill' items - Items which have been added by
715 // an incoming TAP stream and need to be persisted to disk.
718 std::queue<queued_item> items;
719 std::atomic<bool> isBackfillPhase;
723 * Gets the valid StoredValue for the key and deletes an expired item if
724 * desired by the caller. Requires the hash bucket to be locked
726 * @param hbl Reference to the hash bucket lock
728 * @param wantsDeleted
729 * @param trackReference
730 * @param queueExpired Delete an expired item
732 StoredValue* fetchValidValue(HashTable::HashBucketLock& hbl,
734 WantsDeleted wantsDeleted,
735 TrackReference trackReference,
736 QueueExpired queueExpired);
739 * Complete the background fetch for the specified item. Depending on the
740 * state of the item, restore it to the hashtable as appropriate,
741 * potentially queuing it as dirty.
743 * @param key The key of the item
744 * @param fetched_item The item which has been fetched.
745 * @param startTime The time processing of the batch of items started.
747 * @return ENGINE_ERROR_CODE status notified to be to the front end
749 virtual ENGINE_ERROR_CODE completeBGFetchForSingleItem(
751 const VBucketBGFetchItem& fetched_item,
752 const ProcessClock::time_point startTime) = 0;
755 * Retrieve an item from the disk for vkey stats
757 * @param key the key to fetch
758 * @param cookie the connection cookie
759 * @param eviction_policy The eviction policy
760 * @param engine Reference to ep engine
761 * @param bgFetchDelay
763 * @return VBReturnCtx indicates notifyCtx and operation result
765 virtual ENGINE_ERROR_CODE statsVKey(const DocKey& key,
767 EventuallyPersistentEngine& engine,
768 int bgFetchDelay) = 0;
771 * Complete the vkey stats for an item background fetched from disk.
773 * @param key The key of the item
774 * @param gcb Bgfetch cbk obj containing the item from disk
777 virtual void completeStatsVKey(
778 const DocKey& key, const RememberingCallback<GetValue>& gcb) = 0;
781 * Set (add new or update) an item into in-memory structure like
782 * hash table and do not generate a seqno. This is called internally from
783 * ep-engine when we want to update our in-memory data (like in HT) with
784 * another source of truth like disk.
785 * Currently called during rollback.
787 * @param itm Item to be added or updated. Upon success, the itm
788 * revSeqno are updated
790 * @return Result indicating the status of the operation
792 MutationStatus setFromInternal(Item& itm);
795 * Set (add new or update) an item in the vbucket.
797 * @param itm Item to be added or updated. Upon success, the itm
798 * bySeqno, cas and revSeqno are updated
799 * @param cookie the connection cookie
800 * @param engine Reference to ep engine
801 * @param bgFetchDelay
803 * @return ENGINE_ERROR_CODE status notified to be to the front end
805 ENGINE_ERROR_CODE set(Item& itm,
807 EventuallyPersistentEngine& engine,
811 * Replace (overwrite existing) an item in the vbucket.
813 * @param itm Item to be added or updated. Upon success, the itm
814 * bySeqno, cas and revSeqno are updated
815 * @param cookie the connection cookie
816 * @param engine Reference to ep engine
817 * @param bgFetchDelay
819 * @return ENGINE_ERROR_CODE status notified to be to the front end
821 ENGINE_ERROR_CODE replace(Item& itm,
823 EventuallyPersistentEngine& engine,
827 * Add an item directly into its vbucket rather than putting it on a
828 * checkpoint (backfill the item). The can happen during TAP or when a
829 * replica vbucket is receiving backfill items from active vbucket.
831 * @param itm Item to be added/updated from TAP or DCP backfill. Upon
832 * success, the itm revSeqno is updated
833 * @param genBySeqno whether or not to generate sequence number
835 * @return the result of the operation
837 ENGINE_ERROR_CODE addBackfillItem(Item& itm, GenerateBySeqno genBySeqno);
840 * Set an item in the store from a non-front end operation (DCP, XDCR)
842 * @param item the item to set. Upon success, the itm revSeqno is updated
843 * @param cas value to match
844 * @param seqno sequence number of mutation
845 * @param cookie the cookie representing the client to store the item
846 * @param engine Reference to ep engine
847 * @param bgFetchDelay
848 * @param force override vbucket states
849 * @param allowExisting set to false if you want set to fail if the
850 * item exists already
851 * @param genBySeqno whether or not to generate sequence number
853 * @param isReplication set to true if we are to use replication
856 * @return the result of the store operation
858 ENGINE_ERROR_CODE setWithMeta(Item& itm,
862 EventuallyPersistentEngine& engine,
866 GenerateBySeqno genBySeqno,
871 * Delete an item in the vbucket
873 * @param key key to be deleted
874 * @param[in,out] cas value to match; new cas after logical delete
875 * @param cookie the cookie representing the client to store the item
876 * @param engine Reference to ep engine
877 * @param bgFetchDelay
878 * @param[out] itemMeta pointer to item meta data that needs to be returned
879 * as a result the delete. A NULL pointer indicates
880 * that no meta data needs to be returned.
881 * @param[out] mutInfo Info to uniquely identify (and order) the delete
882 * seq. A NULL pointer indicates no info needs to be
885 * @return the result of the operation
887 ENGINE_ERROR_CODE deleteItem(const DocKey& key,
890 EventuallyPersistentEngine& engine,
892 ItemMetaData* itemMeta,
893 mutation_descr_t* mutInfo);
896 * Delete an item in the vbucket from a non-front end operation (DCP, XDCR)
898 * @param key key to be deleted
899 * @param[in, out] cas value to match; new cas after logical delete
900 * @param[out] seqno Pointer to get the seqno generated for the item. A
901 * NULL value is passed if not needed
902 * @param cookie the cookie representing the client to store the item
903 * @param engine Reference to ep engine
904 * @param bgFetchDelay
905 * @param force force a delete in full eviction mode without doing a
907 * @param itemMeta ref to item meta data
908 * @param backfill indicates if the item must be put onto vb queue or
910 * @param genBySeqno whether or not to generate sequence number
911 * @param generateCas whether or not to generate cas
912 * @param bySeqno seqno of the key being deleted
913 * @param isReplication set to true if we are to use replication
916 * @return the result of the operation
918 ENGINE_ERROR_CODE deleteWithMeta(const DocKey& key,
922 EventuallyPersistentEngine& engine,
925 const ItemMetaData& itemMeta,
927 GenerateBySeqno genBySeqno,
928 GenerateCas generateCas,
933 * Delete an expired item
935 * @param it item to be deleted
936 * @param startTime the time to be compared with this item's expiry time
937 * @param source Expiry source
939 void deleteExpiredItem(const Item& it,
944 * Evict a key from memory.
946 * @param key Key to evict
947 * @param[out] msg Updated to point to a string (with static duration)
948 * describing the result of the operation.
950 * @return SUCCESS if key was successfully evicted (or was already
951 * evicted), or the reason why the request failed.
954 virtual protocol_binary_response_status evictKey(const DocKey& key,
955 const char** msg) = 0;
958 * Page out a StoredValue from memory.
960 * The definition of "page out" is up to the underlying VBucket
961 * implementation - this may mean simply ejecting the value from memory
962 * (Value Eviction), removing the entire document from memory (Full Eviction),
963 * or actually deleting the document (Ephemeral Buckets).
965 * @param lh Bucket lock associated with the StoredValue.
966 * @param v[in, out] Ref to the StoredValue to be ejected. Based on the
967 * VBucket type, policy in the vbucket contents of v and
968 * v itself may be changed
970 * @return true if an item is ejected.
972 virtual bool pageOut(const HashTable::HashBucketLock& lh,
973 StoredValue*& v) = 0;
976 * Add an item in the store
978 * @param itm the item to add. On success, this will have its seqno and
980 * @param cookie the cookie representing the client to store the item
981 * @param engine Reference to ep engine
982 * @param bgFetchDelay
984 * @return the result of the operation
986 ENGINE_ERROR_CODE add(Item& itm,
988 EventuallyPersistentEngine& engine,
992 * Retrieve a value, but update its TTL first
994 * @param key the key to fetch
995 * @param cookie the connection cookie
996 * @param engine Reference to ep engine
997 * @param bgFetchDelay
998 * @param exptime the new expiry time for the object
1000 * @return a GetValue representing the result of the request
1002 GetValue getAndUpdateTtl(const DocKey& key,
1004 EventuallyPersistentEngine& engine,
1008 * Queue an Item to the checkpoint and return its seqno
1010 * @param item an Item object to queue, can be any kind of item and will be
1011 * given a CAS and seqno by this function.
1012 * @param seqno An optional sequence number, if not specified checkpoint
1013 * queueing will assign a seqno to the Item.
1015 int64_t queueItem(Item* item, OptionalSeqno seqno);
1018 * Insert an item into the VBucket during warmup. If we're trying to insert
1019 * a partial item we mark it as nonResident
1021 * @param itm Item to insert. itm is not modified. But cannot be passed as
1022 * const because it is passed to functions that can generally
1023 * modify the itm but do not modify it due to the flags passed.
1024 * @param eject true if we should eject the value immediately
1025 * @param keyMetaDataOnly is this just the key and meta-data or a complete
1028 * @return the result of the operation
1030 MutationStatus insertFromWarmup(Item& itm,
1032 bool keyMetaDataOnly);
1035 * Get metadata and value for a given key
1037 * @param key key for which metadata and value should be retrieved
1038 * @param cookie the cookie representing the client
1039 * @param engine Reference to ep engine
1040 * @param bgFetchDelay
1041 * @param options flags indicating some retrieval related info
1042 * @param diskFlushAll
1044 * @return the result of the operation
1046 GetValue getInternal(const DocKey& key,
1048 EventuallyPersistentEngine& engine,
1050 get_options_t options,
1054 * Retrieve the meta data for given key
1056 * @param key the key to get the meta data for
1057 * @param cookie the connection cookie
1058 * @param engine Reference to ep engine
1059 * @param bgFetchDelay Delay in secs before we run the bgFetch task
1060 * @param[out] metadata meta information returned to the caller
1061 * @param[out] deleted specifies the caller whether or not the key is
1063 * @param[out] datatype specifies the datatype of the item
1065 * @return the result of the operation
1067 ENGINE_ERROR_CODE getMetaData(const DocKey& key,
1069 EventuallyPersistentEngine& engine,
1071 ItemMetaData& metadata,
1076 * Looks up the key stats for the given {vbucket, key}.
1078 * @param key The key to lookup
1079 * @param cookie The client's cookie
1080 * @param engine Reference to ep engine
1081 * @param bgFetchDelay
1082 * @param[out] kstats On success the keystats for this item.
1083 * @param wantsDeleted If yes then return keystats even if the item is
1084 * marked as deleted. If no then will return
1085 * ENGINE_KEY_ENOENT for deleted items.
1087 * @return the result of the operation
1089 ENGINE_ERROR_CODE getKeyStats(const DocKey& key,
1091 EventuallyPersistentEngine& engine,
1093 struct key_stats& kstats,
1094 WantsDeleted wantsDeleted);
1097 * Gets a locked item for a given key.
1099 * @param key The key to lookup
1100 * @param currentTime Current time to use for locking the item for a
1101 * duration of lockTimeout
1102 * @param lockTimeout Timeout for the lock on the item
1103 * @param cookie The client's cookie
1104 * @param engine Reference to ep engine
1105 * @param bgFetchDelay Delay in secs before we run the bgFetch task
1107 * @return the result of the operation (contains locked item on success)
1109 GetValue getLocked(const DocKey& key,
1110 rel_time_t currentTime,
1111 uint32_t lockTimeout,
1113 EventuallyPersistentEngine& engine,
1116 * Update in memory data structures after an item is deleted on disk
1118 * @param queuedItem reference to the deleted item
1119 * @param deleted indicates if item actaully deleted or not (in case item
1120 * did not exist on disk)
1122 void deletedOnDiskCbk(const Item& queuedItem, bool deleted);
1125 * Update in memory data structures after a rollback on disk
1127 * @param queuedItem item key
1129 * @return indicates if the operation is succcessful
1131 bool deleteKey(const DocKey& key);
1134 * Creates a DCP backfill object
1136 * @param e ref to EventuallyPersistentEngine
1137 * @param stream ref to the stream for which this backfill obj is created
1138 * @param startSeqno requested start sequence number of the backfill
1139 * @param endSeqno requested end sequence number of the backfill
1141 * @return pointer to the backfill object created. Caller to own this
1142 * object and hence must handle deletion.
1144 virtual std::unique_ptr<DCPBackfill> createDCPBackfill(
1145 EventuallyPersistentEngine& e,
1146 const active_stream_t& stream,
1147 uint64_t startSeqno,
1148 uint64_t endSeqno) = 0;
1151 * Update failovers, checkpoint mgr and other vBucket members after
1154 * @param rollbackResult contains high seqno of the vBucket after rollback,
1155 * snapshot start seqno of the last snapshot in the
1156 * vBucket after the rollback,
1157 * snapshot end seqno of the last snapshot in the
1158 * vBucket after the rollback
1159 * @param prevHighSeqno high seqno before the rollback
1161 void postProcessRollback(const RollbackResult& rollbackResult,
1162 uint64_t prevHighSeqno);
1165 * Debug - print a textual description of the VBucket to stderr.
1167 virtual void dump() const;
1170 * Returns the number of deletes in the memory
1172 * @return number of deletes
1174 size_t getNumInMemoryDeletes() const {
1175 /* couchbase vbuckets: this is generally (after deletes are persisted)
1176 zero as hash table doesn't keep deletes after
1178 ephemeral vbuckets: we keep deletes in both hash table and ordered
1180 return ht.getNumDeletedItems();
1183 static size_t getCheckpointFlushTimeout();
1185 std::queue<queued_item> rejectQueue;
1186 std::unique_ptr<FailoverTable> failovers;
1188 std::atomic<size_t> opsCreate;
1189 std::atomic<size_t> opsUpdate;
1190 std::atomic<size_t> opsDelete;
1191 std::atomic<size_t> opsReject;
1193 cb::NonNegativeCounter<size_t> dirtyQueueSize;
1194 std::atomic<size_t> dirtyQueueMem;
1195 std::atomic<size_t> dirtyQueueFill;
1196 std::atomic<size_t> dirtyQueueDrain;
1197 std::atomic<uint64_t> dirtyQueueAge;
1198 std::atomic<size_t> dirtyQueuePendingWrites;
1199 std::atomic<size_t> metaDataDisk;
1201 std::atomic<size_t> numExpiredItems;
1204 * A custom delete function for deleting VBucket objects. Any thread could
1205 * be the last thread to release a VBucketPtr and deleting a VB will
1206 * eventually hit the I/O sub-system when we unlink the file, to be sure no
1207 * front-end thread does this work, we schedule the deletion to a background
1208 * task. This task scheduling is triggered by the shared_ptr/VBucketPtr
1209 * using this object as the deleter.
1211 struct DeferredDeleter {
1212 DeferredDeleter(EventuallyPersistentEngine& engine) : engine(engine) {
1216 * Called when the VBucketPtr has no more owners and runs delete on
1219 void operator()(VBucket* vb) const;
1221 EventuallyPersistentEngine& engine;
1226 * This function checks for the various states of the value & depending on
1227 * which the calling function can issue a bgfetch as needed.
1229 std::pair<MutationStatus, GetValue> processGetAndUpdateTtl(
1230 HashTable::HashBucketLock& hbl,
1235 * This function checks cas, expiry and other partition (vbucket) related
1236 * rules before setting an item into other in-memory structure like HT,
1237 * and checkpoint mgr. This function assumes that HT bucket lock is grabbed.
1239 * @param hbl Hash table bucket lock that must be held
1240 * @param v Reference to the ptr of StoredValue. This can be changed if a
1241 * new StoredValue is added or just its contents is changed if the
1242 * exisiting StoredValue is updated
1243 * @param itm Item to be added/updated. On success, its revSeqno is updated
1244 * @param cas value to match
1245 * @param allowExisting set to false if you want set to fail if the
1246 * item exists already
1247 * @param hasMetaData
1248 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1249 * backfill queue; NULL if item need not be queued
1250 * @param maybeKeyExists true if bloom filter predicts that key may exist
1251 * @param isReplication true if issued by consumer (for replication)
1253 * @return Result indicating the status of the operation and notification
1256 std::pair<MutationStatus, VBNotifyCtx> processSet(
1257 const HashTable::HashBucketLock& hbl,
1263 const VBQueueItemCtx* queueItmCtx = nullptr,
1264 bool maybeKeyExists = true,
1265 bool isReplication = false);
1268 * This function checks cas, expiry and other partition (vbucket) related
1269 * rules before adding an item into other in-memory structure like HT,
1270 * and checkpoint mgr. This function assumes that HT bucket lock is grabbed.
1272 * @param hbl Hash table bucket lock that must be held
1273 * @param v[in, out] the stored value to do this operation on
1274 * @param itm Item to be added/updated. On success, its revSeqno is updated
1275 * @param isReplication true if issued by consumer (for replication)
1276 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1277 * backfill queue; NULL if item need not be queued
1279 * @return Result indicating the status of the operation and notification
1282 std::pair<AddStatus, VBNotifyCtx> processAdd(
1283 const HashTable::HashBucketLock& hbl,
1286 bool maybeKeyExists,
1288 const VBQueueItemCtx* queueItmCtx = nullptr);
1291 * This function checks cas, eviction policy and other partition
1292 * (vbucket) related rules before logically (soft) deleting an item in
1293 * in-memory structure like HT, and checkpoint mgr.
1294 * Assumes that HT bucket lock is grabbed.
1296 * @param hbl Hash table bucket lock that must be held
1297 * @param v Reference to the StoredValue to be soft deleted
1298 * @param cas the expected CAS of the item (or 0 to override)
1299 * @param metadata ref to item meta data
1300 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1302 * @param use_meta Indicates if v must be updated with the metadata
1303 * @param bySeqno seqno of the key being deleted
1305 * @return pointer to the updated StoredValue. It can be same as that of
1306 * v or different value if a new StoredValue is created for the
1308 * status of the operation.
1309 * notification info.
1311 std::tuple<MutationStatus, StoredValue*, VBNotifyCtx> processSoftDelete(
1312 const HashTable::HashBucketLock& hbl,
1315 const ItemMetaData& metadata,
1316 const VBQueueItemCtx& queueItmCtx,
1321 * Delete a key (associated StoredValue) from ALL in-memory data structures
1323 * Assumes that HT bucket lock is grabbed.
1325 * Currently StoredValues form HashTable intrusively. That is, HashTable
1326 * does not store a reference or a copy of the StoredValue. If any other
1327 * in-memory data strucutures are formed intrusively using StoredValues,
1328 * then it must be decided in this function which data structure deletes
1329 * the StoredValue. Currently it is HashTable that deleted the StoredValue
1331 * @param hbl Hash table bucket lock that must be held
1332 * @param v Reference to the StoredValue to be deleted
1334 * @return true if an object was deleted, false otherwise
1336 bool deleteStoredValue(const HashTable::HashBucketLock& hbl,
1340 * Queue an item for persistence and replication. Maybe track CAS drift
1342 * The caller of this function must hold the lock of the hash table
1343 * partition that contains the StoredValue being Queued.
1345 * @param v the dirty item. The cas and seqno maybe updated based on the
1347 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1348 * backfill queue, whether to track cas, generate seqno,
1351 * @return Notification context containing info needed to notify the
1352 * clients (like connections, flusher)
1354 VBNotifyCtx queueDirty(StoredValue& v, const VBQueueItemCtx& queueItmCtx);
1357 * Queue an item for persistence and replication
1359 * The caller of this function must hold the lock of the hash table
1360 * partition that contains the StoredValue being Queued.
1362 * @param v the dirty item. The cas and seqno maybe updated based on the
1364 * @param generateBySeqno request that the seqno is generated by this call
1365 * @param generateCas request that the CAS is generated by this call
1366 * @param isBackfillItem indicates if the item must be put onto vb queue or
1368 * @param preLinkDocumentContext context object which allows running the
1369 * document pre link callback after the cas is assinged (but
1370 * but document not available for anyone)
1372 * @return Notification context containing info needed to notify the
1373 * clients (like connections, flusher)
1375 VBNotifyCtx queueDirty(
1377 GenerateBySeqno generateBySeqno = GenerateBySeqno::Yes,
1378 GenerateCas generateCas = GenerateCas::Yes,
1379 bool isBackfillItem = false,
1380 PreLinkDocumentContext* preLinkDocumentContext = nullptr);
1383 * Adds a temporary StoredValue in in-memory data structures like HT.
1384 * Assumes that HT bucket lock is grabbed.
1386 * @param hbl Hash table bucket lock that must be held
1387 * @param key the key for which a temporary item needs to be added
1388 * @param isReplication true if issued by consumer (for replication)
1390 * @return Result indicating the status of the operation
1392 AddStatus addTempStoredValue(const HashTable::HashBucketLock& hbl,
1394 bool isReplication = false);
1397 * Internal wrapper function around the callback to be called when a new
1398 * seqno is generated in the vbucket
1400 * @param notifyCtx holds info needed for notification
1402 void notifyNewSeqno(const VBNotifyCtx& notifyCtx);
1405 * VBucket internal function to store high priority requests on the vbucket.
1407 * @param seqnoOrChkId seqno to be seen or checkpoint id to be persisted
1408 * @param cookie cookie of conn to be notified
1409 * @param reqType request type indicating seqno or chk persistence
1411 void addHighPriorityVBEntry(uint64_t seqnoOrChkId,
1413 HighPriorityVBNotify reqType);
1416 * Get all high priority notifications as temporary failures because they
1417 * could not be completed.
1419 * @param engine Ref to ep-engine
1421 * @return map of notifies with conn cookie as the key and notify status as
1424 std::map<const void*, ENGINE_ERROR_CODE> tmpFailAndGetAllHpNotifies(
1425 EventuallyPersistentEngine& engine);
1427 void _addStats(bool details, ADD_STAT add_stat, const void* c);
1429 template <typename T>
1430 void addStat(const char* nm,
1435 /* This member holds the eviction policy used */
1436 const item_eviction_policy_t eviction;
1438 /* Reference to global (EP engine wide) stats */
1441 /* last seqno that is persisted on the disk */
1442 std::atomic<uint64_t> persistenceSeqno;
1444 /* holds all high priority async requests to the vbucket */
1445 std::list<HighPriorityVBEntry> hpVBReqs;
1447 /* synchronizes access to hpVBReqs */
1448 std::mutex hpVBReqsMutex;
1450 /* size of list hpVBReqs (to avoid MB-9434) */
1451 Couchbase::RelaxedAtomic<size_t> numHpVBReqs;
1454 * VBucket sub-classes must implement a function that will schedule
1455 * an appropriate task that will delete the VBucket and its resources.
1457 * @param engine owning engine (required for task construction)
1459 virtual void scheduleDeferredDeletion(
1460 EventuallyPersistentEngine& engine) = 0;
1463 void fireAllOps(EventuallyPersistentEngine& engine, ENGINE_ERROR_CODE code);
1465 void decrDirtyQueueMem(size_t decrementBy);
1467 void decrDirtyQueueAge(uint32_t decrementBy);
1469 void decrDirtyQueuePendingWrites(size_t decrementBy);
1472 * Updates an existing StoredValue in in-memory data structures like HT.
1473 * Assumes that HT bucket lock is grabbed.
1475 * @param htLock Hash table lock that must be held
1476 * @param v Reference to the StoredValue to be updated.
1477 * @param itm Item to be updated.
1478 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1479 * backfill queue; NULL if item need not be queued
1480 * @param justTouch To note that this object is an existing item with
1481 * the same value but with few flags changed.
1482 * @return pointer to the updated StoredValue. It can be same as that of
1483 * v or different value if a new StoredValue is created for the
1485 * status of the operation.
1486 * notification info.
1488 virtual std::tuple<StoredValue*, MutationStatus, VBNotifyCtx>
1489 updateStoredValue(const HashTable::HashBucketLock& hbl,
1492 const VBQueueItemCtx* queueItmCtx,
1493 bool justTouch = false) = 0;
1496 * Adds a new StoredValue in in-memory data structures like HT.
1497 * Assumes that HT bucket lock is grabbed.
1499 * @param hbl Hash table bucket lock that must be held
1500 * @param itm Item to be added.
1501 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1502 * backfill queue; NULL if item need not be queued
1504 * @return Ptr of the StoredValue added and notification info
1506 virtual std::pair<StoredValue*, VBNotifyCtx> addNewStoredValue(
1507 const HashTable::HashBucketLock& hbl,
1509 const VBQueueItemCtx* queueItmCtx) = 0;
1512 * Logically (soft) delete item in all in-memory data structures. Also
1513 * updates revSeqno. Depending on the in-memory data structure the item may
1514 * be marked delete and/or reset and/or a new value (marked as deleted)
1516 * Assumes that HT bucket lock is grabbed.
1517 * Also assumes that v is in the hash table.
1519 * @param hbl Hash table bucket lock that must be held
1520 * @param v Reference to the StoredValue to be soft deleted
1521 * @param onlyMarkDeleted indicates if we must reset the StoredValue or
1523 * @param queueItmCtx holds info needed to queue an item in chkpt or vb
1525 * @param bySeqno seqno of the key being deleted
1527 * @return pointer to the updated StoredValue. It can be same as that of
1528 * v or different value if a new StoredValue is created for the
1530 * notification info.
1532 virtual std::tuple<StoredValue*, VBNotifyCtx> softDeleteStoredValue(
1533 const HashTable::HashBucketLock& hbl,
1535 bool onlyMarkDeleted,
1536 const VBQueueItemCtx& queueItmCtx,
1537 uint64_t bySeqno) = 0;
1540 * This function handles expiry relatead stuff before logically (soft)
1541 * deleting an item in in-memory structures like HT, and checkpoint mgr.
1542 * Assumes that HT bucket lock is grabbed.
1544 * @param hbl Hash table bucket lock that must be held
1545 * @param v Reference to the StoredValue to be soft deleted
1547 * @return status of the operation.
1548 * pointer to the updated StoredValue. It can be same as that of
1549 * v or different value if a new StoredValue is created for the
1551 * notification info.
1553 std::tuple<MutationStatus, StoredValue*, VBNotifyCtx> processExpiredItem(
1554 const HashTable::HashBucketLock& hbl, StoredValue& v);
1557 * Add a temporary item in hash table and enqueue a background fetch for a
1560 * @param hbl Reference to the hash table bucket lock
1561 * @param key the key to be bg fetched
1562 * @param cookie the cookie of the requestor
1563 * @param engine Reference to ep engine
1564 * @param bgFetchDelay Delay in secs before we run the bgFetch task
1565 * @param metadataOnly whether the fetch is for a non-resident value or
1566 * metadata of a (possibly) deleted item
1567 * @param isReplication indicates if the call is for a replica vbucket
1569 * @return ENGINE_ERROR_CODE status notified to be to the front end
1571 virtual ENGINE_ERROR_CODE addTempItemAndBGFetch(
1572 HashTable::HashBucketLock& hbl,
1575 EventuallyPersistentEngine& engine,
1578 bool isReplication = false) = 0;
1581 * Enqueue a background fetch for a key.
1583 * @param key the key to be bg fetched
1584 * @param cookie the cookie of the requestor
1585 * @param engine Reference to ep engine
1586 * @param bgFetchDelay Delay in secs before we run the bgFetch task
1587 * @param isMeta whether the fetch is for a non-resident value or metadata
1588 * of a (possibly) deleted item
1590 virtual void bgFetch(const DocKey& key,
1592 EventuallyPersistentEngine& engine,
1594 bool isMeta = false) = 0;
1597 * Get metadata and value for a non-resident key
1599 * @param key key for which metadata and value should be retrieved
1600 * @param cookie the cookie representing the client
1601 * @param engine Reference to ep engine
1602 * @param bgFetchDelay Delay in secs before we run the bgFetch task
1603 * @param options flags indicating some retrieval related info
1604 * @param v reference to the stored value of the non-resident key
1606 * @return the result of the operation
1608 virtual GetValue getInternalNonResident(const DocKey& key,
1610 EventuallyPersistentEngine& engine,
1612 get_options_t options,
1613 const StoredValue& v) = 0;
1616 * Update the revision seqno of a newly StoredValue item.
1617 * We must ensure that it is greater the maxDeletedRevSeqno
1619 * @param v StoredValue added newly. Its revSeqno is updated
1621 void updateRevSeqNoOfNewStoredValue(StoredValue& v);
1624 * Increase the expiration count global stats and in the vbucket stats
1626 void incExpirationStat(ExpireBy source);
1628 void adjustCheckpointFlushTimeout(size_t wall_time);
1631 * Given a StoredValue with XATTRs - prune the user keys so only system keys
1634 * @param v StoredValue with XATTR value
1635 * @param itemMeta New ItemMetaData to use in item creation
1636 * @return unique_ptr<Item> which matches the StoredValue's meta-data and
1637 * has the XATTR value with only the system-keys. If the pruning
1638 * removed all keys (because no system-keys exist) an empty
1639 * unique_ptr is returned.
1641 std::unique_ptr<Item> pruneXattrDocument(StoredValue& v,
1642 const ItemMetaData& itemMeta);
1645 std::atomic<vbucket_state_t> state;
1646 cb::RWLock stateLock;
1647 vbucket_state_t initialState;
1648 std::mutex pendingOpLock;
1649 std::vector<const void*> pendingOps;
1650 hrtime_t pendingOpsStart;
1651 uint64_t purge_seqno;
1652 std::atomic<bool> takeover_backed_up;
1654 /* snapshotMutex is used to update/read the pair {start, end} atomically,
1655 but not if reading a single field. */
1656 mutable std::mutex snapshotMutex;
1657 uint64_t persisted_snapshot_start;
1658 uint64_t persisted_snapshot_end;
1661 std::unique_ptr<BloomFilter> bFilter;
1662 std::unique_ptr<BloomFilter> tempFilter; // Used during compaction.
1664 std::atomic<uint64_t> rollbackItemCount;
1667 std::string statPrefix;
1668 // The persistence checkpoint ID for this vbucket.
1669 std::atomic<uint64_t> persistenceCheckpointId;
1670 // Flag to indicate the vbucket is being created
1671 std::atomic<bool> bucketCreation;
1672 // Flag to indicate the vbucket deletion is deferred
1673 std::atomic<bool> deferredDeletion;
1674 /// A cookie that can be set when the vbucket is deletion is deferred, the
1675 /// cookie will be notified when the deferred deletion completes
1676 const void* deferredDeletionCookie;
1678 // Ptr to the item conflict resolution module
1679 std::unique_ptr<ConflictResolution> conflictResolver;
1681 // A callback to be called when a new seqno is generated in the vbucket as
1682 // a result of a front end call
1683 NewSeqnoCallback newSeqnoCb;
1685 /// The VBucket collection state
1686 Collections::VB::Manifest manifest;
1688 static std::atomic<size_t> chkFlushTimeout;
1690 friend class VBucketTest;
1692 DISALLOW_COPY_AND_ASSIGN(VBucket);
1695 using VBucketPtr = std::shared_ptr<VBucket>;