diff options
author | Maksim Kita <kitaetoya@gmail.com> | 2023-06-30 08:50:02 +0000 |
---|---|---|
committer | maksim-kita <maksim-kita@yandex-team.com> | 2023-06-30 11:50:02 +0300 |
commit | b1acffe1a8d5d2af1b0098f4e259e924ea33e3a3 (patch) | |
tree | ea7e905c2ecd3543add6f36fa3b8d4d456e20995 | |
parent | 5c695412dab36725221e24bb4c5031789623bc70 (diff) | |
download | ydb-b1acffe1a8d5d2af1b0098f4e259e924ea33e3a3.tar.gz |
SerializedCellVec improve serialization
SerializedCellVec improve serialization
Pull Request resolved: #275
33 files changed, 210 insertions, 170 deletions
diff --git a/ydb/core/grpc_services/rpc_load_rows.cpp b/ydb/core/grpc_services/rpc_load_rows.cpp index 847be3a1634..6f3f442d4de 100644 --- a/ydb/core/grpc_services/rpc_load_rows.cpp +++ b/ydb/core/grpc_services/rpc_load_rows.cpp @@ -246,9 +246,9 @@ private: cost += TUpsertCost::OneRowCost(sz); // Save serialized key and value - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(keyCells)); + TSerializedCellVec serializedKey(keyCells); TString serializedValue = TSerializedCellVec::Serialize(valueCells); - AllRows.emplace_back(serializedKey, serializedValue); + AllRows.emplace_back(std::move(serializedKey), std::move(serializedValue)); } RuCost = TUpsertCost::CostToRu(cost); diff --git a/ydb/core/grpc_services/rpc_read_columns.cpp b/ydb/core/grpc_services/rpc_read_columns.cpp index 4bbc49f88ed..46c3ede0af2 100644 --- a/ydb/core/grpc_services/rpc_read_columns.cpp +++ b/ydb/core/grpc_services/rpc_read_columns.cpp @@ -534,7 +534,7 @@ private: MinKeyInclusive = proto->from_key_inclusive(); } else { TVector<TCell> allNulls(KeyColumnTypes.size()); - MinKey.Parse(TSerializedCellVec::Serialize(allNulls)); + MinKey = TSerializedCellVec(allNulls); MinKeyInclusive = true; } @@ -548,7 +548,7 @@ private: MaxKeyInclusive = proto->to_key_inclusive(); } else { TVector<TCell> infinity; - MaxKey.Parse(TSerializedCellVec::Serialize(infinity)); + MaxKey = TSerializedCellVec(infinity); MaxKeyInclusive = false; } diff --git a/ydb/core/kqp/common/kqp_resolve.cpp b/ydb/core/kqp/common/kqp_resolve.cpp index 473b3574e10..fb5bc8640ad 100644 --- a/ydb/core/kqp/common/kqp_resolve.cpp +++ b/ydb/core/kqp/common/kqp_resolve.cpp @@ -78,7 +78,7 @@ TVector<TPartitionWithRange> GetKeyRangePartitions(const TTableRange& range, if (range.Point) { TPartitionWithRange ret(it); if (!it->Range->IsPoint) { - ret.PointOrRange = TSerializedCellVec(TSerializedCellVec::Serialize(range.From)); + ret.PointOrRange = TSerializedCellVec(range.From); } else { ret.FullRange.emplace(TSerializedTableRange(range)); } @@ -147,7 +147,7 @@ TVector<TPartitionWithRange> GetKeyRangePartitions(const TTableRange& range, rangePartitions.back().FullRange.emplace(TSerializedTableRange(fromValues, true, fromValues, true)); rangePartitions.back().FullRange->Point = true; } else { - rangePartitions.back().PointOrRange = TSerializedCellVec(TSerializedCellVec::Serialize(fromValues)); + rangePartitions.back().PointOrRange = TSerializedCellVec(fromValues); } } else { auto r = TTableRange(fromValues, inclusiveFrom, toValues, inclusiveTo); diff --git a/ydb/core/kqp/executer_actor/kqp_partition_helper.cpp b/ydb/core/kqp/executer_actor/kqp_partition_helper.cpp index de5296f2db5..c29ddd65cf1 100644 --- a/ydb/core/kqp/executer_actor/kqp_partition_helper.cpp +++ b/ydb/core/kqp/executer_actor/kqp_partition_helper.cpp @@ -75,7 +75,7 @@ THashMap<ui64, TShardParamValuesAndRanges> PartitionParamByKey( shardId = key.GetPartitions()[partitionIndex].ShardId; - auto point = TSerializedCellVec(TSerializedCellVec::Serialize(keyValue)); + auto point = TSerializedCellVec(keyValue); auto& shardData = ret[shardId]; if (key.GetPartitions()[partitionIndex].Range->IsPoint) { // singular case when partition is just a point @@ -326,7 +326,7 @@ TSerializedPointOrRange FillOneRange(NUdf::TUnboxedValue& begin, NUdf::TUnboxedV YQL_CLOG(DEBUG, ProviderKqp) << "Formed point [extract predicate]: " << DebugPrintPoint(keyColumnTypes, fromKeyValues, *AppData()->TypeRegistry); - return TSerializedCellVec(TSerializedCellVec::Serialize(fromKeyValues)); + return TSerializedCellVec(fromKeyValues); } auto range = TSerializedTableRange(fromKeyValues, fromInclusive, toKeyValues, toInclusive); @@ -577,7 +577,7 @@ TVector<TSerializedPointOrRange> ExtractRanges(const TKqpTableKeys& tableKeys, const auto& range = source.GetKeyRange(); if (range.GetRangeIsPoint() && range.GetFrom().ValuesSize() == keyColumnTypes.size()) { auto cells = FillKeyValues(keyColumnTypes, range.GetFrom(), stageInfo, holderFactory, typeEnv); - ranges.push_back(TSerializedCellVec(TSerializedCellVec::Serialize(cells))); + ranges.push_back(TSerializedCellVec(cells)); } else { ranges.push_back(MakeKeyRange(keyColumnTypes, range, stageInfo, holderFactory, typeEnv)); } diff --git a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp index 9d00649423b..d0cb21e71ca 100644 --- a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp @@ -548,7 +548,7 @@ private: for (auto& key : read.Keys) { YQL_ENSURE(key.Point); - request->Keys.emplace_back(TSerializedCellVec::Serialize(key.From)); + request->Keys.emplace_back(key.From); } Send(MakePipePeNodeCacheID(false), new TEvPipeCache::TEvForward(request.Release(), shardId, true), diff --git a/ydb/core/load_test/ut_ycsb.cpp b/ydb/core/load_test/ut_ycsb.cpp index 868b8cd94bf..0cffef3a0f0 100644 --- a/ydb/core/load_test/ut_ycsb.cpp +++ b/ydb/core/load_test/ut_ycsb.cpp @@ -184,7 +184,7 @@ void AddRangeQuery( auto fromBuf = TSerializedCellVec::Serialize(fromCells); auto toBuf = TSerializedCellVec::Serialize(toCells); - request.Ranges.emplace_back(fromBuf, toBuf, fromInclusive, toInclusive); + request.Ranges.emplace_back(std::move(fromBuf), std::move(toBuf), fromInclusive, toInclusive); } struct TTableInfo { diff --git a/ydb/core/load_test/ycsb/common.cpp b/ydb/core/load_test/ycsb/common.cpp index 8f408a95581..612592ff2e5 100644 --- a/ydb/core/load_test/ycsb/common.cpp +++ b/ydb/core/load_test/ycsb/common.cpp @@ -211,15 +211,14 @@ void AddRangeQuery( auto fromBuf = TSerializedCellVec::Serialize(fromCells); auto toBuf = TSerializedCellVec::Serialize(toCells); - request.Ranges.emplace_back(fromBuf, toBuf, fromInclusive, toInclusive); + request.Ranges.emplace_back(std::move(fromBuf), std::move(toBuf), fromInclusive, toInclusive); } void AddKeyQuery( TEvDataShard::TEvRead &request, const TOwnedCellVec &key) { - auto buf = TSerializedCellVec::Serialize(key); - request.Keys.emplace_back(buf); + request.Keys.emplace_back(key); } IActor *CreateReadIteratorScan( diff --git a/ydb/core/scheme/scheme_tablecell.cpp b/ydb/core/scheme/scheme_tablecell.cpp index 483402200a3..7643d5e3390 100644 --- a/ydb/core/scheme/scheme_tablecell.cpp +++ b/ydb/core/scheme/scheme_tablecell.cpp @@ -63,6 +63,128 @@ TOwnedCellVec::TInit TOwnedCellVec::Allocate(TOwnedCellVec::TCellVec cells) { }; } +namespace { + + struct TCellHeader { + TCellHeader() = default; + + TCellHeader(ui32 rawValue) : RawValue(rawValue) {} + + TCellHeader(ui32 cellSize, bool isNull) + : RawValue(cellSize | (static_cast<ui32>(isNull) << 31)) + {} + + ui32 CellSize() const { return RawValue & ~(1ULL << 31); } + + bool IsNull() const { return RawValue & (1ULL << 31); }; + + ui32 RawValue = 0; + }; + + static_assert(sizeof(TCellHeader) == sizeof(ui32)); + + Y_FORCE_INLINE void SerializeCellVec(TConstArrayRef<TCell> cells, TString &resultBuffer, TVector<TCell> *resultCells) { + resultBuffer.clear(); + if (resultCells) + resultCells->clear(); + + if (cells.empty()) { + return; + } + + size_t size = sizeof(ui16); + for (auto& cell : cells) { + size += sizeof(TCellHeader) + cell.Size(); + } + + resultBuffer.resize(size); + char* resultBufferData = const_cast<char*>(resultBuffer.data()); + + ui16 cellsSize = cells.size(); + WriteUnaligned<ui16>(resultBufferData, cellsSize); + resultBufferData += sizeof(cellsSize); + + if (resultCells) { + resultCells->resize(cellsSize); + } + + for (size_t i = 0; i < cellsSize; ++i) { + TCellHeader header(cells[i].Size(), cells[i].IsNull()); + WriteUnaligned<ui32>(resultBufferData, header.RawValue); + resultBufferData += sizeof(header); + + const auto & cell = cells[i]; + memcpy(resultBufferData, cell.Data(), cell.Size()); + + if (resultCells && !cell.IsNull()) { + (*resultCells)[i] = TCell(resultBufferData, cell.Size()); + } + + resultBufferData += cell.Size(); + } + } + + Y_FORCE_INLINE bool TryDeserializeCellVec(const TString & data, TString & resultBuffer, TVector<TCell> & resultCells) { + resultBuffer.clear(); + resultCells.clear(); + + if (data.empty()) + return true; + + const char* buf = data.data(); + const char* bufEnd = data.data() + data.size(); + if (bufEnd - buf < static_cast<ptrdiff_t>(sizeof(ui16))) + return false; + + ui16 cellsSize = ReadUnaligned<ui16>(buf); + buf += sizeof(cellsSize); + + resultCells.resize(cellsSize); + + for (ui32 i = 0; i < cellsSize; ++i) { + if (bufEnd - buf < static_cast<ptrdiff_t>(sizeof(TCellHeader))) { + resultCells.clear(); + return false; + } + + TCellHeader cellHeader = ReadUnaligned<TCellHeader>(buf); + buf += sizeof(cellHeader); + + if (bufEnd - buf < static_cast<ptrdiff_t>(cellHeader.CellSize())) { + resultCells.clear(); + return false; + } + + resultCells[i] = cellHeader.IsNull() ? TCell() : TCell(buf, cellHeader.CellSize()); + buf += cellHeader.CellSize(); + } + + resultBuffer = data; + return true; + } + +} + +TSerializedCellVec::TSerializedCellVec(TConstArrayRef<TCell> cells) +{ + SerializeCellVec(cells, Buf, &Cells); +} + +void TSerializedCellVec::Serialize(TString& res, TConstArrayRef<TCell> cells) { + SerializeCellVec(cells, res, nullptr /*resultCells*/); +} + +TString TSerializedCellVec::Serialize(TConstArrayRef<TCell> cells) { + TString result; + SerializeCellVec(cells, result, nullptr /*resultCells*/); + + return result; +} + +bool TSerializedCellVec::DoTryParse(const TString& data) { + return TryDeserializeCellVec(data, Buf, Cells); +} + TString DbgPrintCell(const TCell& r, NScheme::TTypeInfo typeInfo, const NScheme::TTypeRegistry ®) { auto typeId = typeInfo.GetTypeId(); TString res; diff --git a/ydb/core/scheme/scheme_tablecell.h b/ydb/core/scheme/scheme_tablecell.h index ce97552e362..ff19e1da557 100644 --- a/ydb/core/scheme/scheme_tablecell.h +++ b/ydb/core/scheme/scheme_tablecell.h @@ -415,12 +415,14 @@ static_assert(std::is_nothrow_default_constructible_v<TOwnedCellVec>, "Expected // When loading from a buffer the cells will point to the buffer contents class TSerializedCellVec { public: + explicit TSerializedCellVec(TConstArrayRef<TCell> cells); + explicit TSerializedCellVec(const TString& buf) { Parse(buf); } - TSerializedCellVec() {} + TSerializedCellVec() = default; TSerializedCellVec(const TSerializedCellVec &other) : Buf(other.Buf) @@ -457,50 +459,20 @@ public: } static bool TryParse(const TString& data, TSerializedCellVec& vec) { - bool ok = DoTryParse(data, vec); - if (!ok) { - vec.Cells.clear(); - vec.Buf.clear(); - } - return ok; + return vec.DoTryParse(data); } void Parse(const TString &buf) { - Y_VERIFY(TryParse(buf, *this)); + Y_VERIFY(DoTryParse(buf)); } TConstArrayRef<TCell> GetCells() const { return Cells; } - static void Serialize(TString& res, const TConstArrayRef<TCell>& cells) { - size_t sz = sizeof(ui16); - for (auto& c : cells) { - sz += sizeof(TValue) + c.Size(); - } - - if (res.capacity() < sz) { - res.reserve(FastClp2(sz + 1) - 1); - } - - DoSerialize(res, cells); - } + static void Serialize(TString& res, TConstArrayRef<TCell> cells); - static TString Serialize(const TConstArrayRef<TCell>& cells) { - if (cells.empty()) - return TString(); - - size_t sz = sizeof(ui16); - for (auto& c : cells) { - sz += sizeof(TValue) + c.Size(); - } - - TString res; - res.reserve(sz); - - DoSerialize(res, cells); - return res; - } + static TString Serialize(TConstArrayRef<TCell> cells); const TString &GetBuffer() const { return Buf; } @@ -510,58 +482,7 @@ public: } private: - -#pragma pack(push,4) - struct TValue { - ui32 Size : 31; - ui32 IsNull : 1; - }; -#pragma pack(pop) - - static bool DoTryParse(const TString& data, TSerializedCellVec& vec) { - vec.Cells.clear(); - if (data.empty()) - return true; - - if (data.size() < sizeof(ui16)) - return false; - - ui16 count = ReadUnaligned<ui16>(data.data()); - vec.Cells.resize(count); - const char* buf = data.data() + sizeof(count); - const char* bufEnd = data.data() + data.size(); - for (ui32 ki = 0; ki < count; ++ki) { - if (bufEnd - buf < (long)sizeof(TValue)) - return false; - - const TValue v = ReadUnaligned<TValue>((const TValue*)buf); - if (bufEnd - buf < (long)sizeof(TValue) + v.Size) - return false; - vec.Cells[ki] = v.IsNull ? TCell() : TCell((const char*)((const TValue*)buf + 1), v.Size); - buf += sizeof(TValue) + v.Size; - } - - vec.Buf = data; - return true; - } - - // note, that res space should be reserved before this call - static void DoSerialize(TString& res, const TConstArrayRef<TCell>& cells) { - res.clear(); - - if (cells.empty()) - return; - - ui16 cnt = cells.size(); - res.append((const char*)&cnt, sizeof(ui16)); - for (auto& c : cells) { - TValue header; - header.Size = c.Size(); - header.IsNull = c.IsNull(); - res.append((const char*)&header, sizeof(header)); - res.append(c.Data(), c.Size()); - } - } + bool DoTryParse(const TString& data); private: TString Buf; diff --git a/ydb/core/scheme/scheme_tablecell_ut.cpp b/ydb/core/scheme/scheme_tablecell_ut.cpp index 4ff0b48c95f..83520a70ac5 100644 --- a/ydb/core/scheme/scheme_tablecell_ut.cpp +++ b/ydb/core/scheme/scheme_tablecell_ut.cpp @@ -199,7 +199,7 @@ Y_UNIT_TEST_SUITE(Scheme) { UNIT_ASSERT(!TSerializedCellVec::TryParse("\1\1", vec)); const TString buf = TSerializedCellVec::Serialize({TCell(), TCell()}); - UNIT_ASSERT_VALUES_EQUAL(buf.size(), 2 + 2*4); + UNIT_ASSERT_VALUES_EQUAL(buf.size(), sizeof(ui16) + 2 * 4); { for (size_t i = 0; i < buf.size(); ++i) { diff --git a/ydb/core/scheme/scheme_tabledefs.h b/ydb/core/scheme/scheme_tabledefs.h index a15d333964f..4615e5388db 100644 --- a/ydb/core/scheme/scheme_tabledefs.h +++ b/ydb/core/scheme/scheme_tabledefs.h @@ -212,8 +212,8 @@ public: TSerializedTableRange(TConstArrayRef<TCell> fromValues, bool inclusiveFrom, TConstArrayRef<TCell> toValues, bool inclusiveTo) - : From(TSerializedCellVec::Serialize(fromValues)) - , To(TSerializedCellVec::Serialize(toValues)) + : From(fromValues) + , To(toValues) , FromInclusive(inclusiveFrom) , ToInclusive(inclusiveTo) {} diff --git a/ydb/core/tablet_flat/flat_comp_shard.cpp b/ydb/core/tablet_flat/flat_comp_shard.cpp index 89d765129c4..efb8fe189f9 100644 --- a/ydb/core/tablet_flat/flat_comp_shard.cpp +++ b/ydb/core/tablet_flat/flat_comp_shard.cpp @@ -367,16 +367,16 @@ namespace NCompShard { if (viable) { if (reuseActive && lastItem == item && results.Reusable.back().Slice.EndRowId() == start.BeginRowId()) { // Merge adjacent pages as long as it's from the same item - results.Reusable.back().Slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(end.Key)); + results.Reusable.back().Slice.LastKey = TSerializedCellVec(end.Key); results.Reusable.back().Slice.LastRowId = end.RowId; results.Reusable.back().Slice.LastInclusive = end.Inclusive; } else { auto& result = results.Reusable.emplace_back(); result.Part = item->Part; - result.Slice.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(start.Key)); + result.Slice.FirstKey = TSerializedCellVec(start.Key); result.Slice.FirstRowId = start.RowId; result.Slice.FirstInclusive = start.Inclusive; - result.Slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(end.Key)); + result.Slice.LastKey = TSerializedCellVec(end.Key); result.Slice.LastRowId = end.RowId; result.Slice.LastInclusive = end.Inclusive; ++results.ExpectedSlices; @@ -495,7 +495,7 @@ namespace NCompShard { if (bestLeftSize < Stats.Size && bestRightSize < Stats.Size) { // We found a key that splits current shard somewhat evenly and // the size estimate of individual subshards actually decreses - foundKey = TSerializedCellVec(TSerializedCellVec::Serialize(bestSplitKey)); + foundKey = TSerializedCellVec(bestSplitKey); return true; } @@ -565,7 +565,7 @@ namespace NCompShard { // This is the first key >= split key TSerializedCellVec rowKey; if (rowId != currentEnd) { - rowKey = TSerializedCellVec(TSerializedCellVec::Serialize(it.GetRawKey())); + rowKey = TSerializedCellVec(it.GetRawKey()); } else if (Y_UNLIKELY(ComparePartKeys(current.LastKey.GetCells(), keyCells, *keyDefaults) < 0)) { // This shouldn't normally happen, but better safe than sorry // Current split key is actually out of bounds for the slice @@ -588,7 +588,7 @@ namespace NCompShard { // Seek to previous row ready = it.Seek(rowId - 1); Y_VERIFY(ready == EReady::Data, "Unexpected failure, precharge logic may be faulty"); - left.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(it.GetRawKey())); + left.LastKey = TSerializedCellVec(it.GetRawKey()); left.LastRowId = it.GetRowId(); left.LastInclusive = true; if ((rowId - currentBegin) == 1) { diff --git a/ydb/core/tablet_flat/flat_part_keys.h b/ydb/core/tablet_flat/flat_part_keys.h index 457e5c24fb5..dd68f0c7f28 100644 --- a/ydb/core/tablet_flat/flat_part_keys.h +++ b/ydb/core/tablet_flat/flat_part_keys.h @@ -138,7 +138,7 @@ namespace NTable { TSerializedCellVec GetKey() const noexcept { - return TSerializedCellVec(TSerializedCellVec::Serialize(Key)); + return TSerializedCellVec(Key); } bool SeekRow(TRowId rowId) noexcept diff --git a/ydb/core/tablet_flat/flat_part_slice.cpp b/ydb/core/tablet_flat/flat_part_slice.cpp index a2e2e85d81f..473d07e099a 100644 --- a/ydb/core/tablet_flat/flat_part_slice.cpp +++ b/ydb/core/tablet_flat/flat_part_slice.cpp @@ -475,13 +475,13 @@ TIntrusiveConstPtr<TSlices> TSlices::Cut( if (!result->empty()) { auto& first = result->front(); if (first.FirstRowId < beginRowId) { - first.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(beginKey)); + first.FirstKey = TSerializedCellVec(beginKey); first.FirstRowId = beginRowId; first.FirstInclusive = true; } auto& last = result->back(); if (last.LastRowId > endRowId) { - last.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(endKey)); + last.LastKey = TSerializedCellVec(endKey); last.LastRowId = endRowId; last.LastInclusive = false; } else if (last.LastRowId == endRowId && last.LastInclusive) { diff --git a/ydb/core/tablet_flat/flat_part_writer.h b/ydb/core/tablet_flat/flat_part_writer.h index 6239466a2a6..a5dc97c10ec 100644 --- a/ydb/core/tablet_flat/flat_part_writer.h +++ b/ydb/core/tablet_flat/flat_part_writer.h @@ -328,7 +328,7 @@ namespace NTable { if (NextSliceFirstRowId == Max<TRowId>()) { NextSliceFirstRowId = Groups[0].Data.GetLastRowId(); - NextSliceFirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(KeyState.Key)); + NextSliceFirstKey = TSerializedCellVec(KeyState.Key); } } @@ -732,7 +732,7 @@ namespace NTable { TRowId lastRowId = dataPage.BaseRow() + dataPage->Records - 1; InitKey(dataPage->Record(dataPage->Records - 1), groupId); - SaveSlice(lastRowId, TSerializedCellVec(TSerializedCellVec::Serialize(Key))); + SaveSlice(lastRowId, TSerializedCellVec(Key)); if (Phase == 1) { Y_VERIFY_DEBUG(g.Index.CalcSize(Key) == g.LastKeyIndexSize); diff --git a/ydb/core/tablet_flat/flat_scan_lead.h b/ydb/core/tablet_flat/flat_scan_lead.h index 2a0c8123b3d..e3650cba231 100644 --- a/ydb/core/tablet_flat/flat_scan_lead.h +++ b/ydb/core/tablet_flat/flat_scan_lead.h @@ -14,14 +14,14 @@ namespace NTable { Valid = true; Tags.assign(tags.begin(), tags.end()); Relation = seek; - Key = TSerializedCellVec(TSerializedCellVec::Serialize(key)); + Key = TSerializedCellVec(key); StopKey = { }; } void Until(TArrayRef<const TCell> key, bool inclusive) { Y_VERIFY(Valid, "Until must be called after To"); - StopKey = TSerializedCellVec(TSerializedCellVec::Serialize(key)); + StopKey = TSerializedCellVec(key); StopKeyInclusive = inclusive; } diff --git a/ydb/core/tablet_flat/flat_table_part_ut.cpp b/ydb/core/tablet_flat/flat_table_part_ut.cpp index ffb53ed8ab0..77ff5388b4c 100644 --- a/ydb/core/tablet_flat/flat_table_part_ut.cpp +++ b/ydb/core/tablet_flat/flat_table_part_ut.cpp @@ -315,14 +315,14 @@ Y_UNIT_TEST_SUITE(TLegacy) { dbgOut << DbgPrintTuple(key, typeRegistry) << " " << stats.RowCount << " " << stats.DataSize.Size << Endl; - + UNIT_ASSERT_C(CompareTypedCellVectors(key.Columns, prevKey.GetCells().data(), key.Types, key.ColumnCount, prevKey.GetCells().size()) > 0, "Keys must be sorted"); UNIT_ASSERT(prevRowCount < stats.RowCount); - UNIT_ASSERT(prevDataSize < stats.DataSize.Size); + UNIT_ASSERT(prevDataSize < stats.DataSize.Size); - prevKey = TSerializedCellVec(TSerializedCellVec::Serialize(TConstArrayRef<TCell>(key.Columns, key.ColumnCount))); + prevKey = TSerializedCellVec(TConstArrayRef<TCell>(key.Columns, key.ColumnCount)); prevRowCount = stats.RowCount; prevDataSize = stats.DataSize.Size; } diff --git a/ydb/core/tablet_flat/ut/ut_comp_shard.cpp b/ydb/core/tablet_flat/ut/ut_comp_shard.cpp index 08fc181ba93..0a4a1432763 100644 --- a/ydb/core/tablet_flat/ut/ut_comp_shard.cpp +++ b/ydb/core/tablet_flat/ut/ut_comp_shard.cpp @@ -251,8 +251,8 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { } slice.FirstRowId = beginRow - !slice.FirstInclusive; slice.LastRowId = endRow - slice.LastInclusive; - slice.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.FirstRowId]))); - slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.LastRowId]))); + slice.FirstKey = TSerializedCellVec(tool.KeyCells(rows[slice.FirstRowId])); + slice.LastKey = TSerializedCellVec(tool.KeyCells(rows[slice.LastRowId])); for (ui64 splitRow = slice.FirstRowId; splitRow <= slice.LastRowId; ++splitRow) { for (ui64 splitFlags = 0; splitFlags < 2; ++splitFlags) { @@ -268,7 +268,7 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { TTableInfo table; table.RowScheme = lay.RowScheme(); - table.SplitKeys[1] = TSerializedCellVec(TSerializedCellVec::Serialize(splitCells)); + table.SplitKeys[1] = TSerializedCellVec(splitCells); TTableShard left; TTableShard right; left.RightKey = 1; @@ -337,8 +337,8 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { slice.FirstInclusive = true; slice.LastRowId = rows.Size() - 1; slice.LastInclusive = true; - slice.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.FirstRowId]))); - slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.LastRowId]))); + slice.FirstKey = TSerializedCellVec(tool.KeyCells(rows[slice.FirstRowId])); + slice.LastKey = TSerializedCellVec(tool.KeyCells(rows[slice.LastRowId])); TRowId splitRow = rows.Size() / 2; @@ -349,7 +349,7 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { for (ui32 k = 1; k <= 5; ++k) { auto cells = tool.KeyCells(rows[splitRow]); cells[1] = Cimple(k); - table.SplitKeys[k] = TSerializedCellVec(TSerializedCellVec::Serialize(cells)); + table.SplitKeys[k] = TSerializedCellVec(cells); auto* left = shards.Back(); shards.PushBack(new TTableShard); auto* right = shards.Back(); @@ -400,14 +400,14 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { slice.FirstInclusive = false; slice.LastRowId = rows.Size() - 3; // the last 3 rows are not included slice.LastInclusive = false; - slice.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.FirstRowId]))); - slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[slice.LastRowId]))); + slice.FirstKey = TSerializedCellVec(tool.KeyCells(rows[slice.FirstRowId])); + slice.LastKey = TSerializedCellVec(tool.KeyCells(rows[slice.LastRowId])); TRowId splitRow = rows.Size() / 2; TVector<TSerializedCellVec> splitKeys; - splitKeys.emplace_back(TSerializedCellVec::Serialize(tool.KeyCells(*TSchemedCookRow(*lay).Col(500_u64, 42_u32)))); - splitKeys.emplace_back(TSerializedCellVec::Serialize(tool.KeyCells(rows[splitRow]))); - splitKeys.emplace_back(TSerializedCellVec::Serialize(tool.KeyCells(*TSchemedCookRow(*lay).Col(5000_u64, 42_u32)))); + splitKeys.emplace_back(tool.KeyCells(*TSchemedCookRow(*lay).Col(500_u64, 42_u32))); + splitKeys.emplace_back(tool.KeyCells(rows[splitRow])); + splitKeys.emplace_back(tool.KeyCells(*TSchemedCookRow(*lay).Col(5000_u64, 42_u32))); TTableInfo table; table.RowScheme = lay.RowScheme(); @@ -736,8 +736,8 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { TSlice slice = partView1.Slices->at(0); slice.FirstRowId += 2; slice.LastRowId -= 2; - slice.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows1[slice.FirstRowId]))); - slice.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows1[slice.LastRowId]))); + slice.FirstKey = TSerializedCellVec(tool.KeyCells(rows1[slice.FirstRowId])); + slice.LastKey = TSerializedCellVec(tool.KeyCells(rows1[slice.LastRowId])); builder.AddSlice(partView1.Part.Get(), slice, true); } @@ -791,16 +791,16 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { { auto& bounds = underlayMaskValues.emplace_back(); - bounds.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[3]))); - bounds.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[7]))); + bounds.FirstKey = TSerializedCellVec(tool.KeyCells(rows[3])); + bounds.LastKey = TSerializedCellVec(tool.KeyCells(rows[7])); bounds.FirstInclusive = false; bounds.LastInclusive = true; } { auto& bounds = underlayMaskValues.emplace_back(); - bounds.FirstKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[11]))); - bounds.LastKey = TSerializedCellVec(TSerializedCellVec::Serialize(tool.KeyCells(rows[14]))); + bounds.FirstKey = TSerializedCellVec(tool.KeyCells(rows[11])); + bounds.LastKey = TSerializedCellVec(tool.KeyCells(rows[14])); bounds.FirstInclusive = true; bounds.LastInclusive = false; } @@ -852,8 +852,8 @@ Y_UNIT_TEST_SUITE(TShardedCompaction) { } TVector<TSerializedCellVec> splitKeyValues; - splitKeyValues.emplace_back(TSerializedCellVec::Serialize(tool.KeyCells(rows[6]))); - splitKeyValues.emplace_back(TSerializedCellVec::Serialize(tool.KeyCells(rows[14]))); + splitKeyValues.emplace_back(tool.KeyCells(rows[6])); + splitKeyValues.emplace_back(tool.KeyCells(rows[14])); TSplitKeys splitKeys(lay.RowScheme(), std::move(splitKeyValues)); auto born = TCompaction(nullptr, CreateConf(4, &splitKeys)) diff --git a/ydb/core/tablet_flat/ut/ut_slice.cpp b/ydb/core/tablet_flat/ut/ut_slice.cpp index 5833066aadb..de715d39998 100644 --- a/ydb/core/tablet_flat/ut/ut_slice.cpp +++ b/ydb/core/tablet_flat/ut/ut_slice.cpp @@ -25,8 +25,8 @@ namespace { TVector<TCell> firstKey{ TCell::Make(firstRowId) }; TVector<TCell> lastKey{ TCell::Make(lastRowId) }; Run->emplace_back( - TSerializedCellVec(TSerializedCellVec::Serialize(firstKey)), - TSerializedCellVec(TSerializedCellVec::Serialize(lastKey)), + TSerializedCellVec(firstKey), + TSerializedCellVec(lastKey), firstRowId, lastRowId, firstInclusive, diff --git a/ydb/core/tx/datashard/build_index.cpp b/ydb/core/tx/datashard/build_index.cpp index dfd00362aa3..7ec1c9c1ab6 100644 --- a/ydb/core/tx/datashard/build_index.cpp +++ b/ydb/core/tx/datashard/build_index.cpp @@ -323,8 +323,8 @@ public: const TConstArrayRef<TCell> rowCells = *row; - ReadBuf.AddRow(TSerializedCellVec(TSerializedCellVec::Serialize(key)), - TSerializedCellVec(TSerializedCellVec::Serialize(rowCells.Slice(0, TargetDataColumnPos))), + ReadBuf.AddRow(TSerializedCellVec(key), + TSerializedCellVec(rowCells.Slice(0, TargetDataColumnPos)), TSerializedCellVec::Serialize(rowCells.Slice(TargetDataColumnPos))); if (!ReadBuf.IsReachLimits(Limits)) { diff --git a/ydb/core/tx/datashard/cdc_stream_scan.cpp b/ydb/core/tx/datashard/cdc_stream_scan.cpp index 8c39fc6849b..16c28b6825c 100644 --- a/ydb/core/tx/datashard/cdc_stream_scan.cpp +++ b/ydb/core/tx/datashard/cdc_stream_scan.cpp @@ -358,8 +358,8 @@ class TCdcStreamScan: public IActorCallback, public IScan { public: void AddRow(TArrayRef<const TCell> key, TArrayRef<const TCell> value) { const auto& [k, v] = Data.emplace_back( - TSerializedCellVec(TSerializedCellVec::Serialize(key)), - TSerializedCellVec(TSerializedCellVec::Serialize(value)) + TSerializedCellVec(key), + TSerializedCellVec(value) ); ByteSize += k.GetBuffer().size() + v.GetBuffer().size(); } diff --git a/ydb/core/tx/datashard/datashard__read_iterator.cpp b/ydb/core/tx/datashard/datashard__read_iterator.cpp index 24dfc4caeb6..0fb608cd422 100644 --- a/ydb/core/tx/datashard/datashard__read_iterator.cpp +++ b/ydb/core/tx/datashard/datashard__read_iterator.cpp @@ -286,7 +286,7 @@ TSerializedCellVec ExtendWithNulls( } extendedCells.resize(columnCount, TCell()); - return TSerializedCellVec(TSerializedCellVec::Serialize(extendedCells)); + return TSerializedCellVec(extendedCells); } ui64 ResetRowStats(NTable::TIteratorStats& stats) diff --git a/ydb/core/tx/datashard/datashard_distributed_erase.cpp b/ydb/core/tx/datashard/datashard_distributed_erase.cpp index 69e283aeac1..ad3fe66c1df 100644 --- a/ydb/core/tx/datashard/datashard_distributed_erase.cpp +++ b/ydb/core/tx/datashard/datashard_distributed_erase.cpp @@ -652,7 +652,7 @@ class TDistEraser: public TActorBootstrapped<TDistEraser> { indexCells.push_back(keyCells.GetCells()[keyColumnIdToIdx.at(id)]); } - indexColumnValues.push_back(std::move(TSerializedCellVec::Serialize(indexCells))); + indexColumnValues.push_back(TSerializedCellVec::Serialize(indexCells)); } } diff --git a/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp b/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp index 63a1d27cb2f..84c5ae628f8 100644 --- a/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp +++ b/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp @@ -284,8 +284,7 @@ void AddKeyQuery( { // convertion is ugly, but for tests is OK auto cells = ToCells(keys); - auto buf = TSerializedCellVec::Serialize(cells); - request.Keys.emplace_back(buf); + request.Keys.emplace_back(cells); } template <typename TCellType> diff --git a/ydb/core/tx/datashard/datashard_ut_snapshot.cpp b/ydb/core/tx/datashard/datashard_ut_snapshot.cpp index 1256b07ec77..bc309da9f1a 100644 --- a/ydb/core/tx/datashard/datashard_ut_snapshot.cpp +++ b/ydb/core/tx/datashard/datashard_ut_snapshot.cpp @@ -2217,7 +2217,7 @@ Y_UNIT_TEST_SUITE(DataShardSnapshots) { TVector<TCell> key{ TCell::Make(ui32(2)) }; TVector<TCell> values{ TCell::Make(ui32(22)) }; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValues(TSerializedCellVec::Serialize(values)); rows->emplace_back(serializedKey, serializedValues); diff --git a/ydb/core/tx/datashard/datashard_ut_upload_rows.cpp b/ydb/core/tx/datashard/datashard_ut_upload_rows.cpp index d826ff4e91d..2ae7d02e43b 100644 --- a/ydb/core/tx/datashard/datashard_ut_upload_rows.cpp +++ b/ydb/core/tx/datashard/datashard_ut_upload_rows.cpp @@ -38,7 +38,7 @@ static void DoStartUploadTestRows( for (ui32 i = 0; i < 32; i++) { auto key = TVector<TCell>{TCell::Make(1 << i)}; auto value = TVector<TCell>{TCell::Make(i)}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -84,7 +84,7 @@ static TActorId DoStartUploadRows( for (const auto& kv : data) { auto key = TVector<TCell>{TCell::Make(kv.first)}; auto value = TVector<TCell>{TCell::Make(kv.second)}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -289,7 +289,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto value = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -354,7 +354,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto value = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -434,7 +434,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto value = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -532,7 +532,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto value = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -633,7 +633,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto value = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedValue = TSerializedCellVec::Serialize(value); rows->emplace_back(serializedKey, serializedValue); } @@ -685,7 +685,7 @@ Y_UNIT_TEST_SUITE(TTxDataShardUploadRows) { for (ui32 i = 1; i <= 9; i++) { auto key = TVector<TCell>{TCell::Make(ui32(i))}; auto extra = TVector<TCell>{TCell::Make(ui32(i * 10))}; - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); + TSerializedCellVec serializedKey(key); TString serializedExtra = TSerializedCellVec::Serialize(extra); rows->emplace_back(serializedKey, serializedExtra); } diff --git a/ydb/core/tx/schemeshard/ut_base.cpp b/ydb/core/tx/schemeshard/ut_base.cpp index 6cdbd2640ef..cce8f5bc2eb 100644 --- a/ydb/core/tx/schemeshard/ut_base.cpp +++ b/ydb/core/tx/schemeshard/ut_base.cpp @@ -9827,7 +9827,7 @@ Y_UNIT_TEST_SUITE(TSchemeShardTest) { } } - return NKikimr::TSerializedCellVec(NKikimr::TSerializedCellVec::Serialize(cells)); + return NKikimr::TSerializedCellVec(cells); } NKikimrTableStats::THistogram FillHistogram(const TVector<TVector<TString>>& keys) const { diff --git a/ydb/core/tx/schemeshard/ut_split_merge.cpp b/ydb/core/tx/schemeshard/ut_split_merge.cpp index acbcfb959ed..8c6b60feddb 100644 --- a/ydb/core/tx/schemeshard/ut_split_merge.cpp +++ b/ydb/core/tx/schemeshard/ut_split_merge.cpp @@ -413,7 +413,7 @@ using NFmt::TPrintableTypedCells; TString ToSerialized(ui64 key) { const auto cell = TCell::Make(key); - const TSerializedCellVec saved(TSerializedCellVec::Serialize(TArrayRef<const TCell>(&cell, 1))); + const TSerializedCellVec saved(TArrayRef<const TCell>(&cell, 1)); return TString(saved.GetBuffer()); } diff --git a/ydb/core/tx/tx_proxy/datareq.cpp b/ydb/core/tx/tx_proxy/datareq.cpp index 90444545da3..0f8fd2b2081 100644 --- a/ydb/core/tx/tx_proxy/datareq.cpp +++ b/ydb/core/tx/tx_proxy/datareq.cpp @@ -588,9 +588,9 @@ void TKeySpace::Initialize(bool ordered, TConstArrayRef<NScheme::TTypeInfo> keyTypes, const TTableRange &range) { - SpaceRange.From.Parse(TSerializedCellVec::Serialize(range.From)); + SpaceRange.From = TSerializedCellVec(range.From); SpaceRange.FromInclusive = range.InclusiveFrom; - SpaceRange.To.Parse(TSerializedCellVec::Serialize(range.To)); + SpaceRange.To = TSerializedCellVec(range.To); SpaceRange.ToInclusive = range.InclusiveTo; // +INF should not be included @@ -3047,7 +3047,7 @@ bool TDataReq::ParseRangeKey(const NKikimrMiniKQL::TParams &proto, break; } - buf.Parse(TSerializedCellVec::Serialize(key)); + buf = TSerializedCellVec(key); return true; } diff --git a/ydb/core/tx/tx_proxy/read_table_impl.cpp b/ydb/core/tx/tx_proxy/read_table_impl.cpp index 865a7acac72..17a6ba79b26 100644 --- a/ydb/core/tx/tx_proxy/read_table_impl.cpp +++ b/ydb/core/tx/tx_proxy/read_table_impl.cpp @@ -119,7 +119,7 @@ namespace { break; } - buf.Parse(TSerializedCellVec::Serialize(key)); + buf = TSerializedCellVec(key); return true; } diff --git a/ydb/core/tx/tx_proxy/resolvereq.cpp b/ydb/core/tx/tx_proxy/resolvereq.cpp index 6ad8cb64448..1fd439ac9e0 100644 --- a/ydb/core/tx/tx_proxy/resolvereq.cpp +++ b/ydb/core/tx/tx_proxy/resolvereq.cpp @@ -49,7 +49,7 @@ namespace { break; } - buf.Parse(TSerializedCellVec::Serialize(key)); + buf = TSerializedCellVec(key); return true; } diff --git a/ydb/core/tx/tx_proxy/upload_rows_common_impl.h b/ydb/core/tx/tx_proxy/upload_rows_common_impl.h index bb6a85122b7..2cd59aa10d7 100644 --- a/ydb/core/tx/tx_proxy/upload_rows_common_impl.h +++ b/ydb/core/tx/tx_proxy/upload_rows_common_impl.h @@ -75,8 +75,8 @@ public: TConstArrayRef<TCell> keyCells(&cells[0], KeySize); TConstArrayRef<TCell> valueCells(&cells[KeySize], cells.size() - KeySize); - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(keyCells)); - Rows.emplace_back(serializedKey, TSerializedCellVec::Serialize(valueCells)); + TSerializedCellVec serializedKey(keyCells); + Rows.emplace_back(std::move(serializedKey), TSerializedCellVec::Serialize(valueCells)); } float GetRuCost() const { @@ -817,7 +817,6 @@ private: } void FindMinMaxKeys() { - for (const auto& pair : GetRows()) { const auto& serializedKey = pair.first; diff --git a/ydb/services/ext_index/service/add_index.cpp b/ydb/services/ext_index/service/add_index.cpp index 83b7950b1cc..a6f7c45bcfb 100644 --- a/ydb/services/ext_index/service/add_index.cpp +++ b/ydb/services/ext_index/service/add_index.cpp @@ -35,8 +35,8 @@ public: for (auto&& c : cells) { key.emplace_back(c); } - TSerializedCellVec serializedKey(TSerializedCellVec::Serialize(key)); - Rows->emplace_back(serializedKey, ""); + TSerializedCellVec serializedKey(key); + Rows->emplace_back(std::move(serializedKey), ""); ++Index; } }; |