From a5b7b318278c9a2eac137ab729369558a03b03b6 Mon Sep 17 00:00:00 2001 From: Neil Kakkar Date: Thu, 1 Jul 2021 18:14:40 +0100 Subject: [PATCH] Funnel with Strict Step Ordering (#4890) * wip: pagination for persons on clickhouse funnels * wip: added offset support for getting a list of persons; added support for conversion window; * fixed mypy exception * helper function to insert data for local testing * moved generate code into separate class for more functionality later * corrected person_distinct_id to use the person id from postgres * minor corrections to generate local class along with addition of data cleanup via destroy() method * reduce the number of persons who make it to each step * moved funnel queries to a new folder for better organization; separated funnel_persons and funnel_trends_persons into individual classes; * funnel persons and tests * initial implementation * invoke the funnel or funnel trends class respectively * add a test * add breakdown handling and first test * add test stubs * remove repeats * mypy corrections and PR feedback * run funnel test suite on new query implementation * remove imports * corrected tests * minor test updates * correct func name * fix types * func name change * move builder functions to funnel base * add test classe for new funnel * Handle multiple same events in the funnel (#4863) * dedup + tests * deep equality. Tests to come * write test for entity equality * finish testing funnels * clean up comments * add strict option for funnels * typing * use new persons pattern * persons for funnel strict ordering Co-authored-by: Buddy Williams Co-authored-by: eric --- ee/clickhouse/queries/funnels/base.py | 12 +- .../queries/funnels/funnel_event_query.py | 9 +- .../queries/funnels/funnel_strict.py | 73 +++++ .../queries/funnels/funnel_strict_persons.py | 19 ++ .../funnels/test/test_funnel_persons.py | 2 +- .../funnels/test/test_funnel_strict.py | 266 ++++++++++++++++++ .../test/test_funnel_strict_persons.py | 116 ++++++++ .../test/test_funnel_unordered_persons.py | 4 - 8 files changed, 491 insertions(+), 10 deletions(-) create mode 100644 ee/clickhouse/queries/funnels/funnel_strict.py create mode 100644 ee/clickhouse/queries/funnels/funnel_strict_persons.py create mode 100644 ee/clickhouse/queries/funnels/test/test_funnel_strict.py create mode 100644 ee/clickhouse/queries/funnels/test/test_funnel_strict_persons.py diff --git a/ee/clickhouse/queries/funnels/base.py b/ee/clickhouse/queries/funnels/base.py index dee01706287dc..0f60c0770668a 100644 --- a/ee/clickhouse/queries/funnels/base.py +++ b/ee/clickhouse/queries/funnels/base.py @@ -191,15 +191,21 @@ def _get_sorting_condition(self, curr_index: int, max_steps: int): return f"if({' AND '.join(conditions)}, {curr_index}, {self._get_sorting_condition(curr_index - 1, max_steps)})" - def _get_inner_event_query(self, entities=None, entity_name="events") -> str: + def _get_inner_event_query( + self, entities=None, entity_name="events", skip_entity_filter=False, skip_step_filter=False + ) -> str: entities_to_use = entities or self._filter.entities event_query, params = FunnelEventQuery(filter=self._filter, team_id=self._team.pk).get_query( - entities_to_use, entity_name + entities_to_use, entity_name, skip_entity_filter=skip_entity_filter ) self.params.update(params) - steps_conditions = self._get_steps_conditions(length=len(entities_to_use)) + + if skip_step_filter: + steps_conditions = "1=1" + else: + steps_conditions = self._get_steps_conditions(length=len(self._filter.entities)) all_step_cols: List[str] = [] for index, entity in enumerate(entities_to_use): diff --git a/ee/clickhouse/queries/funnels/funnel_event_query.py b/ee/clickhouse/queries/funnels/funnel_event_query.py index 649a80bc5c7b3..fbb82d45c55f9 100644 --- a/ee/clickhouse/queries/funnels/funnel_event_query.py +++ b/ee/clickhouse/queries/funnels/funnel_event_query.py @@ -6,7 +6,7 @@ class FunnelEventQuery(ClickhouseEventQuery): - def get_query(self, entities=None, entity_name="events") -> Tuple[str, Dict[str, Any]]: + def get_query(self, entities=None, entity_name="events", skip_entity_filter=False) -> Tuple[str, Dict[str, Any]]: _fields = ( f"{self.EVENT_TABLE_ALIAS}.event as event, {self.EVENT_TABLE_ALIAS}.team_id as team_id, {self.EVENT_TABLE_ALIAS}.distinct_id as distinct_id, {self.EVENT_TABLE_ALIAS}.timestamp as timestamp, {self.EVENT_TABLE_ALIAS}.properties as properties, {self.EVENT_TABLE_ALIAS}.elements_chain as elements_chain" + (f", {self.DISTINCT_ID_TABLE_ALIAS}.person_id as person_id" if self._should_join_distinct_ids else "") @@ -20,7 +20,12 @@ def get_query(self, entities=None, entity_name="events") -> Tuple[str, Dict[str, prop_query, prop_params = self._get_props(prop_filters) self.params.update(prop_params) - entity_query, entity_params = self._get_entity_query(entities, entity_name) + if skip_entity_filter: + entity_query = "" + entity_params: Dict[str, Any] = {} + else: + entity_query, entity_params = self._get_entity_query(entities, entity_name) + self.params.update(entity_params) query = f""" diff --git a/ee/clickhouse/queries/funnels/funnel_strict.py b/ee/clickhouse/queries/funnels/funnel_strict.py new file mode 100644 index 0000000000000..bec31e91fd675 --- /dev/null +++ b/ee/clickhouse/queries/funnels/funnel_strict.py @@ -0,0 +1,73 @@ +from typing import List + +from ee.clickhouse.queries.funnels.base import ClickhouseFunnelBase + + +class ClickhouseFunnelStrict(ClickhouseFunnelBase): + def get_query(self, format_properties): + max_steps = len(self._filter.entities) + return f""" + SELECT {self._get_count_columns(max_steps)} {self._get_step_time_avgs(max_steps)} FROM ( + {self.get_step_counts_query()} + ) SETTINGS allow_experimental_window_functions = 1 + """ + + def get_step_counts_query(self): + + max_steps = len(self._filter.entities) + + partition_select = self.get_partition_cols(1, max_steps) + sorting_condition = self._get_sorting_condition(max_steps, max_steps) + + inner_query = f""" + SELECT + person_id, + timestamp, + {partition_select} + FROM ({self._get_inner_event_query(skip_entity_filter=True, skip_step_filter=True)}) + """ + + formatted_query = f""" + SELECT *, {sorting_condition} AS steps {self._get_step_times(max_steps)} FROM ( + {inner_query} + ) WHERE step_0 = 1""" + + return f""" + SELECT person_id, max(steps) AS steps {self._get_step_time_avgs(max_steps)} FROM ( + {formatted_query} + ) GROUP BY person_id + """ + + def get_partition_cols(self, level_index: int, max_steps: int): + cols: List[str] = [] + for i in range(0, max_steps): + cols.append(f"step_{i}") + if i < level_index: + cols.append(f"latest_{i}") + else: + cols.append( + f"min(latest_{i}) over (PARTITION by person_id ORDER BY timestamp DESC ROWS BETWEEN {i} PRECEDING AND {i} PRECEDING) latest_{i}" + ) + return ", ".join(cols) + + # TODO: copied from funnel.py. Once the new funnel query replaces old one, the base format_results function can use this + def _format_results(self, results): + # Format of this is [step order, person count (that reached that step), array of person uuids] + steps = [] + total_people = 0 + + for step in reversed(self._filter.entities): + + if results[0] and len(results[0]) > 0: + total_people += results[0][step.order] + + serialized_result = self._serialize_step(step, total_people, []) + if step.order > 0: + serialized_result.update( + {"average_conversion_time": results[0][step.order + len(self._filter.entities) - 1]} + ) + else: + serialized_result.update({"average_conversion_time": None}) + steps.append(serialized_result) + + return steps[::-1] #  reverse diff --git a/ee/clickhouse/queries/funnels/funnel_strict_persons.py b/ee/clickhouse/queries/funnels/funnel_strict_persons.py new file mode 100644 index 0000000000000..473acec53d278 --- /dev/null +++ b/ee/clickhouse/queries/funnels/funnel_strict_persons.py @@ -0,0 +1,19 @@ +from ee.clickhouse.queries.funnels.funnel_strict import ClickhouseFunnelStrict +from ee.clickhouse.sql.funnels.funnel import FUNNEL_PERSONS_BY_STEP_SQL +from posthog.models import Person + + +class ClickhouseFunnelStrictPersons(ClickhouseFunnelStrict): + def get_query(self, format_properties): + return FUNNEL_PERSONS_BY_STEP_SQL.format( + **format_properties, + steps_per_person_query=self.get_step_counts_query(), + persons_steps=self._get_funnel_person_step_condition() + ) + + def _format_results(self, results): + people = Person.objects.filter(team_id=self._team.pk, uuid__in=[val[0] for val in results]) + + from posthog.api.person import PersonSerializer + + return PersonSerializer(people, many=True).data diff --git a/ee/clickhouse/queries/funnels/test/test_funnel_persons.py b/ee/clickhouse/queries/funnels/test/test_funnel_persons.py index 696d09830f533..194c580153fee 100644 --- a/ee/clickhouse/queries/funnels/test/test_funnel_persons.py +++ b/ee/clickhouse/queries/funnels/test/test_funnel_persons.py @@ -26,7 +26,7 @@ def _create_event(**kwargs): create_event(**kwargs) -class TestFunnel(ClickhouseTestMixin, APIBaseTest): +class TestFunnelPersons(ClickhouseTestMixin, APIBaseTest): def _create_sample_data_multiple_dropoffs(self): for i in range(5): _create_person(distinct_ids=[f"user_{i}"], team=self.team) diff --git a/ee/clickhouse/queries/funnels/test/test_funnel_strict.py b/ee/clickhouse/queries/funnels/test/test_funnel_strict.py new file mode 100644 index 0000000000000..c5b98db08aca8 --- /dev/null +++ b/ee/clickhouse/queries/funnels/test/test_funnel_strict.py @@ -0,0 +1,266 @@ +from datetime import datetime, timedelta +from uuid import uuid4 + +from ee.clickhouse.models.event import create_event +from ee.clickhouse.queries.funnels.funnel_strict import ClickhouseFunnelStrict +from ee.clickhouse.queries.funnels.funnel_strict_persons import ClickhouseFunnelStrictPersons +from ee.clickhouse.util import ClickhouseTestMixin +from posthog.constants import INSIGHT_FUNNELS +from posthog.models.action import Action +from posthog.models.action_step import ActionStep +from posthog.models.filters import Filter +from posthog.models.person import Person +from posthog.test.base import APIBaseTest + +FORMAT_TIME = "%Y-%m-%d 00:00:00" + + +def _create_action(**kwargs): + team = kwargs.pop("team") + name = kwargs.pop("name") + properties = kwargs.pop("properties", {}) + action = Action.objects.create(team=team, name=name) + ActionStep.objects.create(action=action, event=name, properties=properties) + return action + + +def _create_person(**kwargs): + person = Person.objects.create(**kwargs) + return Person(id=person.uuid, uuid=person.uuid) + + +def _create_event(**kwargs): + kwargs.update({"event_uuid": uuid4()}) + create_event(**kwargs) + + +class TestFunnelStrictSteps(ClickhouseTestMixin, APIBaseTest): + def _get_people_at_step(self, filter, funnel_step): + person_filter = filter.with_data({"funnel_step": funnel_step}) + result = ClickhouseFunnelStrictPersons(person_filter, self.team)._exec_query() + return [row[0] for row in result] + + def test_basic_strict_funnel(self): + filter = Filter( + data={ + "insight": INSIGHT_FUNNELS, + "events": [ + {"id": "user signed up", "order": 0}, + {"id": "$pageview", "order": 1}, + {"id": "insight viewed", "order": 2}, + ], + } + ) + + funnel = ClickhouseFunnelStrict(filter, self.team) + + person1_stopped_after_signup = _create_person(distinct_ids=["stopped_after_signup1"], team_id=self.team.pk) + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_signup1") + + person2_stopped_after_one_pageview = _create_person( + distinct_ids=["stopped_after_pageview1"], team_id=self.team.pk + ) + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_pageview1") + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_pageview1") + + person3_stopped_after_insight_view = _create_person( + distinct_ids=["stopped_after_insightview"], team_id=self.team.pk + ) + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_insightview") + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview") + _create_event(team=self.team, event="blaah blaa", distinct_id="stopped_after_insightview") + _create_event(team=self.team, event="insight viewed", distinct_id="stopped_after_insightview") + + person4_stopped_after_insight_view_not_strict_order = _create_person( + distinct_ids=["stopped_after_insightview2"], team_id=self.team.pk + ) + _create_event(team=self.team, event="insight viewed", distinct_id="stopped_after_insightview2") + _create_event(team=self.team, event="blaah blaa", distinct_id="stopped_after_insightview2") + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview2") + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_insightview2") + + person5_stopped_after_insight_view_random = _create_person( + distinct_ids=["stopped_after_insightview3"], team_id=self.team.pk + ) + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview3") + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_insightview3") + _create_event(team=self.team, event="blaah blaa", distinct_id="stopped_after_insightview3") + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview3") + _create_event(team=self.team, event="insight viewed", distinct_id="stopped_after_insightview3") + + person6 = _create_person(distinct_ids=["person6"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person6") + _create_event(team=self.team, event="user signed up", distinct_id="person6") + _create_event(team=self.team, event="blaah blaa", distinct_id="person6") + _create_event(team=self.team, event="$pageview", distinct_id="person6") + + person7 = _create_person(distinct_ids=["person7"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person7") + _create_event(team=self.team, event="user signed up", distinct_id="person7") + _create_event(team=self.team, event="$pageview", distinct_id="person7") + _create_event(team=self.team, event="insight viewed", distinct_id="person7") + _create_event(team=self.team, event="blaah blaa", distinct_id="person7") + + person8_didnot_signup = _create_person(distinct_ids=["stopped_after_insightview6"], team_id=self.team.pk) + _create_event(team=self.team, event="insight viewed", distinct_id="stopped_after_insightview6") + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview6") + + result = funnel.run() + + self.assertEqual(result[0]["name"], "user signed up") + self.assertEqual(result[1]["name"], "$pageview") + self.assertEqual(result[2]["name"], "insight viewed") + self.assertEqual(result[0]["count"], 7) + + self.assertCountEqual( + self._get_people_at_step(filter, 1), + [ + person1_stopped_after_signup.uuid, + person2_stopped_after_one_pageview.uuid, + person3_stopped_after_insight_view.uuid, + person4_stopped_after_insight_view_not_strict_order.uuid, + person5_stopped_after_insight_view_random.uuid, + person6.uuid, + person7.uuid, + ], + ) + + self.assertCountEqual( + self._get_people_at_step(filter, 2), [person3_stopped_after_insight_view.uuid, person7.uuid,], + ) + + self.assertCountEqual( + self._get_people_at_step(filter, 3), [person7.uuid], + ) + + def test_advanced_strict_funnel(self): + + sign_up_action = _create_action( + name="sign up", + team=self.team, + properties=[{"key": "key", "type": "event", "value": ["val"], "operator": "exact"}], + ) + + view_action = _create_action( + name="pageview", + team=self.team, + properties=[{"key": "key", "type": "event", "value": ["val"], "operator": "exact"}], + ) + + filters = { + "events": [ + {"id": "user signed up", "type": "events", "order": 0}, + {"id": "$pageview", "type": "events", "order": 2}, + ], + "actions": [ + {"id": sign_up_action.id, "math": "dau", "order": 1}, + {"id": view_action.id, "math": "wau", "order": 3}, + ], + "insight": INSIGHT_FUNNELS, + } + + filter = Filter(data=filters) + funnel = ClickhouseFunnelStrict(filter, self.team) + + person1_stopped_after_signup = _create_person(distinct_ids=["stopped_after_signup1"], team_id=self.team.pk) + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_signup1") + + person2_stopped_after_one_pageview = _create_person( + distinct_ids=["stopped_after_pageview1"], team_id=self.team.pk + ) + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_pageview1") + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_pageview1") + + person3_stopped_after_insight_view = _create_person( + distinct_ids=["stopped_after_insightview"], team_id=self.team.pk + ) + _create_event(team=self.team, event="user signed up", distinct_id="stopped_after_insightview") + _create_event( + team=self.team, event="sign up", distinct_id="stopped_after_insightview", properties={"key": "val"} + ) + _create_event( + team=self.team, event="sign up", distinct_id="stopped_after_insightview", properties={"key": "val2"} + ) + _create_event(team=self.team, event="$pageview", distinct_id="stopped_after_insightview") + _create_event(team=self.team, event="blaah blaa", distinct_id="stopped_after_insightview") + _create_event(team=self.team, event="insight viewed", distinct_id="stopped_after_insightview") + + person4 = _create_person(distinct_ids=["person4"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person4") + _create_event(team=self.team, event="user signed up", distinct_id="person4") + _create_event(team=self.team, event="sign up", distinct_id="person4", properties={"key": "val"}) + _create_event(team=self.team, event="$pageview", distinct_id="person4", properties={"key": "val"}) + _create_event(team=self.team, event="blaah blaa", distinct_id="person4") + + person5 = _create_person(distinct_ids=["person5"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person5") + _create_event(team=self.team, event="user signed up", distinct_id="person5") + _create_event(team=self.team, event="sign up", distinct_id="person5", properties={"key": "val"}) + _create_event(team=self.team, event="$pageview", distinct_id="person5") + _create_event(team=self.team, event="blaah blaa", distinct_id="person5") + + person6 = _create_person(distinct_ids=["person6"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person6") + _create_event(team=self.team, event="user signed up", distinct_id="person6") + _create_event(team=self.team, event="sign up", distinct_id="person6", properties={"key": "val"}) + _create_event(team=self.team, event="$pageview", distinct_id="person6") + _create_event(team=self.team, event="pageview", distinct_id="person6", properties={"key": "val1"}) + + person7 = _create_person(distinct_ids=["person7"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person7") + _create_event(team=self.team, event="user signed up", distinct_id="person7") + _create_event(team=self.team, event="sign up", distinct_id="person7", properties={"key": "val"}) + _create_event(team=self.team, event="$pageview", distinct_id="person7") + _create_event(team=self.team, event="user signed up", distinct_id="person7") + _create_event(team=self.team, event="pageview", distinct_id="person7", properties={"key": "val"}) + + person8 = _create_person(distinct_ids=["person8"], team_id=self.team.pk) + _create_event(team=self.team, event="blaah blaa", distinct_id="person8") + _create_event(team=self.team, event="user signed up", distinct_id="person8") + _create_event(team=self.team, event="user signed up", distinct_id="person8") + _create_event(team=self.team, event="sign up", distinct_id="person8", properties={"key": "val"}) + _create_event(team=self.team, event="$pageview", distinct_id="person8") + _create_event(team=self.team, event="pageview", distinct_id="person8", properties={"key": "val"}) + + result = funnel.run() + + self.assertEqual(result[0]["name"], "user signed up") + self.assertEqual(result[1]["name"], "sign up") + self.assertEqual(result[2]["name"], "$pageview") + self.assertEqual(result[3]["name"], "pageview") + self.assertEqual(result[0]["count"], 8) + + self.assertCountEqual( + self._get_people_at_step(filter, 1), + [ + person1_stopped_after_signup.uuid, + person2_stopped_after_one_pageview.uuid, + person3_stopped_after_insight_view.uuid, + person4.uuid, + person5.uuid, + person6.uuid, + person7.uuid, + person8.uuid, + ], + ) + + self.assertCountEqual( + self._get_people_at_step(filter, 2), + [ + person3_stopped_after_insight_view.uuid, + person4.uuid, + person5.uuid, + person6.uuid, + person7.uuid, + person8.uuid, + ], + ) + + self.assertCountEqual( + self._get_people_at_step(filter, 3), + [person4.uuid, person5.uuid, person6.uuid, person7.uuid, person8.uuid,], + ) + + self.assertCountEqual( + self._get_people_at_step(filter, 4), [person8.uuid,], + ) diff --git a/ee/clickhouse/queries/funnels/test/test_funnel_strict_persons.py b/ee/clickhouse/queries/funnels/test/test_funnel_strict_persons.py new file mode 100644 index 0000000000000..6813c6a67ab8c --- /dev/null +++ b/ee/clickhouse/queries/funnels/test/test_funnel_strict_persons.py @@ -0,0 +1,116 @@ +from uuid import uuid4 + +from ee.clickhouse.models.event import create_event +from ee.clickhouse.queries.funnels.funnel_strict_persons import ClickhouseFunnelStrictPersons +from ee.clickhouse.util import ClickhouseTestMixin +from posthog.constants import INSIGHT_FUNNELS +from posthog.models.filters import Filter +from posthog.models.person import Person +from posthog.test.base import APIBaseTest + +FORMAT_TIME = "%Y-%m-%d 00:00:00" + + +def _create_person(**kwargs): + person = Person.objects.create(**kwargs) + return Person(id=person.uuid, uuid=person.uuid) + + +def _create_event(**kwargs): + kwargs.update({"event_uuid": uuid4()}) + create_event(**kwargs) + + +class TestFunnelStrictStepsPersons(ClickhouseTestMixin, APIBaseTest): + def _create_sample_data_multiple_dropoffs(self): + for i in range(5): + _create_person(distinct_ids=[f"user_{i}"], team=self.team) + _create_event(event="step one", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-01 00:00:00") + _create_event(event="step fake", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-02 00:00:00") + _create_event(event="step two", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-03 00:00:00") + _create_event(event="step three", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-05 00:00:00") + + for i in range(5, 15): + _create_person(distinct_ids=[f"user_{i}"], team=self.team) + _create_event(event="step one", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-01 00:00:00") + _create_event(event="step two", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-03 00:00:00") + + for i in range(15, 35): + _create_person(distinct_ids=[f"user_{i}"], team=self.team) + _create_event(event="step one", distinct_id=f"user_{i}", team=self.team, timestamp="2021-05-01 00:00:00") + + def test_first_step(self): + self._create_sample_data_multiple_dropoffs() + data = { + "insight": INSIGHT_FUNNELS, + "interval": "day", + "date_from": "2021-05-01 00:00:00", + "date_to": "2021-05-07 00:00:00", + "funnel_window_days": 7, + "funnel_step": 1, + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + {"id": "step three", "order": 2}, + ], + } + filter = Filter(data=data) + results = ClickhouseFunnelStrictPersons(filter, self.team).run() + self.assertEqual(35, len(results)) + + def test_second_step(self): + self._create_sample_data_multiple_dropoffs() + data = { + "insight": INSIGHT_FUNNELS, + "interval": "day", + "date_from": "2021-05-01 00:00:00", + "date_to": "2021-05-07 00:00:00", + "funnel_window_days": 7, + "funnel_step": 2, + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + {"id": "step three", "order": 2}, + ], + } + filter = Filter(data=data) + results = ClickhouseFunnelStrictPersons(filter, self.team).run() + self.assertEqual(10, len(results)) + + def test_second_step_dropoff(self): + self._create_sample_data_multiple_dropoffs() + data = { + "insight": INSIGHT_FUNNELS, + "interval": "day", + "date_from": "2021-05-01 00:00:00", + "date_to": "2021-05-07 00:00:00", + "funnel_window_days": 7, + "funnel_step": -2, + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + {"id": "step three", "order": 2}, + ], + } + filter = Filter(data=data) + results = ClickhouseFunnelStrictPersons(filter, self.team).run() + self.assertEqual(25, len(results)) + + def test_third_step(self): + self._create_sample_data_multiple_dropoffs() + data = { + "insight": INSIGHT_FUNNELS, + "interval": "day", + "date_from": "2021-05-01 00:00:00", + "date_to": "2021-05-07 00:00:00", + "funnel_window_days": 7, + "funnel_step": 3, + "events": [ + {"id": "step one", "order": 0}, + {"id": "step two", "order": 1}, + {"id": "step three", "order": 2}, + ], + } + filter = Filter(data=data) + results = ClickhouseFunnelStrictPersons(filter, self.team).run() + self.assertEqual(0, len(results)) diff --git a/ee/clickhouse/queries/funnels/test/test_funnel_unordered_persons.py b/ee/clickhouse/queries/funnels/test/test_funnel_unordered_persons.py index 441f0ec317117..8f74cc2b8f0dc 100644 --- a/ee/clickhouse/queries/funnels/test/test_funnel_unordered_persons.py +++ b/ee/clickhouse/queries/funnels/test/test_funnel_unordered_persons.py @@ -1,10 +1,6 @@ -from datetime import datetime, timedelta from uuid import uuid4 -import pytest - from ee.clickhouse.models.event import create_event -from ee.clickhouse.queries.funnels.funnel_unordered import ClickhouseFunnelUnordered from ee.clickhouse.queries.funnels.funnel_unordered_persons import ClickhouseFunnelUnorderedPersons from ee.clickhouse.util import ClickhouseTestMixin from posthog.constants import INSIGHT_FUNNELS