Merge pull request #2 from martinsumner/initial-setup

Initial setup
This commit is contained in:
martinsumner 2016-11-05 18:26:35 +00:00 committed by GitHub
commit ebf549fba5
22 changed files with 11934 additions and 0 deletions

3
.gitignore vendored Normal file
View file

@ -0,0 +1,3 @@
*.beam
/.eunit
/_build

84
include/leveled.hrl Normal file
View file

@ -0,0 +1,84 @@
%% Tag to be used on standard Riak KV objects
-define(RIAK_TAG, o_rkv).
%% Tag to be used on K/V objects for non-Riak purposes
-define(STD_TAG, o).
%% Tag used for secondary index keys
-define(IDX_TAG, i).
%% Inker key type used for 'normal' objects
-define(INKT_STND, stnd).
%% Inker key type used for objects which contain no value, only key changes
%% This is used currently for objects formed under a 'retain' strategy on Inker
%% compaction, but could be used for special set-type objects
-define(INKT_KEYD, keyd).
%% Inker key type used for tombstones
-define(INKT_TOMB, tomb).
-record(sft_options,
{wait = true :: boolean(),
expire_tombstones = false :: boolean(),
penciller :: pid()}).
-record(penciller_work,
{next_sqn :: integer(),
clerk :: pid(),
src_level :: integer(),
manifest :: list(),
start_time :: tuple(),
ledger_filepath :: string(),
manifest_file :: string(),
new_manifest :: list(),
unreferenced_files :: list(),
target_is_basement = false ::boolean()}).
-record(level,
{level :: integer(),
is_basement = false :: boolean(),
timestamp :: integer()}).
-record(manifest_entry,
{start_key :: tuple(),
end_key :: tuple(),
owner :: pid(),
filename :: string()}).
-record(cdb_options,
{max_size :: integer(),
file_path :: string(),
binary_mode = false :: boolean()}).
-record(inker_options,
{cdb_max_size :: integer(),
root_path :: string(),
cdb_options :: #cdb_options{},
start_snapshot = false :: boolean(),
source_inker :: pid(),
reload_strategy = [] :: list(),
max_run_length}).
-record(penciller_options,
{root_path :: string(),
max_inmemory_tablesize :: integer(),
start_snapshot = false :: boolean(),
source_penciller :: pid()}).
-record(iclerk_options,
{inker :: pid(),
max_run_length :: integer(),
cdb_options :: #cdb_options{},
reload_strategy = [] :: list()}).
-record(r_content, {
metadata,
value :: term()
}).
-record(r_object, {
bucket,
key,
contents :: [#r_content{}],
vclock,
updatemetadata=dict:store(clean, true, dict:new()),
updatevalue :: term()}).

1
rebar.lock Normal file
View file

@ -0,0 +1 @@
[].

12
src/eleveleddb.app.src Normal file
View file

@ -0,0 +1,12 @@
{application, eleveleddb,
[
{description, "Key Value store based on LSM-Tree and designed for larger values"},
{vsn, "1"},
{registered, []},
{applications, [
kernel,
stdlib
]},
{mod, { eleveleddb_app, []}},
{env, [{root_path, "test"}]}
]}.

16
src/eleveleddb_app.erl Normal file
View file

@ -0,0 +1,16 @@
-module(eleveleddb_app).
-behaviour(application).
%% Application callbacks
-export([start/2, stop/1]).
%% ===================================================================
%% Application callbacks
%% ===================================================================
start(_StartType, _StartArgs) ->
eleveleddb_sup:start_link().
stop(_State) ->
ok.

27
src/eleveleddb_sup.erl Normal file
View file

@ -0,0 +1,27 @@
-module(eleveleddb_sup).
-behaviour(supervisor).
%% API
-export([start_link/0]).
%% Supervisor callbacks
-export([init/1]).
%% Helper macro for declaring children of supervisor
-define(CHILD(I, Type), {I, {I, start_link, []}, permanent, 5000, Type, [I]}).
%% ===================================================================
%% API functions
%% ===================================================================
start_link() ->
supervisor:start_link({local, leveled_bookie}, ?MODULE, []).
%% ===================================================================
%% Supervisor callbacks
%% ===================================================================
init([]) ->
{ok, { {one_for_one, 5, 10}, []} }.

1188
src/leveled_bookie.erl Normal file

File diff suppressed because it is too large Load diff

1809
src/leveled_cdb.erl Normal file

File diff suppressed because it is too large Load diff

400
src/leveled_codec.erl Normal file
View file

@ -0,0 +1,400 @@
%% -------- Key Codec ---------
%%
%% Functions for manipulating keys and values within leveled.
%%
%%
%% Within the LEDGER:
%% Keys are of the form -
%% {Tag, Bucket, Key, SubKey|null}
%% Values are of the form
%% {SQN, Status, MD}
%%
%% Within the JOURNAL:
%% Keys are of the form -
%% {SQN, LedgerKey}
%% Values are of the form
%% {Object, IndexSpecs} (as a binary)
%%
%% IndexSpecs are of the form of a Ledger Key/Value
%%
%% Tags need to be set during PUT operations and each Tag used must be
%% supported in an extract_metadata and a build_metadata_object function clause
%%
%% Currently the only tags supported are:
%% - o (standard objects)
%% - o_rkv (riak objects)
%% - i (index entries)
-module(leveled_codec).
-include("include/leveled.hrl").
-include_lib("eunit/include/eunit.hrl").
-export([
inker_reload_strategy/1,
strip_to_keyonly/1,
strip_to_seqonly/1,
strip_to_statusonly/1,
strip_to_keyseqstatusonly/1,
strip_to_keyseqonly/1,
striphead_to_details/1,
is_active/3,
endkey_passed/2,
key_dominates/2,
maybe_reap_expiredkey/2,
print_key/1,
to_ledgerkey/3,
to_ledgerkey/5,
from_ledgerkey/1,
to_inkerkv/4,
from_inkerkv/1,
from_journalkey/1,
compact_inkerkvc/2,
split_inkvalue/1,
check_forinkertype/2,
create_value_for_journal/1,
build_metadata_object/2,
generate_ledgerkv/5,
get_size/2,
get_keyandhash/2,
convert_indexspecs/5,
riakto_keydetails/1,
generate_uuid/0,
integer_now/0]).
%% Credit to
%% https://github.com/afiskon/erlang-uuid-v4/blob/master/src/uuid.erl
generate_uuid() ->
<<A:32, B:16, C:16, D:16, E:48>> = crypto:rand_bytes(16),
io_lib:format("~8.16.0b-~4.16.0b-4~3.16.0b-~4.16.0b-~12.16.0b",
[A, B, C band 16#0fff, D band 16#3fff bor 16#8000, E]).
inker_reload_strategy(AltList) ->
ReloadStrategy0 = [{?RIAK_TAG, retain}, {?STD_TAG, retain}],
lists:foldl(fun({X, Y}, SList) ->
lists:keyreplace(X, 1, SList, {X, Y})
end,
ReloadStrategy0,
AltList).
strip_to_keyonly({keyonly, K}) -> K;
strip_to_keyonly({K, _V}) -> K.
strip_to_keyseqstatusonly({K, {SeqN, St, _MD}}) -> {K, SeqN, St}.
strip_to_statusonly({_, {_, St, _}}) -> St.
strip_to_seqonly({_, {SeqN, _, _}}) -> SeqN.
strip_to_keyseqonly({LK, {SeqN, _, _}}) -> {LK, SeqN}.
striphead_to_details({SeqN, St, MD}) -> {SeqN, St, MD}.
key_dominates(LeftKey, RightKey) ->
case {LeftKey, RightKey} of
{{LK, _LVAL}, {RK, _RVAL}} when LK < RK ->
left_hand_first;
{{LK, _LVAL}, {RK, _RVAL}} when RK < LK ->
right_hand_first;
{{LK, {LSN, _LST, _LMD}}, {RK, {RSN, _RST, _RMD}}}
when LK == RK, LSN >= RSN ->
left_hand_dominant;
{{LK, {LSN, _LST, _LMD}}, {RK, {RSN, _RST, _RMD}}}
when LK == RK, LSN < RSN ->
right_hand_dominant
end.
maybe_reap_expiredkey(KV, LevelD) ->
Status = strip_to_statusonly(KV),
maybe_reap(Status, LevelD).
maybe_reap({_, infinity}, _) ->
false; % key is not set to expire
maybe_reap({_, TS}, {true, CurrTS}) when CurrTS > TS ->
true; % basement and ready to expire
maybe_reap(tomb, {true, _CurrTS}) ->
true; % always expire in basement
maybe_reap(_, _) ->
false.
is_active(Key, Value, Now) ->
case strip_to_statusonly({Key, Value}) of
{active, infinity} ->
true;
tomb ->
false;
{active, TS} when TS >= Now ->
true;
{active, _TS} ->
false
end.
from_ledgerkey({Tag, Bucket, {_IdxField, IdxValue}, Key})
when Tag == ?IDX_TAG ->
{Bucket, Key, IdxValue};
from_ledgerkey({_Tag, Bucket, Key, null}) ->
{Bucket, Key}.
to_ledgerkey(Bucket, Key, Tag, Field, Value) when Tag == ?IDX_TAG ->
{?IDX_TAG, Bucket, {Field, Value}, Key}.
to_ledgerkey(Bucket, Key, Tag) ->
{Tag, Bucket, Key, null}.
%% Return the Key, Value and Hash Option for this object. The hash option
%% indicates whether the key would ever be looked up directly, and so if it
%% requires an entry in the hash table
to_inkerkv(LedgerKey, SQN, to_fetch, null) ->
{{SQN, ?INKT_STND, LedgerKey}, null, true};
to_inkerkv(LedgerKey, SQN, Object, KeyChanges) ->
InkerType = check_forinkertype(LedgerKey, Object),
Value = create_value_for_journal({Object, KeyChanges}),
{{SQN, InkerType, LedgerKey}, Value}.
%% Used when fetching objects, so only handles standard, hashable entries
from_inkerkv(Object) ->
case Object of
{{SQN, ?INKT_STND, PK}, Bin} when is_binary(Bin) ->
{{SQN, PK}, binary_to_term(Bin)};
{{SQN, ?INKT_STND, PK}, Term} ->
{{SQN, PK}, Term};
_ ->
Object
end.
from_journalkey({SQN, _Type, LedgerKey}) ->
{SQN, LedgerKey}.
compact_inkerkvc({_InkerKey, crc_wonky, false}, _Strategy) ->
skip;
compact_inkerkvc({{_SQN, ?INKT_TOMB, _LK}, _V, _CrcCheck}, _Strategy) ->
skip;
compact_inkerkvc({{SQN, ?INKT_KEYD, LK}, V, CrcCheck}, Strategy) ->
{Tag, _, _, _} = LK,
{Tag, TagStrat} = lists:keyfind(Tag, 1, Strategy),
case TagStrat of
retain ->
{retain, {{SQN, ?INKT_KEYD, LK}, V, CrcCheck}};
TagStrat ->
{TagStrat, null}
end;
compact_inkerkvc({{SQN, ?INKT_STND, LK}, V, CrcCheck}, Strategy) ->
{Tag, _, _, _} = LK,
{Tag, TagStrat} = lists:keyfind(Tag, 1, Strategy),
case TagStrat of
retain ->
{_V, KeyDeltas} = split_inkvalue(V),
{TagStrat, {{SQN, ?INKT_KEYD, LK}, {null, KeyDeltas}, CrcCheck}};
TagStrat ->
{TagStrat, null}
end.
split_inkvalue(VBin) ->
case is_binary(VBin) of
true ->
binary_to_term(VBin);
false ->
VBin
end.
check_forinkertype(_LedgerKey, delete) ->
?INKT_TOMB;
check_forinkertype(_LedgerKey, _Object) ->
?INKT_STND.
create_value_for_journal(Value) ->
case Value of
{Object, KeyChanges} ->
term_to_binary({Object, KeyChanges}, [compressed]);
Value when is_binary(Value) ->
Value
end.
hash(Obj) ->
erlang:phash2(term_to_binary(Obj)).
% Return a tuple of strings to ease the printing of keys to logs
print_key(Key) ->
{A_STR, B_TERM, C_TERM} = case Key of
{?STD_TAG, B, K, _SK} ->
{"Object", B, K};
{?RIAK_TAG, B, K, _SK} ->
{"RiakObject", B, K};
{?IDX_TAG, B, {F, _V}, _K} ->
{"Index", B, F}
end,
B_STR = turn_to_string(B_TERM),
C_STR = turn_to_string(C_TERM),
{A_STR, B_STR, C_STR}.
turn_to_string(Item) ->
if
is_binary(Item) == true ->
binary_to_list(Item);
is_integer(Item) == true ->
integer_to_list(Item);
is_list(Item) == true ->
Item;
true ->
[Output] = io_lib:format("~w", [Item]),
Output
end.
% Compare a key against a query key, only comparing elements that are non-null
% in the Query key. This is used for comparing against end keys in queries.
endkey_passed({EK1, null, null, null}, {CK1, _, _, _}) ->
EK1 < CK1;
endkey_passed({EK1, EK2, null, null}, {CK1, CK2, _, _}) ->
{EK1, EK2} < {CK1, CK2};
endkey_passed({EK1, EK2, EK3, null}, {CK1, CK2, CK3, _}) ->
{EK1, EK2, EK3} < {CK1, CK2, CK3};
endkey_passed(EndKey, CheckingKey) ->
EndKey < CheckingKey.
convert_indexspecs(IndexSpecs, Bucket, Key, SQN, TTL) ->
lists:map(fun({IndexOp, IdxField, IdxValue}) ->
Status = case IndexOp of
add ->
{active, TTL};
remove ->
%% TODO: timestamps for delayed reaping
tomb
end,
{to_ledgerkey(Bucket, Key, ?IDX_TAG,
IdxField, IdxValue),
{SQN, Status, null}}
end,
IndexSpecs).
generate_ledgerkv(PrimaryKey, SQN, Obj, Size, TS) ->
{Tag, Bucket, Key, _} = PrimaryKey,
Status = case Obj of
delete ->
tomb;
_ ->
{active, TS}
end,
{Bucket,
Key,
{PrimaryKey, {SQN, Status, extract_metadata(Obj, Size, Tag)}}}.
integer_now() ->
integer_time(os:timestamp()).
integer_time(TS) ->
DT = calendar:now_to_universal_time(TS),
calendar:datetime_to_gregorian_seconds(DT).
extract_metadata(Obj, Size, ?RIAK_TAG) ->
riak_extract_metadata(Obj, Size);
extract_metadata(Obj, Size, ?STD_TAG) ->
{hash(Obj), Size}.
get_size(PK, Value) ->
{Tag, _Bucket, _Key, _} = PK,
{_, _, MD} = Value,
case Tag of
?RIAK_TAG ->
{_RMD, _VC, _Hash, Size} = MD,
Size;
?STD_TAG ->
{_Hash, Size} = MD,
Size
end.
get_keyandhash(LK, Value) ->
{Tag, Bucket, Key, _} = LK,
{_, _, MD} = Value,
case Tag of
?RIAK_TAG ->
{_RMD, _VC, Hash, _Size} = MD,
{Bucket, Key, Hash};
?STD_TAG ->
{Hash, _Size} = MD,
{Bucket, Key, Hash}
end.
build_metadata_object(PrimaryKey, MD) ->
{Tag, Bucket, Key, null} = PrimaryKey,
case Tag of
?RIAK_TAG ->
riak_metadata_object(Bucket, Key, MD);
?STD_TAG ->
MD
end.
riak_metadata_object(Bucket, Key, MD) ->
{RMD, VC, _Hash, _Size} = MD,
Contents = lists:foldl(fun(X, Acc) -> Acc ++ [#r_content{metadata=X}] end,
[],
RMD),
#r_object{contents=Contents, bucket=Bucket, key=Key, vclock=VC}.
riak_extract_metadata(delete, Size) ->
{delete, null, null, Size};
riak_extract_metadata(Obj, Size) ->
{get_metadatas(Obj), vclock(Obj), riak_hash(Obj), Size}.
riak_hash(Obj=#r_object{}) ->
Vclock = vclock(Obj),
UpdObj = set_vclock(Obj, lists:sort(Vclock)),
erlang:phash2(term_to_binary(UpdObj)).
riakto_keydetails(Object) ->
{Object#r_object.bucket, Object#r_object.key}.
get_metadatas(#r_object{contents=Contents}) ->
[Content#r_content.metadata || Content <- Contents].
set_vclock(Object=#r_object{}, VClock) -> Object#r_object{vclock=VClock}.
vclock(#r_object{vclock=VClock}) -> VClock.
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
indexspecs_test() ->
IndexSpecs = [{add, "t1_int", 456},
{add, "t1_bin", "adbc123"},
{remove, "t1_bin", "abdc456"}],
Changes = convert_indexspecs(IndexSpecs, "Bucket", "Key2", 1, infinity),
?assertMatch({{i, "Bucket", {"t1_int", 456}, "Key2"},
{1, {active, infinity}, null}}, lists:nth(1, Changes)),
?assertMatch({{i, "Bucket", {"t1_bin", "adbc123"}, "Key2"},
{1, {active, infinity}, null}}, lists:nth(2, Changes)),
?assertMatch({{i, "Bucket", {"t1_bin", "abdc456"}, "Key2"},
{1, tomb, null}}, lists:nth(3, Changes)).
endkey_passed_test() ->
TestKey = {i, null, null, null},
K1 = {i, 123, {"a", "b"}, <<>>},
K2 = {o, 123, {"a", "b"}, <<>>},
?assertMatch(false, endkey_passed(TestKey, K1)),
?assertMatch(true, endkey_passed(TestKey, K2)).
stringcheck_test() ->
?assertMatch("Bucket", turn_to_string("Bucket")),
?assertMatch("Bucket", turn_to_string(<<"Bucket">>)),
?assertMatch("bucket", turn_to_string(bucket)).
-endif.

818
src/leveled_iclerk.erl Normal file
View file

@ -0,0 +1,818 @@
%% -------- Inker's Clerk ---------
%%
%% The Inker's clerk runs compaction jobs on behalf of the Inker, informing the
%% Inker of any manifest changes when complete.
%%
%% -------- Value Compaction ---------
%%
%% Compaction requires the Inker to have four different types of keys
%% * stnd - A standard key of the form {SQN, stnd, LedgerKey} which maps to a
%% value of {Object, KeyDeltas}
%% * tomb - A tombstone for a LedgerKey {SQN, tomb, LedgerKey}
%% * keyd - An object containing key deltas only of the form
%% {SQN, keyd, LedgerKey} which maps to a value of {KeyDeltas}
%%
%% Each LedgerKey has a Tag, and for each Tag there should be a compaction
%% strategy, which will be set to one of the following:
%% * retain - KeyDeltas must be retained permanently, only values can be
%% compacted (if replaced or not_present in the ledger)
%% * recalc - The full object can be removed through comapction (if replaced or
%% not_present in the ledger), as each object with that tag can have the Key
%% Deltas recreated by passing into an assigned recalc function {LedgerKey,
%% SQN, Object, KeyDeltas, PencillerSnapshot}
%% * recovr - At compaction time this is equivalent to recalc, only KeyDeltas
%% are lost when reloading the Ledger from the Journal, and it is assumed that
%% those deltas will be resolved through external anti-entropy (e.g. read
%% repair or AAE) - or alternatively the risk of loss of persisted data from
%% the ledger is accepted for this data type
%%
%% During the compaction process for the Journal, the file chosen for
%% compaction is scanned in SQN order, and a FilterFun is passed (which will
%% normally perform a check against a snapshot of the persisted part of the
%% Ledger). If the given key is of type stnd, and this object is no longer the
%% active object under the LedgerKey, then the object can be compacted out of
%% the journal. This will lead to either its removal (if the strategy for the
%% Tag is recovr or recalc), or its replacement with a KeyDelta object.
%%
%% Tombstones cannot be reaped through this compaction process.
%%
%% Currently, KeyDeltas are also reaped if the LedgerKey has been updated and
%% the Tag has a recovr strategy. This may be the case when KeyDeltas are used
%% as a way of directly representing a change, and where anti-entropy can
%% recover from a loss.
%%
%% -------- Tombstone Reaping ---------
%%
%% Value compaction does not remove tombstones from the database, and so a
%% separate compaction job is required for this.
%%
%% Tombstones can only be reaped for Tags set to recovr or recalc.
%%
%% The tombstone reaping process should select a file to compact, and then
%% take that file and discover the LedgerKeys of all reapable tombstones.
%% The lesger should then be scanned from SQN 0 looking for unreaped objects
%% before the tombstone. If no ushc objects exist for that tombstone, it can
%% now be reaped as part of the compaction job.
%%
%% Other tombstones cannot be reaped, as otherwis eon laoding a ledger an old
%% version of the object may re-emerge.
-module(leveled_iclerk).
-behaviour(gen_server).
-include("include/leveled.hrl").
-export([init/1,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2,
clerk_new/1,
clerk_compact/6,
clerk_hashtablecalc/3,
clerk_stop/1,
code_change/3]).
-include_lib("eunit/include/eunit.hrl").
-define(JOURNAL_FILEX, "cdb").
-define(PENDING_FILEX, "pnd").
-define(SAMPLE_SIZE, 200).
-define(BATCH_SIZE, 32).
-define(BATCHES_TO_CHECK, 8).
%% How many consecutive files to compact in one run
-define(MAX_COMPACTION_RUN, 4).
%% Sliding scale to allow preference of longer runs up to maximum
-define(SINGLEFILE_COMPACTION_TARGET, 60.0).
-define(MAXRUN_COMPACTION_TARGET, 80.0).
-define(CRC_SIZE, 4).
-define(DEFAULT_RELOAD_STRATEGY, leveled_codec:inker_reload_strategy([])).
-record(state, {inker :: pid(),
max_run_length :: integer(),
cdb_options,
reload_strategy = ?DEFAULT_RELOAD_STRATEGY :: list()}).
-record(candidate, {low_sqn :: integer(),
filename :: string(),
journal :: pid(),
compaction_perc :: float()}).
%%%============================================================================
%%% API
%%%============================================================================
clerk_new(InkerClerkOpts) ->
gen_server:start(?MODULE, [InkerClerkOpts], []).
clerk_compact(Pid, Checker, InitiateFun, FilterFun, Inker, Timeout) ->
gen_server:cast(Pid,
{compact,
Checker,
InitiateFun,
FilterFun,
Inker,
Timeout}).
clerk_hashtablecalc(HashTree, StartPos, CDBpid) ->
{ok, Clerk} = gen_server:start(?MODULE, [#iclerk_options{}], []),
gen_server:cast(Clerk, {hashtable_calc, HashTree, StartPos, CDBpid}).
clerk_stop(Pid) ->
gen_server:cast(Pid, stop).
%%%============================================================================
%%% gen_server callbacks
%%%============================================================================
init([IClerkOpts]) ->
ReloadStrategy = IClerkOpts#iclerk_options.reload_strategy,
case IClerkOpts#iclerk_options.max_run_length of
undefined ->
{ok, #state{max_run_length = ?MAX_COMPACTION_RUN,
inker = IClerkOpts#iclerk_options.inker,
cdb_options = IClerkOpts#iclerk_options.cdb_options,
reload_strategy = ReloadStrategy}};
MRL ->
{ok, #state{max_run_length = MRL,
inker = IClerkOpts#iclerk_options.inker,
cdb_options = IClerkOpts#iclerk_options.cdb_options,
reload_strategy = ReloadStrategy}}
end.
handle_call(_Msg, _From, State) ->
{reply, not_supported, State}.
handle_cast({compact, Checker, InitiateFun, FilterFun, Inker, _Timeout},
State) ->
% Need to fetch manifest at start rather than have it be passed in
% Don't want to process a queued call waiting on an old manifest
[_Active|Manifest] = leveled_inker:ink_getmanifest(Inker),
MaxRunLength = State#state.max_run_length,
{FilterServer, MaxSQN} = InitiateFun(Checker),
CDBopts = State#state.cdb_options,
FP = CDBopts#cdb_options.file_path,
ok = filelib:ensure_dir(FP),
Candidates = scan_all_files(Manifest, FilterFun, FilterServer, MaxSQN),
BestRun0 = assess_candidates(Candidates, MaxRunLength),
case score_run(BestRun0, MaxRunLength) of
Score when Score > 0.0 ->
BestRun1 = sort_run(BestRun0),
print_compaction_run(BestRun1, MaxRunLength),
{ManifestSlice,
PromptDelete} = compact_files(BestRun1,
CDBopts,
FilterFun,
FilterServer,
MaxSQN,
State#state.reload_strategy),
FilesToDelete = lists:map(fun(C) ->
{C#candidate.low_sqn,
C#candidate.filename,
C#candidate.journal}
end,
BestRun1),
leveled_log:log("IC002", [length(FilesToDelete)]),
case is_process_alive(Inker) of
true ->
update_inker(Inker,
ManifestSlice,
FilesToDelete,
PromptDelete),
{noreply, State};
false ->
leveled_log:log("IC001", []),
{stop, normal, State}
end;
Score ->
leveled_log:log("IC003", [Score]),
ok = leveled_inker:ink_compactioncomplete(Inker),
{noreply, State}
end;
handle_cast({hashtable_calc, HashTree, StartPos, CDBpid}, State) ->
{IndexList, HashTreeBin} = leveled_cdb:hashtable_calc(HashTree, StartPos),
ok = leveled_cdb:cdb_returnhashtable(CDBpid, IndexList, HashTreeBin),
{stop, normal, State};
handle_cast(stop, State) ->
{stop, normal, State}.
handle_info(_Info, State) ->
{noreply, State}.
terminate(_Reason, _State) ->
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%%============================================================================
%%% Internal functions
%%%============================================================================
check_single_file(CDB, FilterFun, FilterServer, MaxSQN, SampleSize, BatchSize) ->
FN = leveled_cdb:cdb_filename(CDB),
PositionList = leveled_cdb:cdb_getpositions(CDB, SampleSize),
KeySizeList = fetch_inbatches(PositionList, BatchSize, CDB, []),
R0 = lists:foldl(fun(KS, {ActSize, RplSize}) ->
{{SQN, _Type, PK}, Size} = KS,
Check = FilterFun(FilterServer, PK, SQN),
case {Check, SQN > MaxSQN} of
{true, _} ->
{ActSize + Size - ?CRC_SIZE, RplSize};
{false, true} ->
{ActSize + Size - ?CRC_SIZE, RplSize};
_ ->
{ActSize, RplSize + Size - ?CRC_SIZE}
end end,
{0, 0},
KeySizeList),
{ActiveSize, ReplacedSize} = R0,
Score = case ActiveSize + ReplacedSize of
0 ->
100.0;
_ ->
100 * ActiveSize / (ActiveSize + ReplacedSize)
end,
leveled_log:log("IC004", [FN, Score]),
Score.
scan_all_files(Manifest, FilterFun, FilterServer, MaxSQN) ->
scan_all_files(Manifest, FilterFun, FilterServer, MaxSQN, []).
scan_all_files([], _FilterFun, _FilterServer, _MaxSQN, CandidateList) ->
CandidateList;
scan_all_files([Entry|Tail], FilterFun, FilterServer, MaxSQN, CandidateList) ->
{LowSQN, FN, JournalP} = Entry,
CpctPerc = check_single_file(JournalP,
FilterFun,
FilterServer,
MaxSQN,
?SAMPLE_SIZE,
?BATCH_SIZE),
scan_all_files(Tail,
FilterFun,
FilterServer,
MaxSQN,
CandidateList ++
[#candidate{low_sqn = LowSQN,
filename = FN,
journal = JournalP,
compaction_perc = CpctPerc}]).
fetch_inbatches([], _BatchSize, _CDB, CheckedList) ->
CheckedList;
fetch_inbatches(PositionList, BatchSize, CDB, CheckedList) ->
{Batch, Tail} = if
length(PositionList) >= BatchSize ->
lists:split(BatchSize, PositionList);
true ->
{PositionList, []}
end,
KL_List = leveled_cdb:cdb_directfetch(CDB, Batch, key_size),
fetch_inbatches(Tail, BatchSize, CDB, CheckedList ++ KL_List).
assess_candidates(AllCandidates, MaxRunLength) ->
NaiveBestRun = assess_candidates(AllCandidates, MaxRunLength, [], []),
case length(AllCandidates) of
L when L > MaxRunLength, MaxRunLength > 1 ->
%% Assess with different offsets from the start
SqL = lists:seq(1, MaxRunLength - 1),
lists:foldl(fun(Counter, BestRun) ->
SubList = lists:nthtail(Counter,
AllCandidates),
assess_candidates(SubList,
MaxRunLength,
[],
BestRun)
end,
NaiveBestRun,
SqL);
_ ->
NaiveBestRun
end.
assess_candidates([], _MaxRunLength, _CurrentRun0, BestAssessment) ->
BestAssessment;
assess_candidates([HeadC|Tail], MaxRunLength, CurrentRun0, BestAssessment) ->
CurrentRun1 = choose_best_assessment(CurrentRun0 ++ [HeadC],
[HeadC],
MaxRunLength),
assess_candidates(Tail,
MaxRunLength,
CurrentRun1,
choose_best_assessment(CurrentRun1,
BestAssessment,
MaxRunLength)).
choose_best_assessment(RunToAssess, BestRun, MaxRunLength) ->
case length(RunToAssess) of
LR1 when LR1 > MaxRunLength ->
BestRun;
_ ->
AssessScore = score_run(RunToAssess, MaxRunLength),
BestScore = score_run(BestRun, MaxRunLength),
if
AssessScore > BestScore ->
RunToAssess;
true ->
BestRun
end
end.
score_run([], _MaxRunLength) ->
0.0;
score_run(Run, MaxRunLength) ->
TargetIncr = case MaxRunLength of
1 ->
0.0;
MaxRunSize ->
(?MAXRUN_COMPACTION_TARGET
- ?SINGLEFILE_COMPACTION_TARGET)
/ (MaxRunSize - 1)
end,
Target = ?SINGLEFILE_COMPACTION_TARGET + TargetIncr * (length(Run) - 1),
RunTotal = lists:foldl(fun(Cand, Acc) ->
Acc + Cand#candidate.compaction_perc end,
0.0,
Run),
Target - RunTotal / length(Run).
print_compaction_run(BestRun, MaxRunLength) ->
leveled_log:log("IC005", [length(BestRun),
score_run(BestRun, MaxRunLength)]),
lists:foreach(fun(File) ->
leveled_log:log("IC006", [File#candidate.filename])
end,
BestRun).
sort_run(RunOfFiles) ->
CompareFun = fun(Cand1, Cand2) ->
Cand1#candidate.low_sqn =< Cand2#candidate.low_sqn end,
lists:sort(CompareFun, RunOfFiles).
update_inker(Inker, ManifestSlice, FilesToDelete, PromptDelete) ->
{ok, ManSQN} = leveled_inker:ink_updatemanifest(Inker,
ManifestSlice,
FilesToDelete),
ok = leveled_inker:ink_compactioncomplete(Inker),
leveled_log:log("IC007", []),
case PromptDelete of
true ->
lists:foreach(fun({_SQN, _FN, J2D}) ->
leveled_cdb:cdb_deletepending(J2D,
ManSQN,
Inker)
end,
FilesToDelete),
ok;
false ->
ok
end.
compact_files(BestRun, CDBopts, FilterFun, FilterServer, MaxSQN, RStrategy) ->
BatchesOfPositions = get_all_positions(BestRun, []),
compact_files(BatchesOfPositions,
CDBopts,
null,
FilterFun,
FilterServer,
MaxSQN,
RStrategy,
[],
true).
compact_files([], _CDBopts, null, _FilterFun, _FilterServer, _MaxSQN,
_RStrategy, ManSlice0, PromptDelete0) ->
{ManSlice0, PromptDelete0};
compact_files([], _CDBopts, ActiveJournal0, _FilterFun, _FilterServer, _MaxSQN,
_RStrategy, ManSlice0, PromptDelete0) ->
ManSlice1 = ManSlice0 ++ generate_manifest_entry(ActiveJournal0),
{ManSlice1, PromptDelete0};
compact_files([Batch|T], CDBopts, ActiveJournal0,
FilterFun, FilterServer, MaxSQN,
RStrategy, ManSlice0, PromptDelete0) ->
{SrcJournal, PositionList} = Batch,
KVCs0 = leveled_cdb:cdb_directfetch(SrcJournal,
PositionList,
key_value_check),
R0 = filter_output(KVCs0,
FilterFun,
FilterServer,
MaxSQN,
RStrategy),
{KVCs1, PromptDelete1} = R0,
PromptDelete2 = case {PromptDelete0, PromptDelete1} of
{true, true} ->
true;
_ ->
false
end,
{ActiveJournal1, ManSlice1} = write_values(KVCs1,
CDBopts,
ActiveJournal0,
ManSlice0),
compact_files(T, CDBopts, ActiveJournal1, FilterFun, FilterServer, MaxSQN,
RStrategy, ManSlice1, PromptDelete2).
get_all_positions([], PositionBatches) ->
PositionBatches;
get_all_positions([HeadRef|RestOfBest], PositionBatches) ->
SrcJournal = HeadRef#candidate.journal,
Positions = leveled_cdb:cdb_getpositions(SrcJournal, all),
leveled_log:log("IC008", [HeadRef#candidate.filename, length(Positions)]),
Batches = split_positions_into_batches(lists:sort(Positions),
SrcJournal,
[]),
get_all_positions(RestOfBest, PositionBatches ++ Batches).
split_positions_into_batches([], _Journal, Batches) ->
Batches;
split_positions_into_batches(Positions, Journal, Batches) ->
{ThisBatch, Tail} = if
length(Positions) > ?BATCH_SIZE ->
lists:split(?BATCH_SIZE, Positions);
true ->
{Positions, []}
end,
split_positions_into_batches(Tail,
Journal,
Batches ++ [{Journal, ThisBatch}]).
filter_output(KVCs, FilterFun, FilterServer, MaxSQN, ReloadStrategy) ->
lists:foldl(fun(KVC0, {Acc, PromptDelete}) ->
R = leveled_codec:compact_inkerkvc(KVC0, ReloadStrategy),
case R of
skip ->
{Acc, PromptDelete};
{TStrat, KVC1} ->
{K, _V, CrcCheck} = KVC0,
{SQN, LedgerKey} = leveled_codec:from_journalkey(K),
KeyValid = FilterFun(FilterServer, LedgerKey, SQN),
case {KeyValid, CrcCheck, SQN > MaxSQN, TStrat} of
{true, true, _, _} ->
{Acc ++ [KVC0], PromptDelete};
{false, true, true, _} ->
{Acc ++ [KVC0], PromptDelete};
{false, true, false, retain} ->
{Acc ++ [KVC1], PromptDelete};
{false, true, false, _} ->
{Acc, PromptDelete}
end
end
end,
{[], true},
KVCs).
write_values([], _CDBopts, Journal0, ManSlice0) ->
{Journal0, ManSlice0};
write_values(KVCList, CDBopts, Journal0, ManSlice0) ->
KVList = lists:map(fun({K, V, _C}) ->
{K, leveled_codec:create_value_for_journal(V)}
end,
KVCList),
{ok, Journal1} = case Journal0 of
null ->
{TK, _TV} = lists:nth(1, KVList),
{SQN, _LK} = leveled_codec:from_journalkey(TK),
FP = CDBopts#cdb_options.file_path,
FN = leveled_inker:filepath(FP,
SQN,
compact_journal),
leveled_log:log("IC009", [FN]),
leveled_cdb:cdb_open_writer(FN,
CDBopts);
_ ->
{ok, Journal0}
end,
R = leveled_cdb:cdb_mput(Journal1, KVList),
case R of
ok ->
{Journal1, ManSlice0};
roll ->
ManSlice1 = ManSlice0 ++ generate_manifest_entry(Journal1),
write_values(KVCList, CDBopts, null, ManSlice1)
end.
generate_manifest_entry(ActiveJournal) ->
{ok, NewFN} = leveled_cdb:cdb_complete(ActiveJournal),
{ok, PidR} = leveled_cdb:cdb_open_reader(NewFN),
{StartSQN, _Type, _PK} = leveled_cdb:cdb_firstkey(PidR),
[{StartSQN, NewFN, PidR}].
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
simple_score_test() ->
Run1 = [#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 76.0},
#candidate{compaction_perc = 70.0}],
?assertMatch(6.0, score_run(Run1, 4)),
Run2 = [#candidate{compaction_perc = 75.0}],
?assertMatch(-15.0, score_run(Run2, 4)),
?assertMatch(0.0, score_run([], 4)),
Run3 = [#candidate{compaction_perc = 100.0}],
?assertMatch(-40.0, score_run(Run3, 4)).
score_compare_test() ->
Run1 = [#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 76.0},
#candidate{compaction_perc = 70.0}],
?assertMatch(6.0, score_run(Run1, 4)),
Run2 = [#candidate{compaction_perc = 75.0}],
?assertMatch(Run1, choose_best_assessment(Run1, Run2, 4)),
?assertMatch(Run2, choose_best_assessment(Run1 ++ Run2, Run2, 4)).
find_bestrun_test() ->
%% Tests dependent on these defaults
%% -define(MAX_COMPACTION_RUN, 4).
%% -define(SINGLEFILE_COMPACTION_TARGET, 60.0).
%% -define(MAXRUN_COMPACTION_TARGET, 80.0).
%% Tested first with blocks significant as no back-tracking
Block1 = [#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 85.0},
#candidate{compaction_perc = 62.0},
#candidate{compaction_perc = 70.0}],
Block2 = [#candidate{compaction_perc = 58.0},
#candidate{compaction_perc = 95.0},
#candidate{compaction_perc = 95.0},
#candidate{compaction_perc = 65.0}],
Block3 = [#candidate{compaction_perc = 90.0},
#candidate{compaction_perc = 100.0},
#candidate{compaction_perc = 100.0},
#candidate{compaction_perc = 100.0}],
Block4 = [#candidate{compaction_perc = 75.0},
#candidate{compaction_perc = 76.0},
#candidate{compaction_perc = 76.0},
#candidate{compaction_perc = 60.0}],
Block5 = [#candidate{compaction_perc = 80.0},
#candidate{compaction_perc = 80.0}],
CList0 = Block1 ++ Block2 ++ Block3 ++ Block4 ++ Block5,
?assertMatch(Block4, assess_candidates(CList0, 4, [], [])),
CList1 = CList0 ++ [#candidate{compaction_perc = 20.0}],
?assertMatch([#candidate{compaction_perc = 20.0}],
assess_candidates(CList1, 4, [], [])),
CList2 = Block4 ++ Block3 ++ Block2 ++ Block1 ++ Block5,
?assertMatch(Block4, assess_candidates(CList2, 4, [], [])),
CList3 = Block5 ++ Block1 ++ Block2 ++ Block3 ++ Block4,
?assertMatch([#candidate{compaction_perc = 62.0},
#candidate{compaction_perc = 70.0},
#candidate{compaction_perc = 58.0}],
assess_candidates(CList3, 4, [], [])),
%% Now do some back-tracking to get a genuinely optimal solution without
%% needing to re-order
?assertMatch([#candidate{compaction_perc = 62.0},
#candidate{compaction_perc = 70.0},
#candidate{compaction_perc = 58.0}],
assess_candidates(CList0, 4)),
?assertMatch([#candidate{compaction_perc = 62.0},
#candidate{compaction_perc = 70.0},
#candidate{compaction_perc = 58.0}],
assess_candidates(CList0, 5)),
?assertMatch([#candidate{compaction_perc = 62.0},
#candidate{compaction_perc = 70.0},
#candidate{compaction_perc = 58.0},
#candidate{compaction_perc = 95.0},
#candidate{compaction_perc = 95.0},
#candidate{compaction_perc = 65.0}],
assess_candidates(CList0, 6)).
test_ledgerkey(Key) ->
{o, "Bucket", Key, null}.
test_inkerkv(SQN, Key, V, IdxSpecs) ->
{{SQN, ?INKT_STND, test_ledgerkey(Key)}, term_to_binary({V, IdxSpecs})}.
fetch_testcdb(RP) ->
FN1 = leveled_inker:filepath(RP, 1, new_journal),
{ok, CDB1} = leveled_cdb:cdb_open_writer(FN1, #cdb_options{}),
{K1, V1} = test_inkerkv(1, "Key1", "Value1", []),
{K2, V2} = test_inkerkv(2, "Key2", "Value2", []),
{K3, V3} = test_inkerkv(3, "Key3", "Value3", []),
{K4, V4} = test_inkerkv(4, "Key1", "Value4", []),
{K5, V5} = test_inkerkv(5, "Key1", "Value5", []),
{K6, V6} = test_inkerkv(6, "Key1", "Value6", []),
{K7, V7} = test_inkerkv(7, "Key1", "Value7", []),
{K8, V8} = test_inkerkv(8, "Key1", "Value8", []),
ok = leveled_cdb:cdb_put(CDB1, K1, V1),
ok = leveled_cdb:cdb_put(CDB1, K2, V2),
ok = leveled_cdb:cdb_put(CDB1, K3, V3),
ok = leveled_cdb:cdb_put(CDB1, K4, V4),
ok = leveled_cdb:cdb_put(CDB1, K5, V5),
ok = leveled_cdb:cdb_put(CDB1, K6, V6),
ok = leveled_cdb:cdb_put(CDB1, K7, V7),
ok = leveled_cdb:cdb_put(CDB1, K8, V8),
{ok, FN2} = leveled_cdb:cdb_complete(CDB1),
leveled_cdb:cdb_open_reader(FN2).
check_single_file_test() ->
RP = "../test/journal",
{ok, CDB} = fetch_testcdb(RP),
LedgerSrv1 = [{8, {o, "Bucket", "Key1", null}},
{2, {o, "Bucket", "Key2", null}},
{3, {o, "Bucket", "Key3", null}}],
LedgerFun1 = fun(Srv, Key, ObjSQN) ->
case lists:keyfind(ObjSQN, 1, Srv) of
{ObjSQN, Key} ->
true;
_ ->
false
end end,
Score1 = check_single_file(CDB, LedgerFun1, LedgerSrv1, 9, 8, 4),
?assertMatch(37.5, Score1),
LedgerFun2 = fun(_Srv, _Key, _ObjSQN) -> true end,
Score2 = check_single_file(CDB, LedgerFun2, LedgerSrv1, 9, 8, 4),
?assertMatch(100.0, Score2),
Score3 = check_single_file(CDB, LedgerFun1, LedgerSrv1, 9, 8, 3),
?assertMatch(37.5, Score3),
Score4 = check_single_file(CDB, LedgerFun1, LedgerSrv1, 4, 8, 4),
?assertMatch(75.0, Score4),
ok = leveled_cdb:cdb_deletepending(CDB),
ok = leveled_cdb:cdb_destroy(CDB).
compact_single_file_setup() ->
RP = "../test/journal",
{ok, CDB} = fetch_testcdb(RP),
Candidate = #candidate{journal = CDB,
low_sqn = 1,
filename = "test",
compaction_perc = 37.5},
LedgerSrv1 = [{8, {o, "Bucket", "Key1", null}},
{2, {o, "Bucket", "Key2", null}},
{3, {o, "Bucket", "Key3", null}}],
LedgerFun1 = fun(Srv, Key, ObjSQN) ->
case lists:keyfind(ObjSQN, 1, Srv) of
{ObjSQN, Key} ->
true;
_ ->
false
end end,
CompactFP = leveled_inker:filepath(RP, journal_compact_dir),
ok = filelib:ensure_dir(CompactFP),
{Candidate, LedgerSrv1, LedgerFun1, CompactFP, CDB}.
compact_single_file_recovr_test() ->
{Candidate,
LedgerSrv1,
LedgerFun1,
CompactFP,
CDB} = compact_single_file_setup(),
R1 = compact_files([Candidate],
#cdb_options{file_path=CompactFP},
LedgerFun1,
LedgerSrv1,
9,
[{?STD_TAG, recovr}]),
{ManSlice1, PromptDelete1} = R1,
?assertMatch(true, PromptDelete1),
[{LowSQN, FN, PidR}] = ManSlice1,
io:format("FN of ~s~n", [FN]),
?assertMatch(2, LowSQN),
?assertMatch(probably,
leveled_cdb:cdb_keycheck(PidR,
{8,
stnd,
test_ledgerkey("Key1")})),
?assertMatch(missing, leveled_cdb:cdb_get(PidR,
{7,
stnd,
test_ledgerkey("Key1")})),
?assertMatch(missing, leveled_cdb:cdb_get(PidR,
{1,
stnd,
test_ledgerkey("Key1")})),
{_RK1, RV1} = leveled_cdb:cdb_get(PidR,
{2,
stnd,
test_ledgerkey("Key2")}),
?assertMatch({"Value2", []}, binary_to_term(RV1)),
ok = leveled_cdb:cdb_deletepending(CDB),
ok = leveled_cdb:cdb_destroy(CDB).
compact_single_file_retain_test() ->
{Candidate,
LedgerSrv1,
LedgerFun1,
CompactFP,
CDB} = compact_single_file_setup(),
R1 = compact_files([Candidate],
#cdb_options{file_path=CompactFP},
LedgerFun1,
LedgerSrv1,
9,
[{?STD_TAG, retain}]),
{ManSlice1, PromptDelete1} = R1,
?assertMatch(true, PromptDelete1),
[{LowSQN, FN, PidR}] = ManSlice1,
io:format("FN of ~s~n", [FN]),
?assertMatch(1, LowSQN),
?assertMatch(probably,
leveled_cdb:cdb_keycheck(PidR,
{8,
stnd,
test_ledgerkey("Key1")})),
?assertMatch(missing, leveled_cdb:cdb_get(PidR,
{7,
stnd,
test_ledgerkey("Key1")})),
?assertMatch(missing, leveled_cdb:cdb_get(PidR,
{1,
stnd,
test_ledgerkey("Key1")})),
{_RK1, RV1} = leveled_cdb:cdb_get(PidR,
{2,
stnd,
test_ledgerkey("Key2")}),
?assertMatch({"Value2", []}, binary_to_term(RV1)),
ok = leveled_cdb:cdb_deletepending(CDB),
ok = leveled_cdb:cdb_destroy(CDB).
compact_empty_file_test() ->
RP = "../test/journal",
FN1 = leveled_inker:filepath(RP, 1, new_journal),
CDBopts = #cdb_options{binary_mode=true},
{ok, CDB1} = leveled_cdb:cdb_open_writer(FN1, CDBopts),
ok = leveled_cdb:cdb_put(CDB1, {1, stnd, test_ledgerkey("Key1")}, <<>>),
{ok, FN2} = leveled_cdb:cdb_complete(CDB1),
{ok, CDB2} = leveled_cdb:cdb_open_reader(FN2),
LedgerSrv1 = [{8, {o, "Bucket", "Key1", null}},
{2, {o, "Bucket", "Key2", null}},
{3, {o, "Bucket", "Key3", null}}],
LedgerFun1 = fun(_Srv, _Key, _ObjSQN) -> false end,
Score1 = check_single_file(CDB2, LedgerFun1, LedgerSrv1, 9, 8, 4),
?assertMatch(100.0, Score1).
compare_candidate_test() ->
Candidate1 = #candidate{low_sqn=1},
Candidate2 = #candidate{low_sqn=2},
Candidate3 = #candidate{low_sqn=3},
Candidate4 = #candidate{low_sqn=4},
?assertMatch([Candidate1, Candidate2, Candidate3, Candidate4],
sort_run([Candidate3, Candidate2, Candidate4, Candidate1])).
compact_singlefile_totwosmallfiles_test() ->
RP = "../test/journal",
CP = "../test/journal/journal_file/post_compact/",
ok = filelib:ensure_dir(CP),
FN1 = leveled_inker:filepath(RP, 1, new_journal),
CDBoptsLarge = #cdb_options{binary_mode=true, max_size=30000000},
{ok, CDB1} = leveled_cdb:cdb_open_writer(FN1, CDBoptsLarge),
lists:foreach(fun(X) ->
LK = test_ledgerkey("Key" ++ integer_to_list(X)),
Value = term_to_binary({crypto:rand_bytes(1024), []}),
ok = leveled_cdb:cdb_put(CDB1,
{X, ?INKT_STND, LK},
Value)
end,
lists:seq(1, 1000)),
{ok, NewName} = leveled_cdb:cdb_complete(CDB1),
{ok, CDBr} = leveled_cdb:cdb_open_reader(NewName),
CDBoptsSmall = #cdb_options{binary_mode=true, max_size=400000, file_path=CP},
BestRun1 = [#candidate{low_sqn=1,
filename=leveled_cdb:cdb_filename(CDBr),
journal=CDBr,
compaction_perc=50.0}],
FakeFilterFun = fun(_FS, _LK, SQN) -> SQN rem 2 == 0 end,
{ManifestSlice, PromptDelete} = compact_files(BestRun1,
CDBoptsSmall,
FakeFilterFun,
null,
900,
[{?STD_TAG, recovr}]),
?assertMatch(2, length(ManifestSlice)),
?assertMatch(true, PromptDelete),
lists:foreach(fun({_SQN, _FN, CDB}) ->
ok = leveled_cdb:cdb_deletepending(CDB),
ok = leveled_cdb:cdb_destroy(CDB)
end,
ManifestSlice),
ok = leveled_cdb:cdb_deletepending(CDBr),
ok = leveled_cdb:cdb_destroy(CDBr).
-endif.

916
src/leveled_inker.erl Normal file
View file

@ -0,0 +1,916 @@
%% -------- Inker ---------
%%
%% The Inker is responsible for managing access and updates to the Journal.
%%
%% The Inker maintains a manifest of what files make up the Journal, and which
%% file is the current append-only nursery log to accept new PUTs into the
%% Journal. The Inker also marshals GET requests to the appropriate database
%% file within the Journal (routed by sequence number). The Inker is also
%% responsible for scheduling compaction work to be carried out by the Inker's
%% clerk.
%%
%% -------- Journal Files ---------
%%
%% The Journal is a series of files originally named as <SQN>_<GUID>
%% where the sequence number is the first object sequence number (key) within
%% the given database file. The files will be named *.cdb at the point they
%% have been made immutable (through a rename operation). Prior to this, they
%% will originally start out as a *.pnd file.
%%
%% At some stage in the future compacted versions of old journal cdb files may
%% be produced. These files will be named <SQN>-<NewGUID>.cdb, and once
%% the manifest is updated the original <SQN>_<GUID>.cdb (or
%% <SQN>_<previousGUID>.cdb) files they replace will be erased.
%%
%% The current Journal is made up of a set of files referenced in the manifest.
%% No PUTs are made to files which are not in the manifest.
%%
%% The Journal is ordered by sequence number from front to back both within
%% and across files.
%%
%% On startup the Inker should open the manifest with the highest sequence
%% number, and this will contain the list of filenames that make up the
%% non-recent part of the Journal. All the filenames should then be opened.
%% How they are opened depends on the file extension:
%%
%% - If the file extension is *.cdb the file is opened read only
%% - If the file extension is *.pnd and the file is not the most recent in the
%% manifest, then the file should be completed bfore being opened read-only
%% - If the file extension is *.pnd the file is opened for writing
%%
%% -------- Manifest Files ---------
%%
%% The manifest is just saved as a straight term_to_binary blob, with a
%% filename ordered by the Manifest SQN. The Manifest is first saved with a
%% *.pnd extension, and then renamed to one with a *.man extension.
%%
%% On startup the *.man manifest file with the highest manifest sequence
%% number should be used.
%%
%% -------- Objects ---------
%%
%% From the perspective of the Inker, objects to store are made up of:
%% - An Inker Key formed from
%% - A sequence number (assigned by the Inker)
%% - An Inker key type (stnd, tomb or keyd)
%% - A Ledger Key (as an Erlang term)
%% - A value formed from
%% - An object (an Erlang term) which should be null for tomb types, and
%% maybe null for keyd types
%% - A set of Key Deltas associated with the change (which may be an
%% empty list )
%%
%% Note that only the Inker key type of stnd is directly fetchable, other
%% key types are to be found only in scans and so can be added without being
%% entered into the hashtree
%%
%% -------- Compaction ---------
%%
%% Compaction is a process whereby an Inker's clerk will:
%% - Request a view of the current Inker manifest and a snaphot of the Ledger
%% - Test all files within the Journal to find the approximate comapction
%% potential percentage (the volume of the Journal that has been replaced)
%% - Attempts to find the optimal "run" of files to compact
%% - Compacts those files in the run, by rolling over the files re-writing
%% to a new Journal if and only if the Key is still present in the Ledger (or
%% the sequence number of the Key is higher than the SQN of the snapshot)
%% - Requests the Inker update the manifest with the new changes
%% - Instructs the files to destroy themselves when they are next closed
%%
%% TODO: how to instruct the files to close is tbd
%%
-module(leveled_inker).
-behaviour(gen_server).
-include("include/leveled.hrl").
-export([init/1,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2,
code_change/3,
ink_start/1,
ink_put/4,
ink_get/3,
ink_fetch/3,
ink_keycheck/3,
ink_loadpcl/4,
ink_registersnapshot/2,
ink_confirmdelete/2,
ink_compactjournal/3,
ink_compactioncomplete/1,
ink_compactionpending/1,
ink_getmanifest/1,
ink_updatemanifest/3,
ink_print_manifest/1,
ink_close/1,
build_dummy_journal/0,
simple_manifest_reader/2,
clean_testdir/1,
filepath/2,
filepath/3]).
-include_lib("eunit/include/eunit.hrl").
-define(MANIFEST_FP, "journal_manifest").
-define(FILES_FP, "journal_files").
-define(COMPACT_FP, "post_compact").
-define(JOURNAL_FILEX, "cdb").
-define(MANIFEST_FILEX, "man").
-define(PENDING_FILEX, "pnd").
-define(LOADING_PAUSE, 1000).
-define(LOADING_BATCH, 1000).
-record(state, {manifest = [] :: list(),
manifest_sqn = 0 :: integer(),
journal_sqn = 0 :: integer(),
active_journaldb :: pid(),
pending_removals = [] :: list(),
registered_snapshots = [] :: list(),
root_path :: string(),
cdb_options :: #cdb_options{},
clerk :: pid(),
compaction_pending = false :: boolean(),
is_snapshot = false :: boolean(),
source_inker :: pid()}).
%%%============================================================================
%%% API
%%%============================================================================
ink_start(InkerOpts) ->
gen_server:start(?MODULE, [InkerOpts], []).
ink_put(Pid, PrimaryKey, Object, KeyChanges) ->
gen_server:call(Pid, {put, PrimaryKey, Object, KeyChanges}, infinity).
ink_get(Pid, PrimaryKey, SQN) ->
gen_server:call(Pid, {get, PrimaryKey, SQN}, infinity).
ink_fetch(Pid, PrimaryKey, SQN) ->
gen_server:call(Pid, {fetch, PrimaryKey, SQN}, infinity).
ink_keycheck(Pid, PrimaryKey, SQN) ->
gen_server:call(Pid, {key_check, PrimaryKey, SQN}, infinity).
ink_registersnapshot(Pid, Requestor) ->
gen_server:call(Pid, {register_snapshot, Requestor}, infinity).
ink_releasesnapshot(Pid, Snapshot) ->
gen_server:call(Pid, {release_snapshot, Snapshot}, infinity).
ink_confirmdelete(Pid, ManSQN) ->
gen_server:call(Pid, {confirm_delete, ManSQN}, 1000).
ink_close(Pid) ->
gen_server:call(Pid, close, infinity).
ink_loadpcl(Pid, MinSQN, FilterFun, Penciller) ->
gen_server:call(Pid, {load_pcl, MinSQN, FilterFun, Penciller}, infinity).
ink_compactjournal(Pid, Bookie, Timeout) ->
CheckerInitiateFun = fun initiate_penciller_snapshot/1,
CheckerFilterFun = fun leveled_penciller:pcl_checksequencenumber/3,
gen_server:call(Pid,
{compact,
Bookie,
CheckerInitiateFun,
CheckerFilterFun,
Timeout},
infinity).
%% Allows the Checker to be overriden in test, use something other than a
%% penciller
ink_compactjournal(Pid, Checker, InitiateFun, FilterFun, Timeout) ->
gen_server:call(Pid,
{compact,
Checker,
InitiateFun,
FilterFun,
Timeout},
infinity).
ink_compactioncomplete(Pid) ->
gen_server:call(Pid, compaction_complete, infinity).
ink_compactionpending(Pid) ->
gen_server:call(Pid, compaction_pending, infinity).
ink_getmanifest(Pid) ->
gen_server:call(Pid, get_manifest, infinity).
ink_updatemanifest(Pid, ManifestSnippet, DeletedFiles) ->
gen_server:call(Pid,
{update_manifest,
ManifestSnippet,
DeletedFiles},
infinity).
ink_print_manifest(Pid) ->
gen_server:call(Pid, print_manifest, infinity).
%%%============================================================================
%%% gen_server callbacks
%%%============================================================================
init([InkerOpts]) ->
case {InkerOpts#inker_options.root_path,
InkerOpts#inker_options.start_snapshot} of
{undefined, true} ->
SrcInker = InkerOpts#inker_options.source_inker,
{Manifest,
ActiveJournalDB} = ink_registersnapshot(SrcInker, self()),
{ok, #state{manifest=Manifest,
active_journaldb=ActiveJournalDB,
source_inker=SrcInker,
is_snapshot=true}};
%% Need to do something about timeout
{_RootPath, false} ->
start_from_file(InkerOpts)
end.
handle_call({put, Key, Object, KeyChanges}, _From, State) ->
case put_object(Key, Object, KeyChanges, State) of
{ok, UpdState, ObjSize} ->
{reply, {ok, UpdState#state.journal_sqn, ObjSize}, UpdState};
{rolling, UpdState, ObjSize} ->
ok = leveled_cdb:cdb_roll(State#state.active_journaldb),
{reply, {ok, UpdState#state.journal_sqn, ObjSize}, UpdState}
end;
handle_call({fetch, Key, SQN}, _From, State) ->
case get_object(Key, SQN, State#state.manifest) of
{{SQN, Key}, {Value, _IndexSpecs}} ->
{reply, {ok, Value}, State};
Other ->
leveled_log:log("I0001", [Key, SQN, Other]),
{reply, not_present, State}
end;
handle_call({get, Key, SQN}, _From, State) ->
{reply, get_object(Key, SQN, State#state.manifest), State};
handle_call({key_check, Key, SQN}, _From, State) ->
{reply, key_check(Key, SQN, State#state.manifest), State};
handle_call({load_pcl, StartSQN, FilterFun, Penciller}, _From, State) ->
Manifest = lists:reverse(State#state.manifest),
Reply = load_from_sequence(StartSQN, FilterFun, Penciller, Manifest),
{reply, Reply, State};
handle_call({register_snapshot, Requestor}, _From , State) ->
Rs = [{Requestor,
State#state.manifest_sqn}|State#state.registered_snapshots],
leveled_log:log("I0002", [Requestor, State#state.manifest_sqn]),
{reply, {State#state.manifest,
State#state.active_journaldb},
State#state{registered_snapshots=Rs}};
handle_call({release_snapshot, Snapshot}, _From , State) ->
Rs = lists:keydelete(Snapshot, 1, State#state.registered_snapshots),
leveled_log:log("I0003", [Snapshot]),
leveled_log:log("I0004", [length(Rs)]),
{reply, ok, State#state{registered_snapshots=Rs}};
handle_call({confirm_delete, ManSQN}, _From, State) ->
Reply = lists:foldl(fun({_R, SnapSQN}, Bool) ->
case SnapSQN >= ManSQN of
true ->
Bool;
false ->
false
end end,
true,
State#state.registered_snapshots),
{reply, Reply, State};
handle_call(get_manifest, _From, State) ->
{reply, State#state.manifest, State};
handle_call({update_manifest,
ManifestSnippet,
DeletedFiles}, _From, State) ->
Man0 = lists:foldl(fun(ManEntry, AccMan) ->
remove_from_manifest(AccMan, ManEntry)
end,
State#state.manifest,
DeletedFiles),
Man1 = lists:foldl(fun(ManEntry, AccMan) ->
add_to_manifest(AccMan, ManEntry) end,
Man0,
ManifestSnippet),
NewManifestSQN = State#state.manifest_sqn + 1,
manifest_printer(Man1),
simple_manifest_writer(Man1, NewManifestSQN, State#state.root_path),
{reply,
{ok, NewManifestSQN},
State#state{manifest=Man1,
manifest_sqn=NewManifestSQN,
pending_removals=DeletedFiles}};
handle_call(print_manifest, _From, State) ->
manifest_printer(State#state.manifest),
{reply, ok, State};
handle_call({compact,
Checker,
InitiateFun,
FilterFun,
Timeout},
_From, State) ->
leveled_iclerk:clerk_compact(State#state.clerk,
Checker,
InitiateFun,
FilterFun,
self(),
Timeout),
{reply, ok, State#state{compaction_pending=true}};
handle_call(compaction_complete, _From, State) ->
{reply, ok, State#state{compaction_pending=false}};
handle_call(compaction_pending, _From, State) ->
{reply, State#state.compaction_pending, State};
handle_call(close, _From, State) ->
{stop, normal, ok, State}.
handle_cast(_Msg, State) ->
{noreply, State}.
handle_info(_Info, State) ->
{noreply, State}.
terminate(Reason, State) ->
case State#state.is_snapshot of
true ->
ok = ink_releasesnapshot(State#state.source_inker, self());
false ->
leveled_log:log("I0005", [Reason]),
leveled_log:log("I0006", [State#state.journal_sqn,
State#state.manifest_sqn]),
leveled_iclerk:clerk_stop(State#state.clerk),
lists:foreach(fun({Snap, _SQN}) -> ok = ink_close(Snap) end,
State#state.registered_snapshots),
leveled_log:log("I0007", []),
manifest_printer(State#state.manifest),
ok = close_allmanifest(State#state.manifest)
end.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%%============================================================================
%%% Internal functions
%%%============================================================================
start_from_file(InkOpts) ->
RootPath = InkOpts#inker_options.root_path,
CDBopts = InkOpts#inker_options.cdb_options,
JournalFP = filepath(RootPath, journal_dir),
filelib:ensure_dir(JournalFP),
CompactFP = filepath(RootPath, journal_compact_dir),
filelib:ensure_dir(CompactFP),
ManifestFP = filepath(RootPath, manifest_dir),
ok = filelib:ensure_dir(ManifestFP),
{ok, ManifestFilenames} = file:list_dir(ManifestFP),
IClerkCDBOpts = CDBopts#cdb_options{file_path = CompactFP},
ReloadStrategy = InkOpts#inker_options.reload_strategy,
MRL = InkOpts#inker_options.max_run_length,
IClerkOpts = #iclerk_options{inker = self(),
cdb_options=IClerkCDBOpts,
reload_strategy = ReloadStrategy,
max_run_length = MRL},
{ok, Clerk} = leveled_iclerk:clerk_new(IClerkOpts),
{Manifest,
ManifestSQN,
JournalSQN,
ActiveJournal} = build_manifest(ManifestFilenames,
RootPath,
CDBopts),
{ok, #state{manifest = Manifest,
manifest_sqn = ManifestSQN,
journal_sqn = JournalSQN,
active_journaldb = ActiveJournal,
root_path = RootPath,
cdb_options = CDBopts,
clerk = Clerk}}.
put_object(LedgerKey, Object, KeyChanges, State) ->
NewSQN = State#state.journal_sqn + 1,
{JournalKey, JournalBin} = leveled_codec:to_inkerkv(LedgerKey,
NewSQN,
Object,
KeyChanges),
case leveled_cdb:cdb_put(State#state.active_journaldb,
JournalKey,
JournalBin) of
ok ->
{ok, State#state{journal_sqn=NewSQN}, byte_size(JournalBin)};
roll ->
SW = os:timestamp(),
CDBopts = State#state.cdb_options,
ManEntry = start_new_activejournal(NewSQN,
State#state.root_path,
CDBopts),
{_, _, NewJournalP} = ManEntry,
NewManifest = add_to_manifest(State#state.manifest, ManEntry),
ok = simple_manifest_writer(NewManifest,
State#state.manifest_sqn + 1,
State#state.root_path),
ok = leveled_cdb:cdb_put(NewJournalP,
JournalKey,
JournalBin),
leveled_log:log_timer("I0008", [], SW),
{rolling,
State#state{journal_sqn=NewSQN,
manifest=NewManifest,
manifest_sqn = State#state.manifest_sqn + 1,
active_journaldb=NewJournalP},
byte_size(JournalBin)}
end.
get_object(LedgerKey, SQN, Manifest) ->
JournalP = find_in_manifest(SQN, Manifest),
{InkerKey, _V, true} = leveled_codec:to_inkerkv(LedgerKey,
SQN,
to_fetch,
null),
Obj = leveled_cdb:cdb_get(JournalP, InkerKey),
leveled_codec:from_inkerkv(Obj).
key_check(LedgerKey, SQN, Manifest) ->
JournalP = find_in_manifest(SQN, Manifest),
{InkerKey, _V, true} = leveled_codec:to_inkerkv(LedgerKey,
SQN,
to_fetch,
null),
leveled_cdb:cdb_keycheck(JournalP, InkerKey).
build_manifest(ManifestFilenames,
RootPath,
CDBopts) ->
% Find the manifest with a highest Manifest sequence number
% Open it and read it to get the current Confirmed Manifest
ManifestRegex = "(?<MSQN>[0-9]+)\\." ++ ?MANIFEST_FILEX,
ValidManSQNs = sequencenumbers_fromfilenames(ManifestFilenames,
ManifestRegex,
'MSQN'),
{Manifest,
ManifestSQN} = case length(ValidManSQNs) of
0 ->
{[], 1};
_ ->
PersistedManSQN = lists:max(ValidManSQNs),
M1 = simple_manifest_reader(PersistedManSQN,
RootPath),
{M1, PersistedManSQN}
end,
% Open the manifest files, completing if necessary and ensure there is
% a valid active journal at the head of the manifest
OpenManifest = open_all_manifest(Manifest, RootPath, CDBopts),
{ActiveLowSQN, _FN, ActiveJournal} = lists:nth(1, OpenManifest),
JournalSQN = case leveled_cdb:cdb_lastkey(ActiveJournal) of
empty ->
ActiveLowSQN;
{JSQN, _Type, _LastKey} ->
JSQN
end,
% Update the manifest if it has been changed by the process of laoding
% the manifest (must also increment the manifest SQN).
UpdManifestSQN = if
length(OpenManifest) > length(Manifest) ->
leveled_log:log("I0009", []),
manifest_printer(OpenManifest),
simple_manifest_writer(OpenManifest,
ManifestSQN + 1,
RootPath),
ManifestSQN + 1;
true ->
leveled_log:log("I0010", []),
manifest_printer(OpenManifest),
ManifestSQN
end,
{OpenManifest, UpdManifestSQN, JournalSQN, ActiveJournal}.
close_allmanifest([]) ->
ok;
close_allmanifest([H|ManifestT]) ->
{_, _, Pid} = H,
ok = leveled_cdb:cdb_close(Pid),
close_allmanifest(ManifestT).
open_all_manifest([], RootPath, CDBOpts) ->
leveled_log:log("I0011", []),
add_to_manifest([], start_new_activejournal(1, RootPath, CDBOpts));
open_all_manifest(Man0, RootPath, CDBOpts) ->
Man1 = lists:reverse(lists:sort(Man0)),
[{HeadSQN, HeadFN}|ManifestTail] = Man1,
CompleteHeadFN = HeadFN ++ "." ++ ?JOURNAL_FILEX,
PendingHeadFN = HeadFN ++ "." ++ ?PENDING_FILEX,
Man2 = case filelib:is_file(CompleteHeadFN) of
true ->
leveled_log:log("I0012", [HeadFN]),
{ok, HeadR} = leveled_cdb:cdb_open_reader(CompleteHeadFN),
{LastSQN, _Type, _PK} = leveled_cdb:cdb_lastkey(HeadR),
add_to_manifest(add_to_manifest(ManifestTail,
{HeadSQN, HeadFN, HeadR}),
start_new_activejournal(LastSQN + 1,
RootPath,
CDBOpts));
false ->
{ok, HeadW} = leveled_cdb:cdb_open_writer(PendingHeadFN,
CDBOpts),
add_to_manifest(ManifestTail, {HeadSQN, HeadFN, HeadW})
end,
lists:map(fun(ManEntry) ->
case ManEntry of
{LowSQN, FN} ->
CFN = FN ++ "." ++ ?JOURNAL_FILEX,
PFN = FN ++ "." ++ ?PENDING_FILEX,
case filelib:is_file(CFN) of
true ->
{ok,
Pid} = leveled_cdb:cdb_open_reader(CFN),
{LowSQN, FN, Pid};
false ->
W = leveled_cdb:cdb_open_writer(PFN, CDBOpts),
{ok, Pid} = W,
ok = leveled_cdb:cdb_roll(Pid),
{LowSQN, FN, Pid}
end;
_ ->
ManEntry
end end,
Man2).
start_new_activejournal(SQN, RootPath, CDBOpts) ->
Filename = filepath(RootPath, SQN, new_journal),
{ok, PidW} = leveled_cdb:cdb_open_writer(Filename, CDBOpts),
{SQN, Filename, PidW}.
add_to_manifest(Manifest, Entry) ->
{SQN, FN, PidR} = Entry,
StrippedName = filename:rootname(FN),
lists:reverse(lists:sort([{SQN, StrippedName, PidR}|Manifest])).
remove_from_manifest(Manifest, Entry) ->
{SQN, FN, _PidR} = Entry,
leveled_log:log("I0013", [FN]),
lists:keydelete(SQN, 1, Manifest).
find_in_manifest(SQN, [{LowSQN, _FN, Pid}|_Tail]) when SQN >= LowSQN ->
Pid;
find_in_manifest(SQN, [_Head|Tail]) ->
find_in_manifest(SQN, Tail).
%% Scan between sequence numbers applying FilterFun to each entry where
%% FilterFun{K, V, Acc} -> Penciller Key List
%% Load the output for the CDB file into the Penciller.
load_from_sequence(_MinSQN, _FilterFun, _Penciller, []) ->
ok;
load_from_sequence(MinSQN, FilterFun, Penciller, [{LowSQN, FN, Pid}|Rest])
when LowSQN >= MinSQN ->
load_between_sequence(MinSQN,
MinSQN + ?LOADING_BATCH,
FilterFun,
Penciller,
Pid,
undefined,
FN,
Rest);
load_from_sequence(MinSQN, FilterFun, Penciller, [{_LowSQN, FN, Pid}|Rest]) ->
case Rest of
[] ->
load_between_sequence(MinSQN,
MinSQN + ?LOADING_BATCH,
FilterFun,
Penciller,
Pid,
undefined,
FN,
Rest);
[{NextSQN, _NxtFN, _NxtPid}|_Rest] when NextSQN > MinSQN ->
load_between_sequence(MinSQN,
MinSQN + ?LOADING_BATCH,
FilterFun,
Penciller,
Pid,
undefined,
FN,
Rest);
_ ->
load_from_sequence(MinSQN, FilterFun, Penciller, Rest)
end.
load_between_sequence(MinSQN, MaxSQN, FilterFun, Penciller,
CDBpid, StartPos, FN, Rest) ->
leveled_log:log("I0014", [FN, MinSQN]),
InitAcc = {MinSQN, MaxSQN, gb_trees:empty()},
Res = case leveled_cdb:cdb_scan(CDBpid, FilterFun, InitAcc, StartPos) of
{eof, {AccMinSQN, _AccMaxSQN, AccKL}} ->
ok = push_to_penciller(Penciller, AccKL),
{ok, AccMinSQN};
{LastPosition, {_AccMinSQN, _AccMaxSQN, AccKL}} ->
ok = push_to_penciller(Penciller, AccKL),
NextSQN = MaxSQN + 1,
load_between_sequence(NextSQN,
NextSQN + ?LOADING_BATCH,
FilterFun,
Penciller,
CDBpid,
LastPosition,
FN,
Rest)
end,
case Res of
{ok, LMSQN} ->
load_from_sequence(LMSQN, FilterFun, Penciller, Rest);
ok ->
ok
end.
push_to_penciller(Penciller, KeyTree) ->
% The push to penciller must start as a tree to correctly de-duplicate
% the list by order before becoming a de-duplicated list for loading
R = leveled_penciller:pcl_pushmem(Penciller, KeyTree),
case R of
returned ->
timer:sleep(?LOADING_PAUSE),
push_to_penciller(Penciller, KeyTree);
ok ->
ok
end.
sequencenumbers_fromfilenames(Filenames, Regex, IntName) ->
lists:foldl(fun(FN, Acc) ->
case re:run(FN,
Regex,
[{capture, [IntName], list}]) of
nomatch ->
Acc;
{match, [Int]} when is_list(Int) ->
Acc ++ [list_to_integer(Int)]
end end,
[],
Filenames).
filepath(RootPath, journal_dir) ->
RootPath ++ "/" ++ ?FILES_FP ++ "/";
filepath(RootPath, manifest_dir) ->
RootPath ++ "/" ++ ?MANIFEST_FP ++ "/";
filepath(RootPath, journal_compact_dir) ->
filepath(RootPath, journal_dir) ++ "/" ++ ?COMPACT_FP ++ "/".
filepath(RootPath, NewSQN, new_journal) ->
filename:join(filepath(RootPath, journal_dir),
integer_to_list(NewSQN) ++ "_"
++ leveled_codec:generate_uuid()
++ "." ++ ?PENDING_FILEX);
filepath(CompactFilePath, NewSQN, compact_journal) ->
filename:join(CompactFilePath,
integer_to_list(NewSQN) ++ "_"
++ leveled_codec:generate_uuid()
++ "." ++ ?PENDING_FILEX).
simple_manifest_reader(SQN, RootPath) ->
ManifestPath = filepath(RootPath, manifest_dir),
leveled_log:log("I0015", [ManifestPath, SQN]),
{ok, MBin} = file:read_file(filename:join(ManifestPath,
integer_to_list(SQN)
++ ".man")),
binary_to_term(MBin).
simple_manifest_writer(Manifest, ManSQN, RootPath) ->
ManPath = filepath(RootPath, manifest_dir),
NewFN = filename:join(ManPath,
integer_to_list(ManSQN) ++ "." ++ ?MANIFEST_FILEX),
TmpFN = filename:join(ManPath,
integer_to_list(ManSQN) ++ "." ++ ?PENDING_FILEX),
MBin = term_to_binary(lists:map(fun({SQN, FN, _PID}) -> {SQN, FN} end,
Manifest), [compressed]),
case filelib:is_file(NewFN) of
false ->
leveled_log:log("I0016", [ManSQN]),
ok = file:write_file(TmpFN, MBin),
ok = file:rename(TmpFN, NewFN),
ok
end.
manifest_printer(Manifest) ->
lists:foreach(fun({SQN, FN, _PID}) ->
leveled_log:log("I0017", [SQN, FN]) end,
Manifest).
initiate_penciller_snapshot(Bookie) ->
{ok,
{LedgerSnap, LedgerCache},
_} = leveled_bookie:book_snapshotledger(Bookie, self(), undefined),
ok = leveled_penciller:pcl_loadsnapshot(LedgerSnap, LedgerCache),
MaxSQN = leveled_penciller:pcl_getstartupsequencenumber(LedgerSnap),
{LedgerSnap, MaxSQN}.
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
build_dummy_journal() ->
F = fun(X) -> X end,
build_dummy_journal(F).
build_dummy_journal(KeyConvertF) ->
RootPath = "../test/journal",
clean_testdir(RootPath),
JournalFP = filepath(RootPath, journal_dir),
ManifestFP = filepath(RootPath, manifest_dir),
ok = filelib:ensure_dir(RootPath),
ok = filelib:ensure_dir(JournalFP),
ok = filelib:ensure_dir(ManifestFP),
F1 = filename:join(JournalFP, "nursery_1.pnd"),
{ok, J1} = leveled_cdb:cdb_open_writer(F1),
{K1, V1} = {KeyConvertF("Key1"), "TestValue1"},
{K2, V2} = {KeyConvertF("Key2"), "TestValue2"},
ok = leveled_cdb:cdb_put(J1, {1, stnd, K1}, term_to_binary({V1, []})),
ok = leveled_cdb:cdb_put(J1, {2, stnd, K2}, term_to_binary({V2, []})),
ok = leveled_cdb:cdb_roll(J1),
_LK = leveled_cdb:cdb_lastkey(J1),
ok = leveled_cdb:cdb_close(J1),
F2 = filename:join(JournalFP, "nursery_3.pnd"),
{ok, J2} = leveled_cdb:cdb_open_writer(F2),
{K1, V3} = {KeyConvertF("Key1"), "TestValue3"},
{K4, V4} = {KeyConvertF("Key4"), "TestValue4"},
ok = leveled_cdb:cdb_put(J2, {3, stnd, K1}, term_to_binary({V3, []})),
ok = leveled_cdb:cdb_put(J2, {4, stnd, K4}, term_to_binary({V4, []})),
ok = leveled_cdb:cdb_close(J2),
Manifest = [{1, "../test/journal/journal_files/nursery_1"},
{3, "../test/journal/journal_files/nursery_3"}],
ManifestBin = term_to_binary(Manifest),
{ok, MF1} = file:open(filename:join(ManifestFP, "1.man"),
[binary, raw, read, write]),
ok = file:write(MF1, ManifestBin),
ok = file:close(MF1).
clean_testdir(RootPath) ->
clean_subdir(filepath(RootPath, journal_dir)),
clean_subdir(filepath(RootPath, journal_compact_dir)),
clean_subdir(filepath(RootPath, manifest_dir)).
clean_subdir(DirPath) ->
ok = filelib:ensure_dir(DirPath),
{ok, Files} = file:list_dir(DirPath),
lists:foreach(fun(FN) ->
File = filename:join(DirPath, FN),
case file:delete(File) of
ok -> io:format("Success deleting ~s~n", [File]);
_ -> io:format("Error deleting ~s~n", [File])
end
end,
Files).
simple_inker_test() ->
RootPath = "../test/journal",
build_dummy_journal(),
CDBopts = #cdb_options{max_size=300000},
{ok, Ink1} = ink_start(#inker_options{root_path=RootPath,
cdb_options=CDBopts}),
Obj1 = ink_get(Ink1, "Key1", 1),
?assertMatch({{1, "Key1"}, {"TestValue1", []}}, Obj1),
Obj2 = ink_get(Ink1, "Key4", 4),
?assertMatch({{4, "Key4"}, {"TestValue4", []}}, Obj2),
ink_close(Ink1),
clean_testdir(RootPath).
simple_inker_completeactivejournal_test() ->
RootPath = "../test/journal",
build_dummy_journal(),
CDBopts = #cdb_options{max_size=300000},
JournalFP = filepath(RootPath, journal_dir),
F2 = filename:join(JournalFP, "nursery_3.pnd"),
{ok, PidW} = leveled_cdb:cdb_open_writer(F2),
{ok, _F2} = leveled_cdb:cdb_complete(PidW),
F1 = filename:join(JournalFP, "nursery_1.cdb"),
F1r = filename:join(JournalFP, "nursery_1.pnd"),
ok = file:rename(F1, F1r),
{ok, Ink1} = ink_start(#inker_options{root_path=RootPath,
cdb_options=CDBopts}),
Obj1 = ink_get(Ink1, "Key1", 1),
?assertMatch({{1, "Key1"}, {"TestValue1", []}}, Obj1),
Obj2 = ink_get(Ink1, "Key4", 4),
?assertMatch({{4, "Key4"}, {"TestValue4", []}}, Obj2),
ink_close(Ink1),
clean_testdir(RootPath).
test_ledgerkey(Key) ->
{o, "Bucket", Key, null}.
compact_journal_test() ->
RootPath = "../test/journal",
build_dummy_journal(fun test_ledgerkey/1),
CDBopts = #cdb_options{max_size=300000},
RStrategy = [{?STD_TAG, recovr}],
{ok, Ink1} = ink_start(#inker_options{root_path=RootPath,
cdb_options=CDBopts,
reload_strategy=RStrategy}),
{ok, NewSQN1, _ObjSize} = ink_put(Ink1,
test_ledgerkey("KeyAA"),
"TestValueAA", []),
?assertMatch(NewSQN1, 5),
ok = ink_print_manifest(Ink1),
R0 = ink_get(Ink1, test_ledgerkey("KeyAA"), 5),
?assertMatch(R0, {{5, test_ledgerkey("KeyAA")}, {"TestValueAA", []}}),
FunnyLoop = lists:seq(1, 48),
Checker = lists:map(fun(X) ->
PK = "KeyZ" ++ integer_to_list(X),
{ok, SQN, _} = ink_put(Ink1,
test_ledgerkey(PK),
crypto:rand_bytes(10000),
[]),
{SQN, test_ledgerkey(PK)}
end,
FunnyLoop),
{ok, NewSQN2, _ObjSize} = ink_put(Ink1,
test_ledgerkey("KeyBB"),
"TestValueBB", []),
?assertMatch(NewSQN2, 54),
ActualManifest = ink_getmanifest(Ink1),
ok = ink_print_manifest(Ink1),
?assertMatch(3, length(ActualManifest)),
ok = ink_compactjournal(Ink1,
Checker,
fun(X) -> {X, 55} end,
fun(L, K, SQN) -> lists:member({SQN, K}, L) end,
5000),
timer:sleep(1000),
CompactedManifest1 = ink_getmanifest(Ink1),
?assertMatch(2, length(CompactedManifest1)),
Checker2 = lists:sublist(Checker, 16),
ok = ink_compactjournal(Ink1,
Checker2,
fun(X) -> {X, 55} end,
fun(L, K, SQN) -> lists:member({SQN, K}, L) end,
5000),
timer:sleep(1000),
CompactedManifest2 = ink_getmanifest(Ink1),
R = lists:foldl(fun({_SQN, FN, _P}, Acc) ->
case string:str(FN, "post_compact") of
N when N > 0 ->
true;
0 ->
Acc
end end,
false,
CompactedManifest2),
?assertMatch(true, R),
?assertMatch(2, length(CompactedManifest2)),
ink_close(Ink1),
clean_testdir(RootPath).
empty_manifest_test() ->
RootPath = "../test/journal",
clean_testdir(RootPath),
CDBopts = #cdb_options{max_size=300000},
{ok, Ink1} = ink_start(#inker_options{root_path=RootPath,
cdb_options=CDBopts}),
?assertMatch(not_present, ink_fetch(Ink1, "Key1", 1)),
ok = ink_compactjournal(Ink1,
[],
fun(X) -> {X, 55} end,
fun(L, K, SQN) -> lists:member({SQN, K}, L) end,
5000),
timer:sleep(1000),
?assertMatch(1, length(ink_getmanifest(Ink1))),
ok = ink_close(Ink1),
% Add pending manifest to be ignored
FN = filepath(RootPath, manifest_dir) ++ "999.pnd",
ok = file:write_file(FN, term_to_binary("Hello")),
{ok, Ink2} = ink_start(#inker_options{root_path=RootPath,
cdb_options=CDBopts}),
?assertMatch(not_present, ink_fetch(Ink2, "Key1", 1)),
{ok, SQN, Size} = ink_put(Ink2, "Key1", "Value1", []),
?assertMatch(2, SQN),
?assertMatch(true, Size > 0),
{ok, V} = ink_fetch(Ink2, "Key1", 2),
?assertMatch("Value1", V),
ink_close(Ink2),
clean_testdir(RootPath).
-endif.

293
src/leveled_log.erl Normal file
View file

@ -0,0 +1,293 @@
%% Module to abstract from choice of logger, and allow use of logReferences
%% for fast lookup
-module(leveled_log).
-include("include/leveled.hrl").
-include_lib("eunit/include/eunit.hrl").
-export([log/2,
log_timer/3]).
-define(LOG_LEVEL, [info, warn, error, critical]).
-define(LOGBASE, dict:from_list([
{"G0001",
{info, "Generic log point"}},
{"D0001",
{debug, "Generic debug log"}},
{"B0001",
{info, "Bookie starting with Ink ~w Pcl ~w"}},
{"B0002",
{info, "Snapshot starting with Ink ~w Pcl ~w"}},
{"B0003",
{info, "Bookie closing for reason ~w"}},
{"B0004",
{info, "Length of increment in snapshot is ~w"}},
{"B0005",
{info, "LedgerSQN=~w at startup"}},
{"B0006",
{info, "Reached end of load batch with SQN ~w"}},
{"B0007",
{info, "Skipping as exceeded MaxSQN ~w with SQN ~w"}},
{"P0001",
{info, "Ledger snapshot ~w registered"}},
{"P0003",
{info, "Ledger snapshot ~w released"}},
{"P0004",
{info, "Remaining ledger snapshots are ~w"}},
{"P0005",
{info, "Delete confirmed as file ~s is removed from " ++
"unreferenced files"}},
{"P0006",
{info, "Orphaned reply after timeout on L0 file write ~s"}},
{"P0007",
{debug, "Sent release message for cloned Penciller following close for "
++ "reason ~w"}},
{"P0008",
{info, "Penciller closing for reason ~w"}},
{"P0009",
{info, "Level 0 cache empty at close of Penciller"}},
{"P0010",
{info, "No level zero action on close of Penciller"}},
{"P0011",
{info, "Shutdown complete for Penciller"}},
{"P0012",
{info, "Store to be started based on manifest sequence number of ~w"}},
{"P0013",
{warn, "Seqence number of 0 indicates no valid manifest"}},
{"P0014",
{info, "Maximum sequence number of ~w found in nonzero levels"}},
{"P0015",
{info, "L0 file found ~s"}},
{"P0016",
{info, "L0 file had maximum sequence number of ~w"}},
{"P0017",
{info, "No L0 file found"}},
{"P0018",
{info, "Respone to push_mem of ~w ~s"}},
{"P0019",
{info, "Rolling level zero to filename ~s"}},
{"P0020",
{info, "Work at Level ~w to be scheduled for ~w with ~w "
++ "queue items outstanding at all levels"}},
{"P0021",
{info, "Allocation of work blocked as L0 pending"}},
{"P0022",
{info, "Manifest at Level ~w"}},
{"P0023",
{info, "Manifest entry of startkey ~s ~s ~s endkey ~s ~s ~s "
++ "filename=~s~n"}},
{"P0024",
{info, "Outstanding compaction work items of ~w at level ~w"}},
{"P0025",
{info, "Merge to sqn ~w from Level ~w completed"}},
{"P0026",
{info, "Merge has been commmitted at sequence number ~w"}},
{"P0027",
{info, "Rename of manifest from ~s ~w to ~s ~w"}},
{"P0028",
{info, "Adding cleared file ~s to deletion list"}},
{"P0029",
{info, "L0 completion confirmed and will transition to not pending"}},
{"PC001",
{info, "Penciller's clerk ~w started with owner ~w"}},
{"PC002",
{info, "Request for manifest change from clerk on closing"}},
{"PC003",
{info, "Confirmation of manifest change on closing"}},
{"PC004",
{info, "Prompted confirmation of manifest change"}},
{"PC005",
{info, "Penciller's Clerk ~w shutdown now complete for reason ~w"}},
{"PC006",
{info, "Work prompted but none needed"}},
{"PC007",
{info, "Clerk prompting Penciller regarding manifest change"}},
{"PC008",
{info, "Merge from level ~w to merge into ~w files below"}},
{"PC009",
{info, "File ~s to simply switch levels to level ~w"}},
{"PC010",
{info, "Merge to be commenced for FileToMerge=~s with MSN=~w"}},
{"PC011",
{info, "Merge completed with MSN=~w Level=~w and FileCounter=~w"}},
{"PC012",
{info, "File to be created as part of MSN=~w Filename=~s"}},
{"PC013",
{warn, "Merge resulted in empty file ~s"}},
{"PC014",
{info, "Empty file ~s to be cleared"}},
{"PC015",
{info, "File created"}},
{"I0001",
{info, "Unexpected failure to fetch value for Key=~w SQN=~w "
++ "with reason ~w"}},
{"I0002",
{info, "Journal snapshot ~w registered at SQN ~w"}},
{"I0003",
{info, "Journal snapshot ~w released"}},
{"I0004",
{info, "Remaining number of journal snapshots is ~w"}},
{"I0005",
{info, "Inker closing journal for reason ~w"}},
{"I0006",
{info, "Close triggered with journal_sqn=~w and manifest_sqn=~w"}},
{"I0007",
{info, "Inker manifest when closing is:"}},
{"I0008",
{info, "Put to new active journal required roll and manifest write"}},
{"I0009",
{info, "Updated manifest on startup:"}},
{"I0010",
{info, "Unchanged manifest on startup:"}},
{"I0011",
{info, "Manifest is empty, starting from manifest SQN 1"}},
{"I0012",
{info, "Head manifest entry ~s is complete so new active journal "
++ "required"}},
{"I0013",
{info, "File ~s to be removed from manifest"}},
{"I0014",
{info, "On startup oading from filename ~s from SQN ~w"}},
{"I0015",
{info, "Opening manifest file at ~s with SQN ~w"}},
{"I0016",
{info, "Writing new version of manifest for manifestSQN=~w"}},
{"I0017",
{info, "At SQN=~w journal has filename ~s"}},
{"IC001",
{info, "Inker no longer alive so Clerk to abandon work "
++ "leaving garbage"}},
{"IC002",
{info, "Clerk updating Inker as compaction complete of ~w files"}},
{"IC003",
{info, "No compaction run as highest score=~w"}},
{"IC004",
{info, "Score for filename ~s is ~w"}},
{"IC005",
{info, "Compaction to be performed on ~w files with score of ~w"}},
{"IC006",
{info, "Filename ~s is part of compaction run"}},
{"IC007",
{info, "Clerk has completed compaction process"}},
{"IC008",
{info, "Compaction source ~s has yielded ~w positions"}},
{"IC009",
{info, "Generate journal for compaction with filename ~s"}},
{"PM001",
{info, "Indexed new cache entry with total L0 cache size now ~w"}},
{"PM002",
{info, "Completed dump of L0 cache to list of size ~w"}},
{"SFT01",
{info, "Opened filename with name ~s"}},
{"SFT02",
{info, "File ~s has been set for delete"}},
{"SFT03",
{info, "File creation of L0 file ~s"}},
{"SFT04",
{info, "File ~s prompting for delete status check"}},
{"SFT05",
{info, "Exit called for reason ~w on filename ~s"}},
{"SFT06",
{info, "Exit called and now clearing ~s"}},
{"SFT07",
{info, "Creating file with input of size ~w"}},
{"SFT08",
{info, "Renaming file from ~s to ~s"}},
{"SFT09",
{warn, "Filename ~s already exists"}},
{"SFT10",
{warn, "Rename rogue filename ~s to ~s"}},
{"SFT11",
{error, "Segment filter failed due to ~s"}},
{"SFT12",
{error, "Segment filter failed due to CRC check ~w did not match ~w"}},
{"SFT13",
{error, "Segment filter failed due to ~s"}},
{"CDB01",
{info, "Opening file for writing with filename ~s"}},
{"CDB02",
{info, "Opening file for reading with filename ~s"}},
{"CDB03",
{info, "Re-opening file for reading with filename ~s"}},
{"CDB04",
{info, "Deletion confirmed for file ~s at ManifestSQN ~w"}},
{"CDB05",
{info, "Closing of filename ~s for Reason ~w"}},
{"CDB06",
{info, "File to be truncated at last position of ~w with end of "
++ "file at ~w"}},
{"CDB07",
{info, "Hashtree computed"}},
{"CDB08",
{info, "Renaming file from ~s to ~s for which existence is ~w"}},
{"CDB09",
{info, "Failure to read Key/Value at Position ~w in scan"}},
{"CDB10",
{info, "CRC check failed due to mismatch"}},
{"CDB11",
{info, "CRC check failed due to size"}},
{"CDB12",
{inof, "HashTree written"}}
])).
log(LogReference, Subs) ->
{ok, {LogLevel, LogText}} = dict:find(LogReference, ?LOGBASE),
case lists:member(LogLevel, ?LOG_LEVEL) of
true ->
io:format(LogReference ++ " ~w " ++ LogText ++ "~n",
[self()|Subs]);
false ->
ok
end.
log_timer(LogReference, Subs, StartTime) ->
{ok, {LogLevel, LogText}} = dict:find(LogReference, ?LOGBASE),
case lists:member(LogLevel, ?LOG_LEVEL) of
true ->
MicroS = timer:now_diff(os:timestamp(), StartTime),
{Unit, Time} = case MicroS of
MicroS when MicroS < 1000 ->
{"microsec", MicroS};
MicroS ->
{"ms", MicroS div 1000}
end,
io:format(LogReference ++ " ~w " ++ LogText
++ " with time taken ~w " ++ Unit ++ "~n",
[self()|Subs] ++ [Time]);
false ->
ok
end.
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
log_test() ->
log("D0001", []),
log_timer("D0001", [], os:timestamp()).
-endif.

462
src/leveled_pclerk.erl Normal file
View file

@ -0,0 +1,462 @@
%% -------- PENCILLER's CLERK ---------
%%
%% The Penciller's clerk is responsible for compaction work within the Ledger.
%%
%% The Clerk will periodically poll the Penciller to see if there is work for
%% it to complete, except if the Clerk has informed the Penciller that it has
%% readied a manifest change to be committed - in which case it will wait to
%% be called by the Penciller.
%%
%% -------- COMMITTING MANIFEST CHANGES ---------
%%
%% Once the Penciller has taken a manifest change, the SFT file owners which no
%% longer form part of the manifest will be marked for delete. By marking for
%% deletion, the owners will poll to confirm when it is safe for them to be
%% deleted.
%%
%% It is imperative that the file is not marked for deletion until it is
%% certain that the manifest change has been committed. Some uncollected
%% garbage is considered acceptable.
%%
%% The process of committing a manifest change is as follows:
%%
%% A - The Clerk completes a merge, and casts a prompt to the Penciller with
%% a work item describing the change
%%
%% B - The Penciller commits the change to disk, and then calls the Clerk to
%% confirm the manifest change
%%
%% C - The Clerk replies immediately to acknowledge this call, then marks the
%% removed files for deletion
%%
%% Shutdown < A/B - If the Penciller starts the shutdown process before the
%% merge is complete, in the shutdown the Penciller will call a request for the
%% manifest change which will pick up the pending change. It will then confirm
%% the change, and now the Clerk will mark the files for delete before it
%% replies to the Penciller so it can complete the shutdown process (which will
%% prompt erasing of the removed files).
%%
%% The clerk will not request work on timeout if the committing of a manifest
%% change is pending confirmation.
%%
%% -------- TIMEOUTS ---------
%%
%% The Penciller may prompt the Clerk to callback soon (i.e. reduce the
%% Timeout) if it has urgent work ready (i.e. it has written a L0 file).
%%
%% There will also be a natural quick timeout once the committing of a manifest
%% change has occurred.
%%
-module(leveled_pclerk).
-behaviour(gen_server).
-include("include/leveled.hrl").
-export([init/1,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2,
clerk_new/1,
clerk_prompt/1,
clerk_manifestchange/3,
code_change/3]).
-include_lib("eunit/include/eunit.hrl").
-define(MAX_TIMEOUT, 2000).
-define(MIN_TIMEOUT, 50).
-record(state, {owner :: pid(),
change_pending=false :: boolean(),
work_item :: #penciller_work{}|null}).
%%%============================================================================
%%% API
%%%============================================================================
clerk_new(Owner) ->
{ok, Pid} = gen_server:start(?MODULE, [], []),
ok = gen_server:call(Pid, {register, Owner}, infinity),
leveled_log:log("PC001", [Pid, Owner]),
{ok, Pid}.
clerk_manifestchange(Pid, Action, Closing) ->
gen_server:call(Pid, {manifest_change, Action, Closing}, infinity).
clerk_prompt(Pid) ->
gen_server:cast(Pid, prompt).
%%%============================================================================
%%% gen_server callbacks
%%%============================================================================
init([]) ->
{ok, #state{}}.
handle_call({register, Owner}, _From, State) ->
{reply,
ok,
State#state{owner=Owner},
?MIN_TIMEOUT};
handle_call({manifest_change, return, true}, _From, State) ->
leveled_log:log("PC002", []),
case State#state.change_pending of
true ->
WI = State#state.work_item,
{reply, {ok, WI}, State};
false ->
{stop, normal, no_change, State}
end;
handle_call({manifest_change, confirm, Closing}, From, State) ->
case Closing of
true ->
leveled_log:log("PC003", []),
WI = State#state.work_item,
ok = mark_for_delete(WI#penciller_work.unreferenced_files,
State#state.owner),
{stop, normal, ok, State};
false ->
leveled_log:log("PC004", []),
gen_server:reply(From, ok),
WI = State#state.work_item,
ok = mark_for_delete(WI#penciller_work.unreferenced_files,
State#state.owner),
{noreply,
State#state{work_item=null, change_pending=false},
?MIN_TIMEOUT}
end.
handle_cast(prompt, State) ->
{noreply, State, ?MIN_TIMEOUT}.
handle_info(timeout, State=#state{change_pending=Pnd}) when Pnd == false ->
case requestandhandle_work(State) of
{false, Timeout} ->
{noreply, State, Timeout};
{true, WI} ->
% No timeout now as will wait for call to return manifest
% change
{noreply,
State#state{change_pending=true, work_item=WI}}
end.
terminate(Reason, _State) ->
leveled_log:log("PC005", [self(), Reason]).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%%============================================================================
%%% Internal functions
%%%============================================================================
requestandhandle_work(State) ->
case leveled_penciller:pcl_workforclerk(State#state.owner) of
none ->
leveled_log:log("PC006", []),
{false, ?MAX_TIMEOUT};
WI ->
{NewManifest, FilesToDelete} = merge(WI),
UpdWI = WI#penciller_work{new_manifest=NewManifest,
unreferenced_files=FilesToDelete},
leveled_log:log("PC007", []),
ok = leveled_penciller:pcl_promptmanifestchange(State#state.owner,
UpdWI),
{true, UpdWI}
end.
merge(WI) ->
SrcLevel = WI#penciller_work.src_level,
{SrcF, UpdMFest1} = select_filetomerge(SrcLevel,
WI#penciller_work.manifest),
SinkFiles = get_item(SrcLevel + 1, UpdMFest1, []),
{Candidates, Others} = check_for_merge_candidates(SrcF, SinkFiles),
%% TODO:
%% Need to work out if this is the top level
%% And then tell merge process to create files at the top level
%% Which will include the reaping of expired tombstones
leveled_log:log("PC008", [SrcLevel, length(Candidates)]),
MergedFiles = case length(Candidates) of
0 ->
%% If no overlapping candiates, manifest change only required
%%
%% TODO: need to think still about simply renaming when at
%% lower level
leveled_log:log("PC009",
[SrcF#manifest_entry.filename, SrcLevel + 1]),
[SrcF];
_ ->
perform_merge({SrcF#manifest_entry.owner,
SrcF#manifest_entry.filename},
Candidates,
{SrcLevel, WI#penciller_work.target_is_basement},
{WI#penciller_work.ledger_filepath,
WI#penciller_work.next_sqn})
end,
NewLevel = lists:sort(lists:append(MergedFiles, Others)),
UpdMFest2 = lists:keystore(SrcLevel + 1,
1,
UpdMFest1,
{SrcLevel + 1, NewLevel}),
ok = filelib:ensure_dir(WI#penciller_work.manifest_file),
{ok, Handle} = file:open(WI#penciller_work.manifest_file,
[binary, raw, write]),
ok = file:write(Handle, term_to_binary(UpdMFest2)),
ok = file:close(Handle),
case lists:member(SrcF, MergedFiles) of
true ->
{UpdMFest2, Candidates};
false ->
%% Can rub out src file as it is not part of output
{UpdMFest2, Candidates ++ [SrcF]}
end.
mark_for_delete([], _Penciller) ->
ok;
mark_for_delete([Head|Tail], Penciller) ->
ok = leveled_sft:sft_setfordelete(Head#manifest_entry.owner, Penciller),
mark_for_delete(Tail, Penciller).
check_for_merge_candidates(SrcF, SinkFiles) ->
lists:partition(fun(Ref) ->
case {Ref#manifest_entry.start_key,
Ref#manifest_entry.end_key} of
{_, EK} when SrcF#manifest_entry.start_key > EK ->
false;
{SK, _} when SrcF#manifest_entry.end_key < SK ->
false;
_ ->
true
end end,
SinkFiles).
%% An algorithm for discovering which files to merge ....
%% We can find the most optimal file:
%% - The one with the most overlapping data below?
%% - The one that overlaps with the fewest files below?
%% - The smallest file?
%% We could try and be fair in some way (merge oldest first)
%% Ultimately, there is a lack of certainty that being fair or optimal is
%% genuinely better - eventually every file has to be compacted.
%%
%% Hence, the initial implementation is to select files to merge at random
select_filetomerge(SrcLevel, Manifest) ->
{SrcLevel, LevelManifest} = lists:keyfind(SrcLevel, 1, Manifest),
Selected = lists:nth(random:uniform(length(LevelManifest)),
LevelManifest),
UpdManifest = lists:keyreplace(SrcLevel,
1,
Manifest,
{SrcLevel,
lists:delete(Selected,
LevelManifest)}),
{Selected, UpdManifest}.
%% Assumption is that there is a single SFT from a higher level that needs
%% to be merged into multiple SFTs at a lower level. This should create an
%% entirely new set of SFTs, and the calling process can then update the
%% manifest.
%%
%% Once the FileToMerge has been emptied, the remainder of the candidate list
%% needs to be placed in a remainder SFT that may be of a sub-optimal (small)
%% size. This stops the need to perpetually roll over the whole level if the
%% level consists of already full files. Some smartness may be required when
%% selecting the candidate list so that small files just outside the candidate
%% list be included to avoid a proliferation of small files.
%%
%% FileToMerge should be a tuple of {FileName, Pid} where the Pid is the Pid of
%% the gen_server leveled_sft process representing the file.
%%
%% CandidateList should be a list of {StartKey, EndKey, Pid} tuples
%% representing different gen_server leveled_sft processes, sorted by StartKey.
%%
%% The level is the level which the new files should be created at.
perform_merge({SrcPid, SrcFN}, CandidateList, LevelInfo, {Filepath, MSN}) ->
leveled_log:log("PC010", [SrcFN, MSN]),
PointerList = lists:map(fun(P) ->
{next, P#manifest_entry.owner, all} end,
CandidateList),
do_merge([{next, SrcPid, all}],
PointerList,
LevelInfo,
{Filepath, MSN},
0,
[]).
do_merge([], [], {SrcLevel, _IsB}, {_Filepath, MSN}, FileCounter, OutList) ->
leveled_log:log("PC011", [MSN, SrcLevel, FileCounter]),
OutList;
do_merge(KL1, KL2, {SrcLevel, IsB}, {Filepath, MSN}, FileCounter, OutList) ->
FileName = lists:flatten(io_lib:format(Filepath ++ "_~w_~w.sft",
[SrcLevel + 1, FileCounter])),
leveled_log:log("PC012", [MSN, FileName]),
TS1 = os:timestamp(),
LevelR = case IsB of
true ->
#level{level = SrcLevel + 1,
is_basement = true,
timestamp = leveled_codec:integer_now()};
false ->
SrcLevel + 1
end,
{ok, Pid, Reply} = leveled_sft:sft_new(FileName,
KL1,
KL2,
LevelR),
case Reply of
{{[], []}, null, _} ->
leveled_log:log("PC013", [FileName]),
leveled_log:log("PC014", [FileName]),
ok = leveled_sft:sft_clear(Pid),
OutList;
{{KL1Rem, KL2Rem}, SmallestKey, HighestKey} ->
ExtMan = lists:append(OutList,
[#manifest_entry{start_key=SmallestKey,
end_key=HighestKey,
owner=Pid,
filename=FileName}]),
leveled_log:log_timer("PC015", [], TS1),
do_merge(KL1Rem, KL2Rem,
{SrcLevel, IsB}, {Filepath, MSN},
FileCounter + 1, ExtMan)
end.
get_item(Index, List, Default) ->
case lists:keysearch(Index, 1, List) of
{value, {Index, Value}} ->
Value;
false ->
Default
end.
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
generate_randomkeys(Count, BucketRangeLow, BucketRangeHigh) ->
generate_randomkeys(Count, [], BucketRangeLow, BucketRangeHigh).
generate_randomkeys(0, Acc, _BucketLow, _BucketHigh) ->
Acc;
generate_randomkeys(Count, Acc, BucketLow, BRange) ->
BNumber = string:right(integer_to_list(BucketLow + random:uniform(BRange)),
4, $0),
KNumber = string:right(integer_to_list(random:uniform(1000)), 4, $0),
RandKey = {{o,
"Bucket" ++ BNumber,
"Key" ++ KNumber},
{Count + 1,
{active, infinity}, null}},
generate_randomkeys(Count - 1, [RandKey|Acc], BucketLow, BRange).
choose_pid_toquery([ManEntry|_T], Key) when
Key >= ManEntry#manifest_entry.start_key,
ManEntry#manifest_entry.end_key >= Key ->
ManEntry#manifest_entry.owner;
choose_pid_toquery([_H|T], Key) ->
choose_pid_toquery(T, Key).
find_randomkeys(_FList, 0, _Source) ->
ok;
find_randomkeys(FList, Count, Source) ->
KV1 = lists:nth(random:uniform(length(Source)), Source),
K1 = leveled_codec:strip_to_keyonly(KV1),
P1 = choose_pid_toquery(FList, K1),
FoundKV = leveled_sft:sft_get(P1, K1),
Found = leveled_codec:strip_to_keyonly(FoundKV),
io:format("success finding ~w in ~w~n", [K1, P1]),
?assertMatch(K1, Found),
find_randomkeys(FList, Count - 1, Source).
merge_file_test() ->
KL1_L1 = lists:sort(generate_randomkeys(16000, 0, 1000)),
{ok, PidL1_1, _} = leveled_sft:sft_new("../test/KL1_L1.sft",
KL1_L1, [], 1),
KL1_L2 = lists:sort(generate_randomkeys(16000, 0, 250)),
{ok, PidL2_1, _} = leveled_sft:sft_new("../test/KL1_L2.sft",
KL1_L2, [], 2),
KL2_L2 = lists:sort(generate_randomkeys(16000, 250, 250)),
{ok, PidL2_2, _} = leveled_sft:sft_new("../test/KL2_L2.sft",
KL2_L2, [], 2),
KL3_L2 = lists:sort(generate_randomkeys(16000, 500, 250)),
{ok, PidL2_3, _} = leveled_sft:sft_new("../test/KL3_L2.sft",
KL3_L2, [], 2),
KL4_L2 = lists:sort(generate_randomkeys(16000, 750, 250)),
{ok, PidL2_4, _} = leveled_sft:sft_new("../test/KL4_L2.sft",
KL4_L2, [], 2),
Result = perform_merge({PidL1_1, "../test/KL1_L1.sft"},
[#manifest_entry{owner=PidL2_1},
#manifest_entry{owner=PidL2_2},
#manifest_entry{owner=PidL2_3},
#manifest_entry{owner=PidL2_4}],
{2, false}, {"../test/", 99}),
lists:foreach(fun(ManEntry) ->
{o, B1, K1} = ManEntry#manifest_entry.start_key,
{o, B2, K2} = ManEntry#manifest_entry.end_key,
io:format("Result of ~s ~s and ~s ~s with Pid ~w~n",
[B1, K1, B2, K2, ManEntry#manifest_entry.owner]) end,
Result),
io:format("Finding keys in KL1_L1~n"),
ok = find_randomkeys(Result, 50, KL1_L1),
io:format("Finding keys in KL1_L2~n"),
ok = find_randomkeys(Result, 50, KL1_L2),
io:format("Finding keys in KL2_L2~n"),
ok = find_randomkeys(Result, 50, KL2_L2),
io:format("Finding keys in KL3_L2~n"),
ok = find_randomkeys(Result, 50, KL3_L2),
io:format("Finding keys in KL4_L2~n"),
ok = find_randomkeys(Result, 50, KL4_L2),
leveled_sft:sft_clear(PidL1_1),
leveled_sft:sft_clear(PidL2_1),
leveled_sft:sft_clear(PidL2_2),
leveled_sft:sft_clear(PidL2_3),
leveled_sft:sft_clear(PidL2_4),
lists:foreach(fun(ManEntry) ->
leveled_sft:sft_clear(ManEntry#manifest_entry.owner) end,
Result).
select_merge_candidates_test() ->
Sink1 = #manifest_entry{start_key = {o, "Bucket", "Key1"},
end_key = {o, "Bucket", "Key20000"}},
Sink2 = #manifest_entry{start_key = {o, "Bucket", "Key20001"},
end_key = {o, "Bucket1", "Key1"}},
Src1 = #manifest_entry{start_key = {o, "Bucket", "Key40001"},
end_key = {o, "Bucket", "Key60000"}},
{Candidates, Others} = check_for_merge_candidates(Src1, [Sink1, Sink2]),
?assertMatch([Sink2], Candidates),
?assertMatch([Sink1], Others).
select_merge_file_test() ->
L0 = [{{o, "B1", "K1"}, {o, "B3", "K3"}, dummy_pid}],
L1 = [{{o, "B1", "K1"}, {o, "B2", "K2"}, dummy_pid},
{{o, "B2", "K3"}, {o, "B4", "K4"}, dummy_pid}],
Manifest = [{0, L0}, {1, L1}],
{FileRef, NewManifest} = select_filetomerge(0, Manifest),
?assertMatch(FileRef, {{o, "B1", "K1"}, {o, "B3", "K3"}, dummy_pid}),
?assertMatch(NewManifest, [{0, []}, {1, L1}]).
-endif.

1662
src/leveled_penciller.erl Normal file

File diff suppressed because it is too large Load diff

282
src/leveled_pmem.erl Normal file
View file

@ -0,0 +1,282 @@
%% -------- PENCILLER MEMORY ---------
%%
%% Module that provides functions for maintaining the L0 memory of the
%% Penciller.
%%
%% It is desirable that the L0Mem can efficiently handle the push of new trees
%% whilst maintaining the capability to quickly snapshot the memory for clones
%% of the Penciller.
%%
%% ETS tables are not used due to complications with managing their mutability,
%% as the database is snapshotted.
%%
%% An attempt was made to merge all trees into a single tree on push (in a
%% spawned process), but this proved to have an expensive impact as the tree
%% got larger.
%%
%% This approach is to keep a list of trees which have been received in the
%% order which they were received. There is then a fixed-size array of hashes
%% used to either point lookups at the right tree in the list, or inform the
%% requestor it is not present avoiding any lookups.
%%
%% Tests show this takes one third of the time at push (when compared to
%% merging to a single tree), and is an order of magnitude more efficient as
%% the tree reaches peak size. It is also an order of magnitude more
%% efficient to use the hash index when compared to looking through all the
%% trees.
%%
%% Total time for single_tree 217000 microseconds
%% Total time for array_tree 209000 microseconds
%% Total time for array_list 142000 microseconds
%% Total time for array_filter 69000 microseconds
%% List of 2000 checked without array - success count of 90 in 36000 microsecs
%% List of 2000 checked with array - success count of 90 in 1000 microsecs
%%
%% The trade-off taken with the approach is that the size of the L0Cache is
%% uncertain. The Size count is incremented if the hash is not already
%% present, so the size may be lower than the actual size due to hash
%% collisions
-module(leveled_pmem).
-include("include/leveled.hrl").
-export([
add_to_index/5,
to_list/2,
new_index/0,
check_levelzero/3,
merge_trees/4
]).
-include_lib("eunit/include/eunit.hrl").
-define(SLOT_WIDTH, {4096, 12}).
%%%============================================================================
%%% API
%%%============================================================================
add_to_index(L0Index, L0Size, LevelMinus1, LedgerSQN, TreeList) ->
SW = os:timestamp(),
SlotInTreeList = length(TreeList) + 1,
FoldFun = fun({K, V}, {AccMinSQN, AccMaxSQN, AccCount, HashIndex}) ->
SQN = leveled_codec:strip_to_seqonly({K, V}),
{Hash, Slot} = hash_to_slot(K),
L = array:get(Slot, HashIndex),
Count0 = case lists:keymember(Hash, 1, L) of
true ->
AccCount;
false ->
AccCount + 1
end,
{min(SQN, AccMinSQN),
max(SQN, AccMaxSQN),
Count0,
array:set(Slot, [{Hash, SlotInTreeList}|L], HashIndex)}
end,
LM1List = gb_trees:to_list(LevelMinus1),
StartingT = {infinity, 0, L0Size, L0Index},
{MinSQN, MaxSQN, NewL0Size, UpdL0Index} = lists:foldl(FoldFun,
StartingT,
LM1List),
leveled_log:log_timer("PM001", [NewL0Size], SW),
if
MinSQN > LedgerSQN ->
{MaxSQN,
NewL0Size,
UpdL0Index,
lists:append(TreeList, [LevelMinus1])}
end.
to_list(Slots, FetchFun) ->
SW = os:timestamp(),
SlotList = lists:reverse(lists:seq(1, Slots)),
FullList = lists:foldl(fun(Slot, Acc) ->
Tree = FetchFun(Slot),
L = gb_trees:to_list(Tree),
lists:ukeymerge(1, Acc, L)
end,
[],
SlotList),
leveled_log:log_timer("PM002", [length(FullList)], SW),
FullList.
new_index() ->
array:new(element(1, ?SLOT_WIDTH), [{default, []}, fixed]).
check_levelzero(Key, L0Index, TreeList) ->
{Hash, Slot} = hash_to_slot(Key),
CheckList = array:get(Slot, L0Index),
SlotList = lists:foldl(fun({H0, S0}, SL) ->
case H0 of
Hash ->
[S0|SL];
_ ->
SL
end
end,
[],
CheckList),
lists:foldl(fun(SlotToCheck, {Found, KV}) ->
case Found of
true ->
{Found, KV};
false ->
CheckTree = lists:nth(SlotToCheck, TreeList),
case gb_trees:lookup(Key, CheckTree) of
none ->
{Found, KV};
{value, Value} ->
{true, {Key, Value}}
end
end
end,
{false, not_found},
lists:reverse(lists:usort(SlotList))).
merge_trees(StartKey, EndKey, TreeList, LevelMinus1) ->
lists:foldl(fun(Tree, TreeAcc) ->
merge_nexttree(Tree, TreeAcc, StartKey, EndKey) end,
gb_trees:empty(),
lists:append(TreeList, [LevelMinus1])).
%%%============================================================================
%%% Internal Functions
%%%============================================================================
hash_to_slot(Key) ->
H = erlang:phash2(Key),
{H bsr element(2, ?SLOT_WIDTH), H band (element(1, ?SLOT_WIDTH) - 1)}.
merge_nexttree(Tree, TreeAcc, StartKey, EndKey) ->
Iter = gb_trees:iterator_from(StartKey, Tree),
merge_nexttree(Iter, TreeAcc, EndKey).
merge_nexttree(Iter, TreeAcc, EndKey) ->
case gb_trees:next(Iter) of
none ->
TreeAcc;
{Key, Value, NewIter} ->
case leveled_codec:endkey_passed(EndKey, Key) of
true ->
TreeAcc;
false ->
merge_nexttree(NewIter,
gb_trees:enter(Key, Value, TreeAcc),
EndKey)
end
end.
%%%============================================================================
%%% Test
%%%============================================================================
-ifdef(TEST).
generate_randomkeys(Seqn, Count, BucketRangeLow, BucketRangeHigh) ->
generate_randomkeys(Seqn,
Count,
gb_trees:empty(),
BucketRangeLow,
BucketRangeHigh).
generate_randomkeys(_Seqn, 0, Acc, _BucketLow, _BucketHigh) ->
Acc;
generate_randomkeys(Seqn, Count, Acc, BucketLow, BRange) ->
BNumber = string:right(integer_to_list(BucketLow + random:uniform(BRange)),
4, $0),
KNumber = string:right(integer_to_list(random:uniform(1000)), 4, $0),
{K, V} = {{o, "Bucket" ++ BNumber, "Key" ++ KNumber, null},
{Seqn, {active, infinity}, null}},
generate_randomkeys(Seqn + 1,
Count - 1,
gb_trees:enter(K, V, Acc),
BucketLow,
BRange).
compare_method_test() ->
R = lists:foldl(fun(_X, {LedgerSQN, L0Size, L0Index, L0TreeList}) ->
LM1 = generate_randomkeys(LedgerSQN + 1,
2000, 1, 500),
add_to_index(L0Index, L0Size, LM1, LedgerSQN,
L0TreeList)
end,
{0, 0, new_index(), []},
lists:seq(1, 16)),
{SQN, Size, Index, TreeList} = R,
?assertMatch(32000, SQN),
?assertMatch(true, Size =< 32000),
TestList = gb_trees:to_list(generate_randomkeys(1, 2000, 1, 800)),
S0 = lists:foldl(fun({Key, _V}, Acc) ->
R0 = lists:foldr(fun(Tree, {Found, KV}) ->
case Found of
true ->
{true, KV};
false ->
L0 = gb_trees:lookup(Key, Tree),
case L0 of
none ->
{false, not_found};
{value, Value} ->
{true, {Key, Value}}
end
end
end,
{false, not_found},
TreeList),
[R0|Acc]
end,
[],
TestList),
S1 = lists:foldl(fun({Key, _V}, Acc) ->
R0 = check_levelzero(Key, Index, TreeList),
[R0|Acc]
end,
[],
TestList),
?assertMatch(S0, S1),
StartKey = {o, "Bucket0100", null, null},
EndKey = {o, "Bucket0200", null, null},
SWa = os:timestamp(),
FetchFun = fun(Slot) -> lists:nth(Slot, TreeList) end,
DumpList = to_list(length(TreeList), FetchFun),
Q0 = lists:foldl(fun({K, V}, Acc) ->
P = leveled_codec:endkey_passed(EndKey, K),
case {K, P} of
{K, false} when K >= StartKey ->
gb_trees:enter(K, V, Acc);
_ ->
Acc
end
end,
gb_trees:empty(),
DumpList),
Sz0 = gb_trees:size(Q0),
io:format("Crude method took ~w microseconds resulting in tree of " ++
"size ~w~n",
[timer:now_diff(os:timestamp(), SWa), Sz0]),
SWb = os:timestamp(),
Q1 = merge_trees(StartKey, EndKey, TreeList, gb_trees:empty()),
Sz1 = gb_trees:size(Q1),
io:format("Merge method took ~w microseconds resulting in tree of " ++
"size ~w~n",
[timer:now_diff(os:timestamp(), SWb), Sz1]),
?assertMatch(Sz0, Sz1).
-endif.

1921
src/leveled_sft.erl Normal file

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,544 @@
-module(basic_SUITE).
-include_lib("common_test/include/ct.hrl").
-include("include/leveled.hrl").
-export([all/0]).
-export([simple_put_fetch_head_delete/1,
many_put_fetch_head/1,
journal_compaction/1,
fetchput_snapshot/1,
load_and_count/1,
load_and_count_withdelete/1,
space_clear_ondelete/1
]).
all() -> [
simple_put_fetch_head_delete,
many_put_fetch_head,
journal_compaction,
fetchput_snapshot,
load_and_count,
load_and_count_withdelete,
space_clear_ondelete
].
simple_put_fetch_head_delete(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
testutil:check_formissingobject(Bookie1, "Bucket1", "Key2"),
ok = leveled_bookie:book_close(Bookie1),
StartOpts2 = [{root_path, RootPath},
{max_journalsize, 3000000}],
{ok, Bookie2} = leveled_bookie:book_start(StartOpts2),
testutil:check_forobject(Bookie2, TestObject),
ObjList1 = testutil:generate_objects(5000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie2, Obj, Spc) end,
ObjList1),
ChkList1 = lists:sublist(lists:sort(ObjList1), 100),
testutil:check_forlist(Bookie2, ChkList1),
testutil:check_forobject(Bookie2, TestObject),
testutil:check_formissingobject(Bookie2, "Bucket1", "Key2"),
ok = leveled_bookie:book_put(Bookie2, "Bucket1", "Key2", "Value2",
[{add, "Index1", "Term1"}]),
{ok, "Value2"} = leveled_bookie:book_get(Bookie2, "Bucket1", "Key2"),
{ok, {62888926, 56}} = leveled_bookie:book_head(Bookie2,
"Bucket1",
"Key2"),
testutil:check_formissingobject(Bookie2, "Bucket1", "Key2"),
ok = leveled_bookie:book_put(Bookie2, "Bucket1", "Key2", <<"Value2">>,
[{remove, "Index1", "Term1"},
{add, "Index1", <<"Term2">>}]),
{ok, <<"Value2">>} = leveled_bookie:book_get(Bookie2, "Bucket1", "Key2"),
ok = leveled_bookie:book_close(Bookie2),
{ok, Bookie3} = leveled_bookie:book_start(StartOpts2),
{ok, <<"Value2">>} = leveled_bookie:book_get(Bookie3, "Bucket1", "Key2"),
ok = leveled_bookie:book_delete(Bookie3, "Bucket1", "Key2",
[{remove, "Index1", "Term1"}]),
not_found = leveled_bookie:book_get(Bookie3, "Bucket1", "Key2"),
not_found = leveled_bookie:book_head(Bookie3, "Bucket1", "Key2"),
ok = leveled_bookie:book_close(Bookie3),
{ok, Bookie4} = leveled_bookie:book_start(StartOpts2),
not_found = leveled_bookie:book_get(Bookie4, "Bucket1", "Key2"),
ok = leveled_bookie:book_close(Bookie4),
testutil:reset_filestructure().
many_put_fetch_head(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}, {max_pencillercachesize, 16000}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
ok = leveled_bookie:book_close(Bookie1),
StartOpts2 = [{root_path, RootPath},
{max_journalsize, 1000000000},
{max_pencillercachesize, 32000}],
{ok, Bookie2} = leveled_bookie:book_start(StartOpts2),
testutil:check_forobject(Bookie2, TestObject),
GenList = [2, 20002, 40002, 60002, 80002,
100002, 120002, 140002, 160002, 180002],
CLs = testutil:load_objects(20000, GenList, Bookie2, TestObject,
fun testutil:generate_smallobjects/2),
CL1A = lists:nth(1, CLs),
ChkListFixed = lists:nth(length(CLs), CLs),
testutil:check_forlist(Bookie2, CL1A),
ObjList2A = testutil:generate_objects(5000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie2, Obj, Spc) end,
ObjList2A),
ChkList2A = lists:sublist(lists:sort(ObjList2A), 1000),
testutil:check_forlist(Bookie2, ChkList2A),
testutil:check_forlist(Bookie2, ChkListFixed),
testutil:check_forobject(Bookie2, TestObject),
testutil:check_forlist(Bookie2, ChkList2A),
testutil:check_forlist(Bookie2, ChkListFixed),
testutil:check_forobject(Bookie2, TestObject),
ok = leveled_bookie:book_close(Bookie2),
{ok, Bookie3} = leveled_bookie:book_start(StartOpts2),
testutil:check_forlist(Bookie3, ChkList2A),
testutil:check_forobject(Bookie3, TestObject),
ok = leveled_bookie:book_close(Bookie3),
testutil:reset_filestructure().
journal_compaction(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath},
{max_journalsize, 10000000},
{max_run_length, 1}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
ok = leveled_bookie:book_compactjournal(Bookie1, 30000),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
ObjList1 = testutil:generate_objects(20000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjList1),
ChkList1 = lists:sublist(lists:sort(ObjList1), 10000),
testutil:check_forlist(Bookie1, ChkList1),
testutil:check_forobject(Bookie1, TestObject),
{B2, K2, V2, Spec2, MD} = {"Bucket1",
"Key1",
"Value1",
[],
{"MDK1", "MDV1"}},
{TestObject2, TestSpec2} = testutil:generate_testobject(B2, K2,
V2, Spec2, MD),
ok = leveled_bookie:book_riakput(Bookie1, TestObject2, TestSpec2),
ok = leveled_bookie:book_compactjournal(Bookie1, 30000),
testutil:check_forlist(Bookie1, ChkList1),
testutil:check_forobject(Bookie1, TestObject),
testutil:check_forobject(Bookie1, TestObject2),
testutil:check_forlist(Bookie1, ChkList1),
testutil:check_forobject(Bookie1, TestObject),
testutil:check_forobject(Bookie1, TestObject2),
%% Delete some of the objects
ObjListD = testutil:generate_objects(10000, 2),
lists:foreach(fun({_R, O, _S}) ->
ok = leveled_bookie:book_riakdelete(Bookie1,
O#r_object.bucket,
O#r_object.key,
[])
end,
ObjListD),
%% Now replace all the other objects
ObjList2 = testutil:generate_objects(40000, 10002),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjList2),
ok = leveled_bookie:book_compactjournal(Bookie1, 30000),
F = fun leveled_bookie:book_islastcompactionpending/1,
lists:foldl(fun(X, Pending) ->
case Pending of
false ->
false;
true ->
io:format("Loop ~w waiting for journal "
++ "compaction to complete~n", [X]),
timer:sleep(20000),
F(Bookie1)
end end,
true,
lists:seq(1, 15)),
ChkList3 = lists:sublist(lists:sort(ObjList2), 500),
testutil:check_forlist(Bookie1, ChkList3),
ok = leveled_bookie:book_close(Bookie1),
% Restart
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
testutil:check_forobject(Bookie2, TestObject),
testutil:check_forlist(Bookie2, ChkList3),
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure(10000).
fetchput_snapshot(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}, {max_journalsize, 30000000}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
ObjList1 = testutil:generate_objects(5000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjList1),
SnapOpts1 = [{snapshot_bookie, Bookie1}],
{ok, SnapBookie1} = leveled_bookie:book_start(SnapOpts1),
ChkList1 = lists:sublist(lists:sort(ObjList1), 100),
testutil:check_forlist(Bookie1, ChkList1),
testutil:check_forlist(SnapBookie1, ChkList1),
ok = leveled_bookie:book_close(SnapBookie1),
testutil:check_forlist(Bookie1, ChkList1),
ok = leveled_bookie:book_close(Bookie1),
io:format("Closed initial bookies~n"),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
SnapOpts2 = [{snapshot_bookie, Bookie2}],
{ok, SnapBookie2} = leveled_bookie:book_start(SnapOpts2),
io:format("Bookies restarted~n"),
testutil:check_forlist(Bookie2, ChkList1),
io:format("Check active bookie still contains original data~n"),
testutil:check_forlist(SnapBookie2, ChkList1),
io:format("Check snapshot still contains original data~n"),
ObjList2 = testutil:generate_objects(5000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie2, Obj, Spc) end,
ObjList2),
io:format("Replacement objects put~n"),
ChkList2 = lists:sublist(lists:sort(ObjList2), 100),
testutil:check_forlist(Bookie2, ChkList2),
testutil:check_forlist(SnapBookie2, ChkList1),
io:format("Checked for replacement objects in active bookie" ++
", old objects in snapshot~n"),
ok = filelib:ensure_dir(RootPath ++ "/ledger/ledger_files"),
{ok, FNsA} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
ObjList3 = testutil:generate_objects(15000, 5002),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie2, Obj, Spc) end,
ObjList3),
ChkList3 = lists:sublist(lists:sort(ObjList3), 100),
testutil:check_forlist(Bookie2, ChkList3),
testutil:check_formissinglist(SnapBookie2, ChkList3),
GenList = [20002, 40002, 60002, 80002],
CLs2 = testutil:load_objects(20000, GenList, Bookie2, TestObject,
fun testutil:generate_smallobjects/2),
io:format("Loaded significant numbers of new objects~n"),
testutil:check_forlist(Bookie2, lists:nth(length(CLs2), CLs2)),
io:format("Checked active bookie has new objects~n"),
{ok, SnapBookie3} = leveled_bookie:book_start(SnapOpts2),
testutil:check_forlist(SnapBookie3, lists:nth(length(CLs2), CLs2)),
testutil:check_formissinglist(SnapBookie2, ChkList3),
testutil:check_formissinglist(SnapBookie2, lists:nth(length(CLs2), CLs2)),
testutil:check_forlist(Bookie2, ChkList2),
testutil:check_forlist(SnapBookie3, ChkList2),
testutil:check_forlist(SnapBookie2, ChkList1),
io:format("Started new snapshot and check for new objects~n"),
CLs3 = testutil:load_objects(20000, GenList, Bookie2, TestObject,
fun testutil:generate_smallobjects/2),
testutil:check_forlist(Bookie2, lists:nth(length(CLs3), CLs3)),
testutil:check_forlist(Bookie2, lists:nth(1, CLs3)),
io:format("Starting 15s sleep in which snap2 should block deletion~n"),
timer:sleep(15000),
{ok, FNsB} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
ok = leveled_bookie:book_close(SnapBookie2),
io:format("Starting 15s sleep as snap2 close should unblock deletion~n"),
timer:sleep(15000),
io:format("Pause for deletion has ended~n"),
testutil:check_forlist(Bookie2, lists:nth(length(CLs3), CLs3)),
ok = leveled_bookie:book_close(SnapBookie3),
io:format("Starting 15s sleep as snap3 close should unblock deletion~n"),
timer:sleep(15000),
io:format("Pause for deletion has ended~n"),
testutil:check_forlist(Bookie2, lists:nth(length(CLs3), CLs3)),
testutil:check_forlist(Bookie2, lists:nth(1, CLs3)),
{ok, FNsC} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
true = length(FNsB) > length(FNsA),
true = length(FNsB) > length(FNsC),
{B1Size, B1Count} = testutil:check_bucket_stats(Bookie2, "Bucket1"),
true = B1Size > 0,
true = B1Count == 1,
{B1Size, B1Count} = testutil:check_bucket_stats(Bookie2, "Bucket1"),
{BSize, BCount} = testutil:check_bucket_stats(Bookie2, "Bucket"),
true = BSize > 0,
true = BCount == 100000,
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure().
load_and_count(_Config) ->
% Use artificially small files, and the load keys, counting they're all
% present
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}, {max_journalsize, 50000000}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
io:format("Loading initial small objects~n"),
G1 = fun testutil:generate_smallobjects/2,
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
TestObject,
G1),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Acc + 5000 == Count ->
ok
end,
Acc + 5000 end,
0,
lists:seq(1, 20)),
testutil:check_forobject(Bookie1, TestObject),
io:format("Loading larger compressible objects~n"),
G2 = fun testutil:generate_compressibleobjects/2,
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
TestObject,
G2),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Acc + 5000 == Count ->
ok
end,
Acc + 5000 end,
100000,
lists:seq(1, 20)),
testutil:check_forobject(Bookie1, TestObject),
io:format("Replacing small objects~n"),
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
TestObject,
G1),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Count == 200000 ->
ok
end,
Acc + 5000 end,
0,
lists:seq(1, 20)),
testutil:check_forobject(Bookie1, TestObject),
io:format("Loading more small objects~n"),
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
TestObject,
G2),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Acc + 5000 == Count ->
ok
end,
Acc + 5000 end,
200000,
lists:seq(1, 20)),
testutil:check_forobject(Bookie1, TestObject),
ok = leveled_bookie:book_close(Bookie1),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
{_, 300000} = testutil:check_bucket_stats(Bookie2, "Bucket"),
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure().
load_and_count_withdelete(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}, {max_journalsize, 50000000}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
io:format("Loading initial small objects~n"),
G1 = fun testutil:generate_smallobjects/2,
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
TestObject,
G1),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Acc + 5000 == Count ->
ok
end,
Acc + 5000 end,
0,
lists:seq(1, 20)),
testutil:check_forobject(Bookie1, TestObject),
{BucketD, KeyD} = leveled_codec:riakto_keydetails(TestObject),
{_, 1} = testutil:check_bucket_stats(Bookie1, BucketD),
ok = leveled_bookie:book_riakdelete(Bookie1, BucketD, KeyD, []),
not_found = leveled_bookie:book_riakget(Bookie1, BucketD, KeyD),
{_, 0} = testutil:check_bucket_stats(Bookie1, BucketD),
io:format("Loading larger compressible objects~n"),
G2 = fun testutil:generate_compressibleobjects/2,
lists:foldl(fun(_X, Acc) ->
testutil:load_objects(5000,
[Acc + 2],
Bookie1,
no_check,
G2),
{_S, Count} = testutil:check_bucket_stats(Bookie1,
"Bucket"),
if
Acc + 5000 == Count ->
ok
end,
Acc + 5000 end,
100000,
lists:seq(1, 20)),
not_found = leveled_bookie:book_riakget(Bookie1, BucketD, KeyD),
ok = leveled_bookie:book_close(Bookie1),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
testutil:check_formissingobject(Bookie2, BucketD, KeyD),
{_BSize, 0} = testutil:check_bucket_stats(Bookie2, BucketD),
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure().
space_clear_ondelete(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath}, {max_journalsize, 20000000}],
{ok, Book1} = leveled_bookie:book_start(StartOpts1),
G2 = fun testutil:generate_compressibleobjects/2,
testutil:load_objects(20000,
[uuid, uuid, uuid, uuid],
Book1,
no_check,
G2),
{async, F1} = leveled_bookie:book_returnfolder(Book1, {keylist, o_rkv}),
SW1 = os:timestamp(),
KL1 = F1(),
ok = case length(KL1) of
80000 ->
io:format("Key list took ~w microseconds for 80K keys~n",
[timer:now_diff(os:timestamp(), SW1)]),
ok
end,
timer:sleep(10000), % Allow for any L0 file to be rolled
{ok, FNsA_L} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
{ok, FNsA_J} = file:list_dir(RootPath ++ "/journal/journal_files"),
io:format("Bookie created ~w journal files and ~w ledger files~n",
[length(FNsA_J), length(FNsA_L)]),
% Get an iterator to lock the inker during compaction
FoldObjectsFun = fun(B, K, V, Acc) -> [{B, K, testutil:riak_hash(V)}|Acc]
end,
{async, HTreeF1} = leveled_bookie:book_returnfolder(Book1,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
% Delete the keys
SW2 = os:timestamp(),
lists:foreach(fun({Bucket, Key}) ->
ok = leveled_bookie:book_riakdelete(Book1,
Bucket,
Key,
[])
end,
KL1),
io:format("Deletion took ~w microseconds for 80K keys~n",
[timer:now_diff(os:timestamp(), SW2)]),
ok = leveled_bookie:book_compactjournal(Book1, 30000),
F = fun leveled_bookie:book_islastcompactionpending/1,
lists:foldl(fun(X, Pending) ->
case Pending of
false ->
false;
true ->
io:format("Loop ~w waiting for journal "
++ "compaction to complete~n", [X]),
timer:sleep(20000),
F(Book1)
end end,
true,
lists:seq(1, 15)),
io:format("Waiting for journal deletes - blocked~n"),
timer:sleep(20000),
KeyHashList1 = HTreeF1(),
io:format("Key Hash List returned of length ~w~n", [length(KeyHashList1)]),
true = length(KeyHashList1) == 80000,
io:format("Waiting for journal deletes - unblocked~n"),
timer:sleep(20000),
{ok, FNsB_L} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
{ok, FNsB_J} = file:list_dir(RootPath ++ "/journal/journal_files"),
{ok, FNsB_PC} = file:list_dir(RootPath
++ "/journal/journal_files/post_compact"),
PointB_Journals = length(FNsB_J) + length(FNsB_PC),
io:format("Bookie has ~w journal files and ~w ledger files " ++
"after deletes~n",
[PointB_Journals, length(FNsB_L)]),
{async, F2} = leveled_bookie:book_returnfolder(Book1, {keylist, o_rkv}),
SW3 = os:timestamp(),
KL2 = F2(),
ok = case length(KL2) of
0 ->
io:format("Key list took ~w microseconds for no keys~n",
[timer:now_diff(os:timestamp(), SW3)]),
ok
end,
ok = leveled_bookie:book_close(Book1),
{ok, Book2} = leveled_bookie:book_start(StartOpts1),
{async, F3} = leveled_bookie:book_returnfolder(Book2, {keylist, o_rkv}),
SW4 = os:timestamp(),
KL3 = F3(),
ok = case length(KL3) of
0 ->
io:format("Key list took ~w microseconds for no keys~n",
[timer:now_diff(os:timestamp(), SW4)]),
ok
end,
ok = leveled_bookie:book_close(Book2),
{ok, FNsC_L} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
io:format("Bookie has ~w ledger files " ++
"after close~n", [length(FNsC_L)]),
{ok, Book3} = leveled_bookie:book_start(StartOpts1),
io:format("This should cause a final ledger merge event~n"),
io:format("Will require the penciller to resolve the issue of creating" ++
" an empty file as all keys compact on merge~n"),
timer:sleep(12000),
ok = leveled_bookie:book_close(Book3),
{ok, FNsD_L} = file:list_dir(RootPath ++ "/ledger/ledger_files"),
io:format("Bookie has ~w ledger files " ++
"after second close~n", [length(FNsD_L)]),
true = PointB_Journals < length(FNsA_J),
true = length(FNsD_L) < length(FNsA_L),
true = length(FNsD_L) < length(FNsB_L),
true = length(FNsD_L) < length(FNsC_L),
true = length(FNsD_L) == 0.

View file

@ -0,0 +1,360 @@
-module(iterator_SUITE).
-include_lib("common_test/include/ct.hrl").
-include("include/leveled.hrl").
-define(KEY_ONLY, {false, undefined}).
-export([all/0]).
-export([small_load_with2i/1,
query_count/1,
rotating_objects/1]).
all() -> [
small_load_with2i,
query_count,
rotating_objects
].
small_load_with2i(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath},
{max_journalsize, 5000000}],
% low journal size to make sure > 1 created
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
testutil:check_formissingobject(Bookie1, "Bucket1", "Key2"),
testutil:check_forobject(Bookie1, TestObject),
ObjectGen = testutil:get_compressiblevalue_andinteger(),
IndexGen = testutil:get_randomindexes_generator(8),
ObjL1 = testutil:generate_objects(10000,
uuid,
[],
ObjectGen,
IndexGen),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjL1),
ChkList1 = lists:sublist(lists:sort(ObjL1), 100),
testutil:check_forlist(Bookie1, ChkList1),
testutil:check_forobject(Bookie1, TestObject),
%% Delete the objects from the ChkList removing the indexes
lists:foreach(fun({_RN, Obj, Spc}) ->
DSpc = lists:map(fun({add, F, T}) -> {remove, F, T}
end,
Spc),
{B, K} = leveled_codec:riakto_keydetails(Obj),
leveled_bookie:book_riakdelete(Bookie1, B, K, DSpc)
end,
ChkList1),
%% Get the Buckets Keys and Hashes for the whole bucket
FoldObjectsFun = fun(B, K, V, Acc) -> [{B, K, testutil:riak_hash(V)}|Acc]
end,
{async, HTreeF1} = leveled_bookie:book_returnfolder(Bookie1,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
KeyHashList1 = HTreeF1(),
{async, HTreeF2} = leveled_bookie:book_returnfolder(Bookie1,
{foldobjects_bybucket,
?RIAK_TAG,
"Bucket",
FoldObjectsFun}),
KeyHashList2 = HTreeF2(),
{async, HTreeF3} = leveled_bookie:book_returnfolder(Bookie1,
{foldobjects_byindex,
?RIAK_TAG,
"Bucket",
{"idx1_bin",
"#", "~"},
FoldObjectsFun}),
KeyHashList3 = HTreeF3(),
true = 9901 == length(KeyHashList1), % also includes the test object
true = 9900 == length(KeyHashList2),
true = 9900 == length(KeyHashList3),
SumIntegerFun = fun(_B, _K, V, Acc) ->
[C] = V#r_object.contents,
{I, _Bin} = C#r_content.value,
Acc + I
end,
{async, Sum1} = leveled_bookie:book_returnfolder(Bookie1,
{foldobjects_bybucket,
?RIAK_TAG,
"Bucket",
{SumIntegerFun,
0}}),
Total1 = Sum1(),
true = Total1 > 100000,
ok = leveled_bookie:book_close(Bookie1),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
{async, Sum2} = leveled_bookie:book_returnfolder(Bookie2,
{foldobjects_bybucket,
?RIAK_TAG,
"Bucket",
{SumIntegerFun,
0}}),
Total2 = Sum2(),
true = Total2 == Total1,
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure().
query_count(_Config) ->
RootPath = testutil:reset_filestructure(),
{ok, Book1} = leveled_bookie:book_start(RootPath, 2000, 50000000),
{TestObject, TestSpec} = testutil:generate_testobject("Bucket",
"Key1",
"Value1",
[],
{"MDK1", "MDV1"}),
ok = leveled_bookie:book_riakput(Book1, TestObject, TestSpec),
testutil:check_forobject(Book1, TestObject),
testutil:check_formissingobject(Book1, "Bucket1", "Key2"),
testutil:check_forobject(Book1, TestObject),
lists:foreach(fun(_X) ->
V = testutil:get_compressiblevalue(),
Indexes = testutil:get_randomindexes_generator(8),
SW = os:timestamp(),
ObjL1 = testutil:generate_objects(10000,
uuid,
[],
V,
Indexes),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Book1,
Obj,
Spc)
end,
ObjL1),
io:format("Put of 10000 objects with 8 index entries "
++
"each completed in ~w microseconds~n",
[timer:now_diff(os:timestamp(), SW)])
end,
lists:seq(1, 8)),
testutil:check_forobject(Book1, TestObject),
Total = lists:foldl(fun(X, Acc) ->
IdxF = "idx" ++ integer_to_list(X) ++ "_bin",
T = count_termsonindex("Bucket",
IdxF,
Book1,
?KEY_ONLY),
io:format("~w terms found on index ~s~n",
[T, IdxF]),
Acc + T
end,
0,
lists:seq(1, 8)),
ok = case Total of
640000 ->
ok
end,
Index1Count = count_termsonindex("Bucket",
"idx1_bin",
Book1,
?KEY_ONLY),
ok = leveled_bookie:book_close(Book1),
{ok, Book2} = leveled_bookie:book_start(RootPath, 1000, 50000000),
Index1Count = count_termsonindex("Bucket",
"idx1_bin",
Book2,
?KEY_ONLY),
NameList = testutil:name_list(),
TotalNameByName = lists:foldl(fun({_X, Name}, Acc) ->
{ok, Regex} = re:compile("[0-9]+" ++
Name),
SW = os:timestamp(),
T = count_termsonindex("Bucket",
"idx1_bin",
Book2,
{false,
Regex}),
TD = timer:now_diff(os:timestamp(),
SW),
io:format("~w terms found on " ++
"index idx1 with a " ++
"regex in ~w " ++
"microseconds~n",
[T, TD]),
Acc + T
end,
0,
NameList),
ok = case TotalNameByName of
Index1Count ->
ok
end,
{ok, RegMia} = re:compile("[0-9]+Mia"),
{async,
Mia2KFolder1} = leveled_bookie:book_returnfolder(Book2,
{index_query,
"Bucket",
{"idx2_bin",
"2000",
"2000~"},
{false,
RegMia}}),
Mia2000Count1 = length(Mia2KFolder1()),
{async,
Mia2KFolder2} = leveled_bookie:book_returnfolder(Book2,
{index_query,
"Bucket",
{"idx2_bin",
"2000",
"2001"},
{true,
undefined}}),
Mia2000Count2 = lists:foldl(fun({Term, _Key}, Acc) ->
case re:run(Term, RegMia) of
nomatch ->
Acc;
_ ->
Acc + 1
end end,
0,
Mia2KFolder2()),
ok = case Mia2000Count2 of
Mia2000Count1 when Mia2000Count1 > 0 ->
io:format("Mia2000 counts match at ~w~n",
[Mia2000Count1]),
ok
end,
{ok, RxMia2K} = re:compile("^2000[0-9]+Mia"),
{async,
Mia2KFolder3} = leveled_bookie:book_returnfolder(Book2,
{index_query,
"Bucket",
{"idx2_bin",
"1980",
"2100"},
{false,
RxMia2K}}),
Mia2000Count1 = length(Mia2KFolder3()),
V9 = testutil:get_compressiblevalue(),
Indexes9 = testutil:get_randomindexes_generator(8),
[{_RN, Obj9, Spc9}] = testutil:generate_objects(1, uuid, [], V9, Indexes9),
ok = leveled_bookie:book_riakput(Book2, Obj9, Spc9),
R9 = lists:map(fun({add, IdxF, IdxT}) ->
R = leveled_bookie:book_returnfolder(Book2,
{index_query,
"Bucket",
{IdxF,
IdxT,
IdxT},
?KEY_ONLY}),
{async, Fldr} = R,
case length(Fldr()) of
X when X > 0 ->
{IdxF, IdxT, X}
end
end,
Spc9),
Spc9Del = lists:map(fun({add, IdxF, IdxT}) -> {remove, IdxF, IdxT} end,
Spc9),
ok = leveled_bookie:book_riakput(Book2, Obj9, Spc9Del),
lists:foreach(fun({IdxF, IdxT, X}) ->
R = leveled_bookie:book_returnfolder(Book2,
{index_query,
"Bucket",
{IdxF,
IdxT,
IdxT},
?KEY_ONLY}),
{async, Fldr} = R,
case length(Fldr()) of
Y ->
Y = X - 1
end
end,
R9),
ok = leveled_bookie:book_close(Book2),
{ok, Book3} = leveled_bookie:book_start(RootPath, 2000, 50000000),
lists:foreach(fun({IdxF, IdxT, X}) ->
R = leveled_bookie:book_returnfolder(Book3,
{index_query,
"Bucket",
{IdxF,
IdxT,
IdxT},
?KEY_ONLY}),
{async, Fldr} = R,
case length(Fldr()) of
Y ->
Y = X - 1
end
end,
R9),
ok = leveled_bookie:book_riakput(Book3, Obj9, Spc9),
ok = leveled_bookie:book_close(Book3),
{ok, Book4} = leveled_bookie:book_start(RootPath, 2000, 50000000),
lists:foreach(fun({IdxF, IdxT, X}) ->
R = leveled_bookie:book_returnfolder(Book4,
{index_query,
"Bucket",
{IdxF,
IdxT,
IdxT},
?KEY_ONLY}),
{async, Fldr} = R,
case length(Fldr()) of
X ->
ok
end
end,
R9),
testutil:check_forobject(Book4, TestObject),
ok = leveled_bookie:book_close(Book4),
testutil:reset_filestructure().
count_termsonindex(Bucket, IdxField, Book, QType) ->
lists:foldl(fun(X, Acc) ->
SW = os:timestamp(),
ST = integer_to_list(X),
ET = ST ++ "~",
R = leveled_bookie:book_returnfolder(Book,
{index_query,
Bucket,
{IdxField,
ST,
ET},
QType}),
{async, Folder} = R,
Items = length(Folder()),
io:format("2i query from term ~s on index ~s took " ++
"~w microseconds~n",
[ST,
IdxField,
timer:now_diff(os:timestamp(), SW)]),
Acc + Items
end,
0,
lists:seq(190, 221)).
rotating_objects(_Config) ->
RootPath = testutil:reset_filestructure(),
ok = testutil:rotating_object_check(RootPath, "Bucket1", 10),
ok = testutil:rotating_object_check(RootPath, "Bucket2", 200),
ok = testutil:rotating_object_check(RootPath, "Bucket3", 800),
ok = testutil:rotating_object_check(RootPath, "Bucket4", 1600),
ok = testutil:rotating_object_check(RootPath, "Bucket5", 3200),
ok = testutil:rotating_object_check(RootPath, "Bucket6", 9600),
testutil:reset_filestructure().

View file

@ -0,0 +1,314 @@
-module(recovery_SUITE).
-include_lib("common_test/include/ct.hrl").
-include("include/leveled.hrl").
-export([all/0]).
-export([retain_strategy/1,
aae_bustedjournal/1,
journal_compaction_bustedjournal/1
]).
all() -> [
retain_strategy,
aae_bustedjournal,
journal_compaction_bustedjournal
].
retain_strategy(_Config) ->
RootPath = testutil:reset_filestructure(),
BookOpts = [{root_path, RootPath},
{cache_size, 1000},
{max_journalsize, 5000000},
{reload_strategy, [{?RIAK_TAG, retain}]}],
BookOptsAlt = [{root_path, RootPath},
{cache_size, 1000},
{max_journalsize, 100000},
{reload_strategy, [{?RIAK_TAG, retain}]},
{max_run_length, 8}],
{ok, Spcl3, LastV3} = rotating_object_check(BookOpts, "Bucket3", 800),
ok = restart_from_blankledger(BookOpts, [{"Bucket3", Spcl3, LastV3}]),
{ok, Spcl4, LastV4} = rotating_object_check(BookOpts, "Bucket4", 1600),
ok = restart_from_blankledger(BookOpts, [{"Bucket3", Spcl3, LastV3},
{"Bucket4", Spcl4, LastV4}]),
{ok, Spcl5, LastV5} = rotating_object_check(BookOpts, "Bucket5", 3200),
ok = restart_from_blankledger(BookOptsAlt, [{"Bucket3", Spcl3, LastV3},
{"Bucket5", Spcl5, LastV5}]),
{ok, Spcl6, LastV6} = rotating_object_check(BookOpts, "Bucket6", 6400),
ok = restart_from_blankledger(BookOpts, [{"Bucket3", Spcl3, LastV3},
{"Bucket4", Spcl4, LastV4},
{"Bucket5", Spcl5, LastV5},
{"Bucket6", Spcl6, LastV6}]),
testutil:reset_filestructure().
aae_bustedjournal(_Config) ->
RootPath = testutil:reset_filestructure(),
StartOpts = [{root_path, RootPath},
{max_journalsize, 20000000}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
GenList = [2],
_CLs = testutil:load_objects(20000, GenList, Bookie1, TestObject,
fun testutil:generate_objects/2),
ok = leveled_bookie:book_close(Bookie1),
CDBFiles = testutil:find_journals(RootPath),
[HeadF|_Rest] = CDBFiles,
io:format("Selected Journal for corruption of ~s~n", [HeadF]),
testutil:corrupt_journal(RootPath, HeadF, 1000, 2048, 1000),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts),
{async, KeyF} = leveled_bookie:book_returnfolder(Bookie2,
{keylist, ?RIAK_TAG}),
KeyList = KeyF(),
20001 = length(KeyList),
HeadCount = lists:foldl(fun({B, K}, Acc) ->
case leveled_bookie:book_riakhead(Bookie2,
B,
K) of
{ok, _} -> Acc + 1;
not_found -> Acc
end
end,
0,
KeyList),
20001 = HeadCount,
GetCount = lists:foldl(fun({B, K}, Acc) ->
case leveled_bookie:book_riakget(Bookie2,
B,
K) of
{ok, _} -> Acc + 1;
not_found -> Acc
end
end,
0,
KeyList),
true = GetCount > 19000,
true = GetCount < HeadCount,
{async, HashTreeF1} = leveled_bookie:book_returnfolder(Bookie2,
{hashtree_query,
?RIAK_TAG,
false}),
KeyHashList1 = HashTreeF1(),
20001 = length(KeyHashList1),
{async, HashTreeF2} = leveled_bookie:book_returnfolder(Bookie2,
{hashtree_query,
?RIAK_TAG,
check_presence}),
KeyHashList2 = HashTreeF2(),
% The file is still there, and the hashtree is not corrupted
KeyHashList2 = KeyHashList1,
% Will need to remove the file or corrupt the hashtree to get presence to
% fail
FoldObjectsFun = fun(B, K, V, Acc) -> [{B, K, testutil:riak_hash(V)}|Acc]
end,
SW = os:timestamp(),
{async, HashTreeF3} = leveled_bookie:book_returnfolder(Bookie2,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
KeyHashList3 = HashTreeF3(),
true = length(KeyHashList3) > 19000,
true = length(KeyHashList3) < HeadCount,
Delta = length(lists:subtract(KeyHashList1, KeyHashList3)),
true = Delta < 1001,
io:format("Fetch of hashtree using fold objects took ~w microseconds" ++
" and found a Delta of ~w and an objects count of ~w~n",
[timer:now_diff(os:timestamp(), SW),
Delta,
length(KeyHashList3)]),
ok = leveled_bookie:book_close(Bookie2),
{ok, BytesCopied} = testutil:restore_file(RootPath, HeadF),
io:format("File restored is of size ~w~n", [BytesCopied]),
{ok, Bookie3} = leveled_bookie:book_start(StartOpts),
SW4 = os:timestamp(),
{async, HashTreeF4} = leveled_bookie:book_returnfolder(Bookie3,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
KeyHashList4 = HashTreeF4(),
true = length(KeyHashList4) == 20001,
io:format("Fetch of hashtree using fold objects took ~w microseconds" ++
" and found an object count of ~w~n",
[timer:now_diff(os:timestamp(), SW4), length(KeyHashList4)]),
ok = leveled_bookie:book_close(Bookie3),
testutil:corrupt_journal(RootPath, HeadF, 500, BytesCopied - 8000, 14),
{ok, Bookie4} = leveled_bookie:book_start(StartOpts),
SW5 = os:timestamp(),
{async, HashTreeF5} = leveled_bookie:book_returnfolder(Bookie4,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
KeyHashList5 = HashTreeF5(),
true = length(KeyHashList5) > 19000,
true = length(KeyHashList5) < HeadCount,
Delta5 = length(lists:subtract(KeyHashList1, KeyHashList5)),
true = Delta5 < 1001,
io:format("Fetch of hashtree using fold objects took ~w microseconds" ++
" and found a Delta of ~w and an objects count of ~w~n",
[timer:now_diff(os:timestamp(), SW5),
Delta5,
length(KeyHashList5)]),
{async, HashTreeF6} = leveled_bookie:book_returnfolder(Bookie4,
{hashtree_query,
?RIAK_TAG,
check_presence}),
KeyHashList6 = HashTreeF6(),
true = length(KeyHashList6) > 19000,
true = length(KeyHashList6) < HeadCount,
ok = leveled_bookie:book_close(Bookie4),
testutil:restore_topending(RootPath, HeadF),
{ok, Bookie5} = leveled_bookie:book_start(StartOpts),
SW6 = os:timestamp(),
{async, HashTreeF7} = leveled_bookie:book_returnfolder(Bookie5,
{foldobjects_allkeys,
?RIAK_TAG,
FoldObjectsFun}),
KeyHashList7 = HashTreeF7(),
true = length(KeyHashList7) == 20001,
io:format("Fetch of hashtree using fold objects took ~w microseconds" ++
" and found an object count of ~w~n",
[timer:now_diff(os:timestamp(), SW6), length(KeyHashList7)]),
ok = leveled_bookie:book_close(Bookie5),
testutil:reset_filestructure().
journal_compaction_bustedjournal(_Config) ->
% Simply confirms that none of this causes a crash
RootPath = testutil:reset_filestructure(),
StartOpts1 = [{root_path, RootPath},
{max_journalsize, 10000000},
{max_run_length, 10}],
{ok, Bookie1} = leveled_bookie:book_start(StartOpts1),
{TestObject, TestSpec} = testutil:generate_testobject(),
ok = leveled_bookie:book_riakput(Bookie1, TestObject, TestSpec),
testutil:check_forobject(Bookie1, TestObject),
ObjList1 = testutil:generate_objects(50000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjList1),
%% Now replace all the objects
ObjList2 = testutil:generate_objects(50000, 2),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie1, Obj, Spc) end,
ObjList2),
ok = leveled_bookie:book_close(Bookie1),
CDBFiles = testutil:find_journals(RootPath),
lists:foreach(fun(FN) ->
testutil:corrupt_journal(RootPath, FN, 100, 2048, 1000)
end,
CDBFiles),
{ok, Bookie2} = leveled_bookie:book_start(StartOpts1),
ok = leveled_bookie:book_compactjournal(Bookie2, 30000),
F = fun leveled_bookie:book_islastcompactionpending/1,
lists:foldl(fun(X, Pending) ->
case Pending of
false ->
false;
true ->
io:format("Loop ~w waiting for journal "
++ "compaction to complete~n", [X]),
timer:sleep(20000),
F(Bookie2)
end end,
true,
lists:seq(1, 15)),
ok = leveled_bookie:book_close(Bookie2),
testutil:reset_filestructure(10000).
rotating_object_check(BookOpts, B, NumberOfObjects) ->
{ok, Book1} = leveled_bookie:book_start(BookOpts),
{KSpcL1, V1} = testutil:put_indexed_objects(Book1, B, NumberOfObjects),
ok = testutil:check_indexed_objects(Book1,
B,
KSpcL1,
V1),
{KSpcL2, V2} = testutil:put_altered_indexed_objects(Book1,
B,
KSpcL1,
false),
ok = testutil:check_indexed_objects(Book1,
B,
KSpcL1 ++ KSpcL2,
V2),
{KSpcL3, V3} = testutil:put_altered_indexed_objects(Book1,
B,
KSpcL2,
false),
ok = leveled_bookie:book_close(Book1),
{ok, Book2} = leveled_bookie:book_start(BookOpts),
ok = testutil:check_indexed_objects(Book2,
B,
KSpcL1 ++ KSpcL2 ++ KSpcL3,
V3),
{KSpcL4, V4} = testutil:put_altered_indexed_objects(Book2,
B,
KSpcL3,
false),
io:format("Bucket complete - checking index before compaction~n"),
ok = testutil:check_indexed_objects(Book2,
B,
KSpcL1 ++ KSpcL2 ++ KSpcL3 ++ KSpcL4,
V4),
ok = leveled_bookie:book_compactjournal(Book2, 30000),
F = fun leveled_bookie:book_islastcompactionpending/1,
lists:foldl(fun(X, Pending) ->
case Pending of
false ->
false;
true ->
io:format("Loop ~w waiting for journal "
++ "compaction to complete~n", [X]),
timer:sleep(20000),
F(Book2)
end end,
true,
lists:seq(1, 15)),
io:format("Waiting for journal deletes~n"),
timer:sleep(20000),
io:format("Checking index following compaction~n"),
ok = testutil:check_indexed_objects(Book2,
B,
KSpcL1 ++ KSpcL2 ++ KSpcL3 ++ KSpcL4,
V4),
ok = leveled_bookie:book_close(Book2),
{ok, KSpcL1 ++ KSpcL2 ++ KSpcL3 ++ KSpcL4, V4}.
restart_from_blankledger(BookOpts, B_SpcL) ->
leveled_penciller:clean_testdir(proplists:get_value(root_path, BookOpts) ++
"/ledger"),
{ok, Book1} = leveled_bookie:book_start(BookOpts),
io:format("Checking index following restart~n"),
lists:foreach(fun({B, SpcL, V}) ->
ok = testutil:check_indexed_objects(Book1, B, SpcL, V)
end,
B_SpcL),
ok = leveled_bookie:book_close(Book1),
ok.

View file

@ -0,0 +1,440 @@
-module(testutil).
-include("../include/leveled.hrl").
-export([reset_filestructure/0,
reset_filestructure/1,
check_bucket_stats/2,
check_forlist/2,
check_forlist/3,
check_formissinglist/2,
check_forobject/2,
check_formissingobject/3,
generate_testobject/0,
generate_testobject/5,
generate_compressibleobjects/2,
generate_smallobjects/2,
generate_objects/2,
generate_objects/5,
generate_objects/6,
set_object/5,
get_key/1,
get_value/1,
get_compressiblevalue/0,
get_compressiblevalue_andinteger/0,
get_randomindexes_generator/1,
name_list/0,
load_objects/5,
put_indexed_objects/3,
put_altered_indexed_objects/3,
put_altered_indexed_objects/4,
check_indexed_objects/4,
rotating_object_check/3,
corrupt_journal/5,
restore_file/2,
restore_topending/2,
find_journals/1,
riak_hash/1]).
-define(RETURN_TERMS, {true, undefined}).
reset_filestructure() ->
reset_filestructure(0).
reset_filestructure(Wait) ->
io:format("Waiting ~w ms to give a chance for all file closes " ++
"to complete~n", [Wait]),
timer:sleep(Wait),
RootPath = "test",
filelib:ensure_dir(RootPath ++ "/journal/"),
filelib:ensure_dir(RootPath ++ "/ledger/"),
leveled_inker:clean_testdir(RootPath ++ "/journal"),
leveled_penciller:clean_testdir(RootPath ++ "/ledger"),
RootPath.
check_bucket_stats(Bookie, Bucket) ->
FoldSW1 = os:timestamp(),
io:format("Checking bucket size~n"),
{async, Folder1} = leveled_bookie:book_returnfolder(Bookie,
{riakbucket_stats,
Bucket}),
{B1Size, B1Count} = Folder1(),
io:format("Bucket fold completed in ~w microseconds~n",
[timer:now_diff(os:timestamp(), FoldSW1)]),
io:format("Bucket ~s has size ~w and count ~w~n",
[Bucket, B1Size, B1Count]),
{B1Size, B1Count}.
check_forlist(Bookie, ChkList) ->
check_forlist(Bookie, ChkList, false).
check_forlist(Bookie, ChkList, Log) ->
SW = os:timestamp(),
lists:foreach(fun({_RN, Obj, _Spc}) ->
if
Log == true ->
io:format("Fetching Key ~s~n", [Obj#r_object.key]);
true ->
ok
end,
R = leveled_bookie:book_riakget(Bookie,
Obj#r_object.bucket,
Obj#r_object.key),
ok = case R of
{ok, Obj} ->
ok;
not_found ->
io:format("Object not found for key ~s~n",
[Obj#r_object.key]),
error
end
end,
ChkList),
io:format("Fetch check took ~w microseconds checking list of length ~w~n",
[timer:now_diff(os:timestamp(), SW), length(ChkList)]).
check_formissinglist(Bookie, ChkList) ->
SW = os:timestamp(),
lists:foreach(fun({_RN, Obj, _Spc}) ->
R = leveled_bookie:book_riakget(Bookie,
Obj#r_object.bucket,
Obj#r_object.key),
R = not_found end,
ChkList),
io:format("Miss check took ~w microseconds checking list of length ~w~n",
[timer:now_diff(os:timestamp(), SW), length(ChkList)]).
check_forobject(Bookie, TestObject) ->
{ok, TestObject} = leveled_bookie:book_riakget(Bookie,
TestObject#r_object.bucket,
TestObject#r_object.key),
{ok, HeadObject} = leveled_bookie:book_riakhead(Bookie,
TestObject#r_object.bucket,
TestObject#r_object.key),
ok = case {HeadObject#r_object.bucket,
HeadObject#r_object.key,
HeadObject#r_object.vclock} of
{B1, K1, VC1} when B1 == TestObject#r_object.bucket,
K1 == TestObject#r_object.key,
VC1 == TestObject#r_object.vclock ->
ok
end.
check_formissingobject(Bookie, Bucket, Key) ->
not_found = leveled_bookie:book_riakget(Bookie, Bucket, Key),
not_found = leveled_bookie:book_riakhead(Bookie, Bucket, Key).
generate_testobject() ->
{B1, K1, V1, Spec1, MD} = {"Bucket1",
"Key1",
"Value1",
[],
{"MDK1", "MDV1"}},
generate_testobject(B1, K1, V1, Spec1, MD).
generate_testobject(B, K, V, Spec, MD) ->
Content = #r_content{metadata=MD, value=V},
{#r_object{bucket=B, key=K, contents=[Content], vclock=[{'a',1}]},
Spec}.
generate_compressibleobjects(Count, KeyNumber) ->
V = get_compressiblevalue(),
generate_objects(Count, KeyNumber, [], V).
get_compressiblevalue_andinteger() ->
{random:uniform(1000), get_compressiblevalue()}.
get_compressiblevalue() ->
S1 = "111111111111111",
S2 = "222222222222222",
S3 = "333333333333333",
S4 = "aaaaaaaaaaaaaaa",
S5 = "AAAAAAAAAAAAAAA",
S6 = "GGGGGGGGGGGGGGG",
S7 = "===============",
S8 = "...............",
Selector = [{1, S1}, {2, S2}, {3, S3}, {4, S4},
{5, S5}, {6, S6}, {7, S7}, {8, S8}],
L = lists:seq(1, 1024),
lists:foldl(fun(_X, Acc) ->
{_, Str} = lists:keyfind(random:uniform(8), 1, Selector),
Acc ++ Str end,
"",
L).
generate_smallobjects(Count, KeyNumber) ->
generate_objects(Count, KeyNumber, [], crypto:rand_bytes(512)).
generate_objects(Count, KeyNumber) ->
generate_objects(Count, KeyNumber, [], crypto:rand_bytes(4096)).
generate_objects(Count, KeyNumber, ObjL, Value) ->
generate_objects(Count, KeyNumber, ObjL, Value, fun() -> [] end).
generate_objects(Count, KeyNumber, ObjL, Value, IndexGen) ->
generate_objects(Count, KeyNumber, ObjL, Value, IndexGen, "Bucket").
generate_objects(0, _KeyNumber, ObjL, _Value, _IndexGen, _Bucket) ->
ObjL;
generate_objects(Count, uuid, ObjL, Value, IndexGen, Bucket) ->
{Obj1, Spec1} = set_object(Bucket,
leveled_codec:generate_uuid(),
Value,
IndexGen),
generate_objects(Count - 1,
uuid,
ObjL ++ [{random:uniform(), Obj1, Spec1}],
Value,
IndexGen,
Bucket);
generate_objects(Count, KeyNumber, ObjL, Value, IndexGen, Bucket) ->
{Obj1, Spec1} = set_object(Bucket,
"Key" ++ integer_to_list(KeyNumber),
Value,
IndexGen),
generate_objects(Count - 1,
KeyNumber + 1,
ObjL ++ [{random:uniform(), Obj1, Spec1}],
Value,
IndexGen,
Bucket).
set_object(Bucket, Key, Value, IndexGen) ->
set_object(Bucket, Key, Value, IndexGen, []).
set_object(Bucket, Key, Value, IndexGen, Indexes2Remove) ->
Obj = {Bucket,
Key,
Value,
IndexGen() ++ lists:map(fun({add, IdxF, IdxV}) ->
{remove, IdxF, IdxV} end,
Indexes2Remove),
[{"MDK", "MDV" ++ Key},
{"MDK2", "MDV" ++ Key}]},
{B1, K1, V1, Spec1, MD} = Obj,
Content = #r_content{metadata=MD, value=V1},
{#r_object{bucket=B1, key=K1, contents=[Content], vclock=[{'a',1}]},
Spec1}.
get_key(Object) ->
Object#r_object.key.
get_value(Object) ->
[Content] = Object#r_object.contents,
Content#r_content.value.
load_objects(ChunkSize, GenList, Bookie, TestObject, Generator) ->
lists:map(fun(KN) ->
ObjListA = Generator(ChunkSize, KN),
StartWatchA = os:timestamp(),
lists:foreach(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Bookie, Obj, Spc)
end,
ObjListA),
Time = timer:now_diff(os:timestamp(), StartWatchA),
io:format("~w objects loaded in ~w seconds~n",
[ChunkSize, Time/1000000]),
if
TestObject == no_check ->
ok;
true ->
check_forobject(Bookie, TestObject)
end,
lists:sublist(ObjListA, 1000) end,
GenList).
get_randomindexes_generator(Count) ->
Generator = fun() ->
lists:map(fun(X) ->
{add,
"idx" ++ integer_to_list(X) ++ "_bin",
get_randomdate() ++ get_randomname()} end,
lists:seq(1, Count))
end,
Generator.
name_list() ->
[{1, "Sophia"}, {2, "Emma"}, {3, "Olivia"}, {4, "Ava"},
{5, "Isabella"}, {6, "Mia"}, {7, "Zoe"}, {8, "Lily"},
{9, "Emily"}, {10, "Madelyn"}, {11, "Madison"}, {12, "Chloe"},
{13, "Charlotte"}, {14, "Aubrey"}, {15, "Avery"},
{16, "Abigail"}].
get_randomname() ->
NameList = name_list(),
N = random:uniform(16),
{N, Name} = lists:keyfind(N, 1, NameList),
Name.
get_randomdate() ->
LowTime = 60000000000,
HighTime = 70000000000,
RandPoint = LowTime + random:uniform(HighTime - LowTime),
Date = calendar:gregorian_seconds_to_datetime(RandPoint),
{{Year, Month, Day}, {Hour, Minute, Second}} = Date,
lists:flatten(io_lib:format("~4..0w~2..0w~2..0w~2..0w~2..0w~2..0w",
[Year, Month, Day, Hour, Minute, Second])).
check_indexed_objects(Book, B, KSpecL, V) ->
% Check all objects match, return what should be the results of an all
% index query
IdxR = lists:map(fun({K, Spc}) ->
{ok, O} = leveled_bookie:book_riakget(Book, B, K),
V = testutil:get_value(O),
{add,
"idx1_bin",
IdxVal} = lists:keyfind(add, 1, Spc),
{IdxVal, K} end,
KSpecL),
% Check the all index query matches expectations
R = leveled_bookie:book_returnfolder(Book,
{index_query,
B,
{"idx1_bin",
"0",
"~"},
?RETURN_TERMS}),
SW = os:timestamp(),
{async, Fldr} = R,
QR0 = Fldr(),
io:format("Query match found of length ~w in ~w microseconds " ++
"expected ~w ~n",
[length(QR0),
timer:now_diff(os:timestamp(), SW),
length(IdxR)]),
QR = lists:sort(QR0),
ER = lists:sort(IdxR),
ok = if
ER == QR ->
ok
end,
ok.
put_indexed_objects(Book, Bucket, Count) ->
V = testutil:get_compressiblevalue(),
IndexGen = testutil:get_randomindexes_generator(1),
SW = os:timestamp(),
ObjL1 = testutil:generate_objects(Count,
uuid,
[],
V,
IndexGen,
Bucket),
KSpecL = lists:map(fun({_RN, Obj, Spc}) ->
leveled_bookie:book_riakput(Book,
Obj,
Spc),
{testutil:get_key(Obj), Spc}
end,
ObjL1),
io:format("Put of ~w objects with ~w index entries "
++
"each completed in ~w microseconds~n",
[Count, 1, timer:now_diff(os:timestamp(), SW)]),
{KSpecL, V}.
put_altered_indexed_objects(Book, Bucket, KSpecL) ->
put_altered_indexed_objects(Book, Bucket, KSpecL, true).
put_altered_indexed_objects(Book, Bucket, KSpecL, RemoveOld2i) ->
IndexGen = testutil:get_randomindexes_generator(1),
V = testutil:get_compressiblevalue(),
RplKSpecL = lists:map(fun({K, Spc}) ->
AddSpc = if
RemoveOld2i == true ->
[lists:keyfind(add, 1, Spc)];
RemoveOld2i == false ->
[]
end,
{O, AltSpc} = testutil:set_object(Bucket,
K,
V,
IndexGen,
AddSpc),
ok = leveled_bookie:book_riakput(Book,
O,
AltSpc),
{K, AltSpc} end,
KSpecL),
{RplKSpecL, V}.
rotating_object_check(RootPath, B, NumberOfObjects) ->
BookOpts = [{root_path, RootPath},
{cache_size, 1000},
{max_journalsize, 5000000}],
{ok, Book1} = leveled_bookie:book_start(BookOpts),
{KSpcL1, V1} = testutil:put_indexed_objects(Book1, B, NumberOfObjects),
ok = testutil:check_indexed_objects(Book1, B, KSpcL1, V1),
{KSpcL2, V2} = testutil:put_altered_indexed_objects(Book1, B, KSpcL1),
ok = testutil:check_indexed_objects(Book1, B, KSpcL2, V2),
{KSpcL3, V3} = testutil:put_altered_indexed_objects(Book1, B, KSpcL2),
ok = leveled_bookie:book_close(Book1),
{ok, Book2} = leveled_bookie:book_start(BookOpts),
ok = testutil:check_indexed_objects(Book2, B, KSpcL3, V3),
{KSpcL4, V4} = testutil:put_altered_indexed_objects(Book2, B, KSpcL3),
ok = testutil:check_indexed_objects(Book2, B, KSpcL4, V4),
ok = leveled_bookie:book_close(Book2),
ok.
corrupt_journal(RootPath, FileName, Corruptions, BasePosition, GapSize) ->
OriginalPath = RootPath ++ "/journal/journal_files/" ++ FileName,
BackupPath = RootPath ++ "/journal/journal_files/" ++
filename:basename(FileName, ".cdb") ++ ".bak",
{ok, _BytesCopied} = file:copy(OriginalPath, BackupPath),
{ok, Handle} = file:open(OriginalPath, [binary, raw, read, write]),
lists:foreach(fun(X) ->
Position = X * GapSize + BasePosition,
ok = file:pwrite(Handle, Position, <<0:8/integer>>)
end,
lists:seq(1, Corruptions)),
ok = file:close(Handle).
restore_file(RootPath, FileName) ->
OriginalPath = RootPath ++ "/journal/journal_files/" ++ FileName,
BackupPath = RootPath ++ "/journal/journal_files/" ++
filename:basename(FileName, ".cdb") ++ ".bak",
file:copy(BackupPath, OriginalPath).
restore_topending(RootPath, FileName) ->
OriginalPath = RootPath ++ "/journal/journal_files/" ++ FileName,
PndPath = RootPath ++ "/journal/journal_files/" ++
filename:basename(FileName, ".cdb") ++ ".pnd",
ok = file:rename(OriginalPath, PndPath),
false = filelib:is_file(OriginalPath).
find_journals(RootPath) ->
{ok, FNsA_J} = file:list_dir(RootPath ++ "/journal/journal_files"),
{ok, Regex} = re:compile(".*\.cdb"),
CDBFiles = lists:foldl(fun(FN, Acc) -> case re:run(FN, Regex) of
nomatch ->
Acc;
_ ->
[FN|Acc]
end
end,
[],
FNsA_J),
CDBFiles.
riak_hash(Obj=#r_object{}) ->
Vclock = vclock(Obj),
UpdObj = set_vclock(Obj, lists:sort(Vclock)),
erlang:phash2(term_to_binary(UpdObj)).
set_vclock(Object=#r_object{}, VClock) -> Object#r_object{vclock=VClock}.
vclock(#r_object{vclock=VClock}) -> VClock.

323
test/lookup_test.erl Normal file
View file

@ -0,0 +1,323 @@
-module(lookup_test).
-export([go_dict/1,
go_ets/1,
go_gbtree/1,
go_arrayofdict/1,
go_arrayofgbtree/1,
go_arrayofdict_withcache/1,
create_blocks/3,
size_testblocks/1,
test_testblocks/2]).
-define(CACHE_SIZE, 512).
hash(Key) ->
H = 5381,
hash1(H,Key) band 16#FFFFFFFF.
hash1(H,[]) ->H;
hash1(H,[B|Rest]) ->
H1 = H * 33,
H2 = H1 bxor B,
hash1(H2,Rest).
% Get the least significant 8 bits from the hash.
hash_to_index(Hash) ->
Hash band 255.
%%
%% Timings (microseconds):
%%
%% go_dict(200000) : 1569894
%% go_dict(1000000) : 17191365
%% go_dict(5000000) : forever
go_dict(N) ->
go_dict(dict:new(), N, N).
go_dict(_, 0, _) ->
{erlang:memory(), statistics(garbage_collection)};
go_dict(D, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
dict:find(LookupHash, D),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
case dict:find(Hash, D) of
error ->
go_dict(dict:store(Hash, [N], D), N-1, M);
{ok, List} ->
go_dict(dict:store(Hash, [N|List], D), N-1, M)
end.
%%
%% Timings (microseconds):
%%
%% go_ets(200000) : 609119
%% go_ets(1000000) : 3520757
%% go_ets(5000000) : 19974562
go_ets(N) ->
go_ets(ets:new(ets_test, [private, bag]), N, N).
go_ets(_, 0, _) ->
{erlang:memory(), statistics(garbage_collection)};
go_ets(Ets, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
ets:lookup(Ets, LookupHash),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
ets:insert(Ets, {Hash, N}),
go_ets(Ets, N - 1, M).
%%
%% Timings (microseconds):
%%
%% go_gbtree(200000) : 1393936
%% go_gbtree(1000000) : 8430997
%% go_gbtree(5000000) : 45630810
go_gbtree(N) ->
go_gbtree(gb_trees:empty(), N, N).
go_gbtree(_, 0, _) ->
{erlang:memory(), statistics(garbage_collection)};
go_gbtree(Tree, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
gb_trees:lookup(LookupHash, Tree),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
case gb_trees:lookup(Hash, Tree) of
none ->
go_gbtree(gb_trees:insert(Hash, [N], Tree), N - 1, M);
{value, List} ->
go_gbtree(gb_trees:update(Hash, [N|List], Tree), N - 1, M)
end.
%%
%% Timings (microseconds):
%%
%% go_arrayofidict(200000) : 1266931
%% go_arrayofidict(1000000) : 7387219
%% go_arrayofidict(5000000) : 49511484
go_arrayofdict(N) ->
go_arrayofdict(array:new(256, {default, dict:new()}), N, N).
go_arrayofdict(_, 0, _) ->
% dict:to_list(array:get(0, Array)),
% dict:to_list(array:get(1, Array)),
% dict:to_list(array:get(2, Array)),
% dict:to_list(array:get(3, Array)),
% dict:to_list(array:get(4, Array)),
% dict:to_list(array:get(5, Array)),
% dict:to_list(array:get(6, Array)),
% dict:to_list(array:get(7, Array)),
% dict:to_list(array:get(8, Array)),
% dict:to_list(array:get(9, Array)),
{erlang:memory(), statistics(garbage_collection)};
go_arrayofdict(Array, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
LookupIndex = hash_to_index(LookupHash),
dict:find(LookupHash, array:get(LookupIndex, Array)),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
Index = hash_to_index(Hash),
D = array:get(Index, Array),
case dict:find(Hash, D) of
error ->
go_arrayofdict(array:set(Index,
dict:store(Hash, [N], D), Array), N-1, M);
{ok, List} ->
go_arrayofdict(array:set(Index,
dict:store(Hash, [N|List], D), Array), N-1, M)
end.
%%
%% Timings (microseconds):
%%
%% go_arrayofgbtree(200000) : 1176224
%% go_arrayofgbtree(1000000) : 7480653
%% go_arrayofgbtree(5000000) : 41266701
go_arrayofgbtree(N) ->
go_arrayofgbtree(array:new(256, {default, gb_trees:empty()}), N, N).
go_arrayofgbtree(_, 0, _) ->
% gb_trees:to_list(array:get(0, Array)),
% gb_trees:to_list(array:get(1, Array)),
% gb_trees:to_list(array:get(2, Array)),
% gb_trees:to_list(array:get(3, Array)),
% gb_trees:to_list(array:get(4, Array)),
% gb_trees:to_list(array:get(5, Array)),
% gb_trees:to_list(array:get(6, Array)),
% gb_trees:to_list(array:get(7, Array)),
% gb_trees:to_list(array:get(8, Array)),
% gb_trees:to_list(array:get(9, Array)),
{erlang:memory(), statistics(garbage_collection)};
go_arrayofgbtree(Array, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
LookupIndex = hash_to_index(LookupHash),
gb_trees:lookup(LookupHash, array:get(LookupIndex, Array)),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
Index = hash_to_index(Hash),
Tree = array:get(Index, Array),
case gb_trees:lookup(Hash, Tree) of
none ->
go_arrayofgbtree(array:set(Index,
gb_trees:insert(Hash, [N], Tree), Array), N - 1, M);
{value, List} ->
go_arrayofgbtree(array:set(Index,
gb_trees:update(Hash, [N|List], Tree), Array), N - 1, M)
end.
%%
%% Timings (microseconds):
%%
%% go_arrayofdict_withcache(200000) : 1432951
%% go_arrayofdict_withcache(1000000) : 9140169
%% go_arrayofdict_withcache(5000000) : 59435511
go_arrayofdict_withcache(N) ->
go_arrayofdict_withcache({array:new(256, {default, dict:new()}),
array:new(256, {default, dict:new()})}, N, N).
go_arrayofdict_withcache(_, 0, _) ->
{erlang:memory(), statistics(garbage_collection)};
go_arrayofdict_withcache({MArray, CArray}, N, M) ->
% Lookup a random key - which may not be present
LookupKey = lists:concat(["key-", random:uniform(M)]),
LookupHash = hash(LookupKey),
LookupIndex = hash_to_index(LookupHash),
dict:find(LookupHash, array:get(LookupIndex, CArray)),
dict:find(LookupHash, array:get(LookupIndex, MArray)),
% Add a new key - which may be present so value to be appended
Key = lists:concat(["key-", N]),
Hash = hash(Key),
Index = hash_to_index(Hash),
Cache = array:get(Index, CArray),
case dict:find(Hash, Cache) of
error ->
UpdCache = dict:store(Hash, [N], Cache);
{ok, _} ->
UpdCache = dict:append(Hash, N, Cache)
end,
case dict:size(UpdCache) of
?CACHE_SIZE ->
UpdCArray = array:set(Index, dict:new(), CArray),
UpdMArray = array:set(Index, dict:merge(fun merge_values/3, UpdCache, array:get(Index, MArray)), MArray),
go_arrayofdict_withcache({UpdMArray, UpdCArray}, N - 1, M);
_ ->
UpdCArray = array:set(Index, UpdCache, CArray),
go_arrayofdict_withcache({MArray, UpdCArray}, N - 1, M)
end.
merge_values(_, Value1, Value2) ->
lists:append(Value1, Value2).
%% Some functions for testing options compressing term_to_binary
create_block(N, BlockType) ->
case BlockType of
keylist ->
create_block(N, BlockType, []);
keygbtree ->
create_block(N, BlockType, gb_trees:empty())
end.
create_block(0, _, KeyStruct) ->
KeyStruct;
create_block(N, BlockType, KeyStruct) ->
Bucket = <<"pdsRecord">>,
case N of
20 ->
Key = lists:concat(["key-20-special"]);
_ ->
Key = lists:concat(["key-", N, "-", random:uniform(1000)])
end,
SequenceNumber = random:uniform(1000000000),
Indexes = [{<<"DateOfBirth_int">>, random:uniform(10000)}, {<<"index1_bin">>, lists:concat([random:uniform(1000), "SomeCommonText"])}, {<<"index2_bin">>, <<"RepetitionRepetitionRepetition">>}],
case BlockType of
keylist ->
Term = {o, Bucket, Key, {Indexes, SequenceNumber}},
create_block(N-1, BlockType, [Term|KeyStruct]);
keygbtree ->
create_block(N-1, BlockType, gb_trees:insert({o, Bucket, Key}, {Indexes, SequenceNumber}, KeyStruct))
end.
create_blocks(N, Compression, BlockType) ->
create_blocks(N, Compression, BlockType, 10000, []).
create_blocks(_, _, _, 0, BlockList) ->
BlockList;
create_blocks(N, Compression, BlockType, TestLoops, BlockList) ->
NewBlock = term_to_binary(create_block(N, BlockType), [{compressed, Compression}]),
create_blocks(N, Compression, BlockType, TestLoops - 1, [NewBlock|BlockList]).
size_testblocks(BlockList) ->
size_testblocks(BlockList,0).
size_testblocks([], Acc) ->
Acc;
size_testblocks([H|T], Acc) ->
size_testblocks(T, Acc + byte_size(H)).
test_testblocks([], _) ->
true;
test_testblocks([H|T], BlockType) ->
Block = binary_to_term(H),
case findkey("key-20-special", Block, BlockType) of
true ->
test_testblocks(T, BlockType);
not_found ->
false
end.
findkey(_, [], keylist) ->
not_found;
findkey(Key, [H|T], keylist) ->
case H of
{o, <<"pdsRecord">>, Key, _} ->
true;
_ ->
findkey(Key,T, keylist)
end;
findkey(Key, Tree, keygbtree) ->
case gb_trees:lookup({o, <<"pdsRecord">>, Key}, Tree) of
none ->
not_found;
_ ->
true
end.

59
test/rice_test.erl Normal file
View file

@ -0,0 +1,59 @@
%% Test performance and accuracy of rice-encoded bloom filters
%%
%% Calling check_negative(2048, 1000000) should return about 122 false
%% positives in around 11 seconds, with a size below 4KB
%%
%% The equivalent positive check is check_positive(2048, 488) and this
%% should take around 6 seconds.
%%
%% So a blooom with 2048 members should support o(100K) checks per second
%% on a modern CPU, whilst requiring 2 bytes per member.
-module(rice_test).
-export([check_positive/2, check_negative/2, calc_hash/2]).
check_positive(KeyCount, LoopCount) ->
KeyList = produce_keylist(KeyCount),
Bloom = leveled_rice:create_bloom(KeyList),
check_positive(KeyList, Bloom, LoopCount).
check_positive(_, Bloom, 0) ->
{ok, byte_size(Bloom)};
check_positive(KeyList, Bloom, LoopCount) ->
true = leveled_rice:check_keys(KeyList, Bloom),
check_positive(KeyList, Bloom, LoopCount - 1).
produce_keylist(KeyCount) ->
KeyPrefix = lists:concat(["PositiveKey-", random:uniform(KeyCount)]),
produce_keylist(KeyCount, [], KeyPrefix).
produce_keylist(0, KeyList, _) ->
KeyList;
produce_keylist(KeyCount, KeyList, KeyPrefix) ->
Key = lists:concat([KeyPrefix, KeyCount]),
produce_keylist(KeyCount - 1, [Key|KeyList], KeyPrefix).
check_negative(KeyCount, CheckCount) ->
KeyList = produce_keylist(KeyCount),
Bloom = leveled_rice:create_bloom(KeyList),
check_negative(Bloom, CheckCount, 0).
check_negative(Bloom, 0, FalsePos) ->
{byte_size(Bloom), FalsePos};
check_negative(Bloom, CheckCount, FalsePos) ->
Key = lists:concat(["NegativeKey-", CheckCount, random:uniform(CheckCount)]),
case leveled_rice:check_key(Key, Bloom) of
true -> check_negative(Bloom, CheckCount - 1, FalsePos + 1);
false -> check_negative(Bloom, CheckCount - 1, FalsePos)
end.
calc_hash(_, 0) ->
ok;
calc_hash(Key, Count) ->
erlang:phash2(lists:concat([Key, Count, "sometxt"])),
calc_hash(Key, Count -1).