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:
parent
30ec9214ac
commit
5db277b82d
8 changed files with 311 additions and 176 deletions
|
@ -1722,11 +1722,11 @@ build_hashtree_binary(SlotMap, IndexLength) ->
|
|||
|
||||
build_hashtree_binary([], IdxLen, SlotPos, Bin) ->
|
||||
case SlotPos of
|
||||
IdxLen ->
|
||||
lists:reverse(Bin);
|
||||
N when N == IdxLen ->
|
||||
Bin;
|
||||
N when N < IdxLen ->
|
||||
ZeroLen = (IdxLen - N) * 64,
|
||||
lists:reverse([<<0:ZeroLen>>|Bin])
|
||||
[<<0:ZeroLen>>|Bin]
|
||||
end;
|
||||
build_hashtree_binary([{TopSlot, TopBin}|SlotMapTail], IdxLen, SlotPos, Bin) ->
|
||||
case TopSlot of
|
||||
|
@ -1774,7 +1774,7 @@ write_hash_tables([], _HashTree, _CurrPos, _BasePos,
|
|||
IndexList, HT_BinList, {T1, T2, T3}) ->
|
||||
leveled_log:log(cdb14, [T1, T2, T3]),
|
||||
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,
|
||||
IndexList, HT_BinList, Timers) ->
|
||||
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),
|
||||
case SlotMap of
|
||||
[] ->
|
||||
write_hash_tables(Rest,
|
||||
write_hash_tables(
|
||||
Rest,
|
||||
HashTree,
|
||||
CurrPos,
|
||||
BasePos,
|
||||
[{Index, BasePos, 0}|IndexList],
|
||||
HT_BinList,
|
||||
Timers);
|
||||
Timers
|
||||
);
|
||||
_ ->
|
||||
SW2 = os:timestamp(),
|
||||
IndexLength = length(SlotMap) * 2,
|
||||
|
@ -1797,13 +1799,15 @@ write_hash_tables([Index|Rest], HashTree, CurrPos, BasePos,
|
|||
SW3 = os:timestamp(),
|
||||
NewSlotBin = build_hashtree_binary(SortedMap, IndexLength),
|
||||
T3 = timer:now_diff(os:timestamp(), SW3) + element(3, Timers),
|
||||
write_hash_tables(Rest,
|
||||
write_hash_tables(
|
||||
Rest,
|
||||
HashTree,
|
||||
CurrPos + IndexLength * ?DWORD_SIZE,
|
||||
BasePos,
|
||||
[{Index, CurrPos, IndexLength}|IndexList],
|
||||
HT_BinList ++ NewSlotBin,
|
||||
{T1, T2, T3})
|
||||
lists:append(NewSlotBin, HT_BinList),
|
||||
{T1, T2, T3}
|
||||
)
|
||||
end.
|
||||
|
||||
|
||||
|
@ -1949,7 +1953,12 @@ build_hashtree_bunchedatend_binary_test() ->
|
|||
{14, <<15:32, 500:32>>},
|
||||
{15, <<16:32, 600: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>>,
|
||||
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>>,
|
||||
|
@ -1965,7 +1974,12 @@ build_hashtree_bunchedatstart_binary_test() ->
|
|||
{6, <<15:32, 500:32>>},
|
||||
{7, <<16:32, 600: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>>,
|
||||
ExpBinP2 = <<13:32, 300:32, 14:32, 400:32, 15:32, 500:32, 16:32, 600:32>>,
|
||||
ExpBinP3 = <<17:32, 700:32, 0:448>>,
|
||||
|
@ -1988,7 +2002,7 @@ build_hashtree_test() ->
|
|||
[<<2424915712:32, 300:32>>, <<0:64>>] ++
|
||||
[<<2424903936:32, 400:32>>, <<2424907008:32, 500:32>>] ++
|
||||
[<<2424913408:32, 600:32>>],
|
||||
?assertMatch(ExpOut, BinList).
|
||||
?assertMatch(ExpOut, lists:reverse(BinList)).
|
||||
|
||||
|
||||
find_firstzero_test() ->
|
||||
|
|
|
@ -92,11 +92,11 @@ check_hash({_SegHash, Hash}, BloomBin) when is_binary(BloomBin)->
|
|||
map_hashes([], HashListTuple, _SlotCount) ->
|
||||
HashListTuple;
|
||||
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),
|
||||
map_hashes(
|
||||
Rest,
|
||||
setelement(Slot + 1, HashListTuple, Hashes ++ SlotHL),
|
||||
setelement(Slot + 1, HashListTuple, [H0, H1 | SlotHL]),
|
||||
SlotCount).
|
||||
|
||||
-spec split_hash(external_hash(), slot_count())
|
||||
|
|
|
@ -421,8 +421,8 @@ decode_maybe_binary(<<0, Bin/binary>>) ->
|
|||
decode_maybe_binary(<<_Other:8, Bin/binary>>) ->
|
||||
Bin.
|
||||
|
||||
-spec diff_index_data([{binary(), index_value()}],
|
||||
[{binary(), index_value()}]) ->
|
||||
-spec diff_index_data(
|
||||
[{binary(), index_value()}], [{binary(), index_value()}]) ->
|
||||
[{index_op(), binary(), index_value()}].
|
||||
diff_index_data(OldIndexes, AllIndexes) ->
|
||||
OldIndexSet = ordsets:from_list(OldIndexes),
|
||||
|
@ -435,17 +435,19 @@ diff_specs_core(AllIndexSet, OldIndexSet) ->
|
|||
RemoveIndexSet =
|
||||
ordsets:subtract(OldIndexSet, AllIndexSet),
|
||||
NewIndexSpecs =
|
||||
assemble_index_specs(ordsets:subtract(NewIndexSet, OldIndexSet),
|
||||
add),
|
||||
assemble_index_specs(
|
||||
ordsets:subtract(NewIndexSet, OldIndexSet),
|
||||
add
|
||||
),
|
||||
RemoveIndexSpecs =
|
||||
assemble_index_specs(RemoveIndexSet,
|
||||
remove),
|
||||
assemble_index_specs(RemoveIndexSet, remove),
|
||||
NewIndexSpecs ++ RemoveIndexSpecs.
|
||||
|
||||
%% @doc Assemble a list of index specs in the
|
||||
%% form of triplets of the form
|
||||
%% {IndexOperation, IndexField, IndexValue}.
|
||||
-spec assemble_index_specs([{binary(), binary()}], index_op()) ->
|
||||
-spec assemble_index_specs(
|
||||
[{binary(), binary()}], index_op()) ->
|
||||
[{index_op(), binary(), binary()}].
|
||||
assemble_index_specs(Indexes, IndexOp) ->
|
||||
[{IndexOp, Index, Value} || {Index, Value} <- Indexes].
|
||||
|
|
|
@ -217,10 +217,13 @@ merge(SrcLevel, Manifest, RootPath, OptsSST) ->
|
|||
Src =
|
||||
leveled_pmanifest:mergefile_selector(Manifest, SrcLevel, SelectMethod),
|
||||
NewSQN = leveled_pmanifest:get_manifest_sqn(Manifest) + 1,
|
||||
SinkList = leveled_pmanifest:merge_lookup(Manifest,
|
||||
SinkList =
|
||||
leveled_pmanifest:merge_lookup(
|
||||
Manifest,
|
||||
SrcLevel + 1,
|
||||
Src#manifest_entry.start_key,
|
||||
Src#manifest_entry.end_key),
|
||||
Src#manifest_entry.end_key
|
||||
),
|
||||
Candidates = length(SinkList),
|
||||
leveled_log:log(pc008, [SrcLevel, Candidates]),
|
||||
case Candidates of
|
||||
|
@ -228,10 +231,13 @@ merge(SrcLevel, Manifest, RootPath, OptsSST) ->
|
|||
NewLevel = SrcLevel + 1,
|
||||
leveled_log:log(pc009, [Src#manifest_entry.filename, NewLevel]),
|
||||
leveled_sst:sst_switchlevels(Src#manifest_entry.owner, NewLevel),
|
||||
Man0 = leveled_pmanifest:switch_manifest_entry(Manifest,
|
||||
Man0 =
|
||||
leveled_pmanifest:switch_manifest_entry(
|
||||
Manifest,
|
||||
NewSQN,
|
||||
SrcLevel,
|
||||
Src),
|
||||
Src
|
||||
),
|
||||
{Man0, []};
|
||||
_ ->
|
||||
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
|
||||
|
||||
perform_merge(Manifest,
|
||||
Src, SinkList, SrcLevel,
|
||||
RootPath, NewSQN,
|
||||
OptsSST) ->
|
||||
perform_merge(Manifest, Src, SinkList, SrcLevel, RootPath, NewSQN, OptsSST) ->
|
||||
leveled_log:log(pc010, [Src#manifest_entry.filename, NewSQN]),
|
||||
SrcList = [{next, Src, all}],
|
||||
MaxSQN = leveled_sst:sst_getmaxsequencenumber(Src#manifest_entry.owner),
|
||||
SinkLevel = SrcLevel + 1,
|
||||
SinkBasement = leveled_pmanifest:is_basement(Manifest, SinkLevel),
|
||||
Additions =
|
||||
do_merge(SrcList, SinkList,
|
||||
do_merge(
|
||||
SrcList, SinkList,
|
||||
SinkLevel, SinkBasement,
|
||||
RootPath, NewSQN, MaxSQN,
|
||||
OptsSST,
|
||||
[]),
|
||||
[]
|
||||
),
|
||||
RevertPointerFun =
|
||||
fun({next, ME, _SK}) ->
|
||||
ME
|
||||
end,
|
||||
SinkManifestList = lists:map(RevertPointerFun, SinkList),
|
||||
Man0 = leveled_pmanifest:replace_manifest_entry(Manifest,
|
||||
Man0 =
|
||||
leveled_pmanifest:replace_manifest_entry(
|
||||
Manifest,
|
||||
NewSQN,
|
||||
SinkLevel,
|
||||
SinkManifestList,
|
||||
Additions),
|
||||
Man2 = leveled_pmanifest:remove_manifest_entry(Man0,
|
||||
Additions
|
||||
),
|
||||
Man2 =
|
||||
leveled_pmanifest:remove_manifest_entry(
|
||||
Man0,
|
||||
NewSQN,
|
||||
SrcLevel,
|
||||
Src),
|
||||
Src
|
||||
),
|
||||
{Man2, [Src|SinkManifestList]}.
|
||||
|
||||
do_merge([], [], SinkLevel, _SinkB, _RP, NewSQN, _MaxSQN, _Opts, Additions) ->
|
||||
leveled_log:log(pc011, [NewSQN, SinkLevel, length(Additions)]),
|
||||
Additions;
|
||||
lists:reverse(Additions);
|
||||
do_merge(KL1, KL2, SinkLevel, SinkB, RP, NewSQN, MaxSQN, OptsSST, Additions) ->
|
||||
FileName = leveled_penciller:sst_filename(NewSQN,
|
||||
SinkLevel,
|
||||
length(Additions)),
|
||||
FileName =
|
||||
leveled_penciller:sst_filename(
|
||||
NewSQN, SinkLevel, length(Additions)
|
||||
),
|
||||
leveled_log:log(pc012, [NewSQN, FileName, SinkB]),
|
||||
TS1 = os:timestamp(),
|
||||
case leveled_sst:sst_newmerge(RP, FileName,
|
||||
|
@ -298,24 +310,31 @@ do_merge(KL1, KL2, SinkLevel, SinkB, RP, NewSQN, MaxSQN, OptsSST, Additions) ->
|
|||
OptsSST) of
|
||||
empty ->
|
||||
leveled_log:log(pc013, [FileName]),
|
||||
do_merge([], [],
|
||||
do_merge(
|
||||
[], [],
|
||||
SinkLevel, SinkB,
|
||||
RP, NewSQN, MaxSQN,
|
||||
OptsSST,
|
||||
Additions);
|
||||
Additions
|
||||
);
|
||||
{ok, Pid, Reply, Bloom} ->
|
||||
{{KL1Rem, KL2Rem}, SmallestKey, HighestKey} = Reply,
|
||||
Entry = #manifest_entry{start_key=SmallestKey,
|
||||
Entry =
|
||||
#manifest_entry{
|
||||
start_key=SmallestKey,
|
||||
end_key=HighestKey,
|
||||
owner=Pid,
|
||||
filename=FileName,
|
||||
bloom=Bloom},
|
||||
bloom=Bloom
|
||||
},
|
||||
leveled_log:log_timer(pc015, [], TS1),
|
||||
do_merge(KL1Rem, KL2Rem,
|
||||
do_merge(
|
||||
KL1Rem, KL2Rem,
|
||||
SinkLevel, SinkB,
|
||||
RP, NewSQN, MaxSQN,
|
||||
OptsSST,
|
||||
Additions ++ [Entry])
|
||||
[Entry|Additions]
|
||||
)
|
||||
end.
|
||||
|
||||
-spec grooming_scorer(
|
||||
|
|
|
@ -2002,6 +2002,9 @@ format_status_test() ->
|
|||
?assertMatch(redacted, ST#state.levelzero_cache),
|
||||
?assertMatch(redacted, ST#state.levelzero_index),
|
||||
?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).
|
||||
|
||||
close_no_crash_test_() ->
|
||||
|
|
|
@ -57,7 +57,6 @@
|
|||
%% the transaction log replay; or by using a higher level for of anti-entropy
|
||||
%% (i.e. make Riak responsible).
|
||||
|
||||
|
||||
-module(leveled_sst).
|
||||
|
||||
-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_range(StartKey, EndKey, Summary, FilterFun, true) ->
|
||||
{Slots, RTrim} =
|
||||
lookup_slots(
|
||||
StartKey,
|
||||
EndKey,
|
||||
Summary#summary.index,
|
||||
FilterFun),
|
||||
lookup_slots(StartKey, EndKey, Summary#summary.index, FilterFun),
|
||||
Self = self(),
|
||||
SL = length(Slots),
|
||||
case SL of
|
||||
1 ->
|
||||
[Slot] = Slots,
|
||||
case RTrim of
|
||||
true ->
|
||||
[{pointer, Self, Slot, StartKey, EndKey}];
|
||||
false ->
|
||||
[{pointer, Self, Slot, StartKey, all}]
|
||||
end;
|
||||
N ->
|
||||
{LSlot, MidSlots, RSlot} =
|
||||
{hd(Slots), lists:sublist(Slots, 2, N - 2), lists:last(Slots)},
|
||||
case {Slots, if not RTrim -> all; RTrim -> EndKey end} of
|
||||
{[Slot], LastKey} ->
|
||||
[{pointer, Self, Slot, StartKey, LastKey}];
|
||||
{[Hd|Rest], all} ->
|
||||
RightPointers =
|
||||
lists:map(
|
||||
fun(S) -> {pointer, Self, S, all, all} end,
|
||||
Rest),
|
||||
[{pointer, Self, Hd, StartKey, all}|RightPointers];
|
||||
{[Hd|Rest], LastKey} ->
|
||||
{MidSlots, [Last]} = lists:split(length(Rest) - 1, Rest),
|
||||
MidSlotPointers =
|
||||
lists:map(
|
||||
fun(S) -> {pointer, Self, S, all, all} end,
|
||||
MidSlots),
|
||||
case RTrim of
|
||||
true ->
|
||||
[{pointer, Self, LSlot, StartKey, all}] ++
|
||||
MidSlotPointers ++
|
||||
[{pointer, Self, RSlot, all, EndKey}];
|
||||
false ->
|
||||
[{pointer, Self, LSlot, StartKey, all}] ++
|
||||
MidSlotPointers ++
|
||||
[{pointer, Self, RSlot, all, all}]
|
||||
end
|
||||
[{pointer, Self, Hd, StartKey, all}|MidSlotPointers]
|
||||
++ [{pointer, Self, Last, all, LastKey}]
|
||||
end;
|
||||
fetch_range(_StartKey, _EndKey, _Summary, _FilterFun, false) ->
|
||||
%% This has been pre-checked to be uninteresting (i.e. due to modified date)
|
||||
[].
|
||||
|
||||
-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
|
||||
% index block cache isn't cached for any part this may be
|
||||
% 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} ->
|
||||
% If there is a BlockIndex cached then we can use it to
|
||||
% check to see if any of the expected segments are
|
||||
|
@ -2242,10 +2232,14 @@ read_slots(Handle, SlotList, {SegChecker, LowLastMod, BlockIndexCache},
|
|||
case SegChecker of
|
||||
false ->
|
||||
% No SegChecker - need all the slot now
|
||||
{NeededBlockIdx,
|
||||
read_slotlist([Pointer], Handle) ++ Acc
|
||||
{
|
||||
NeededBlockIdx,
|
||||
append(
|
||||
read_slotlist([Pointer], Handle),
|
||||
Acc
|
||||
)
|
||||
};
|
||||
SegChecker ->
|
||||
_ ->
|
||||
TrimmedKVL =
|
||||
checkblocks_segandrange(
|
||||
BlockIdx,
|
||||
|
@ -2255,7 +2249,7 @@ read_slots(Handle, SlotList, {SegChecker, LowLastMod, BlockIndexCache},
|
|||
IdxModDate,
|
||||
SegChecker,
|
||||
{SK, EK}),
|
||||
{NeededBlockIdx, TrimmedKVL ++ Acc}
|
||||
{NeededBlockIdx, append(TrimmedKVL, Acc)}
|
||||
end
|
||||
end
|
||||
end
|
||||
|
@ -2318,10 +2312,14 @@ binaryslot_reader(
|
|||
% 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
|
||||
% loop state), and those caches can be used for future queries.
|
||||
{Acc, BIAcc} =
|
||||
binaryslot_reader(
|
||||
SlotBinsToFetch, PressMethod, IdxModDate, SegChecker, [], []),
|
||||
{lists:reverse(lists:reverse(SlotsToPoint) ++ Acc), BIAcc}.
|
||||
lists:reverse(SlotBinsToFetch),
|
||||
PressMethod,
|
||||
IdxModDate,
|
||||
SegChecker,
|
||||
SlotsToPoint,
|
||||
[]
|
||||
).
|
||||
|
||||
binaryslot_reader([], _PressMethod, _IdxModDate, _SegChecker, Acc, BIAcc) ->
|
||||
{Acc, BIAcc};
|
||||
|
@ -2335,17 +2333,18 @@ binaryslot_reader(
|
|||
% entries in this slot to be trimmed from either or both sides.
|
||||
{TrimmedL, BICache} =
|
||||
binaryslot_trimmed(
|
||||
SlotBin, SK, EK, PressMethod, IdxModDate, SegChecker),
|
||||
SlotBin, SK, EK, PressMethod, IdxModDate, SegChecker, Acc
|
||||
),
|
||||
binaryslot_reader(
|
||||
Tail, PressMethod, IdxModDate, SegChecker,
|
||||
lists:reverse(TrimmedL) ++ Acc, [{ID, BICache}|BIAcc]);
|
||||
binaryslot_reader(L, PressMethod, IdxModDate, SegChecker, Acc, BIAcc) ->
|
||||
{KVs, Tail} = lists:splitwith(fun(SR) -> tuple_size(SR) == 2 end, L),
|
||||
% These entries must already have been filtered for membership inside any
|
||||
% range used in the query.
|
||||
binaryslot_reader(
|
||||
Tail, PressMethod, IdxModDate, SegChecker,
|
||||
lists:reverse(KVs) ++ Acc, BIAcc).
|
||||
Tail,
|
||||
PressMethod,
|
||||
IdxModDate,
|
||||
SegChecker,
|
||||
TrimmedL,
|
||||
[{ID, BICache}|BIAcc]
|
||||
);
|
||||
binaryslot_reader([{K, V}|Tail], PM, IMD, SC, Acc, BIAcc) ->
|
||||
binaryslot_reader(Tail, PM, IMD, SC, [{K, V}|Acc], BIAcc).
|
||||
|
||||
|
||||
read_length_list(Handle, LengthList) ->
|
||||
|
@ -2386,28 +2385,13 @@ binaryslot_get(FullBin, Key, Hash, PressMethod, IdxModDate) ->
|
|||
none}
|
||||
end.
|
||||
|
||||
|
||||
-spec binaryslot_blockstolist(
|
||||
list(non_neg_integer()),
|
||||
-spec binaryslot_tolist(
|
||||
binary(),
|
||||
press_method(),
|
||||
list(leveled_codec:ledger_kv())) -> list(leveled_codec:ledger_kv()).
|
||||
binaryslot_blockstolist([], _Bin, _PressMethod, Acc) ->
|
||||
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())
|
||||
boolean(),
|
||||
list(leveled_codec:ledger_kv()|expandable_pointer()))
|
||||
-> list(leveled_codec:ledger_kv()).
|
||||
binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
|
||||
binaryslot_tolist(FullBin, PressMethod, IdxModDate, InitAcc) ->
|
||||
case crc_check_slot(FullBin) of
|
||||
{Header, Blocks} ->
|
||||
{BlockLengths, _LMD, _PosBinIndex} =
|
||||
|
@ -2417,10 +2401,20 @@ binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
|
|||
B3L:32/integer,
|
||||
B4L:32/integer,
|
||||
B5L:32/integer>> = BlockLengths,
|
||||
binaryslot_blockstolist(
|
||||
[B1L, B2L, B3L, B4L, B5L], Blocks, PressMethod, []);
|
||||
<<B1:B1L/binary,
|
||||
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 ->
|
||||
[]
|
||||
InitAcc
|
||||
end.
|
||||
|
||||
-spec binaryslot_trimmed(
|
||||
|
@ -2429,16 +2423,19 @@ binaryslot_tolist(FullBin, PressMethod, IdxModDate) ->
|
|||
range_endpoint(),
|
||||
press_method(),
|
||||
boolean(),
|
||||
segment_check_fun()) ->
|
||||
segment_check_fun(),
|
||||
list(leveled_codec:ledger_kv()|expandable_pointer())
|
||||
) ->
|
||||
{list(leveled_codec:ledger_kv()),
|
||||
list({integer(), binary()})|none}.
|
||||
%% @doc
|
||||
%% Must return a trimmed and reversed list of results in the range
|
||||
binaryslot_trimmed(
|
||||
FullBin, all, all, PressMethod, IdxModDate, false) ->
|
||||
{binaryslot_tolist(FullBin, PressMethod, IdxModDate), none};
|
||||
FullBin, all, all, PressMethod, IdxModDate, false, Acc) ->
|
||||
{binaryslot_tolist(FullBin, PressMethod, IdxModDate, Acc), none};
|
||||
binaryslot_trimmed(
|
||||
FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker) ->
|
||||
FullBin, StartKey, EndKey, PressMethod, IdxModDate, SegmentChecker, Acc
|
||||
) ->
|
||||
case {crc_check_slot(FullBin), SegmentChecker} of
|
||||
% Get a trimmed list of keys in the slot based on the range, trying
|
||||
% to minimise the number of blocks which are deserialised by
|
||||
|
@ -2459,7 +2456,7 @@ binaryslot_trimmed(
|
|||
{StartKey, EndKey},
|
||||
Block1, Block2, MidBlock, Block4, Block5,
|
||||
PressMethod),
|
||||
{TrimmedKVL, none};
|
||||
{append(TrimmedKVL, Acc), none};
|
||||
{{Header, _Blocks}, SegmentChecker} ->
|
||||
{BlockLengths, _LMD, BlockIdx} =
|
||||
extract_header(Header, IdxModDate),
|
||||
|
@ -2472,9 +2469,9 @@ binaryslot_trimmed(
|
|||
IdxModDate,
|
||||
SegmentChecker,
|
||||
{StartKey, EndKey}),
|
||||
{TrimmedKVL, Header};
|
||||
{append(TrimmedKVL, Acc), Header};
|
||||
{crc_wonky, _} ->
|
||||
{[], none}
|
||||
{Acc, none}
|
||||
end.
|
||||
|
||||
-spec blocks_required(
|
||||
|
@ -2487,27 +2484,35 @@ blocks_required(
|
|||
case filterby_midblock(
|
||||
fetchends_rawblock(MidBlockList), {StartKey, EndKey}) of
|
||||
empty ->
|
||||
in_range(deserialise_block(B1, PressMethod), StartKey, EndKey)
|
||||
++ in_range(deserialise_block(B2, PressMethod), StartKey, EndKey)
|
||||
++ in_range(deserialise_block(B4, PressMethod), StartKey, EndKey)
|
||||
++ in_range(deserialise_block(B5, PressMethod), StartKey, EndKey);
|
||||
append(
|
||||
in_range(deserialise_block(B1, PressMethod), StartKey, EndKey),
|
||||
in_range(deserialise_block(B2, PressMethod), StartKey, EndKey),
|
||||
in_range(deserialise_block(B4, PressMethod), StartKey, EndKey),
|
||||
in_range(deserialise_block(B5, PressMethod), StartKey, EndKey)
|
||||
);
|
||||
all_blocks ->
|
||||
get_lefthand_blocks(B1, B2, PressMethod, StartKey)
|
||||
++ MidBlockList
|
||||
++ get_righthand_blocks(B4, B5, PressMethod, EndKey);
|
||||
append(
|
||||
get_lefthand_blocks(B1, B2, PressMethod, StartKey),
|
||||
MidBlockList,
|
||||
get_righthand_blocks(B4, B5, PressMethod, EndKey)
|
||||
);
|
||||
lt_mid ->
|
||||
in_range(
|
||||
get_lefthand_blocks(B1, B2, PressMethod, StartKey),
|
||||
all,
|
||||
EndKey);
|
||||
le_mid ->
|
||||
get_lefthand_blocks(B1, B2, PressMethod, StartKey)
|
||||
++ in_range(MidBlockList, all, EndKey);
|
||||
append(
|
||||
get_lefthand_blocks(B1, B2, PressMethod, StartKey),
|
||||
in_range(MidBlockList, all, EndKey)
|
||||
);
|
||||
mid_only ->
|
||||
in_range(MidBlockList, StartKey, EndKey);
|
||||
ge_mid ->
|
||||
in_range(MidBlockList, StartKey, all)
|
||||
++ get_righthand_blocks(B4, B5, PressMethod, EndKey);
|
||||
append(
|
||||
in_range(MidBlockList, StartKey, all),
|
||||
get_righthand_blocks(B4, B5, PressMethod, EndKey)
|
||||
);
|
||||
gt_mid ->
|
||||
in_range(
|
||||
get_righthand_blocks(B4, B5, PressMethod, EndKey),
|
||||
|
@ -2695,6 +2700,34 @@ revert_position(Pos) ->
|
|||
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
|
||||
%%%============================================================================
|
||||
|
@ -3060,6 +3093,15 @@ maybelog_fetch_timing({Pid, _SlotFreq}, Level, Type, SW) ->
|
|||
|
||||
-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, false, 0).
|
||||
|
@ -3146,11 +3188,60 @@ generate_indexkeys(Count, IndexList) ->
|
|||
|
||||
generate_indexkey(Term, Count) ->
|
||||
IndexSpecs = [{add, "t1_int", Term}],
|
||||
leveled_codec:idx_indexspecs(IndexSpecs,
|
||||
leveled_codec:idx_indexspecs(
|
||||
IndexSpecs,
|
||||
"Bucket",
|
||||
"Key" ++ integer_to_list(Count),
|
||||
Count,
|
||||
infinity).
|
||||
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_tester(1),
|
||||
|
@ -3957,6 +4048,9 @@ fetch_status_test() ->
|
|||
ST = maps:get(state, Status),
|
||||
?assertMatch(redacted, ST#state.blockindex_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 = file:delete(filename:join(RP, Filename ++ ".sst")).
|
||||
|
||||
|
|
|
@ -418,9 +418,9 @@ generate_segmentfilter_list(SegmentList, xsmall) ->
|
|||
A1 = 1 bsl 14,
|
||||
ExpandSegFun =
|
||||
fun(X, Acc) ->
|
||||
Acc ++ [X, X + A0, X + A1, X + A0 + A1]
|
||||
[X, X + A0, X + A1, X + A0 + A1] ++ Acc
|
||||
end,
|
||||
lists:foldl(ExpandSegFun, [], SegmentList);
|
||||
lists:foldr(ExpandSegFun, [], SegmentList);
|
||||
false ->
|
||||
false
|
||||
end;
|
||||
|
|
|
@ -5,12 +5,15 @@
|
|||
|
||||
-module(leveled_util).
|
||||
|
||||
-export([generate_uuid/0,
|
||||
-export([
|
||||
generate_uuid/0,
|
||||
integer_now/0,
|
||||
integer_time/1,
|
||||
magic_hash/1,
|
||||
t2b/1,
|
||||
safe_rename/4]).
|
||||
safe_rename/4
|
||||
]
|
||||
).
|
||||
|
||||
-define(WRITE_OPS, [binary, raw, read, write]).
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue