YugabyteDB (2.13.0.0-b42, bfc6a6643e7399ac8a0e81d06a3ee6d6571b33ab)

Coverage Report

Created: 2022-03-09 17:30

/Users/deen/code/yugabyte-db/src/yb/tablet/tablet_metadata.h
Line
Count
Source (jump to first uncovered line)
1
// Licensed to the Apache Software Foundation (ASF) under one
2
// or more contributor license agreements.  See the NOTICE file
3
// distributed with this work for additional information
4
// regarding copyright ownership.  The ASF licenses this file
5
// to you under the Apache License, Version 2.0 (the
6
// "License"); you may not use this file except in compliance
7
// with the License.  You may obtain a copy of the License at
8
//
9
//   http://www.apache.org/licenses/LICENSE-2.0
10
//
11
// Unless required by applicable law or agreed to in writing,
12
// software distributed under the License is distributed on an
13
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14
// KIND, either express or implied.  See the License for the
15
// specific language governing permissions and limitations
16
// under the License.
17
//
18
// The following only applies to changes made to this file as part of YugaByte development.
19
//
20
// Portions Copyright (c) YugaByte, Inc.
21
//
22
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
23
// in compliance with the License.  You may obtain a copy of the License at
24
//
25
// http://www.apache.org/licenses/LICENSE-2.0
26
//
27
// Unless required by applicable law or agreed to in writing, software distributed under the License
28
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
29
// or implied.  See the License for the specific language governing permissions and limitations
30
// under the License.
31
//
32
#ifndef YB_TABLET_TABLET_METADATA_H
33
#define YB_TABLET_TABLET_METADATA_H
34
35
#include <memory>
36
#include <string>
37
#include <unordered_set>
38
#include <vector>
39
40
#include <boost/optional/optional_fwd.hpp>
41
42
#include "yb/common/common_fwd.h"
43
#include "yb/common/constants.h"
44
#include "yb/common/entity_ids.h"
45
#include "yb/common/hybrid_time.h"
46
#include "yb/common/partition.h"
47
#include "yb/common/snapshot.h"
48
49
#include "yb/fs/fs_manager.h"
50
51
#include "yb/gutil/dynamic_annotations.h"
52
#include "yb/gutil/macros.h"
53
#include "yb/gutil/ref_counted.h"
54
55
#include "yb/tablet/tablet_fwd.h"
56
#include "yb/tablet/metadata.pb.h"
57
58
#include "yb/util/status_fwd.h"
59
#include "yb/util/locks.h"
60
#include "yb/util/mutex.h"
61
#include "yb/util/opid.h"
62
#include "yb/util/opid.pb.h"
63
64
namespace yb {
65
namespace tablet {
66
67
extern const int64 kNoDurableMemStore;
68
extern const std::string kIntentsSubdir;
69
extern const std::string kIntentsDBSuffix;
70
extern const std::string kSnapshotsDirSuffix;
71
72
  // Table info.
73
struct TableInfo {
74
  // Table id, name and type.
75
  std::string table_id;
76
  std::string namespace_name;
77
  std::string table_name;
78
  TableType table_type;
79
80
  // The table schema, secondary index map, index info (for index table only) and schema version.
81
  std::unique_ptr<Schema> schema;
82
  std::unique_ptr<IndexMap> index_map;
83
  std::unique_ptr<IndexInfo> index_info;
84
  uint32_t schema_version = 0;
85
86
  // Partition schema of the table.
87
  PartitionSchema partition_schema;
88
89
  // A vector of column IDs that have been deleted, so that the compaction filter can free the
90
  // associated memory. As of 01/2019, deleted column IDs are persisted forever, even if all the
91
  // associated data has been discarded. In the future, we can garbage collect such column IDs to
92
  // make sure this vector doesn't grow too large.
93
  std::vector<DeletedColumn> deleted_cols;
94
95
  // We use the retention time from the primary table.
96
  uint32_t wal_retention_secs = 0;
97
98
  TableInfo();
99
  TableInfo(std::string table_id,
100
            std::string namespace_name,
101
            std::string table_name,
102
            TableType table_type,
103
            const Schema& schema,
104
            const IndexMap& index_map,
105
            const boost::optional<IndexInfo>& index_info,
106
            uint32_t schema_version,
107
            PartitionSchema partition_schema);
108
  TableInfo(const TableInfo& other,
109
            const Schema& schema,
110
            const IndexMap& index_map,
111
            const std::vector<DeletedColumn>& deleted_cols,
112
            uint32_t schema_version);
113
114
  CHECKED_STATUS LoadFromPB(const TableInfoPB& pb);
115
  void ToPB(TableInfoPB* pb) const;
116
117
0
  std::string ToString() const {
118
0
    TableInfoPB pb;
119
0
    ToPB(&pb);
120
0
    return pb.ShortDebugString();
121
0
  }
122
};
123
124
// Describes KV-store. Single KV-store is backed by one or two RocksDB instances, depending on
125
// whether distributed transactions are enabled for the table. KV-store for sys catalog could
126
// contain multiple tables.
127
struct KvStoreInfo {
128
6.32k
  explicit KvStoreInfo(const KvStoreId& kv_store_id_) : kv_store_id(kv_store_id_) {}
129
130
  KvStoreInfo(const KvStoreId& kv_store_id_, const std::string& rocksdb_dir_,
131
              const std::vector<SnapshotScheduleId>& snapshot_schedules_)
132
      : kv_store_id(kv_store_id_),
133
        rocksdb_dir(rocksdb_dir_),
134
88.8k
        snapshot_schedules(snapshot_schedules_.begin(), snapshot_schedules_.end()) {}
135
136
  CHECKED_STATUS LoadFromPB(const KvStoreInfoPB& pb,
137
                            const TableId& primary_table_id,
138
                            bool local_superblock);
139
140
  CHECKED_STATUS LoadTablesFromPB(
141
      const google::protobuf::RepeatedPtrField<TableInfoPB>& pbs, const TableId& primary_table_id);
142
143
  void ToPB(const TableId& primary_table_id, KvStoreInfoPB* pb) const;
144
145
  KvStoreId kv_store_id;
146
147
  // The directory where the regular RocksDB data for this KV-store is stored. For KV-stores having
148
  // tables with distributed transactions enabled an additional RocksDB is created in directory at
149
  // `rocksdb_dir + kIntentsDBSuffix` path.
150
  std::string rocksdb_dir;
151
152
  // Optional inclusive lower bound and exclusive upper bound for keys served by this KV-store.
153
  // See docdb::KeyBounds.
154
  std::string lower_bound_key;
155
  std::string upper_bound_key;
156
157
  // See KvStoreInfoPB field with the same name.
158
  bool has_been_fully_compacted = false;
159
160
  // Map of tables sharing this KV-store indexed by the table id.
161
  // If pieces of the same table live in the same Raft group they should be located in different
162
  // KV-stores.
163
  std::unordered_map<TableId, TableInfoPtr> tables;
164
165
  std::unordered_set<SnapshotScheduleId, SnapshotScheduleIdHash> snapshot_schedules;
166
};
167
168
struct RaftGroupMetadataData {
169
  FsManager* fs_manager;
170
  TableInfoPtr table_info;
171
  RaftGroupId raft_group_id;
172
  Partition partition;
173
  TabletDataState tablet_data_state;
174
  bool colocated = false;
175
  std::vector<SnapshotScheduleId> snapshot_schedules;
176
};
177
178
// At startup, the TSTabletManager will load a RaftGroupMetadata for each
179
// super block found in the tablets/ directory, and then instantiate
180
// Raft groups from this data.
181
class RaftGroupMetadata : public RefCountedThreadSafe<RaftGroupMetadata> {
182
 public:
183
  // Create metadata for a new Raft group. This assumes that the given superblock
184
  // has not been written before, and writes out the initial superblock with
185
  // the provided parameters.
186
  // data_root_dir and wal_root_dir dictates which disk this Raft group will
187
  // use in the respective directories.
188
  // If empty string is passed in, it will be randomly chosen.
189
  static Result<RaftGroupMetadataPtr> CreateNew(
190
      const RaftGroupMetadataData& data, const std::string& data_root_dir = std::string(),
191
      const std::string& wal_root_dir = std::string());
192
193
  // Load existing metadata from disk.
194
  static Result<RaftGroupMetadataPtr> Load(FsManager* fs_manager, const RaftGroupId& raft_group_id);
195
196
  // Try to load an existing Raft group. If it does not exist, create it.
197
  // If it already existed, verifies that the schema of the Raft group matches the
198
  // provided 'schema'.
199
  //
200
  // This is mostly useful for tests which instantiate Raft groups directly.
201
  static Result<RaftGroupMetadataPtr> LoadOrCreate(const RaftGroupMetadataData& data);
202
203
  Result<TableInfoPtr> GetTableInfo(const TableId& table_id) const;
204
  Result<TableInfoPtr> GetTableInfoUnlocked(const TableId& table_id) const;
205
206
4.08M
  const RaftGroupId& raft_group_id() const {
207
4.08M
    DCHECK_NE(state_, kNotLoadedYet);
208
4.08M
    return raft_group_id_;
209
4.08M
  }
210
211
  // Returns the partition of the Raft group.
212
436k
  std::shared_ptr<Partition> partition() const {
213
436k
    DCHECK_NE(state_, kNotLoadedYet);
214
436k
    std::lock_guard<MutexType> lock(data_mutex_);
215
436k
    return partition_;
216
436k
  }
217
218
  // Returns the primary table id. For co-located tables, the primary table is the table this Raft
219
  // group was first created for. For single-tenant table, it is the primary table.
220
235k
  TableId table_id() const {
221
235k
    DCHECK_NE(state_, kNotLoadedYet);
222
235k
    std::lock_guard<MutexType> lock(data_mutex_);
223
235k
    return primary_table_id_;
224
235k
  }
225
226
  // Returns the name, type, schema, index map, schema, etc of the table.
227
  std::string namespace_name(const TableId& table_id = "") const;
228
229
  std::string table_name(const TableId& table_id = "") const;
230
231
  TableType table_type(const TableId& table_id = "") const;
232
233
  yb::SchemaPtr schema(const TableId& table_id = "") const;
234
235
  std::shared_ptr<IndexMap> index_map(const TableId& table_id = "") const;
236
237
  uint32_t schema_version(const TableId& table_id = "") const;
238
239
  const std::string& indexed_table_id(const TableId& table_id = "") const;
240
241
  bool is_local_index(const TableId& table_id = "") const;
242
243
  bool is_unique_index(const TableId& table_id = "") const;
244
245
  std::vector<ColumnId> index_key_column_ids(const TableId& table_id = "") const;
246
247
  // Returns the partition schema of the Raft group's tables.
248
3.21k
  std::shared_ptr<PartitionSchema> partition_schema() const {
249
3.21k
    DCHECK_NE(state_, kNotLoadedYet);
250
3.21k
    const TableInfoPtr table_info = primary_table_info();
251
3.21k
    return std::shared_ptr<PartitionSchema>(table_info, &table_info->partition_schema);
252
3.21k
  }
253
254
  std::shared_ptr<std::vector<DeletedColumn>> deleted_cols(
255
      const TableId& table_id = "") const;
256
257
360k
  const std::string& rocksdb_dir() const { return kv_store_.rocksdb_dir; }
258
47.6k
  std::string intents_rocksdb_dir() const { return kv_store_.rocksdb_dir + kIntentsDBSuffix; }
259
47.6k
  std::string snapshots_dir() const { return kv_store_.rocksdb_dir + kSnapshotsDirSuffix; }
260
261
220k
  const std::string& lower_bound_key() const { return kv_store_.lower_bound_key; }
262
220k
  const std::string& upper_bound_key() const { return kv_store_.upper_bound_key; }
263
264
279k
  const std::string& wal_dir() const { return wal_dir_; }
265
266
  // Set the WAL retention time for the primary table.
267
  void set_wal_retention_secs(uint32 wal_retention_secs);
268
269
  // Returns the wal retention time for the primary table.
270
  uint32_t wal_retention_secs() const;
271
272
  CHECKED_STATUS set_cdc_min_replicated_index(int64 cdc_min_replicated_index);
273
274
  int64_t cdc_min_replicated_index() const;
275
276
  CHECKED_STATUS SetIsUnderTwodcReplicationAndFlush(bool is_under_twodc_replication);
277
278
  bool is_under_twodc_replication() const;
279
280
4.60k
  bool has_been_fully_compacted() const {
281
4.60k
    std::lock_guard<MutexType> lock(data_mutex_);
282
4.60k
    return kv_store_.has_been_fully_compacted;
283
4.60k
  }
284
285
104
  void set_has_been_fully_compacted(const bool& value) {
286
104
    std::lock_guard<MutexType> lock(data_mutex_);
287
104
    kv_store_.has_been_fully_compacted = value;
288
104
  }
289
290
0
  bool AddSnapshotSchedule(const SnapshotScheduleId& schedule_id) {
291
0
    std::lock_guard<MutexType> lock(data_mutex_);
292
0
    return kv_store_.snapshot_schedules.insert(schedule_id).second;
293
0
  }
294
295
0
  bool RemoveSnapshotSchedule(const SnapshotScheduleId& schedule_id) {
296
0
    std::lock_guard<MutexType> lock(data_mutex_);
297
0
    return kv_store_.snapshot_schedules.erase(schedule_id) != 0;
298
0
  }
299
300
412
  std::vector<SnapshotScheduleId> SnapshotSchedules() const {
301
412
    std::lock_guard<MutexType> lock(data_mutex_);
302
412
    return std::vector<SnapshotScheduleId>(
303
412
        kv_store_.snapshot_schedules.begin(), kv_store_.snapshot_schedules.end());
304
412
  }
305
306
  // Returns the data root dir for this Raft group, for example:
307
  // /mnt/d0/yb-data/tserver/data
308
  // TODO(#79): rework when we have more than one KV-store (and data roots) per Raft group.
309
  std::string data_root_dir() const;
310
311
  // Returns the WAL root dir for this Raft group, for example:
312
  // /mnt/d0/yb-data/tserver/wals
313
  std::string wal_root_dir() const;
314
315
  // Set table_id for altering the schema of a colocated user table.
316
  void SetSchema(const Schema& schema,
317
                 const IndexMap& index_map,
318
                 const std::vector<DeletedColumn>& deleted_cols,
319
                 const uint32_t version,
320
                 const TableId& table_id = "");
321
322
  void SetPartitionSchema(const PartitionSchema& partition_schema);
323
324
  void SetTableName(
325
      const std::string& namespace_name, const std::string& table_name,
326
      const TableId& table_id = "");
327
328
  void AddTable(const std::string& table_id,
329
                const std::string& namespace_name,
330
                const std::string& table_name,
331
                const TableType table_type,
332
                const Schema& schema,
333
                const IndexMap& index_map,
334
                const PartitionSchema& partition_schema,
335
                const boost::optional<IndexInfo>& index_info,
336
                const uint32_t schema_version);
337
338
  void RemoveTable(const TableId& table_id);
339
340
  // Set / get the remote bootstrap / tablet data state.
341
  void set_tablet_data_state(TabletDataState state);
342
  TabletDataState tablet_data_state() const;
343
344
  void SetHidden(bool value);
345
  bool hidden() const;
346
347
  void SetRestorationHybridTime(HybridTime value);
348
  HybridTime restoration_hybrid_time() const;
349
350
  CHECKED_STATUS Flush();
351
352
  // Mark the superblock to be in state 'delete_type', sync it to disk, and
353
  // then delete all of the rowsets in this tablet.
354
  // The metadata (superblock) is not deleted. For that, call DeleteSuperBlock().
355
  //
356
  // 'delete_type' must be one of TABLET_DATA_DELETED or TABLET_DATA_TOMBSTONED.
357
  // 'last_logged_opid' should be set to the last opid in the log, if any is known.
358
  // If 'last_logged_opid' is not set, then the current value of
359
  // last_logged_opid is not modified. This is important for roll-forward of
360
  // partially-tombstoned tablets during crash recovery.
361
  //
362
  // Returns only once all data has been removed.
363
  // The OUT parameter 'was_deleted' can be used by caller to determine if the tablet data was
364
  // actually deleted from disk or not. For example, in some cases, the tablet may have been
365
  // already deleted (and are here on a retry) and this operation essentially ends up being a no-op;
366
  // in such a case, 'was_deleted' will be set to FALSE.
367
  CHECKED_STATUS DeleteTabletData(TabletDataState delete_type, const yb::OpId& last_logged_opid);
368
369
  // Return true if this metadata references no regular data DB nor intents DB and is
370
  // already marked as tombstoned. If this is the case, then calling DeleteTabletData
371
  // would be a no-op.
372
  bool IsTombstonedWithNoRocksDBData() const;
373
374
  // Permanently deletes the superblock from the disk.
375
  // DeleteTabletData() must first be called and the tablet data state must be
376
  // TABLET_DATA_DELETED.
377
  // Returns Status::InvalidArgument if the list of orphaned blocks is not empty.
378
  // Returns Status::IllegalState if the tablet data state is not TABLET_DATA_DELETED.
379
  CHECKED_STATUS DeleteSuperBlock();
380
381
1.39M
  FsManager *fs_manager() const { return fs_manager_; }
382
383
  OpId tombstone_last_logged_opid() const;
384
385
  // Loads the currently-flushed superblock from disk into the given protobuf.
386
  CHECKED_STATUS ReadSuperBlockFromDisk(RaftGroupReplicaSuperBlockPB* superblock) const;
387
388
  // Sets *superblock to the serialized form of the current metadata.
389
  void ToSuperBlock(RaftGroupReplicaSuperBlockPB* superblock) const;
390
391
  // Fully replace a superblock (used for bootstrap).
392
  CHECKED_STATUS ReplaceSuperBlock(const RaftGroupReplicaSuperBlockPB &pb);
393
394
  // Returns a new WAL dir path to be used for new Raft group `raft_group_id` which will be created
395
  // as a result of this Raft group splitting.
396
  // Uses the same root dir as for `this` Raft group.
397
  std::string GetSubRaftGroupWalDir(const RaftGroupId& raft_group_id) const;
398
399
  // Returns a new Data dir path to be used for new Raft group `raft_group_id` which will be created
400
  // as a result of this Raft group splitting.
401
  // Uses the same root dir as for `this` Raft group.
402
  std::string GetSubRaftGroupDataDir(const RaftGroupId& raft_group_id) const;
403
404
  // Creates a new Raft group metadata for the part of existing tablet contained in this Raft group.
405
  // Assigns specified Raft group ID, partition and key bounds for a new tablet.
406
  Result<RaftGroupMetadataPtr> CreateSubtabletMetadata(
407
      const RaftGroupId& raft_group_id, const Partition& partition,
408
      const std::string& lower_bound_key, const std::string& upper_bound_key) const;
409
410
30.9M
  TableInfoPtr primary_table_info() const {
411
30.9M
    std::lock_guard<MutexType> lock(data_mutex_);
412
30.9M
    return primary_table_info_unlocked();
413
30.9M
  }
414
415
  bool colocated() const;
416
417
  Result<std::string> TopSnapshotsDir() const;
418
419
  // Return standard "T xxx P yyy" log prefix.
420
  std::string LogPrefix() const;
421
422
  std::array<TabletId, kNumSplitParts> split_child_tablet_ids() const;
423
424
  OpId split_op_id() const;
425
426
  // If this tablet should be deleted, returns op id that should be applied to all replicas,
427
  // before performing such deletion.
428
  OpId GetOpIdToDeleteAfterAllApplied() const;
429
430
  void SetSplitDone(const OpId& op_id, const TabletId& child1, const TabletId& child2);
431
432
  bool has_active_restoration() const;
433
434
  void RegisterRestoration(const TxnSnapshotRestorationId& restoration_id);
435
  void UnregisterRestoration(const TxnSnapshotRestorationId& restoration_id);
436
437
  // Find whether some of active restorations complete. Returns max complete hybrid time of such
438
  // restoration.
439
  HybridTime CheckCompleteRestorations(const RestorationCompleteTimeMap& restoration_complete_time);
440
441
  // Removes all complete or unknown restorations.
442
  bool CleanupRestorations(const RestorationCompleteTimeMap& restoration_complete_time);
443
444
  bool UsePartialRangeKeyIntents() const;
445
446
 private:
447
  typedef simple_spinlock MutexType;
448
449
  friend class RefCountedThreadSafe<RaftGroupMetadata>;
450
  friend class MetadataTest;
451
452
  // Compile time assert that no one deletes RaftGroupMetadata objects.
453
  ~RaftGroupMetadata();
454
455
  // Constructor for creating a new Raft group.
456
  explicit RaftGroupMetadata(
457
      const RaftGroupMetadataData& data, const std::string& data_dir,
458
      const std::string& wal_dir);
459
460
  // Constructor for loading an existing Raft group.
461
  RaftGroupMetadata(FsManager* fs_manager, RaftGroupId raft_group_id);
462
463
  CHECKED_STATUS LoadFromDisk();
464
465
  // Update state of metadata to that of the given superblock PB.
466
  CHECKED_STATUS LoadFromSuperBlock(const RaftGroupReplicaSuperBlockPB& superblock,
467
                                    bool local_superblock);
468
469
  CHECKED_STATUS ReadSuperBlock(RaftGroupReplicaSuperBlockPB *pb);
470
471
  // Fully replace superblock.
472
  // Requires 'flush_lock_'.
473
  CHECKED_STATUS SaveToDiskUnlocked(const RaftGroupReplicaSuperBlockPB &pb);
474
475
  // Requires 'data_mutex_'.
476
  void ToSuperBlockUnlocked(RaftGroupReplicaSuperBlockPB* superblock) const REQUIRES(data_mutex_);
477
478
32.2M
  const TableInfoPtr primary_table_info_unlocked() const {
479
32.2M
    const auto& tables = kv_store_.tables;
480
32.2M
    const auto itr = tables.find(primary_table_id_);
481
32.2M
    CHECK(itr != tables.end());
482
32.2M
    return itr->second;
483
32.2M
  }
484
485
  enum State {
486
    kNotLoadedYet,
487
    kNotWrittenYet,
488
    kInitialized
489
  };
490
  State state_;
491
492
  // Lock protecting the underlying data.
493
  // TODO: consider switching to RW mutex.
494
  mutable MutexType data_mutex_;
495
496
  // Lock protecting flushing the data to disk.
497
  // If taken together with 'data_mutex_', must be acquired first.
498
  mutable Mutex flush_lock_;
499
500
  RaftGroupId raft_group_id_ GUARDED_BY(data_mutex_);
501
  std::shared_ptr<Partition> partition_ GUARDED_BY(data_mutex_);
502
503
  // The primary table id. Primary table is the first table this Raft group is created for.
504
  // Additional tables can be added to this Raft group to co-locate with this table.
505
  TableId primary_table_id_ GUARDED_BY(data_mutex_);
506
507
  // KV-store for this Raft group.
508
  KvStoreInfo kv_store_;
509
510
  FsManager* const fs_manager_;
511
512
  // The directory where the write-ahead log for this Raft group is stored.
513
  std::string wal_dir_ GUARDED_BY(data_mutex_);
514
515
  // The current state of remote bootstrap for the tablet.
516
  TabletDataState tablet_data_state_ GUARDED_BY(data_mutex_) = TABLET_DATA_UNKNOWN;
517
518
  // Record of the last opid logged by the tablet before it was last tombstoned. Has no meaning for
519
  // non-tombstoned tablets.
520
  OpId tombstone_last_logged_opid_ GUARDED_BY(data_mutex_);
521
522
  // True if the raft group is for a colocated tablet.
523
  bool colocated_ GUARDED_BY(data_mutex_) = false;
524
525
  // The minimum index that has been replicated by the cdc service.
526
  int64_t cdc_min_replicated_index_ GUARDED_BY(data_mutex_) = std::numeric_limits<int64_t>::max();
527
528
  bool is_under_twodc_replication_ GUARDED_BY(data_mutex_) = false;
529
530
  bool hidden_ GUARDED_BY(data_mutex_) = false;
531
532
  HybridTime restoration_hybrid_time_ GUARDED_BY(data_mutex_) = HybridTime::kMin;
533
534
  OpId split_op_id_ GUARDED_BY(data_mutex_);
535
  std::array<TabletId, kNumSplitParts> split_child_tablet_ids_ GUARDED_BY(data_mutex_);
536
537
  std::vector<TxnSnapshotRestorationId> active_restorations_;
538
539
  DISALLOW_COPY_AND_ASSIGN(RaftGroupMetadata);
540
};
541
542
CHECKED_STATUS MigrateSuperblock(RaftGroupReplicaSuperBlockPB* superblock);
543
544
// Checks whether tablet data storage is ready for function, i.e. its creation or bootstrap process
545
// has been completed and tablet is not deleted and not in process of being deleted.
546
16.7M
inline bool CanServeTabletData(TabletDataState state) {
547
16.7M
  return state == TabletDataState::TABLET_DATA_READY ||
548
4.26k
         state == TabletDataState::TABLET_DATA_SPLIT_COMPLETED;
549
16.7M
}
550
551
} // namespace tablet
552
} // namespace yb
553
554
#endif /* YB_TABLET_TABLET_METADATA_H */