diff options
author | Ilnaz Nizametdinov <i.nizametdinov@gmail.com> | 2022-02-08 16:43:13 +0300 |
---|---|---|
committer | Ilnaz Nizametdinov <i.nizametdinov@gmail.com> | 2022-02-08 16:43:13 +0300 |
commit | 60312eaed2e51caa7ed482270d3545a9d27cb284 (patch) | |
tree | 4f1ac3a703f6cd5cca85d7cb6e685c9b2504cca2 | |
parent | dbe30c72d28c228273a89cef849419be41f11161 (diff) | |
download | ydb-60312eaed2e51caa7ed482270d3545a9d27cb284.tar.gz |
Schema snapshots KIKIMR-14255
ref:a69f168261946ccb0d81518c0b1a54ee8d2bc888
-rw-r--r-- | ydb/core/tx/datashard/create_cdc_stream_unit.cpp | 15 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard.cpp | 1 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard__init.cpp | 8 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard_impl.h | 24 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard_schema_snapshots.cpp | 143 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard_schema_snapshots.h | 50 | ||||
-rw-r--r-- | ydb/core/tx/datashard/datashard_snapshots.h | 43 | ||||
-rw-r--r-- | ydb/core/tx/datashard/snapshot_key.h | 101 | ||||
-rw-r--r-- | ydb/core/tx/datashard/ya.make | 1 | ||||
-rw-r--r-- | ydb/tests/functional/scheme_tests/canondata/tablet_scheme_tests.TestTabletSchemes.test_tablet_schemes_flat_datashard_/flat_datashard.schema | 67 |
10 files changed, 407 insertions, 46 deletions
diff --git a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp index c7061f31d2..8f577f03b4 100644 --- a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp +++ b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp @@ -38,13 +38,26 @@ public: const auto streamPathId = TPathId(streamDesc.GetPathId().GetOwnerId(), streamDesc.GetPathId().GetLocalId()); Y_VERIFY(streamPathId.OwnerId == DataShard.GetPathOwnerId()); - auto tableInfo = DataShard.AlterTableAddCdcStream(ctx, txc, pathId, params.GetTableSchemaVersion(), streamDesc); + const auto version = params.GetTableSchemaVersion(); + Y_VERIFY(version); + + auto tableInfo = DataShard.AlterTableAddCdcStream(ctx, txc, pathId, version, streamDesc); DataShard.AddUserTable(pathId, tableInfo); AddSender.Reset(new TEvChangeExchange::TEvAddSender( pathId, TEvChangeExchange::ESenderType::CdcStream, streamPathId )); + /* TODO(ilnaz) + if (streamDesc.GetFormat() == NKikimrSchemeOp::ECdcStreamFormatJson) { + auto& manager = DataShard.GetSchemaSnapshotManager(); + + const auto key = TSchemaSnapshotKey(pathId.OwnerId, pathId.LocalPathId, version); + manager.AddSnapshot(txc.DB, key, TSchemaSnapshot(tableInfo, op->GetStep(), op->GetTxId())); + manager.AcquireReference(key); + } + */ + BuildResult(op, NKikimrTxDataShard::TEvProposeTransactionResult::COMPLETE); op->Result()->SetStepOrderId(op->GetStepOrder().ToPair()); diff --git a/ydb/core/tx/datashard/datashard.cpp b/ydb/core/tx/datashard/datashard.cpp index 3a0de1dc81..297c966a54 100644 --- a/ydb/core/tx/datashard/datashard.cpp +++ b/ydb/core/tx/datashard/datashard.cpp @@ -123,6 +123,7 @@ TDataShard::TDataShard(const TActorId &tablet, TTabletStorageInfo *info) , Pipeline(this) , SysLocks(this) , SnapshotManager(this) + , SchemaSnapshotManager(this) , DisableByKeyFilter(0, 0, 1) , MaxTxInFly(15000, 0, 100000) , MaxTxLagMilliseconds(5*60*1000, 0, 30*24*3600*1000ll) diff --git a/ydb/core/tx/datashard/datashard__init.cpp b/ydb/core/tx/datashard/datashard__init.cpp index 7c46a7d1a0..f5592073d1 100644 --- a/ydb/core/tx/datashard/datashard__init.cpp +++ b/ydb/core/tx/datashard/datashard__init.cpp @@ -25,6 +25,7 @@ bool TDataShard::TTxInit::Execute(TTransactionContext& txc, const TActorContext& Self->LastChangeRecordGroup = 1; Self->TransQueue.Reset(); Self->SnapshotManager.Reset(); + Self->SchemaSnapshotManager.Reset(); Self->S3Uploads.Reset(); Self->S3Downloads.Reset(); @@ -162,6 +163,7 @@ bool TDataShard::TTxInit::ReadEverything(TTransactionContext &txc) { PRECHARGE_SYS_TABLE(Schema::ReplicationSourceOffsets); PRECHARGE_SYS_TABLE(Schema::DstReplicationSourceOffsetsReceived); PRECHARGE_SYS_TABLE(Schema::UserTablesStats); + PRECHARGE_SYS_TABLE(Schema::SchemaSnapshots); if (!ready) return false; @@ -470,6 +472,12 @@ bool TDataShard::TTxInit::ReadEverything(TTransactionContext &txc) { } } + if (Self->State != TShardState::Offline && txc.DB.GetScheme().GetTableInfo(Schema::SchemaSnapshots::TableId)) { + if (!Self->SchemaSnapshotManager.Load(db)) { + return false; + } + } + return true; } diff --git a/ydb/core/tx/datashard/datashard_impl.h b/ydb/core/tx/datashard/datashard_impl.h index ba574ae5cb..3b76663036 100644 --- a/ydb/core/tx/datashard/datashard_impl.h +++ b/ydb/core/tx/datashard/datashard_impl.h @@ -5,6 +5,7 @@ #include "datashard_trans_queue.h" #include "datashard_outreadset.h" #include "datashard_pipeline.h" +#include "datashard_schema_snapshots.h" #include "datashard_snapshots.h" #include "datashard_s3_downloads.h" #include "datashard_s3_uploads.h" @@ -239,6 +240,7 @@ class TDataShard friend class TEngineBay; friend class NMiniKQL::TKqpScanComputeContext; friend class TSnapshotManager; + friend class TSchemaSnapshotManager; friend class TReplicationSourceOffsetsClient; friend class TReplicationSourceOffsetsServer; @@ -589,8 +591,8 @@ class TDataShard }; struct Snapshots : Table<14> { - struct Oid : Column<1, NScheme::NTypeIds::Uint64> {}; - struct Tid : Column<2, NScheme::NTypeIds::Uint64> {}; + struct Oid : Column<1, NScheme::NTypeIds::Uint64> {}; // PathOwnerId + struct Tid : Column<2, NScheme::NTypeIds::Uint64> {}; // LocalPathId struct Step : Column<3, NScheme::NTypeIds::Uint64> {}; struct TxId : Column<4, NScheme::NTypeIds::Uint64> {}; struct Name : Column<5, NScheme::NTypeIds::String> {}; @@ -734,12 +736,24 @@ class TDataShard using TColumns = TableColumns<Tid, FullCompactionTs>; }; + struct SchemaSnapshots : Table<28> { + struct PathOwnerId : Column<1, NScheme::NTypeIds::Uint64> {}; + struct LocalPathId : Column<2, NScheme::NTypeIds::Uint64> {}; + struct SchemaVersion : Column<3, NScheme::NTypeIds::Uint64> {}; + struct Step : Column<4, NScheme::NTypeIds::Uint64> {}; + struct TxId : Column<5, NScheme::NTypeIds::Uint64> {}; + struct Schema : Column<6, NScheme::NTypeIds::String> {}; + + using TKey = TableKey<PathOwnerId, LocalPathId, SchemaVersion>; + using TColumns = TableColumns<PathOwnerId, LocalPathId, SchemaVersion, Step, TxId, Schema>; + }; + using TTables = SchemaTables<Sys, UserTables, TxMain, TxDetails, InReadSets, OutReadSets, PlanQueue, DeadlineQueue, SchemaOperations, SplitSrcSnapshots, SplitDstReceivedSnapshots, TxArtifacts, ScanProgress, Snapshots, S3Uploads, S3Downloads, ChangeRecords, ChangeRecordDetails, ChangeSenders, S3UploadedParts, SrcChangeSenderActivations, DstChangeSenderActivations, ReplicationSourceOffsets, ReplicationSources, DstReplicationSourceOffsetsReceived, - UserTablesStats>; + UserTablesStats, SchemaSnapshots>; // These settings are persisted on each Init. So we use empty settings in order not to overwrite what // was changed by the user @@ -1388,6 +1402,9 @@ public: TSnapshotManager& GetSnapshotManager() { return SnapshotManager; } const TSnapshotManager& GetSnapshotManager() const { return SnapshotManager; } + TSchemaSnapshotManager& GetSchemaSnapshotManager() { return SchemaSnapshotManager; } + const TSchemaSnapshotManager& GetSchemaSnapshotManager() const { return SchemaSnapshotManager; } + template <typename... Args> bool PromoteCompleteEdge(Args&&... args) { return SnapshotManager.PromoteCompleteEdge(std::forward<Args>(args)...); @@ -1942,6 +1959,7 @@ private: TSysLocks SysLocks; TSnapshotManager SnapshotManager; + TSchemaSnapshotManager SchemaSnapshotManager; TReplicationSourceOffsetsServerLink ReplicationSourceOffsetsServer; diff --git a/ydb/core/tx/datashard/datashard_schema_snapshots.cpp b/ydb/core/tx/datashard/datashard_schema_snapshots.cpp new file mode 100644 index 0000000000..d4c7ab0510 --- /dev/null +++ b/ydb/core/tx/datashard/datashard_schema_snapshots.cpp @@ -0,0 +1,143 @@ +#include "datashard_impl.h" +#include "datashard_schema_snapshots.h" + +#include <ydb/core/util/pb.h> + +namespace NKikimr { +namespace NDataShard { + +TSchemaSnapshot::TSchemaSnapshot(TUserTable::TCPtr schema, ui64 step, ui64 txId) + : Schema(schema) + , Step(step) + , TxId(txId) +{ +} + +TSchemaSnapshotManager::TSchemaSnapshotManager(const TDataShard* self) + : Self(self) +{ +} + +void TSchemaSnapshotManager::Reset() { + Snapshots.clear(); +} + +bool TSchemaSnapshotManager::Load(NIceDb::TNiceDb& db) { + using Schema = TDataShard::Schema; + + auto rowset = db.Table<Schema::SchemaSnapshots>().Range().Select(); + if (!rowset.IsReady()) { + return false; + } + + const auto& tables = Self->GetUserTables(); + while (!rowset.EndOfSet()) { + const ui64 oid = rowset.GetValue<Schema::SchemaSnapshots::PathOwnerId>(); + const ui64 tid = rowset.GetValue<Schema::SchemaSnapshots::LocalPathId>(); + const ui64 version = rowset.GetValue<Schema::SchemaSnapshots::SchemaVersion>(); + const ui64 step = rowset.GetValue<Schema::SchemaSnapshots::Step>(); + const ui64 txId = rowset.GetValue<Schema::SchemaSnapshots::TxId>(); + const TString schema = rowset.GetValue<Schema::SchemaSnapshots::Schema>(); + + NKikimrSchemeOp::TTableDescription desc; + const bool ok = ParseFromStringNoSizeLimit(desc, schema); + Y_VERIFY(ok); + + auto it = tables.find(tid); + Y_VERIFY_S(it != tables.end(), "Cannot find table: " << tid); + + const auto res = Snapshots.emplace( + std::piecewise_construct, + std::forward_as_tuple(oid, tid, version), + std::forward_as_tuple(new TUserTable(it->second->LocalTid, desc, 0), step, txId) + ); + Y_VERIFY_S(res.second, "Duplicate schema snapshot: " << res.first->first); + + if (!rowset.Next()) { + return false; + } + } + + return true; +} + +bool TSchemaSnapshotManager::AddSnapshot(NTable::TDatabase& db, const TSchemaSnapshotKey& key, const TSchemaSnapshot& snapshot) { + if (auto it = Snapshots.find(key); it != Snapshots.end()) { + Y_VERIFY_DEBUG_S(false, "Duplicate schema snapshot: " << key); + return false; + } + + auto it = Self->GetUserTables().find(key.PathId); + Y_VERIFY_S(it != Self->GetUserTables().end(), "Cannot find table: " << key.PathId); + + const auto res = Snapshots.emplace(key, snapshot); + Y_VERIFY_S(res.second, "Duplicate schema snapshot: " << key); + + NIceDb::TNiceDb nicedb(db); + PersistAddSnapshot(nicedb, key, snapshot); + + return true; +} + +const TSchemaSnapshot* TSchemaSnapshotManager::FindSnapshot(const TSchemaSnapshotKey& key) const { + return Snapshots.FindPtr(key); +} + +bool TSchemaSnapshotManager::AcquireReference(const TSchemaSnapshotKey& key) { + auto it = Snapshots.find(key); + if (it == Snapshots.end()) { + return false; + } + + ++References[key]; + return true; +} + +bool TSchemaSnapshotManager::ReleaseReference(const TSchemaSnapshotKey& key, NIceDb::TNiceDb& db) { + auto refIt = References.find(key); + + if (refIt == References.end() || refIt->second <= 0) { + Y_VERIFY_DEBUG(false, "ReleaseReference underflow, check acquire/release pairs"); + return false; + } + + if (--refIt->second) { + return false; + } + + References.erase(refIt); + + auto it = Snapshots.find(key); + if (it == Snapshots.end()) { + Y_VERIFY_DEBUG(false, "ReleaseReference on an already removed snapshot"); + return false; + } + + PersistRemoveSnapshot(db, key); + return true; +} + +void TSchemaSnapshotManager::PersistAddSnapshot(NIceDb::TNiceDb& db, const TSchemaSnapshotKey& key, const TSchemaSnapshot& snapshot) { + using Schema = TDataShard::Schema; + db.Table<Schema::SchemaSnapshots>() + .Key(key.OwnerId, key.PathId, key.Version) + .Update( + NIceDb::TUpdate<Schema::SchemaSnapshots::Step>(snapshot.Step), + NIceDb::TUpdate<Schema::SchemaSnapshots::TxId>(snapshot.TxId), + NIceDb::TUpdate<Schema::SchemaSnapshots::Schema>(snapshot.Schema->GetSchema()) + ); +} + +void TSchemaSnapshotManager::PersistRemoveSnapshot(NIceDb::TNiceDb& db, const TSchemaSnapshotKey& key) { + using Schema = TDataShard::Schema; + db.Table<Schema::SchemaSnapshots>() + .Key(key.OwnerId, key.PathId, key.Version) + .Delete(); +} + +} // NDataShard +} // NKikimr + +Y_DECLARE_OUT_SPEC(, NKikimr::NDataShard::TSchemaSnapshotKey, stream, value) { + stream << "{ table " << value.OwnerId << ":" << value.PathId << " version " << value.Version << " }"; +} diff --git a/ydb/core/tx/datashard/datashard_schema_snapshots.h b/ydb/core/tx/datashard/datashard_schema_snapshots.h new file mode 100644 index 0000000000..a8218a2259 --- /dev/null +++ b/ydb/core/tx/datashard/datashard_schema_snapshots.h @@ -0,0 +1,50 @@ +#pragma once + +#include "datashard_user_table.h" +#include "snapshot_key.h" + +#include <ydb/core/tablet_flat/flat_cxx_database.h> + +#include <util/generic/hash.h> +#include <util/generic/map.h> + +namespace NKikimr { +namespace NDataShard { + +class TDataShard; +class TSchemaSnapshotManager; + +struct TSchemaSnapshot { + TUserTable::TCPtr Schema; + ui64 Step = 0; + ui64 TxId = 0; + + explicit TSchemaSnapshot(TUserTable::TCPtr schema, ui64 step, ui64 txId); +}; + +class TSchemaSnapshotManager { +public: + explicit TSchemaSnapshotManager(const TDataShard* self); + + void Reset(); + bool Load(NIceDb::TNiceDb& db); + + bool AddSnapshot(NTable::TDatabase& db, const TSchemaSnapshotKey& key, const TSchemaSnapshot& snapshot); + const TSchemaSnapshot* FindSnapshot(const TSchemaSnapshotKey& key) const; + + bool AcquireReference(const TSchemaSnapshotKey& key); + bool ReleaseReference(const TSchemaSnapshotKey& key, NIceDb::TNiceDb& db); + +private: + void PersistAddSnapshot(NIceDb::TNiceDb& db, const TSchemaSnapshotKey& key, const TSchemaSnapshot& snapshot); + void PersistRemoveSnapshot(NIceDb::TNiceDb& db, const TSchemaSnapshotKey& key); + +private: + const TDataShard* Self; + TMap<TSchemaSnapshotKey, TSchemaSnapshot, TLess<void>> Snapshots; + THashMap<TSchemaSnapshotKey, size_t> References; + +}; // TSchemaSnapshotManager + +} // NDataShard +} // NKikimr diff --git a/ydb/core/tx/datashard/datashard_snapshots.h b/ydb/core/tx/datashard/datashard_snapshots.h index 8472c5e274..8746834112 100644 --- a/ydb/core/tx/datashard/datashard_snapshots.h +++ b/ydb/core/tx/datashard/datashard_snapshots.h @@ -1,6 +1,7 @@ #pragma once #include "const.h" +#include "snapshot_key.h" #include <ydb/core/tablet_flat/flat_cxx_database.h> #include <ydb/core/tablet_flat/tablet_flat_executor.h> @@ -19,48 +20,6 @@ using NTabletFlatExecutor::TTransactionContext; class TOperation; class TDataShard; -struct TSnapshotTableKey { - ui64 OwnerId = 0; - ui64 PathId = 0; - - TSnapshotTableKey() = default; - - TSnapshotTableKey(ui64 ownerId, ui64 pathId) - : OwnerId(ownerId) - , PathId(pathId) - { } - - friend inline bool operator<(const TSnapshotTableKey& a, const TSnapshotTableKey& b) { - return std::tie(a.OwnerId, a.PathId) < std::tie(b.OwnerId, b.PathId); - } -}; - -struct TSnapshotKey : public TSnapshotTableKey { - ui64 Step = 0; - ui64 TxId = 0; - - TSnapshotKey() = default; - - TSnapshotKey(ui64 ownerId, ui64 pathId, ui64 step, ui64 txId) - : TSnapshotTableKey(ownerId, pathId) - , Step(step) - , TxId(txId) - { } - - friend inline bool operator<(const TSnapshotKey& a, const TSnapshotKey& b) { - return std::tie(a.OwnerId, a.PathId, a.Step, a.TxId) < std::tie(b.OwnerId, b.PathId, b.Step, b.TxId); - } - - friend inline bool operator==(const TSnapshotKey& a, const TSnapshotKey& b) { - return std::tie(a.OwnerId, a.PathId, a.Step, a.TxId) == std::tie(b.OwnerId, b.PathId, b.Step, b.TxId); - } - - // Hash for using snapshot keys in a hash table - explicit operator size_t() const noexcept { - return ((OwnerId * 31 + PathId) * 31 + Step) * 31 + TxId; - } -}; - class TSnapshot { friend class TSnapshotExpireQueue; friend class TSnapshotManager; diff --git a/ydb/core/tx/datashard/snapshot_key.h b/ydb/core/tx/datashard/snapshot_key.h new file mode 100644 index 0000000000..7a742a6e8a --- /dev/null +++ b/ydb/core/tx/datashard/snapshot_key.h @@ -0,0 +1,101 @@ +#pragma once + +#include <util/generic/fwd.h> + +#include <tuple> + +namespace NKikimr { +namespace NDataShard { + +namespace NPrivate { + +template <typename T, size_t I = std::tuple_size<T>::value - 1> +size_t HashTuple(const T& tp) { + if constexpr (I != 0) { + return std::get<I>(tp) + 31 * HashTuple<T, I - 1>(tp); + } else { + return std::get<I>(tp); + } +} + +template <typename TDerived> +struct TCommonOps { + friend inline bool operator<(const TDerived& a, const TDerived& b) { + return a.ToTuple() < b.ToTuple(); + } + + friend inline bool operator==(const TDerived& a, const TDerived& b) { + return a.ToTuple() == b.ToTuple(); + } + + explicit operator size_t() const noexcept { + return HashTuple(static_cast<const TDerived*>(this)->ToTuple()); + } +}; + +} // NPrivate + +struct TSnapshotTableKey : public NPrivate::TCommonOps<TSnapshotTableKey> { + ui64 OwnerId = 0; + ui64 PathId = 0; + + TSnapshotTableKey() = default; + + TSnapshotTableKey(ui64 ownerId, ui64 pathId) + : OwnerId(ownerId) + , PathId(pathId) + { } + + auto ToTuple() const { + return std::make_tuple(OwnerId, PathId); + } + + using TCommonOps<TSnapshotTableKey>::operator size_t; +}; + +struct TDataSnapshotKey + : public TSnapshotTableKey + , public NPrivate::TCommonOps<TDataSnapshotKey> +{ + ui64 Step = 0; + ui64 TxId = 0; + + TDataSnapshotKey() = default; + + TDataSnapshotKey(ui64 ownerId, ui64 pathId, ui64 step, ui64 txId) + : TSnapshotTableKey(ownerId, pathId) + , Step(step) + , TxId(txId) + { } + + auto ToTuple() const { + return std::tuple_cat(TSnapshotTableKey::ToTuple(), std::make_tuple(Step, TxId)); + } + + using TCommonOps<TDataSnapshotKey>::operator size_t; +}; + +using TSnapshotKey = TDataSnapshotKey; + +struct TSchemaSnapshotKey + : public TSnapshotTableKey + , public NPrivate::TCommonOps<TSchemaSnapshotKey> +{ + ui64 Version = 0; + + TSchemaSnapshotKey() = default; + + TSchemaSnapshotKey(ui64 ownerId, ui64 pathId, ui64 version) + : TSnapshotTableKey(ownerId, pathId) + , Version(version) + { } + + auto ToTuple() const { + return std::tuple_cat(TSnapshotTableKey::ToTuple(), std::make_tuple(Version)); + } + + using TCommonOps<TSchemaSnapshotKey>::operator size_t; +}; + +} // NDataShard +} // NKikimr diff --git a/ydb/core/tx/datashard/ya.make b/ydb/core/tx/datashard/ya.make index 2a4334cf73..9c98e8e79b 100644 --- a/ydb/core/tx/datashard/ya.make +++ b/ydb/core/tx/datashard/ya.make @@ -99,6 +99,7 @@ SRCS( datashard_pipeline.h datashard_s3_downloads.cpp datashard_s3_uploads.cpp + datashard_schema_snapshots.cpp datashard_snapshots.cpp datashard_unsafe_upload.cpp datashard_user_table.cpp diff --git a/ydb/tests/functional/scheme_tests/canondata/tablet_scheme_tests.TestTabletSchemes.test_tablet_schemes_flat_datashard_/flat_datashard.schema b/ydb/tests/functional/scheme_tests/canondata/tablet_scheme_tests.TestTabletSchemes.test_tablet_schemes_flat_datashard_/flat_datashard.schema index fb3438c1d7..10d5f2ef24 100644 --- a/ydb/tests/functional/scheme_tests/canondata/tablet_scheme_tests.TestTabletSchemes.test_tablet_schemes_flat_datashard_/flat_datashard.schema +++ b/ydb/tests/functional/scheme_tests/canondata/tablet_scheme_tests.TestTabletSchemes.test_tablet_schemes_flat_datashard_/flat_datashard.schema @@ -1465,5 +1465,72 @@ "Blobs": 1 } } + }, + { + "TableId": 28, + "TableName": "SchemaSnapshots", + "TableKey": [ + 1, + 2, + 3 + ], + "ColumnsAdded": [ + { + "ColumnId": 1, + "ColumnName": "PathOwnerId", + "ColumnType": "Uint64" + }, + { + "ColumnId": 2, + "ColumnName": "LocalPathId", + "ColumnType": "Uint64" + }, + { + "ColumnId": 3, + "ColumnName": "SchemaVersion", + "ColumnType": "Uint64" + }, + { + "ColumnId": 4, + "ColumnName": "Step", + "ColumnType": "Uint64" + }, + { + "ColumnId": 5, + "ColumnName": "TxId", + "ColumnType": "Uint64" + }, + { + "ColumnId": 6, + "ColumnName": "Schema", + "ColumnType": "String" + } + ], + "ColumnsDropped": [], + "ColumnFamilies": { + "0": { + "Columns": [ + 1, + 2, + 3, + 4, + 5, + 6 + ], + "RoomID": 0, + "Codec": 0, + "InMemory": false, + "Cache": 0, + "Small": 4294967295, + "Large": 4294967295 + } + }, + "Rooms": { + "0": { + "Main": 1, + "Outer": 1, + "Blobs": 1 + } + } } ]
\ No newline at end of file |