Mas d34 ms.i446 plusplus (#448)

* Minor optimisations

Try to reduce the calls to ++, and ensure that where possible the shorted list is being copied.

* Pass Acc into function

So that the list can be accumulated efficiently, without an additional copy to add back the accumulator at the end.

* prepend to accumulators

Code review to make sure we prepend to accumulators everywhere, to reduce the copying involved.

attempt to further optimise in leveled_sst (where most expensive ++ occurs).  This optimises for the case when Acc is [], and enforces a series of '++' to start from the right, prepending in turn.  Some shell testing indicated that this was not necessarily the case (although this doesn't seem tobe consistently reproducible).

```
6> element(1, timer:tc(fun() -> KL1 ++ KL2 ++ KL3 ++ KL4 end)).
28
7> element(1, timer:tc(fun() -> KL1 ++ KL2 ++ KL3 ++ KL4 end)).
174
8> element(1, timer:tc(fun() -> KL1 ++ KL2 ++ KL3 ++ KL4 end)).
96
9> element(1, timer:tc(fun() -> KL1 ++ KL2 ++ KL3 ++ KL4 end)).
106
10> element(1, timer:tc(fun() -> KL1 ++ KL2 ++ KL3 ++ KL4 end)).
112

17> element(1, timer:tc(fun() -> lists:foldr(fun(KL0, KLAcc) -> KL0 ++ KLAcc end, [], [KL1, KL2, KL3, KL4]) end)).
21
18> element(1, timer:tc(fun() -> lists:foldr(fun(KL0, KLAcc) -> KL0 ++ KLAcc end, [], [KL1, KL2, KL3, KL4]) end)).
17
19> element(1, timer:tc(fun() -> lists:foldr(fun(KL0, KLAcc) -> KL0 ++ KLAcc end, [], [KL1, KL2, KL3, KL4]) end)).
12
20> element(1, timer:tc(fun() -> lists:foldr(fun(KL0, KLAcc) -> KL0 ++ KLAcc end, [], [KL1, KL2, KL3, KL4]) end)).
11
```

running eprof indicates that '++' and lists:reverse have been reduced (however impact had only previously been 1-2%)

* Add unit test to confirm (limited) merit of optimised list function

No difference in unit test with/without inline compilation, so this has been removed

* Update src/leveled_sst.erl

These functions had previously used inline compilation - but this didn't appear to improve performance

Co-authored-by: Thomas Arts <thomas.arts@quviq.com>

* Update src/leveled_sst.erl

Co-authored-by: Thomas Arts <thomas.arts@quviq.com>

* Update src/leveled_ebloom.erl

Co-authored-by: Thomas Arts <thomas.arts@quviq.com>

* Update following review

Also fix code coverage issues

* Update src/leveled_sst.erl

Co-authored-by: Thomas Arts <thomas.arts@quviq.com>

---------

Co-authored-by: Thomas Arts <thomas.arts@quviq.com>
This commit is contained in:
Martin Sumner 2024-09-05 15:08:05 +01:00 committed by GitHub
parent 30ec9214ac
commit 5db277b82d
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
8 changed files with 311 additions and 176 deletions

View file

@ -1722,11 +1722,11 @@ build_hashtree_binary(SlotMap, IndexLength) ->
build_hashtree_binary([], IdxLen, SlotPos, Bin) -> build_hashtree_binary([], IdxLen, SlotPos, Bin) ->
case SlotPos of case SlotPos of
IdxLen -> N when N == IdxLen ->
lists:reverse(Bin); Bin;
N when N < IdxLen -> N when N < IdxLen ->
ZeroLen = (IdxLen - N) * 64, ZeroLen = (IdxLen - N) * 64,
lists:reverse([<<0:ZeroLen>>|Bin]) [<<0:ZeroLen>>|Bin]
end; end;
build_hashtree_binary([{TopSlot, TopBin}|SlotMapTail], IdxLen, SlotPos, Bin) -> build_hashtree_binary([{TopSlot, TopBin}|SlotMapTail], IdxLen, SlotPos, Bin) ->
case TopSlot of case TopSlot of
@ -1774,7 +1774,7 @@ write_hash_tables([], _HashTree, _CurrPos, _BasePos,
IndexList, HT_BinList, {T1, T2, T3}) -> IndexList, HT_BinList, {T1, T2, T3}) ->
leveled_log:log(cdb14, [T1, T2, T3]), leveled_log:log(cdb14, [T1, T2, T3]),
IL = lists:reverse(IndexList), IL = lists:reverse(IndexList),
{IL, list_to_binary(HT_BinList)}; {IL, list_to_binary(lists:reverse(HT_BinList))};
write_hash_tables([Index|Rest], HashTree, CurrPos, BasePos, write_hash_tables([Index|Rest], HashTree, CurrPos, BasePos,
IndexList, HT_BinList, Timers) -> IndexList, HT_BinList, Timers) ->
SW1 = os:timestamp(), SW1 = os:timestamp(),
@ -1782,13 +1782,15 @@ write_hash_tables([Index|Rest], HashTree, CurrPos, BasePos,
T1 = timer:now_diff(os:timestamp(), SW1) + element(1, Timers), T1 = timer:now_diff(os:timestamp(), SW1) + element(1, Timers),
case SlotMap of case SlotMap of
[] -> [] ->
write_hash_tables(Rest, write_hash_tables(
HashTree, Rest,
CurrPos, HashTree,
BasePos, CurrPos,
[{Index, BasePos, 0}|IndexList], BasePos,
HT_BinList, [{Index, BasePos, 0}|IndexList],
Timers); HT_BinList,
Timers
);
_ -> _ ->
SW2 = os:timestamp(), SW2 = os:timestamp(),
IndexLength = length(SlotMap) * 2, IndexLength = length(SlotMap) * 2,
@ -1797,13 +1799,15 @@ write_hash_tables([Index|Rest], HashTree, CurrPos, BasePos,
SW3 = os:timestamp(), SW3 = os:timestamp(),
NewSlotBin = build_hashtree_binary(SortedMap, IndexLength), NewSlotBin = build_hashtree_binary(SortedMap, IndexLength),
T3 = timer:now_diff(os:timestamp(), SW3) + element(3, Timers), T3 = timer:now_diff(os:timestamp(), SW3) + element(3, Timers),
write_hash_tables(Rest, write_hash_tables(
HashTree, Rest,
CurrPos + IndexLength * ?DWORD_SIZE, HashTree,
BasePos, CurrPos + IndexLength * ?DWORD_SIZE,
[{Index, CurrPos, IndexLength}|IndexList], BasePos,
HT_BinList ++ NewSlotBin, [{Index, CurrPos, IndexLength}|IndexList],
{T1, T2, T3}) lists:append(NewSlotBin, HT_BinList),
{T1, T2, T3}
)
end. end.
@ -1949,7 +1953,12 @@ build_hashtree_bunchedatend_binary_test() ->
{14, <<15:32, 500:32>>}, {14, <<15:32, 500:32>>},
{15, <<16:32, 600:32>>}, {15, <<16:32, 600:32>>},
{15, <<17:32, 700:32>>}], {15, <<17:32, 700:32>>}],
Bin = list_to_binary(build_hashtree_binary(SlotMap, 16)), Bin =
list_to_binary(
lists:reverse(
build_hashtree_binary(SlotMap, 16)
)
),
ExpBinP1 = <<16:32, 600:32, 10:32, 0:32, 17:32, 700:32, 0:64>>, ExpBinP1 = <<16:32, 600:32, 10:32, 0:32, 17:32, 700:32, 0:64>>,
ExpBinP2 = <<11:32, 100:32, 0:192, 12:32, 200:32, 13:32, 300:32, 0:256>>, ExpBinP2 = <<11:32, 100:32, 0:192, 12:32, 200:32, 13:32, 300:32, 0:256>>,
ExpBinP3 = <<14:32, 400:32, 15:32, 500:32>>, ExpBinP3 = <<14:32, 400:32, 15:32, 500:32>>,
@ -1965,7 +1974,12 @@ build_hashtree_bunchedatstart_binary_test() ->
{6, <<15:32, 500:32>>}, {6, <<15:32, 500:32>>},
{7, <<16:32, 600:32>>}, {7, <<16:32, 600:32>>},
{8, <<17:32, 700:32>>}], {8, <<17:32, 700:32>>}],
Bin = list_to_binary(build_hashtree_binary(SlotMap, 16)), Bin =
list_to_binary(
lists:reverse(
build_hashtree_binary(SlotMap, 16)
)
),
ExpBinP1 = <<0:64, 10:32, 0:32, 11:32, 100:32, 12:32, 200:32>>, ExpBinP1 = <<0:64, 10:32, 0:32, 11:32, 100:32, 12:32, 200:32>>,
ExpBinP2 = <<13:32, 300:32, 14:32, 400:32, 15:32, 500:32, 16:32, 600:32>>, ExpBinP2 = <<13:32, 300:32, 14:32, 400:32, 15:32, 500:32, 16:32, 600:32>>,
ExpBinP3 = <<17:32, 700:32, 0:448>>, ExpBinP3 = <<17:32, 700:32, 0:448>>,
@ -1988,7 +2002,7 @@ build_hashtree_test() ->
[<<2424915712:32, 300:32>>, <<0:64>>] ++ [<<2424915712:32, 300:32>>, <<0:64>>] ++
[<<2424903936:32, 400:32>>, <<2424907008:32, 500:32>>] ++ [<<2424903936:32, 400:32>>, <<2424907008:32, 500:32>>] ++
[<<2424913408:32, 600:32>>], [<<2424913408:32, 600:32>>],
?assertMatch(ExpOut, BinList). ?assertMatch(ExpOut, lists:reverse(BinList)).
find_firstzero_test() -> find_firstzero_test() ->

View file

@ -92,11 +92,11 @@ check_hash({_SegHash, Hash}, BloomBin) when is_binary(BloomBin)->
map_hashes([], HashListTuple, _SlotCount) -> map_hashes([], HashListTuple, _SlotCount) ->
HashListTuple; HashListTuple;
map_hashes([Hash|Rest], HashListTuple, SlotCount) -> map_hashes([Hash|Rest], HashListTuple, SlotCount) ->
{Slot, Hashes} = split_hash(element(2, Hash), SlotCount), {Slot, [H0, H1]} = split_hash(element(2, Hash), SlotCount),
SlotHL = element(Slot + 1, HashListTuple), SlotHL = element(Slot + 1, HashListTuple),
map_hashes( map_hashes(
Rest, Rest,
setelement(Slot + 1, HashListTuple, Hashes ++ SlotHL), setelement(Slot + 1, HashListTuple, [H0, H1 | SlotHL]),
SlotCount). SlotCount).
-spec split_hash(external_hash(), slot_count()) -spec split_hash(external_hash(), slot_count())

View file

@ -421,9 +421,9 @@ decode_maybe_binary(<<0, Bin/binary>>) ->
decode_maybe_binary(<<_Other:8, Bin/binary>>) -> decode_maybe_binary(<<_Other:8, Bin/binary>>) ->
Bin. Bin.
-spec diff_index_data([{binary(), index_value()}], -spec diff_index_data(
[{binary(), index_value()}]) -> [{binary(), index_value()}], [{binary(), index_value()}]) ->
[{index_op(), binary(), index_value()}]. [{index_op(), binary(), index_value()}].
diff_index_data(OldIndexes, AllIndexes) -> diff_index_data(OldIndexes, AllIndexes) ->
OldIndexSet = ordsets:from_list(OldIndexes), OldIndexSet = ordsets:from_list(OldIndexes),
AllIndexSet = ordsets:from_list(AllIndexes), AllIndexSet = ordsets:from_list(AllIndexes),
@ -435,18 +435,20 @@ diff_specs_core(AllIndexSet, OldIndexSet) ->
RemoveIndexSet = RemoveIndexSet =
ordsets:subtract(OldIndexSet, AllIndexSet), ordsets:subtract(OldIndexSet, AllIndexSet),
NewIndexSpecs = NewIndexSpecs =
assemble_index_specs(ordsets:subtract(NewIndexSet, OldIndexSet), assemble_index_specs(
add), ordsets:subtract(NewIndexSet, OldIndexSet),
add
),
RemoveIndexSpecs = RemoveIndexSpecs =
assemble_index_specs(RemoveIndexSet, assemble_index_specs(RemoveIndexSet, remove),
remove),
NewIndexSpecs ++ RemoveIndexSpecs. NewIndexSpecs ++ RemoveIndexSpecs.
%% @doc Assemble a list of index specs in the %% @doc Assemble a list of index specs in the
%% form of triplets of the form %% form of triplets of the form
%% {IndexOperation, IndexField, IndexValue}. %% {IndexOperation, IndexField, IndexValue}.
-spec assemble_index_specs([{binary(), binary()}], index_op()) -> -spec assemble_index_specs(
[{index_op(), binary(), binary()}]. [{binary(), binary()}], index_op()) ->
[{index_op(), binary(), binary()}].
assemble_index_specs(Indexes, IndexOp) -> assemble_index_specs(Indexes, IndexOp) ->
[{IndexOp, Index, Value} || {Index, Value} <- Indexes]. [{IndexOp, Index, Value} || {Index, Value} <- Indexes].

View file

@ -217,10 +217,13 @@ merge(SrcLevel, Manifest, RootPath, OptsSST) ->
Src = Src =
leveled_pmanifest:mergefile_selector(Manifest, SrcLevel, SelectMethod), leveled_pmanifest:mergefile_selector(Manifest, SrcLevel, SelectMethod),
NewSQN = leveled_pmanifest:get_manifest_sqn(Manifest) + 1, NewSQN = leveled_pmanifest:get_manifest_sqn(Manifest) + 1,
SinkList = leveled_pmanifest:merge_lookup(Manifest, SinkList =
SrcLevel + 1, leveled_pmanifest:merge_lookup(
Src#manifest_entry.start_key, Manifest,
Src#manifest_entry.end_key), SrcLevel + 1,
Src#manifest_entry.start_key,
Src#manifest_entry.end_key
),
Candidates = length(SinkList), Candidates = length(SinkList),
leveled_log:log(pc008, [SrcLevel, Candidates]), leveled_log:log(pc008, [SrcLevel, Candidates]),
case Candidates of case Candidates of
@ -228,10 +231,13 @@ merge(SrcLevel, Manifest, RootPath, OptsSST) ->
NewLevel = SrcLevel + 1, NewLevel = SrcLevel + 1,
leveled_log:log(pc009, [Src#manifest_entry.filename, NewLevel]), leveled_log:log(pc009, [Src#manifest_entry.filename, NewLevel]),
leveled_sst:sst_switchlevels(Src#manifest_entry.owner, NewLevel), leveled_sst:sst_switchlevels(Src#manifest_entry.owner, NewLevel),
Man0 = leveled_pmanifest:switch_manifest_entry(Manifest, Man0 =
NewSQN, leveled_pmanifest:switch_manifest_entry(
SrcLevel, Manifest,
Src), NewSQN,
SrcLevel,
Src
),
{Man0, []}; {Man0, []};
_ -> _ ->
SST_RP = leveled_penciller:sst_rootpath(RootPath), SST_RP = leveled_penciller:sst_rootpath(RootPath),
@ -253,44 +259,50 @@ notify_deletions([Head|Tail], Penciller) ->
%% %%
%% SrcLevel is the level of the src sst file, the sink should be srcLevel + 1 %% SrcLevel is the level of the src sst file, the sink should be srcLevel + 1
perform_merge(Manifest, perform_merge(Manifest, Src, SinkList, SrcLevel, RootPath, NewSQN, OptsSST) ->
Src, SinkList, SrcLevel,
RootPath, NewSQN,
OptsSST) ->
leveled_log:log(pc010, [Src#manifest_entry.filename, NewSQN]), leveled_log:log(pc010, [Src#manifest_entry.filename, NewSQN]),
SrcList = [{next, Src, all}], SrcList = [{next, Src, all}],
MaxSQN = leveled_sst:sst_getmaxsequencenumber(Src#manifest_entry.owner), MaxSQN = leveled_sst:sst_getmaxsequencenumber(Src#manifest_entry.owner),
SinkLevel = SrcLevel + 1, SinkLevel = SrcLevel + 1,
SinkBasement = leveled_pmanifest:is_basement(Manifest, SinkLevel), SinkBasement = leveled_pmanifest:is_basement(Manifest, SinkLevel),
Additions = Additions =
do_merge(SrcList, SinkList, do_merge(
SinkLevel, SinkBasement, SrcList, SinkList,
RootPath, NewSQN, MaxSQN, SinkLevel, SinkBasement,
OptsSST, RootPath, NewSQN, MaxSQN,
[]), OptsSST,
[]
),
RevertPointerFun = RevertPointerFun =
fun({next, ME, _SK}) -> fun({next, ME, _SK}) ->
ME ME
end, end,
SinkManifestList = lists:map(RevertPointerFun, SinkList), SinkManifestList = lists:map(RevertPointerFun, SinkList),
Man0 = leveled_pmanifest:replace_manifest_entry(Manifest, Man0 =
NewSQN, leveled_pmanifest:replace_manifest_entry(
SinkLevel, Manifest,
SinkManifestList, NewSQN,
Additions), SinkLevel,
Man2 = leveled_pmanifest:remove_manifest_entry(Man0, SinkManifestList,
NewSQN, Additions
SrcLevel, ),
Src), Man2 =
leveled_pmanifest:remove_manifest_entry(
Man0,
NewSQN,
SrcLevel,
Src
),
{Man2, [Src|SinkManifestList]}. {Man2, [Src|SinkManifestList]}.
do_merge([], [], SinkLevel, _SinkB, _RP, NewSQN, _MaxSQN, _Opts, Additions) -> do_merge([], [], SinkLevel, _SinkB, _RP, NewSQN, _MaxSQN, _Opts, Additions) ->
leveled_log:log(pc011, [NewSQN, SinkLevel, length(Additions)]), leveled_log:log(pc011, [NewSQN, SinkLevel, length(Additions)]),
Additions; lists:reverse(Additions);
do_merge(KL1, KL2, SinkLevel, SinkB, RP, NewSQN, MaxSQN, OptsSST, Additions) -> do_merge(KL1, KL2, SinkLevel, SinkB, RP, NewSQN, MaxSQN, OptsSST, Additions) ->
FileName = leveled_penciller:sst_filename(NewSQN, FileName =
SinkLevel, leveled_penciller:sst_filename(
length(Additions)), NewSQN, SinkLevel, length(Additions)
),
leveled_log:log(pc012, [NewSQN, FileName, SinkB]), leveled_log:log(pc012, [NewSQN, FileName, SinkB]),
TS1 = os:timestamp(), TS1 = os:timestamp(),
case leveled_sst:sst_newmerge(RP, FileName, case leveled_sst:sst_newmerge(RP, FileName,
@ -298,24 +310,31 @@ do_merge(KL1, KL2, SinkLevel, SinkB, RP, NewSQN, MaxSQN, OptsSST, Additions) ->
OptsSST) of OptsSST) of
empty -> empty ->
leveled_log:log(pc013, [FileName]), leveled_log:log(pc013, [FileName]),
do_merge([], [], do_merge(
SinkLevel, SinkB, [], [],
RP, NewSQN, MaxSQN, SinkLevel, SinkB,
OptsSST, RP, NewSQN, MaxSQN,
Additions); OptsSST,
Additions
);
{ok, Pid, Reply, Bloom} -> {ok, Pid, Reply, Bloom} ->
{{KL1Rem, KL2Rem}, SmallestKey, HighestKey} = Reply, {{KL1Rem, KL2Rem}, SmallestKey, HighestKey} = Reply,
Entry = #manifest_entry{start_key=SmallestKey, Entry =
end_key=HighestKey, #manifest_entry{
owner=Pid, start_key=SmallestKey,
filename=FileName, end_key=HighestKey,
bloom=Bloom}, owner=Pid,
filename=FileName,
bloom=Bloom
},
leveled_log:log_timer(pc015, [], TS1), leveled_log:log_timer(pc015, [], TS1),
do_merge(KL1Rem, KL2Rem, do_merge(
SinkLevel, SinkB, KL1Rem, KL2Rem,
RP, NewSQN, MaxSQN, SinkLevel, SinkB,
OptsSST, RP, NewSQN, MaxSQN,
Additions ++ [Entry]) OptsSST,
[Entry|Additions]
)
end. end.
-spec grooming_scorer( -spec grooming_scorer(

View file

@ -2002,6 +2002,9 @@ format_status_test() ->
?assertMatch(redacted, ST#state.levelzero_cache), ?assertMatch(redacted, ST#state.levelzero_cache),
?assertMatch(redacted, ST#state.levelzero_index), ?assertMatch(redacted, ST#state.levelzero_index),
?assertMatch(redacted, ST#state.levelzero_astree), ?assertMatch(redacted, ST#state.levelzero_astree),
NormStatus = format_status(#{reason => normal, state => S}),
NST = maps:get(state, NormStatus),
?assert(is_integer(array:size(element(2, NST#state.manifest)))),
clean_testdir(RootPath). clean_testdir(RootPath).
close_no_crash_test_() -> close_no_crash_test_() ->

View file

@ -57,7 +57,6 @@
%% the transaction log replay; or by using a higher level for of anti-entropy %% the transaction log replay; or by using a higher level for of anti-entropy
%% (i.e. make Riak responsible). %% (i.e. make Riak responsible).
-module(leveled_sst). -module(leveled_sst).
-behaviour(gen_statem). -behaviour(gen_statem).
@ -1395,41 +1394,28 @@ fetch(LedgerKey, Hash,
%% Fetch pointers to the slots the SST file covered by a given key range. %% Fetch pointers to the slots the SST file covered by a given key range.
fetch_range(StartKey, EndKey, Summary, FilterFun, true) -> fetch_range(StartKey, EndKey, Summary, FilterFun, true) ->
{Slots, RTrim} = {Slots, RTrim} =
lookup_slots( lookup_slots(StartKey, EndKey, Summary#summary.index, FilterFun),
StartKey,
EndKey,
Summary#summary.index,
FilterFun),
Self = self(), Self = self(),
SL = length(Slots), case {Slots, if not RTrim -> all; RTrim -> EndKey end} of
case SL of {[Slot], LastKey} ->
1 -> [{pointer, Self, Slot, StartKey, LastKey}];
[Slot] = Slots, {[Hd|Rest], all} ->
case RTrim of RightPointers =
true -> lists:map(
[{pointer, Self, Slot, StartKey, EndKey}]; fun(S) -> {pointer, Self, S, all, all} end,
false -> Rest),
[{pointer, Self, Slot, StartKey, all}] [{pointer, Self, Hd, StartKey, all}|RightPointers];
end; {[Hd|Rest], LastKey} ->
N -> {MidSlots, [Last]} = lists:split(length(Rest) - 1, Rest),
{LSlot, MidSlots, RSlot} =
{hd(Slots), lists:sublist(Slots, 2, N - 2), lists:last(Slots)},
MidSlotPointers = MidSlotPointers =
lists:map( lists:map(
fun(S) -> {pointer, Self, S, all, all} end, fun(S) -> {pointer, Self, S, all, all} end,
MidSlots), MidSlots),
case RTrim of [{pointer, Self, Hd, StartKey, all}|MidSlotPointers]
true -> ++ [{pointer, Self, Last, all, LastKey}]
[{pointer, Self, LSlot, StartKey, all}] ++
MidSlotPointers ++
[{pointer, Self, RSlot, all, EndKey}];
false ->
[{pointer, Self, LSlot, StartKey, all}] ++
MidSlotPointers ++
[{pointer, Self, RSlot, all, all}]
end
end; end;
fetch_range(_StartKey, _EndKey, _Summary, _FilterFun, false) -> fetch_range(_StartKey, _EndKey, _Summary, _FilterFun, false) ->
%% This has been pre-checked to be uninteresting (i.e. due to modified date)
[]. [].
-spec compress_level( -spec compress_level(
@ -2224,7 +2210,11 @@ read_slots(Handle, SlotList, {SegChecker, LowLastMod, BlockIndexCache},
% If there is an attempt to use the seg list query and the % If there is an attempt to use the seg list query and the
% index block cache isn't cached for any part this may be % index block cache isn't cached for any part this may be
% slower as each slot will be read in turn % slower as each slot will be read in turn
{true, read_slotlist([Pointer], Handle) ++ Acc}; {
true,
append(
read_slotlist([Pointer], Handle), Acc)
};
{BlockLengths, LMD, BlockIdx} -> {BlockLengths, LMD, BlockIdx} ->
% If there is a BlockIndex cached then we can use it to % If there is a BlockIndex cached then we can use it to
% check to see if any of the expected segments are % check to see if any of the expected segments are
@ -2242,10 +2232,14 @@ read_slots(Handle, SlotList, {SegChecker, LowLastMod, BlockIndexCache},
case SegChecker of case SegChecker of
false -> false ->
% No SegChecker - need all the slot now % No SegChecker - need all the slot now
{NeededBlockIdx, {
read_slotlist([Pointer], Handle) ++ Acc NeededBlockIdx,
}; append(
SegChecker -> read_slotlist([Pointer], Handle),
Acc
)
};
_ ->
TrimmedKVL = TrimmedKVL =
checkblocks_segandrange( checkblocks_segandrange(
BlockIdx, BlockIdx,
@ -2255,7 +2249,7 @@ read_slots(Handle, SlotList, {SegChecker, LowLastMod, BlockIndexCache},
IdxModDate, IdxModDate,
SegChecker, SegChecker,
{SK, EK}), {SK, EK}),
{NeededBlockIdx, TrimmedKVL ++ Acc} {NeededBlockIdx, append(TrimmedKVL, Acc)}
end end
end end
end end
@ -2318,15 +2312,19 @@ binaryslot_reader(
% of get_kvreader calls. This means that slots which are only used in % of get_kvreader calls. This means that slots which are only used in
% range queries can still populate their block_index caches (on the FSM % range queries can still populate their block_index caches (on the FSM
% loop state), and those caches can be used for future queries. % loop state), and those caches can be used for future queries.
{Acc, BIAcc} = binaryslot_reader(
binaryslot_reader( lists:reverse(SlotBinsToFetch),
SlotBinsToFetch, PressMethod, IdxModDate, SegChecker, [], []), PressMethod,
{lists:reverse(lists:reverse(SlotsToPoint) ++ Acc), BIAcc}. IdxModDate,
SegChecker,
SlotsToPoint,
[]
).
binaryslot_reader([], _PressMethod, _IdxModDate, _SegChecker, Acc, BIAcc) -> binaryslot_reader([], _PressMethod, _IdxModDate, _SegChecker, Acc, BIAcc) ->
{Acc, BIAcc}; {Acc, BIAcc};
binaryslot_reader( binaryslot_reader(
[{SlotBin, ID, SK, EK}|Tail], [{SlotBin, ID, SK, EK}|Tail],
PressMethod, IdxModDate, SegChecker, Acc, BIAcc) -> PressMethod, IdxModDate, SegChecker, Acc, BIAcc) ->
% The start key and end key here, may not the start key and end key the % The start key and end key here, may not the start key and end key the
% application passed into the query. If the slot is known to lie entirely % application passed into the query. If the slot is known to lie entirely
@ -2335,17 +2333,18 @@ binaryslot_reader(
% entries in this slot to be trimmed from either or both sides. % entries in this slot to be trimmed from either or both sides.
{TrimmedL, BICache} = {TrimmedL, BICache} =
binaryslot_trimmed( binaryslot_trimmed(
SlotBin, SK, EK, PressMethod, IdxModDate, SegChecker), SlotBin, SK, EK, PressMethod, IdxModDate, SegChecker, Acc
),
binaryslot_reader( binaryslot_reader(
Tail, PressMethod, IdxModDate, SegChecker, Tail,
lists:reverse(TrimmedL) ++ Acc, [{ID, BICache}|BIAcc]); PressMethod,
binaryslot_reader(L, PressMethod, IdxModDate, SegChecker, Acc, BIAcc) -> IdxModDate,
{KVs, Tail} = lists:splitwith(fun(SR) -> tuple_size(SR) == 2 end, L), SegChecker,
% These entries must already have been filtered for membership inside any TrimmedL,
% range used in the query. [{ID, BICache}|BIAcc]
binaryslot_reader( );
Tail, PressMethod, IdxModDate, SegChecker, binaryslot_reader([{K, V}|Tail], PM, IMD, SC, Acc, BIAcc) ->
lists:reverse(KVs) ++ Acc, BIAcc). binaryslot_reader(Tail, PM, IMD, SC, [{K, V}|Acc], BIAcc).
read_length_list(Handle, LengthList) -> read_length_list(Handle, LengthList) ->
@ -2386,28 +2385,13 @@ binaryslot_get(FullBin, Key, Hash, PressMethod, IdxModDate) ->
none} none}
end. end.
-spec binaryslot_tolist(
-spec binaryslot_blockstolist(
list(non_neg_integer()),
binary(), binary(),
press_method(), press_method(),
list(leveled_codec:ledger_kv())) -> list(leveled_codec:ledger_kv()). boolean(),
binaryslot_blockstolist([], _Bin, _PressMethod, Acc) -> list(leveled_codec:ledger_kv()|expandable_pointer()))
Acc;
binaryslot_blockstolist([0|RestLengths], RestBin, PressMethod, Acc) ->
binaryslot_blockstolist(RestLengths, RestBin, PressMethod, Acc);
binaryslot_blockstolist([L|RestLengths], Bin, PressMethod, Acc) ->
<<Block:L/binary, RestBin/binary>> = Bin,
binaryslot_blockstolist(
RestLengths,
RestBin,
PressMethod,
Acc ++ deserialise_block(Block, PressMethod)).
-spec binaryslot_tolist(
binary(), press_method(), boolean())
-> list(leveled_codec:ledger_kv()). -> list(leveled_codec:ledger_kv()).
binaryslot_tolist(FullBin, PressMethod, IdxModDate) -> binaryslot_tolist(FullBin, PressMethod, IdxModDate, InitAcc) ->
case crc_check_slot(FullBin) of case crc_check_slot(FullBin) of
{Header, Blocks} -> {Header, Blocks} ->
{BlockLengths, _LMD, _PosBinIndex} = {BlockLengths, _LMD, _PosBinIndex} =
@ -2417,10 +2401,20 @@ binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
B3L:32/integer, B3L:32/integer,
B4L:32/integer, B4L:32/integer,
B5L:32/integer>> = BlockLengths, B5L:32/integer>> = BlockLengths,
binaryslot_blockstolist( <<B1:B1L/binary,
[B1L, B2L, B3L, B4L, B5L], Blocks, PressMethod, []); B2:B2L/binary,
B3:B3L/binary,
B4:B4L/binary,
B5:B5L/binary>> = Blocks,
lists:foldl(
fun(B, Acc) ->
append(deserialise_block(B, PressMethod), Acc)
end,
InitAcc,
[B5, B4, B3, B2, B1]
);
crc_wonky -> crc_wonky ->
[] InitAcc
end. end.
-spec binaryslot_trimmed( -spec binaryslot_trimmed(
@ -2429,16 +2423,19 @@ binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
range_endpoint(), range_endpoint(),
press_method(), press_method(),
boolean(), boolean(),
segment_check_fun()) -> segment_check_fun(),
list(leveled_codec:ledger_kv()|expandable_pointer())
) ->
{list(leveled_codec:ledger_kv()), {list(leveled_codec:ledger_kv()),
list({integer(), binary()})|none}. list({integer(), binary()})|none}.
%% @doc %% @doc
%% Must return a trimmed and reversed list of results in the range %% Must return a trimmed and reversed list of results in the range
binaryslot_trimmed( binaryslot_trimmed(
FullBin, all, all, PressMethod, IdxModDate, false) -> FullBin, all, all, PressMethod, IdxModDate, false, Acc) ->
{binaryslot_tolist(FullBin, PressMethod, IdxModDate), none}; {binaryslot_tolist(FullBin, PressMethod, IdxModDate, Acc), none};
binaryslot_trimmed( binaryslot_trimmed(
FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker) -> FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker, Acc
) ->
case {crc_check_slot(FullBin), SegmentChecker} of case {crc_check_slot(FullBin), SegmentChecker} of
% Get a trimmed list of keys in the slot based on the range, trying % Get a trimmed list of keys in the slot based on the range, trying
% to minimise the number of blocks which are deserialised by % to minimise the number of blocks which are deserialised by
@ -2459,7 +2456,7 @@ binaryslot_trimmed(
{StartKey, EndKey}, {StartKey, EndKey},
Block1, Block2, MidBlock, Block4, Block5, Block1, Block2, MidBlock, Block4, Block5,
PressMethod), PressMethod),
{TrimmedKVL, none}; {append(TrimmedKVL, Acc), none};
{{Header, _Blocks}, SegmentChecker} -> {{Header, _Blocks}, SegmentChecker} ->
{BlockLengths, _LMD, BlockIdx} = {BlockLengths, _LMD, BlockIdx} =
extract_header(Header, IdxModDate), extract_header(Header, IdxModDate),
@ -2472,9 +2469,9 @@ binaryslot_trimmed(
IdxModDate, IdxModDate,
SegmentChecker, SegmentChecker,
{StartKey, EndKey}), {StartKey, EndKey}),
{TrimmedKVL, Header}; {append(TrimmedKVL, Acc), Header};
{crc_wonky, _} -> {crc_wonky, _} ->
{[], none} {Acc, none}
end. end.
-spec blocks_required( -spec blocks_required(
@ -2487,27 +2484,35 @@ blocks_required(
case filterby_midblock( case filterby_midblock(
fetchends_rawblock(MidBlockList), {StartKey, EndKey}) of fetchends_rawblock(MidBlockList), {StartKey, EndKey}) of
empty -> empty ->
in_range(deserialise_block(B1, PressMethod), StartKey, EndKey) append(
++ in_range(deserialise_block(B2, PressMethod), StartKey, EndKey) in_range(deserialise_block(B1, PressMethod), StartKey, EndKey),
++ in_range(deserialise_block(B4, PressMethod), StartKey, EndKey) in_range(deserialise_block(B2, PressMethod), StartKey, EndKey),
++ in_range(deserialise_block(B5, PressMethod), StartKey, EndKey); in_range(deserialise_block(B4, PressMethod), StartKey, EndKey),
in_range(deserialise_block(B5, PressMethod), StartKey, EndKey)
);
all_blocks -> all_blocks ->
get_lefthand_blocks(B1, B2, PressMethod, StartKey) append(
++ MidBlockList get_lefthand_blocks(B1, B2, PressMethod, StartKey),
++ get_righthand_blocks(B4, B5, PressMethod, EndKey); MidBlockList,
get_righthand_blocks(B4, B5, PressMethod, EndKey)
);
lt_mid -> lt_mid ->
in_range( in_range(
get_lefthand_blocks(B1, B2, PressMethod, StartKey), get_lefthand_blocks(B1, B2, PressMethod, StartKey),
all, all,
EndKey); EndKey);
le_mid -> le_mid ->
get_lefthand_blocks(B1, B2, PressMethod, StartKey) append(
++ in_range(MidBlockList, all, EndKey); get_lefthand_blocks(B1, B2, PressMethod, StartKey),
in_range(MidBlockList, all, EndKey)
);
mid_only -> mid_only ->
in_range(MidBlockList, StartKey, EndKey); in_range(MidBlockList, StartKey, EndKey);
ge_mid -> ge_mid ->
in_range(MidBlockList, StartKey, all) append(
++ get_righthand_blocks(B4, B5, PressMethod, EndKey); in_range(MidBlockList, StartKey, all),
get_righthand_blocks(B4, B5, PressMethod, EndKey)
);
gt_mid -> gt_mid ->
in_range( in_range(
get_righthand_blocks(B4, B5, PressMethod, EndKey), get_righthand_blocks(B4, B5, PressMethod, EndKey),
@ -2695,6 +2700,34 @@ revert_position(Pos) ->
end end
end. end.
%%%============================================================================
%%% Optimised list functions
%%%============================================================================
%% @doc See eunit test append_performance_test_/0 and also
%% https://github.com/erlang/otp/pull/8743
%% On OTP 26.2.1 -
%% Time for plus plus 16565
%% Time for append 7453
%% Time for lists:append 16428
%% Time for right associative plus plus 15928
%% ... this is all about optimising the case where there is an empty list
%% on the RHS ... and the absolute benefit is marginal
-spec append(list(), list()) -> list().
append(L1, []) ->
L1;
append(L1, L2) ->
L1 ++ L2.
-spec append(list(), list(), list()) -> list().
append(L1, L2, L3) ->
append(L1, append(L2, L3)).
-spec append(list(), list(), list(), list()) -> list().
append(L1, L2, L3, L4) ->
append(L1, append(L2, L3, L4)).
%%%============================================================================ %%%============================================================================
%%% Merge Functions %%% Merge Functions
%%%============================================================================ %%%============================================================================
@ -3060,6 +3093,15 @@ maybelog_fetch_timing({Pid, _SlotFreq}, Level, Type, SW) ->
-define(TEST_AREA, "test/test_area/"). -define(TEST_AREA, "test/test_area/").
binaryslot_trimmed(
FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker) ->
binaryslot_trimmed(
FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker, []
).
binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
binaryslot_tolist(FullBin, PressMethod, IdxModDate, []).
sst_getkvrange(Pid, StartKey, EndKey, ScanWidth) -> sst_getkvrange(Pid, StartKey, EndKey, ScanWidth) ->
sst_getkvrange(Pid, StartKey, EndKey, ScanWidth, false, 0). sst_getkvrange(Pid, StartKey, EndKey, ScanWidth, false, 0).
@ -3146,11 +3188,60 @@ generate_indexkeys(Count, IndexList) ->
generate_indexkey(Term, Count) -> generate_indexkey(Term, Count) ->
IndexSpecs = [{add, "t1_int", Term}], IndexSpecs = [{add, "t1_int", Term}],
leveled_codec:idx_indexspecs(IndexSpecs, leveled_codec:idx_indexspecs(
"Bucket", IndexSpecs,
"Key" ++ integer_to_list(Count), "Bucket",
Count, "Key" ++ integer_to_list(Count),
infinity). Count,
infinity
).
append_performance_test_() ->
{timeout, 300, fun append_performance_tester/0}.
append_performance_tester() ->
KVL1 = generate_randomkeys(0, 128, 1, 4),
KVL2 = generate_randomkeys(129, 256, 1, 4),
KVL3 = generate_randomkeys(257, 384, 1, 4),
KVL4 = generate_randomkeys(385, 512, 1, 4),
SW0 = os:system_time(microsecond),
lists:foreach(
fun(_I) ->
_KVL = KVL1 ++ KVL2 ++ KVL3 ++ KVL4 ++ []
end,
lists:seq(1, 5000)
),
SW1 = os:system_time(microsecond),
lists:foreach(
fun(_I) ->
_KVL = append(KVL1, KVL2, KVL3, append(KVL4, []))
end,
lists:seq(1, 5000)
),
SW2 = os:system_time(microsecond),
lists:foreach(
fun(_I) ->
_KVL = lists:append([KVL1, KVL2, KVL3, KVL4, []])
end,
lists:seq(1, 5000)
),
SW3 = os:system_time(microsecond),
lists:foreach(
fun(_I) ->
_KVL = KVL1 ++ (KVL2 ++ (KVL3 ++ (KVL4 ++ [])))
end,
lists:seq(1, 5000)
),
SW4 = os:system_time(microsecond),
io:format(
user,
"~nTime for plus plus ~w "
"Time for inline append ~w "
"Time for lists:append ~w "
"Time for bidmas plus plus ~w~n",
[SW1 - SW0, SW2 - SW1, SW3 - SW2, SW4 - SW3]
).
tombcount_test() -> tombcount_test() ->
tombcount_tester(1), tombcount_tester(1),
@ -3957,6 +4048,9 @@ fetch_status_test() ->
ST = maps:get(state, Status), ST = maps:get(state, Status),
?assertMatch(redacted, ST#state.blockindex_cache), ?assertMatch(redacted, ST#state.blockindex_cache),
?assertMatch(redacted, ST#state.fetch_cache), ?assertMatch(redacted, ST#state.fetch_cache),
NormStatus = format_status(#{reason => normal, state => S}),
NST = maps:get(state, NormStatus),
?assert(is_integer(array:size(NST#state.fetch_cache))),
ok = sst_close(Pid), ok = sst_close(Pid),
ok = file:delete(filename:join(RP, Filename ++ ".sst")). ok = file:delete(filename:join(RP, Filename ++ ".sst")).

View file

@ -418,9 +418,9 @@ generate_segmentfilter_list(SegmentList, xsmall) ->
A1 = 1 bsl 14, A1 = 1 bsl 14,
ExpandSegFun = ExpandSegFun =
fun(X, Acc) -> fun(X, Acc) ->
Acc ++ [X, X + A0, X + A1, X + A0 + A1] [X, X + A0, X + A1, X + A0 + A1] ++ Acc
end, end,
lists:foldl(ExpandSegFun, [], SegmentList); lists:foldr(ExpandSegFun, [], SegmentList);
false -> false ->
false false
end; end;

View file

@ -5,12 +5,15 @@
-module(leveled_util). -module(leveled_util).
-export([generate_uuid/0, -export([
generate_uuid/0,
integer_now/0, integer_now/0,
integer_time/1, integer_time/1,
magic_hash/1, magic_hash/1,
t2b/1, t2b/1,
safe_rename/4]). safe_rename/4
]
).
-define(WRITE_OPS, [binary, raw, read, write]). -define(WRITE_OPS, [binary, raw, read, write]).