mirror of
https://github.com/processone/ejabberd.git
synced 2024-11-20 16:15:59 +01:00
Replace code using p1_time_compat wrapper with native functions
Since we now require R19, we shouldn't need that anymore. There are still couple places where p1_time_compat:unique_timestamp() is used as there is no direct equivalent.
This commit is contained in:
parent
77ac0584ed
commit
538f35d05a
@ -21,7 +21,7 @@
|
||||
-record(archive_msg,
|
||||
{us = {<<"">>, <<"">>} :: {binary(), binary()} | '$2',
|
||||
id = <<>> :: binary() | '_',
|
||||
timestamp = p1_time_compat:timestamp() :: erlang:timestamp() | '_' | '$1',
|
||||
timestamp = erlang:timestamp() :: erlang:timestamp() | '_' | '$1',
|
||||
peer = {<<"">>, <<"">>, <<"">>} :: ljid() | '_' | '$3' | undefined,
|
||||
bare_peer = {<<"">>, <<"">>, <<"">>} :: ljid() | '_' | '$3',
|
||||
packet = #xmlel{} :: xmlel() | message() | '_',
|
||||
|
@ -113,7 +113,7 @@
|
||||
history :: lqueue(),
|
||||
subject = [] :: [text()],
|
||||
subject_author = <<"">> :: binary(),
|
||||
just_created = p1_time_compat:os_system_time(micro_seconds) :: true | integer(),
|
||||
just_created = erlang:system_time(microsecond) :: true | integer(),
|
||||
activity = treap:empty() :: treap:treap(),
|
||||
room_shaper = none :: shaper:shaper(),
|
||||
room_queue :: p1_queue:queue() | undefined
|
||||
|
@ -18,7 +18,7 @@
|
||||
%%%----------------------------------------------------------------------
|
||||
-record(push_session,
|
||||
{us = {<<"">>, <<"">>} :: {binary(), binary()},
|
||||
timestamp = p1_time_compat:timestamp() :: erlang:timestamp(),
|
||||
timestamp = erlang:timestamp() :: erlang:timestamp(),
|
||||
service = {<<"">>, <<"">>, <<"">>} :: ljid(),
|
||||
node = <<"">> :: binary(),
|
||||
xml :: undefined | xmlel()}).
|
||||
|
@ -325,7 +325,7 @@ wait_for_session(#body{attrs = Attrs} = Req, From,
|
||||
Type = get_attr(type, Attrs),
|
||||
Requests = Hold + 1,
|
||||
{PollTime, Polling} = if Wait == 0, Hold == 0 ->
|
||||
{p1_time_compat:timestamp(), [{polling, ?DEFAULT_POLLING}]};
|
||||
{erlang:timestamp(), [{polling, ?DEFAULT_POLLING}]};
|
||||
true -> {undefined, []}
|
||||
end,
|
||||
MaxPause = gen_mod:get_module_opt(State#state.host,
|
||||
@ -479,7 +479,7 @@ active1(#body{attrs = Attrs} = Req, From, State) ->
|
||||
Pause = get_attr(pause, Attrs, undefined),
|
||||
NewPoll = case State#state.prev_poll of
|
||||
undefined -> undefined;
|
||||
_ -> p1_time_compat:timestamp()
|
||||
_ -> erlang:timestamp()
|
||||
end,
|
||||
State5 = State4#state{prev_poll = NewPoll,
|
||||
prev_key = NewKey},
|
||||
@ -736,7 +736,7 @@ is_valid_key(PrevKey, Key) ->
|
||||
|
||||
is_overactivity(undefined) -> false;
|
||||
is_overactivity(PrevPoll) ->
|
||||
PollPeriod = timer:now_diff(p1_time_compat:timestamp(), PrevPoll) div
|
||||
PollPeriod = timer:now_diff(erlang:timestamp(), PrevPoll) div
|
||||
1000000,
|
||||
if PollPeriod < (?DEFAULT_POLLING) -> true;
|
||||
true -> false
|
||||
|
@ -725,7 +725,7 @@ process_self_presence(#{lserver := LServer} = State,
|
||||
{Pres1, State1} = ejabberd_hooks:run_fold(
|
||||
c2s_self_presence, LServer, {Pres, State}, []),
|
||||
State2 = State1#{pres_last => Pres1,
|
||||
pres_timestamp => p1_time_compat:timestamp()},
|
||||
pres_timestamp => erlang:timestamp()},
|
||||
FromUnavailable = PreviousPres == undefined,
|
||||
broadcast_presence_available(State2, Pres1, FromUnavailable);
|
||||
process_self_presence(State, _Pres) ->
|
||||
@ -888,7 +888,7 @@ bounce_message_queue() ->
|
||||
new_uniq_id() ->
|
||||
iolist_to_binary(
|
||||
[p1_rand:get_string(),
|
||||
integer_to_binary(p1_time_compat:unique_integer([positive]))]).
|
||||
integer_to_binary(erlang:unique_integer([positive]))]).
|
||||
|
||||
-spec get_conn_type(state()) -> c2s | c2s_tls | c2s_compressed | websocket |
|
||||
c2s_compressed_tls | http_bind.
|
||||
|
@ -589,7 +589,7 @@ callback(_, _, _) ->
|
||||
ok.
|
||||
|
||||
now_priority() ->
|
||||
-p1_time_compat:system_time(micro_seconds).
|
||||
-erlang:system_time(microsecond).
|
||||
|
||||
-spec opt_type(atom()) -> fun((any()) -> any()) | [atom()].
|
||||
opt_type(captcha_cmd) ->
|
||||
|
@ -73,7 +73,7 @@ start() ->
|
||||
[named_table, public, {read_concurrency, true}]),
|
||||
case load_file(ConfigFile) of
|
||||
{ok, State1} ->
|
||||
UnixTime = p1_time_compat:system_time(seconds),
|
||||
UnixTime = erlang:system_time(second),
|
||||
SharedKey = case erlang:get_cookie() of
|
||||
nocookie ->
|
||||
str:sha(p1_rand:get_string());
|
||||
@ -113,7 +113,7 @@ start(Hosts, Opts) ->
|
||||
[named_table, public, {read_concurrency, true}]),
|
||||
catch ets:new(ejabberd_db_modules,
|
||||
[named_table, public, {read_concurrency, true}]),
|
||||
UnixTime = p1_time_compat:system_time(seconds),
|
||||
UnixTime = erlang:system_time(second),
|
||||
SharedKey = case erlang:get_cookie() of
|
||||
nocookie ->
|
||||
str:sha(p1_rand:get_string());
|
||||
|
@ -110,7 +110,7 @@ code_change(_OldVsn, State, _Extra) ->
|
||||
%%%===================================================================
|
||||
-spec current_time() -> non_neg_integer().
|
||||
current_time() ->
|
||||
p1_time_compat:system_time(milli_seconds).
|
||||
erlang:system_time(millisecond).
|
||||
|
||||
-spec clean({non_neg_integer(), binary()} | '$end_of_table')
|
||||
-> non_neg_integer() | infinity.
|
||||
|
@ -366,7 +366,7 @@ init([I]) ->
|
||||
|
||||
handle_call(connect, From, #state{connection = undefined,
|
||||
pending_q = Q} = State) ->
|
||||
CurrTime = p1_time_compat:monotonic_time(milli_seconds),
|
||||
CurrTime = erlang:monotonic_time(millisecond),
|
||||
Q2 = try p1_queue:in({From, CurrTime}, Q)
|
||||
catch error:full ->
|
||||
Q1 = clean_queue(Q, CurrTime),
|
||||
@ -590,7 +590,7 @@ get_queue_type() ->
|
||||
|
||||
-spec flush_queue(p1_queue:queue()) -> p1_queue:queue().
|
||||
flush_queue(Q) ->
|
||||
CurrTime = p1_time_compat:monotonic_time(milli_seconds),
|
||||
CurrTime = erlang:monotonic_time(millisecond),
|
||||
p1_queue:dropwhile(
|
||||
fun({From, Time}) ->
|
||||
if (CurrTime - Time) >= ?CALL_TIMEOUT ->
|
||||
|
@ -413,8 +413,8 @@ get_component_number(LDomain) ->
|
||||
-spec get_domain_balancing(jid(), jid(), binary()) -> any().
|
||||
get_domain_balancing(From, To, LDomain) ->
|
||||
case ejabberd_config:get_option({domain_balancing, LDomain}) of
|
||||
undefined -> p1_time_compat:system_time();
|
||||
random -> p1_time_compat:system_time();
|
||||
undefined -> erlang:system_time();
|
||||
random -> erlang:system_time();
|
||||
source -> jid:tolower(From);
|
||||
destination -> jid:tolower(To);
|
||||
bare_source -> jid:remove_resource(jid:tolower(From));
|
||||
|
@ -112,7 +112,7 @@ external_host_overloaded(Host) ->
|
||||
"seconds",
|
||||
[Host, ?S2S_OVERLOAD_BLOCK_PERIOD]),
|
||||
mnesia:transaction(fun () ->
|
||||
Time = p1_time_compat:monotonic_time(),
|
||||
Time = erlang:monotonic_time(),
|
||||
mnesia:write(#temporarily_blocked{host = Host,
|
||||
timestamp = Time})
|
||||
end).
|
||||
@ -123,8 +123,8 @@ is_temporarly_blocked(Host) ->
|
||||
case mnesia:dirty_read(temporarily_blocked, Host) of
|
||||
[] -> false;
|
||||
[#temporarily_blocked{timestamp = T} = Entry] ->
|
||||
Diff = p1_time_compat:monotonic_time() - T,
|
||||
case p1_time_compat:convert_time_unit(Diff, native, micro_seconds) of
|
||||
Diff = erlang:monotonic_time() - T,
|
||||
case erlang:convert_time_unit(Diff, native, microsecond) of
|
||||
N when N > (?S2S_OVERLOAD_BLOCK_PERIOD) * 1000 * 1000 ->
|
||||
mnesia:dirty_delete_object(Entry), false;
|
||||
_ -> true
|
||||
|
@ -167,7 +167,7 @@ sql_call(Host, Msg) ->
|
||||
none -> {error, <<"Unknown Host">>};
|
||||
Pid ->
|
||||
sync_send_event(Pid,{sql_cmd, Msg,
|
||||
p1_time_compat:monotonic_time(milli_seconds)},
|
||||
erlang:monotonic_time(millisecond)},
|
||||
query_timeout(Host))
|
||||
end;
|
||||
_State -> nested_op(Msg)
|
||||
@ -176,7 +176,7 @@ sql_call(Host, Msg) ->
|
||||
keep_alive(Host, PID) ->
|
||||
case sync_send_event(PID,
|
||||
{sql_cmd, {sql_query, ?KEEPALIVE_QUERY},
|
||||
p1_time_compat:monotonic_time(milli_seconds)},
|
||||
erlang:monotonic_time(millisecond)},
|
||||
query_timeout(Host)) of
|
||||
{selected,_,[[<<"1">>]]} ->
|
||||
ok;
|
||||
@ -450,7 +450,7 @@ print_state(State) -> State.
|
||||
|
||||
run_sql_cmd(Command, From, State, Timestamp) ->
|
||||
QueryTimeout = query_timeout(State#state.host),
|
||||
case p1_time_compat:monotonic_time(milli_seconds) - Timestamp of
|
||||
case erlang:monotonic_time(millisecond) - Timestamp of
|
||||
Age when Age < QueryTimeout ->
|
||||
put(?NESTING_KEY, ?TOP_LEVEL_TXN),
|
||||
put(?STATE_KEY, State),
|
||||
|
@ -1463,7 +1463,7 @@ user_parse_query1(Action, User, Server, Query) ->
|
||||
end.
|
||||
|
||||
list_last_activity(Host, Lang, Integral, Period) ->
|
||||
TimeStamp = p1_time_compat:system_time(seconds),
|
||||
TimeStamp = erlang:system_time(second),
|
||||
case Period of
|
||||
<<"all">> -> TS = 0, Days = infinity;
|
||||
<<"year">> -> TS = TimeStamp - 366 * 86400, Days = 366;
|
||||
|
@ -103,7 +103,7 @@ normalize_pid(Metadata) ->
|
||||
|
||||
%% Return timestamp with milliseconds
|
||||
timestamp(Time, UTCLog) ->
|
||||
{_, _, Micro} = p1_time_compat:timestamp(),
|
||||
{_, _, Micro} = erlang:timestamp(),
|
||||
{Date, {Hours, Minutes, Seconds}} =
|
||||
case UTCLog of
|
||||
true -> calendar:now_to_universal_time(Time);
|
||||
|
@ -171,7 +171,7 @@ code_change(_OldVsn, State, _Extra) ->
|
||||
%%%===================================================================
|
||||
-spec curr_time() -> non_neg_integer().
|
||||
curr_time() ->
|
||||
p1_time_compat:monotonic_time(milli_seconds).
|
||||
erlang:monotonic_time(millisecond).
|
||||
|
||||
-spec start_port(string()) -> {port(), integer() | undefined}.
|
||||
start_port(Path) ->
|
||||
@ -188,7 +188,7 @@ call_port(Server, Args) ->
|
||||
call_port(Server, Args, ?CALL_TIMEOUT).
|
||||
|
||||
call_port(Server, Args, Timeout) ->
|
||||
StartTime = p1_time_compat:monotonic_time(milli_seconds),
|
||||
StartTime = erlang:monotonic_time(millisecond),
|
||||
Pool = pool_name(Server),
|
||||
PoolSize = pool_size(Server),
|
||||
I = p1_rand:round_robin(PoolSize),
|
||||
|
@ -858,7 +858,7 @@ delete_old_users_vhost(Host, Days) ->
|
||||
|
||||
delete_old_users(Days, Users) ->
|
||||
SecOlder = Days*24*60*60,
|
||||
TimeStamp_now = p1_time_compat:system_time(seconds),
|
||||
TimeStamp_now = erlang:system_time(second),
|
||||
TimeStamp_oldest = TimeStamp_now - SecOlder,
|
||||
F = fun({LUser, LServer}) ->
|
||||
case catch delete_or_not(LUser, LServer, TimeStamp_oldest) of
|
||||
@ -1360,12 +1360,12 @@ get_last(User, Server) ->
|
||||
[] ->
|
||||
case mod_last:get_last_info(User, Server) of
|
||||
not_found ->
|
||||
{p1_time_compat:timestamp(), "NOT FOUND"};
|
||||
{erlang:timestamp(), "NOT FOUND"};
|
||||
{ok, Shift, Status1} ->
|
||||
{{Shift div 1000000, Shift rem 1000000, 0}, Status1}
|
||||
end;
|
||||
_ ->
|
||||
{p1_time_compat:timestamp(), "ONLINE"}
|
||||
{erlang:timestamp(), "ONLINE"}
|
||||
end,
|
||||
{xmpp_util:encode_timestamp(Now), Status}.
|
||||
|
||||
|
@ -35,7 +35,7 @@
|
||||
-include("logger.hrl").
|
||||
|
||||
-record(bosh, {sid = <<"">> :: binary() | '_',
|
||||
timestamp = p1_time_compat:timestamp() :: erlang:timestamp() | '_',
|
||||
timestamp = erlang:timestamp() :: erlang:timestamp() | '_',
|
||||
pid = self() :: pid() | '$1'}).
|
||||
|
||||
-record(state, {}).
|
||||
@ -60,7 +60,7 @@ use_cache() ->
|
||||
false.
|
||||
|
||||
open_session(SID, Pid) ->
|
||||
Session = #bosh{sid = SID, timestamp = p1_time_compat:timestamp(), pid = Pid},
|
||||
Session = #bosh{sid = SID, timestamp = erlang:timestamp(), pid = Pid},
|
||||
lists:foreach(
|
||||
fun(Node) when Node == node() ->
|
||||
gen_server:call(?MODULE, {write, Session});
|
||||
|
@ -379,7 +379,7 @@ queue_new() ->
|
||||
-spec queue_in(csi_key(), stanza(), csi_queue()) -> csi_queue().
|
||||
queue_in(Key, Stanza, {Seq, Q}) ->
|
||||
Seq1 = Seq + 1,
|
||||
Time = {Seq1, p1_time_compat:timestamp()},
|
||||
Time = {Seq1, erlang:timestamp()},
|
||||
Q1 = maps:put(Key, {Time, Stanza}, Q),
|
||||
{Seq1, Q1}.
|
||||
|
||||
|
@ -62,7 +62,7 @@ c2s_auth_result(#{ip := {Addr, _}, lserver := LServer} = State, false, _User) ->
|
||||
LServer, ?MODULE, c2s_auth_ban_lifetime),
|
||||
MaxFailures = gen_mod:get_module_opt(
|
||||
LServer, ?MODULE, c2s_max_auth_failures),
|
||||
UnbanTS = p1_time_compat:system_time(seconds) + BanLifetime,
|
||||
UnbanTS = erlang:system_time(second) + BanLifetime,
|
||||
Attempts = case ets:lookup(failed_auth, Addr) of
|
||||
[{Addr, N, _, _}] ->
|
||||
ets:insert(failed_auth,
|
||||
@ -88,7 +88,7 @@ c2s_auth_result(#{ip := {Addr, _}} = State, true, _User) ->
|
||||
c2s_stream_started(#{ip := {Addr, _}} = State, _) ->
|
||||
case ets:lookup(failed_auth, Addr) of
|
||||
[{Addr, N, TS, MaxFailures}] when N >= MaxFailures ->
|
||||
case TS > p1_time_compat:system_time(seconds) of
|
||||
case TS > erlang:system_time(second) of
|
||||
true ->
|
||||
log_and_disconnect(State, N, TS);
|
||||
false ->
|
||||
@ -143,7 +143,7 @@ handle_cast(_Msg, State) ->
|
||||
|
||||
handle_info(clean, State) ->
|
||||
?DEBUG("cleaning ~p ETS table", [failed_auth]),
|
||||
Now = p1_time_compat:system_time(seconds),
|
||||
Now = erlang:system_time(second),
|
||||
ets:select_delete(
|
||||
failed_auth,
|
||||
ets:fun2ms(fun({_, _, UnbanTS, _}) -> UnbanTS =< Now end)),
|
||||
|
@ -117,7 +117,7 @@ get_node_uptime() ->
|
||||
undefined ->
|
||||
trunc(element(1, erlang:statistics(wall_clock)) / 1000);
|
||||
Now ->
|
||||
p1_time_compat:system_time(seconds) - Now
|
||||
erlang:system_time(second) - Now
|
||||
end.
|
||||
|
||||
%%%
|
||||
@ -209,7 +209,7 @@ get_last_iq(#iq{lang = Lang} = IQ, LUser, LServer) ->
|
||||
Txt = <<"No info about last activity found">>,
|
||||
xmpp:make_error(IQ, xmpp:err_service_unavailable(Txt, Lang));
|
||||
{ok, TimeStamp, Status} ->
|
||||
TimeStamp2 = p1_time_compat:system_time(seconds),
|
||||
TimeStamp2 = erlang:system_time(second),
|
||||
Sec = TimeStamp2 - TimeStamp,
|
||||
xmpp:make_iq_result(IQ, #last{seconds = Sec, status = Status})
|
||||
end;
|
||||
@ -227,7 +227,7 @@ register_user(User, Server) ->
|
||||
|
||||
-spec on_presence_update(binary(), binary(), binary(), binary()) -> any().
|
||||
on_presence_update(User, Server, _Resource, Status) ->
|
||||
TimeStamp = p1_time_compat:system_time(seconds),
|
||||
TimeStamp = erlang:system_time(second),
|
||||
store_last_info(User, Server, TimeStamp, Status).
|
||||
|
||||
-spec store_last_info(binary(), binary(), non_neg_integer(), binary()) -> any().
|
||||
|
@ -440,7 +440,7 @@ muc_filter_message(Acc, _MUCState, _FromNick) ->
|
||||
|
||||
-spec make_id() -> binary().
|
||||
make_id() ->
|
||||
p1_time_compat:system_time(micro_seconds).
|
||||
erlang:system_time(microsecond).
|
||||
|
||||
-spec get_stanza_id(stanza()) -> integer().
|
||||
get_stanza_id(#message{meta = #{stanza_id := ID}}) ->
|
||||
|
@ -138,7 +138,7 @@ send_metrics(Host, Probe, Peer, Port) ->
|
||||
[_, FQDN] = binary:split(misc:atom_to_binary(node()), <<"@">>),
|
||||
[Node|_] = binary:split(FQDN, <<".">>),
|
||||
BaseId = <<Host/binary, "/", Node/binary, ".">>,
|
||||
TS = integer_to_binary(p1_time_compat:system_time(seconds)),
|
||||
TS = integer_to_binary(erlang:system_time(second)),
|
||||
case get_socket(?SOCKET_REGISTER_RETRIES) of
|
||||
{ok, Socket} ->
|
||||
case Probe of
|
||||
|
@ -87,7 +87,7 @@ set_channel(_LServer, Channel, Service, CreatorJID, Hidden, Key) ->
|
||||
creator = jid:remove_resource(CreatorJID),
|
||||
hidden = Hidden,
|
||||
hmac_key = Key,
|
||||
created_at = p1_time_compat:timestamp()}).
|
||||
created_at = erlang:timestamp()}).
|
||||
|
||||
get_channels(_LServer, Service) ->
|
||||
Ret = mnesia:dirty_index_read(mix_channel, Service, #mix_channel.service),
|
||||
@ -127,7 +127,7 @@ set_participant(_LServer, Channel, Service, JID, ID, Nick) ->
|
||||
jid = jid:remove_resource(JID),
|
||||
id = ID,
|
||||
nick = Nick,
|
||||
created_at = p1_time_compat:timestamp()}).
|
||||
created_at = erlang:timestamp()}).
|
||||
|
||||
get_participant(_LServer, Channel, Service, JID) ->
|
||||
{User, Domain, _} = jid:tolower(JID),
|
||||
|
@ -1061,11 +1061,11 @@ topic_alias_maximum(Host) ->
|
||||
%%%===================================================================
|
||||
-spec current_time() -> milli_seconds().
|
||||
current_time() ->
|
||||
p1_time_compat:monotonic_time(milli_seconds).
|
||||
erlang:monotonic_time(millisecond).
|
||||
|
||||
-spec unix_time() -> seconds().
|
||||
unix_time() ->
|
||||
p1_time_compat:system_time(seconds).
|
||||
erlang:system_time(second).
|
||||
|
||||
-spec set_keep_alive(state(), seconds()) -> state().
|
||||
set_keep_alive(State, 0) ->
|
||||
|
@ -584,7 +584,7 @@ process_muc_unique(#iq{type = set, lang = Lang} = IQ) ->
|
||||
xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang));
|
||||
process_muc_unique(#iq{from = From, type = get,
|
||||
sub_els = [#muc_unique{}]} = IQ) ->
|
||||
Name = str:sha(term_to_binary([From, p1_time_compat:timestamp(),
|
||||
Name = str:sha(term_to_binary([From, erlang:timestamp(),
|
||||
p1_rand:get_string()])),
|
||||
xmpp:make_iq_result(IQ, #muc_unique{name = Name}).
|
||||
|
||||
|
@ -819,7 +819,7 @@ decide_room(unused, {_Room_name, _Host, Room_pid}, ServerHost, Last_allowed) ->
|
||||
true when (HistorySize == 0) or (JustCreated == true) ->
|
||||
{false, 0};
|
||||
true ->
|
||||
Ts_diff = (p1_time_compat:os_system_time(micro_seconds)
|
||||
Ts_diff = (erlang:system_time(microsecond)
|
||||
- S#state.just_created) div 1000000,
|
||||
{false, Ts_diff};
|
||||
false ->
|
||||
|
@ -309,7 +309,7 @@ add_message_to_log(Nick1, Message, RoomJID, Opts,
|
||||
Room = get_room_info(RoomJID, Opts),
|
||||
Nick = htmlize(Nick1, FileFormat),
|
||||
Nick2 = htmlize_nick(Nick1, FileFormat),
|
||||
Now = p1_time_compat:timestamp(),
|
||||
Now = erlang:timestamp(),
|
||||
TimeStamp = case Timezone of
|
||||
local -> calendar:now_to_local_time(Now);
|
||||
universal -> calendar:now_to_universal_time(Now)
|
||||
@ -625,7 +625,7 @@ put_header_script(F) ->
|
||||
put_room_config(_F, _RoomConfig, _Lang, plaintext) ->
|
||||
ok;
|
||||
put_room_config(F, RoomConfig, Lang, _FileFormat) ->
|
||||
{_, Now2, _} = p1_time_compat:timestamp(),
|
||||
{_, Now2, _} = erlang:timestamp(),
|
||||
fw(F, <<"<div class=\"rc\">">>),
|
||||
fw(F,
|
||||
<<"<div class=\"rct\" onclick=\"sh('a~p');return "
|
||||
@ -642,7 +642,7 @@ put_room_occupants(_F, _RoomOccupants, _Lang,
|
||||
ok;
|
||||
put_room_occupants(F, RoomOccupants, Lang,
|
||||
_FileFormat) ->
|
||||
{_, Now2, _} = p1_time_compat:timestamp(),
|
||||
{_, Now2, _} = erlang:timestamp(),
|
||||
%% htmlize
|
||||
%% The default behaviour is to ignore the nofollow spam prevention on links
|
||||
%% (NoFollow=false)
|
||||
|
@ -162,7 +162,7 @@ normal_state({route, <<"">>,
|
||||
is_user_allowed_message_nonparticipant(From, StateData) of
|
||||
true when Type == groupchat ->
|
||||
Activity = get_user_activity(From, StateData),
|
||||
Now = p1_time_compat:system_time(micro_seconds),
|
||||
Now = erlang:system_time(microsecond),
|
||||
MinMessageInterval = trunc(gen_mod:get_module_opt(
|
||||
StateData#state.server_host,
|
||||
mod_muc, min_message_interval)
|
||||
@ -344,7 +344,7 @@ normal_state({route, <<"">>, #iq{} = IQ}, StateData) ->
|
||||
end;
|
||||
normal_state({route, Nick, #presence{from = From} = Packet}, StateData) ->
|
||||
Activity = get_user_activity(From, StateData),
|
||||
Now = p1_time_compat:system_time(micro_seconds),
|
||||
Now = erlang:system_time(microsecond),
|
||||
MinPresenceInterval =
|
||||
trunc(gen_mod:get_module_opt(StateData#state.server_host,
|
||||
mod_muc, min_presence_interval)
|
||||
@ -903,7 +903,7 @@ process_voice_request(From, Pkt, StateData) ->
|
||||
true ->
|
||||
MinInterval = (StateData#state.config)#config.voice_request_min_interval,
|
||||
BareFrom = jid:remove_resource(jid:tolower(From)),
|
||||
NowPriority = -p1_time_compat:system_time(micro_seconds),
|
||||
NowPriority = -erlang:system_time(microsecond),
|
||||
CleanPriority = NowPriority + MinInterval * 1000000,
|
||||
Times = clean_treap(StateData#state.last_voice_request_time,
|
||||
CleanPriority),
|
||||
@ -1572,7 +1572,7 @@ store_user_activity(JID, UserActivity, StateData) ->
|
||||
mod_muc, min_presence_interval)
|
||||
* 1000),
|
||||
Key = jid:tolower(JID),
|
||||
Now = p1_time_compat:system_time(micro_seconds),
|
||||
Now = erlang:system_time(microsecond),
|
||||
Activity1 = clean_treap(StateData#state.activity,
|
||||
{1, -Now}),
|
||||
Activity = case treap:lookup(Key, Activity1) of
|
||||
@ -1960,7 +1960,7 @@ add_new_user(From, Nick, Packet, StateData) ->
|
||||
ResultState =
|
||||
case NewStateData#state.just_created of
|
||||
true ->
|
||||
NewStateData#state{just_created = p1_time_compat:os_system_time(micro_seconds)};
|
||||
NewStateData#state{just_created = erlang:system_time(microsecond)};
|
||||
_ ->
|
||||
Robots = maps:remove(From, StateData#state.robots),
|
||||
NewStateData#state{robots = Robots}
|
||||
@ -2103,7 +2103,7 @@ extract_password(#iq{} = IQ) ->
|
||||
get_history(Nick, Packet, #state{history = History}) ->
|
||||
case xmpp:get_subtag(Packet, #muc{}) of
|
||||
#muc{history = #muc_history{} = MUCHistory} ->
|
||||
Now = p1_time_compat:timestamp(),
|
||||
Now = erlang:timestamp(),
|
||||
Q = History#lqueue.queue,
|
||||
filter_history(Q, Now, Nick, MUCHistory);
|
||||
_ ->
|
||||
@ -2518,7 +2518,7 @@ add_message_to_history(FromNick, FromJID, Packet, StateData) ->
|
||||
add_to_log(text, {FromNick, Packet}, StateData),
|
||||
case check_subject(Packet) of
|
||||
[] ->
|
||||
TimeStamp = p1_time_compat:timestamp(),
|
||||
TimeStamp = erlang:timestamp(),
|
||||
AddrPacket = case (StateData#state.config)#config.anonymous of
|
||||
true -> Packet;
|
||||
false ->
|
||||
|
@ -413,7 +413,7 @@ store_packet({_Action, #message{from = From, to = To} = Packet} = Acc) ->
|
||||
drop ->
|
||||
Acc;
|
||||
NewPacket ->
|
||||
TimeStamp = p1_time_compat:timestamp(),
|
||||
TimeStamp = erlang:timestamp(),
|
||||
Expire = find_x_expire(TimeStamp, NewPacket),
|
||||
OffMsg = #offline_msg{us = {LUser, LServer},
|
||||
timestamp = TimeStamp,
|
||||
@ -538,7 +538,7 @@ route_offline_message(#{lserver := LServer} = State,
|
||||
|
||||
-spec is_message_expired(erlang:timestamp() | never, message()) -> boolean().
|
||||
is_message_expired(Expire, Msg) ->
|
||||
TS = p1_time_compat:timestamp(),
|
||||
TS = erlang:timestamp(),
|
||||
Expire1 = case Expire of
|
||||
undefined -> find_x_expire(TS, Msg);
|
||||
_ -> Expire
|
||||
@ -807,7 +807,7 @@ count_offline_messages(User, Server) ->
|
||||
undefined | erlang:timestamp()) -> message().
|
||||
add_delay_info(Packet, LServer, TS) ->
|
||||
NewTS = case TS of
|
||||
undefined -> p1_time_compat:timestamp();
|
||||
undefined -> erlang:timestamp();
|
||||
_ -> TS
|
||||
end,
|
||||
Packet1 = xmpp:put_meta(Packet, from_offline, true),
|
||||
@ -840,7 +840,7 @@ import(LServer, {sql, _}, DBType, <<"spool">>,
|
||||
#delay{stamp = {MegaSecs, Secs, _}} ->
|
||||
{MegaSecs, Secs, 0};
|
||||
false ->
|
||||
p1_time_compat:timestamp()
|
||||
erlang:timestamp()
|
||||
end,
|
||||
US = {LUser, LServer},
|
||||
Expire = find_x_expire(TS, Msg),
|
||||
|
@ -63,7 +63,7 @@ pop_messages(LUser, LServer) ->
|
||||
end.
|
||||
|
||||
remove_expired_messages(_LServer) ->
|
||||
TimeStamp = p1_time_compat:timestamp(),
|
||||
TimeStamp = erlang:timestamp(),
|
||||
F = fun () ->
|
||||
mnesia:write_lock_table(offline_msg),
|
||||
mnesia:foldl(fun (Rec, _Acc) ->
|
||||
@ -81,7 +81,7 @@ remove_expired_messages(_LServer) ->
|
||||
mnesia:transaction(F).
|
||||
|
||||
remove_old_messages(Days, _LServer) ->
|
||||
S = p1_time_compat:system_time(seconds) - 60 * 60 * 24 * Days,
|
||||
S = erlang:system_time(second) - 60 * 60 * 24 * Days,
|
||||
MegaSecs1 = S div 1000000,
|
||||
Secs1 = S rem 1000000,
|
||||
TimeStamp = {MegaSecs1, Secs1, 0},
|
||||
|
@ -80,7 +80,7 @@ check_packet(Acc, _, _, _) ->
|
||||
update(Server, JID, Dir) ->
|
||||
StormCount = gen_mod:get_module_opt(Server, ?MODULE, count),
|
||||
TimeInterval = gen_mod:get_module_opt(Server, ?MODULE, interval),
|
||||
TimeStamp = p1_time_compat:system_time(seconds),
|
||||
TimeStamp = erlang:system_time(second),
|
||||
case read(Dir) of
|
||||
undefined ->
|
||||
write(Dir,
|
||||
|
@ -2571,7 +2571,7 @@ sub_option_can_deliver(nodes, _, {subscription_type, items}) -> false;
|
||||
sub_option_can_deliver(_, _, {subscription_depth, all}) -> true;
|
||||
sub_option_can_deliver(_, Depth, {subscription_depth, D}) -> Depth =< D;
|
||||
sub_option_can_deliver(_, _, {deliver, false}) -> false;
|
||||
sub_option_can_deliver(_, _, {expire, When}) -> p1_time_compat:timestamp() < When;
|
||||
sub_option_can_deliver(_, _, {expire, When}) -> erlang:timestamp() < When;
|
||||
sub_option_can_deliver(_, _, _) -> true.
|
||||
|
||||
-spec presence_can_deliver(ljid(), boolean()) -> boolean().
|
||||
@ -3371,7 +3371,7 @@ set_cached_item({_, ServerHost, _}, Nidx, ItemId, Publisher, Payload) ->
|
||||
set_cached_item(Host, Nidx, ItemId, Publisher, Payload) ->
|
||||
case is_last_item_cache_enabled(Host) of
|
||||
true ->
|
||||
Stamp = {p1_time_compat:timestamp(), jid:tolower(jid:remove_resource(Publisher))},
|
||||
Stamp = {erlang:timestamp(), jid:tolower(jid:remove_resource(Publisher))},
|
||||
Item = #pubsub_last_item{nodeid = {Host, Nidx},
|
||||
itemid = ItemId,
|
||||
creation = Stamp,
|
||||
@ -3748,7 +3748,7 @@ err_unsupported_access_model() ->
|
||||
|
||||
-spec uniqid() -> mod_pubsub:itemId().
|
||||
uniqid() ->
|
||||
{T1, T2, T3} = p1_time_compat:timestamp(),
|
||||
{T1, T2, T3} = erlang:timestamp(),
|
||||
(str:format("~.16B~.16B~.16B", [T1, T2, T3])).
|
||||
|
||||
-spec add_message_type(message(), message_type()) -> message().
|
||||
|
@ -163,7 +163,7 @@ get_commands_spec() ->
|
||||
|
||||
-spec delete_old_sessions(non_neg_integer()) -> ok | any().
|
||||
delete_old_sessions(Days) ->
|
||||
CurrentTime = p1_time_compat:system_time(micro_seconds),
|
||||
CurrentTime = erlang:system_time(microsecond),
|
||||
Diff = Days * 24 * 60 * 60 * 1000000,
|
||||
TimeStamp = misc:usec_to_now(CurrentTime - Diff),
|
||||
DBTypes = lists:usort(
|
||||
|
@ -422,7 +422,7 @@ check_timeout(undefined) -> true;
|
||||
check_timeout(Source) ->
|
||||
Timeout = ejabberd_config:get_option(registration_timeout, 600),
|
||||
if is_integer(Timeout) ->
|
||||
Priority = -p1_time_compat:system_time(seconds),
|
||||
Priority = -erlang:system_time(second),
|
||||
CleanPriority = Priority + Timeout,
|
||||
F = fun () ->
|
||||
Treap = case mnesia:read(mod_register_ip, treap, write)
|
||||
|
@ -251,7 +251,7 @@ write_roster_version_t(LUser, LServer) ->
|
||||
write_roster_version(LUser, LServer, true).
|
||||
|
||||
write_roster_version(LUser, LServer, InTransaction) ->
|
||||
Ver = str:sha(term_to_binary(p1_time_compat:unique_integer())),
|
||||
Ver = str:sha(term_to_binary(erlang:unique_integer())),
|
||||
Mod = gen_mod:db_mod(LServer, ?MODULE),
|
||||
Mod:write_roster_version(LUser, LServer, InTransaction, Ver),
|
||||
if InTransaction -> ok;
|
||||
|
@ -302,7 +302,7 @@ add_record_route_and_set_uri(URI, LServer, #sip{hdrs = Hdrs} = Req) ->
|
||||
case need_record_route(LServer) of
|
||||
true ->
|
||||
RR_URI = get_configured_record_route(LServer),
|
||||
TS = (integer_to_binary(p1_time_compat:system_time(seconds))),
|
||||
TS = (integer_to_binary(erlang:system_time(second))),
|
||||
Sign = make_sign(TS, Hdrs),
|
||||
User = <<TS/binary, $-, Sign/binary>>,
|
||||
NewRR_URI = RR_URI#uri{user = User},
|
||||
@ -339,7 +339,7 @@ is_signed_by_me(TS_Sign, Hdrs) ->
|
||||
try
|
||||
[TSBin, Sign] = str:tokens(TS_Sign, <<"-">>),
|
||||
TS = (binary_to_integer(TSBin)),
|
||||
NowTS = p1_time_compat:system_time(seconds),
|
||||
NowTS = erlang:system_time(second),
|
||||
true = (NowTS - TS) =< ?SIGN_LIFETIME,
|
||||
Sign == make_sign(TSBin, Hdrs)
|
||||
catch _:_ ->
|
||||
|
@ -50,7 +50,7 @@
|
||||
socket = #sip_socket{} :: #sip_socket{},
|
||||
call_id = <<"">> :: binary(),
|
||||
cseq = 0 :: non_neg_integer(),
|
||||
timestamp = p1_time_compat:timestamp() :: erlang:timestamp(),
|
||||
timestamp = erlang:timestamp() :: erlang:timestamp(),
|
||||
contact :: {binary(), #uri{}, [{binary(), binary()}]},
|
||||
flow_tref :: reference() | undefined,
|
||||
reg_tref = make_ref() :: reference(),
|
||||
@ -243,7 +243,7 @@ register_session(US, SIPSocket, CallID, CSeq, IsOutboundSupported,
|
||||
socket = SIPSocket,
|
||||
call_id = CallID,
|
||||
cseq = CSeq,
|
||||
timestamp = p1_time_compat:timestamp(),
|
||||
timestamp = erlang:timestamp(),
|
||||
contact = Contact,
|
||||
expires = Expires}
|
||||
end, ContactsWithExpires),
|
||||
|
@ -523,7 +523,7 @@ mgmt_queue_add(#{mgmt_stanzas_out := NumStanzasOut,
|
||||
4294967295 -> 0;
|
||||
Num -> Num + 1
|
||||
end,
|
||||
Queue1 = p1_queue:in({NewNum, p1_time_compat:timestamp(), Pkt}, Queue),
|
||||
Queue1 = p1_queue:in({NewNum, erlang:timestamp(), Pkt}, Queue),
|
||||
State1 = State#{mgmt_queue => Queue1, mgmt_stanzas_out => NewNum},
|
||||
check_queue_length(State1).
|
||||
|
||||
|
@ -54,7 +54,7 @@ process_local_iq(#iq{type = set, lang = Lang} = IQ) ->
|
||||
Txt = <<"Value 'set' of 'type' attribute is not allowed">>,
|
||||
xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang));
|
||||
process_local_iq(#iq{type = get} = IQ) ->
|
||||
Now = p1_time_compat:timestamp(),
|
||||
Now = erlang:timestamp(),
|
||||
Now_universal = calendar:now_to_universal_time(Now),
|
||||
Now_local = calendar:universal_time_to_local_time(Now_universal),
|
||||
Seconds_diff =
|
||||
|
@ -376,7 +376,7 @@ publish_item(Nidx, Publisher, PublishModel, MaxItems, ItemId, Payload,
|
||||
{error, xmpp:err_forbidden()};
|
||||
true ->
|
||||
if MaxItems > 0 ->
|
||||
Now = p1_time_compat:timestamp(),
|
||||
Now = erlang:timestamp(),
|
||||
case get_item(Nidx, ItemId) of
|
||||
{result, #pubsub_item{creation = {_, GenKey}} = OldItem} ->
|
||||
set_item(OldItem#pubsub_item{
|
||||
|
@ -247,7 +247,7 @@ publish_item(Nidx, Publisher, PublishModel, MaxItems, ItemId, Payload,
|
||||
{error, xmpp:err_forbidden()};
|
||||
true ->
|
||||
if MaxItems > 0 ->
|
||||
Now = p1_time_compat:timestamp(),
|
||||
Now = erlang:timestamp(),
|
||||
case get_item(Nidx, ItemId) of
|
||||
{result, #pubsub_item{creation = {_, GenKey}} = OldItem} ->
|
||||
set_item(OldItem#pubsub_item{
|
||||
|
@ -59,7 +59,7 @@ update_node_database(Host, ServerHost) ->
|
||||
{unknown,
|
||||
Publisher},
|
||||
M =
|
||||
{p1_time_compat:timestamp(),
|
||||
{erlang:timestamp(),
|
||||
Publisher},
|
||||
mnesia:write(#pubsub_item{itemid
|
||||
=
|
||||
|
@ -169,7 +169,7 @@ write_subscription(_JID, _NodeId, SubID, Options) ->
|
||||
|
||||
-spec make_subid() -> SubId::mod_pubsub:subId().
|
||||
make_subid() ->
|
||||
{T1, T2, T3} = p1_time_compat:timestamp(),
|
||||
{T1, T2, T3} = erlang:timestamp(),
|
||||
(str:format("~.16B~.16B~.16B", [T1, T2, T3])).
|
||||
|
||||
%%
|
||||
|
@ -134,7 +134,7 @@ create_table() -> ok.
|
||||
|
||||
-spec make_subid() -> mod_pubsub:subId().
|
||||
make_subid() ->
|
||||
{T1, T2, T3} = p1_time_compat:timestamp(),
|
||||
{T1, T2, T3} = erlang:timestamp(),
|
||||
(str:format("~.16B~.16B~.16B", [T1, T2, T3])).
|
||||
|
||||
%%
|
||||
|
Loading…
Reference in New Issue
Block a user