aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMaksim Kita <kitaetoya@gmail.com>2023-06-30 08:50:02 +0000
committermaksim-kita <maksim-kita@yandex-team.com>2023-06-30 11:50:02 +0300
commitb1acffe1a8d5d2af1b0098f4e259e924ea33e3a3 (patch)
treeea7e905c2ecd3543add6f36fa3b8d4d456e20995
parent5c695412dab36725221e24bb4c5031789623bc70 (diff)
downloadydb-b1acffe1a8d5d2af1b0098f4e259e924ea33e3a3.tar.gz
SerializedCellVec improve serialization
SerializedCellVec improve serialization Pull Request resolved: #275
-rw-r--r--ydb/core/grpc_services/rpc_load_rows.cpp4
-rw-r--r--ydb/core/grpc_services/rpc_read_columns.cpp4
-rw-r--r--ydb/core/kqp/common/kqp_resolve.cpp4
-rw-r--r--ydb/core/kqp/executer_actor/kqp_partition_helper.cpp6
-rw-r--r--ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp2
-rw-r--r--ydb/core/load_test/ut_ycsb.cpp2
-rw-r--r--ydb/core/load_test/ycsb/common.cpp5
-rw-r--r--ydb/core/scheme/scheme_tablecell.cpp122
-rw-r--r--ydb/core/scheme/scheme_tablecell.h95
-rw-r--r--ydb/core/scheme/scheme_tablecell_ut.cpp2
-rw-r--r--ydb/core/scheme/scheme_tabledefs.h4
-rw-r--r--ydb/core/tablet_flat/flat_comp_shard.cpp12
-rw-r--r--ydb/core/tablet_flat/flat_part_keys.h2
-rw-r--r--ydb/core/tablet_flat/flat_part_slice.cpp4
-rw-r--r--ydb/core/tablet_flat/flat_part_writer.h4
-rw-r--r--ydb/core/tablet_flat/flat_scan_lead.h4
-rw-r--r--ydb/core/tablet_flat/flat_table_part_ut.cpp6
-rw-r--r--ydb/core/tablet_flat/ut/ut_comp_shard.cpp38
-rw-r--r--ydb/core/tablet_flat/ut/ut_slice.cpp4
-rw-r--r--ydb/core/tx/datashard/build_index.cpp4
-rw-r--r--ydb/core/tx/datashard/cdc_stream_scan.cpp4
-rw-r--r--ydb/core/tx/datashard/datashard__read_iterator.cpp2
-rw-r--r--ydb/core/tx/datashard/datashard_distributed_erase.cpp2
-rw-r--r--ydb/core/tx/datashard/datashard_ut_read_iterator.cpp3
-rw-r--r--ydb/core/tx/datashard/datashard_ut_snapshot.cpp2
-rw-r--r--ydb/core/tx/datashard/datashard_ut_upload_rows.cpp16
-rw-r--r--ydb/core/tx/schemeshard/ut_base.cpp2
-rw-r--r--ydb/core/tx/schemeshard/ut_split_merge.cpp2
-rw-r--r--ydb/core/tx/tx_proxy/datareq.cpp6
-rw-r--r--ydb/core/tx/tx_proxy/read_table_impl.cpp2
-rw-r--r--ydb/core/tx/tx_proxy/resolvereq.cpp2
-rw-r--r--ydb/core/tx/tx_proxy/upload_rows_common_impl.h5
-rw-r--r--ydb/services/ext_index/service/add_index.cpp4
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 &reg) {
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;
}
};