You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@superset.apache.org by el...@apache.org on 2023/07/11 00:31:21 UTC

[superset] branch 2.1 updated (6b965978d7 -> 893097b68c)

This is an automated email from the ASF dual-hosted git repository.

elizabeth pushed a change to branch 2.1
in repository https://gitbox.apache.org/repos/asf/superset.git


    from 6b965978d7 chore(metastore-cache): add codec support (#24586)
     new 0426367437 fix(permalink): migrate to marshmallow codec (#24166)
     new f6b5d4e84d fix: dashboard ownership check (#24249)
     new 893097b68c chore(key-value): use json serialization for main resources (#23888)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../src/dashboard/util/permissionUtils.ts          |   2 +-
 superset/dashboards/permalink/api.py               |   6 +-
 superset/dashboards/permalink/commands/base.py     |   9 +-
 superset/dashboards/permalink/commands/create.py   |   3 +
 superset/dashboards/permalink/commands/get.py      |   7 +-
 superset/dashboards/permalink/schemas.py           |  11 +-
 superset/explore/permalink/api.py                  |   6 +-
 superset/explore/permalink/commands/base.py        |   9 +-
 superset/explore/permalink/commands/create.py      |   3 +
 superset/explore/permalink/commands/get.py         |   7 +-
 superset/explore/permalink/schemas.py              |  26 ++++-
 superset/extensions/metastore_cache.py             |   1 +
 superset/key_value/types.py                        |  36 +++++-
 ...2a5681ddfd_convert_key_value_entries_to_json.py |  21 ++++
 .../explore/permalink/api_tests.py                 |  20 +++-
 tests/unit_tests/key_value/codec_test.py           | 122 +++++++++++++++++++++
 16 files changed, 268 insertions(+), 21 deletions(-)
 create mode 100644 tests/unit_tests/key_value/codec_test.py


[superset] 01/03: fix(permalink): migrate to marshmallow codec (#24166)

Posted by el...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

elizabeth pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/superset.git

commit 04263674378e64989eb76d2fdababd2166be7b03
Author: Ville Brofeldt <33...@users.noreply.github.com>
AuthorDate: Mon May 22 13:35:58 2023 +0300

    fix(permalink): migrate to marshmallow codec (#24166)
    
    (cherry picked from commit 71d0543f28711ed70fcfb8086489661596aa2075)
---
 superset/dashboards/permalink/api.py               |   6 +-
 superset/dashboards/permalink/commands/base.py     |   9 +-
 superset/dashboards/permalink/commands/create.py   |   3 +
 superset/dashboards/permalink/commands/get.py      |   7 +-
 superset/dashboards/permalink/schemas.py           |  11 +-
 superset/explore/permalink/api.py                  |   6 +-
 superset/explore/permalink/commands/base.py        |   9 +-
 superset/explore/permalink/commands/create.py      |   3 +
 superset/explore/permalink/commands/get.py         |   7 +-
 superset/explore/permalink/schemas.py              |  26 ++++-
 superset/key_value/types.py                        |  36 +++++-
 .../explore/permalink/api_tests.py                 |  16 ++-
 tests/unit_tests/key_value/codec_test.py           | 122 +++++++++++++++++++++
 13 files changed, 239 insertions(+), 22 deletions(-)

diff --git a/superset/dashboards/permalink/api.py b/superset/dashboards/permalink/api.py
index a8664f0ddd..d9211df2aa 100644
--- a/superset/dashboards/permalink/api.py
+++ b/superset/dashboards/permalink/api.py
@@ -30,7 +30,7 @@ from superset.dashboards.permalink.commands.create import (
 )
 from superset.dashboards.permalink.commands.get import GetDashboardPermalinkCommand
 from superset.dashboards.permalink.exceptions import DashboardPermalinkInvalidStateError
-from superset.dashboards.permalink.schemas import DashboardPermalinkPostSchema
+from superset.dashboards.permalink.schemas import DashboardPermalinkStateSchema
 from superset.extensions import event_logger
 from superset.key_value.exceptions import KeyValueAccessDeniedError
 from superset.views.base_api import BaseSupersetApi, requires_json
@@ -39,13 +39,13 @@ logger = logging.getLogger(__name__)
 
 
 class DashboardPermalinkRestApi(BaseSupersetApi):
-    add_model_schema = DashboardPermalinkPostSchema()
+    add_model_schema = DashboardPermalinkStateSchema()
     method_permission_name = MODEL_API_RW_METHOD_PERMISSION_MAP
     allow_browser_login = True
     class_permission_name = "DashboardPermalinkRestApi"
     resource_name = "dashboard"
     openapi_spec_tag = "Dashboard Permanent Link"
-    openapi_spec_component_schemas = (DashboardPermalinkPostSchema,)
+    openapi_spec_component_schemas = (DashboardPermalinkStateSchema,)
 
     @expose("/<pk>/permalink", methods=["POST"])
     @protect()
diff --git a/superset/dashboards/permalink/commands/base.py b/superset/dashboards/permalink/commands/base.py
index 82e24264ca..4bfb78ea26 100644
--- a/superset/dashboards/permalink/commands/base.py
+++ b/superset/dashboards/permalink/commands/base.py
@@ -17,13 +17,18 @@
 from abc import ABC
 
 from superset.commands.base import BaseCommand
+from superset.dashboards.permalink.schemas import DashboardPermalinkSchema
 from superset.key_value.shared_entries import get_permalink_salt
-from superset.key_value.types import JsonKeyValueCodec, KeyValueResource, SharedKey
+from superset.key_value.types import (
+    KeyValueResource,
+    MarshmallowKeyValueCodec,
+    SharedKey,
+)
 
 
 class BaseDashboardPermalinkCommand(BaseCommand, ABC):
     resource = KeyValueResource.DASHBOARD_PERMALINK
-    codec = JsonKeyValueCodec()
+    codec = MarshmallowKeyValueCodec(DashboardPermalinkSchema())
 
     @property
     def salt(self) -> str:
diff --git a/superset/dashboards/permalink/commands/create.py b/superset/dashboards/permalink/commands/create.py
index 2b6151fbb2..0487041070 100644
--- a/superset/dashboards/permalink/commands/create.py
+++ b/superset/dashboards/permalink/commands/create.py
@@ -23,6 +23,7 @@ from superset.dashboards.permalink.commands.base import BaseDashboardPermalinkCo
 from superset.dashboards.permalink.exceptions import DashboardPermalinkCreateFailedError
 from superset.dashboards.permalink.types import DashboardPermalinkState
 from superset.key_value.commands.upsert import UpsertKeyValueCommand
+from superset.key_value.exceptions import KeyValueCodecEncodeException
 from superset.key_value.utils import encode_permalink_key, get_deterministic_uuid
 from superset.utils.core import get_user_id
 
@@ -62,6 +63,8 @@ class CreateDashboardPermalinkCommand(BaseDashboardPermalinkCommand):
             ).run()
             assert key.id  # for type checks
             return encode_permalink_key(key=key.id, salt=self.salt)
+        except KeyValueCodecEncodeException as ex:
+            raise DashboardPermalinkCreateFailedError(str(ex)) from ex
         except SQLAlchemyError as ex:
             logger.exception("Error running create command")
             raise DashboardPermalinkCreateFailedError() from ex
diff --git a/superset/dashboards/permalink/commands/get.py b/superset/dashboards/permalink/commands/get.py
index 4206263a37..da54ae0b66 100644
--- a/superset/dashboards/permalink/commands/get.py
+++ b/superset/dashboards/permalink/commands/get.py
@@ -25,7 +25,11 @@ from superset.dashboards.permalink.commands.base import BaseDashboardPermalinkCo
 from superset.dashboards.permalink.exceptions import DashboardPermalinkGetFailedError
 from superset.dashboards.permalink.types import DashboardPermalinkValue
 from superset.key_value.commands.get import GetKeyValueCommand
-from superset.key_value.exceptions import KeyValueGetFailedError, KeyValueParseKeyError
+from superset.key_value.exceptions import (
+    KeyValueCodecDecodeException,
+    KeyValueGetFailedError,
+    KeyValueParseKeyError,
+)
 from superset.key_value.utils import decode_permalink_id
 
 logger = logging.getLogger(__name__)
@@ -51,6 +55,7 @@ class GetDashboardPermalinkCommand(BaseDashboardPermalinkCommand):
             return None
         except (
             DashboardNotFoundError,
+            KeyValueCodecDecodeException,
             KeyValueGetFailedError,
             KeyValueParseKeyError,
         ) as ex:
diff --git a/superset/dashboards/permalink/schemas.py b/superset/dashboards/permalink/schemas.py
index ce222d7ed6..acbfec5a17 100644
--- a/superset/dashboards/permalink/schemas.py
+++ b/superset/dashboards/permalink/schemas.py
@@ -17,7 +17,7 @@
 from marshmallow import fields, Schema
 
 
-class DashboardPermalinkPostSchema(Schema):
+class DashboardPermalinkStateSchema(Schema):
     dataMask = fields.Dict(
         required=False,
         allow_none=True,
@@ -48,3 +48,12 @@ class DashboardPermalinkPostSchema(Schema):
         allow_none=True,
         description="Optional anchor link added to url hash",
     )
+
+
+class DashboardPermalinkSchema(Schema):
+    dashboardId = fields.String(
+        required=True,
+        allow_none=False,
+        metadata={"description": "The id or slug of the dasbhoard"},
+    )
+    state = fields.Nested(DashboardPermalinkStateSchema())
diff --git a/superset/explore/permalink/api.py b/superset/explore/permalink/api.py
index 88e819aa2b..2a8ff1998d 100644
--- a/superset/explore/permalink/api.py
+++ b/superset/explore/permalink/api.py
@@ -32,7 +32,7 @@ from superset.datasets.commands.exceptions import (
 from superset.explore.permalink.commands.create import CreateExplorePermalinkCommand
 from superset.explore.permalink.commands.get import GetExplorePermalinkCommand
 from superset.explore.permalink.exceptions import ExplorePermalinkInvalidStateError
-from superset.explore.permalink.schemas import ExplorePermalinkPostSchema
+from superset.explore.permalink.schemas import ExplorePermalinkStateSchema
 from superset.extensions import event_logger
 from superset.key_value.exceptions import KeyValueAccessDeniedError
 from superset.views.base_api import BaseSupersetApi, requires_json, statsd_metrics
@@ -41,13 +41,13 @@ logger = logging.getLogger(__name__)
 
 
 class ExplorePermalinkRestApi(BaseSupersetApi):
-    add_model_schema = ExplorePermalinkPostSchema()
+    add_model_schema = ExplorePermalinkStateSchema()
     method_permission_name = MODEL_API_RW_METHOD_PERMISSION_MAP
     allow_browser_login = True
     class_permission_name = "ExplorePermalinkRestApi"
     resource_name = "explore"
     openapi_spec_tag = "Explore Permanent Link"
-    openapi_spec_component_schemas = (ExplorePermalinkPostSchema,)
+    openapi_spec_component_schemas = (ExplorePermalinkStateSchema,)
 
     @expose("/permalink", methods=["POST"])
     @protect()
diff --git a/superset/explore/permalink/commands/base.py b/superset/explore/permalink/commands/base.py
index a87183b7e9..0b7cfbb8ec 100644
--- a/superset/explore/permalink/commands/base.py
+++ b/superset/explore/permalink/commands/base.py
@@ -17,13 +17,18 @@
 from abc import ABC
 
 from superset.commands.base import BaseCommand
+from superset.explore.permalink.schemas import ExplorePermalinkSchema
 from superset.key_value.shared_entries import get_permalink_salt
-from superset.key_value.types import JsonKeyValueCodec, KeyValueResource, SharedKey
+from superset.key_value.types import (
+    KeyValueResource,
+    MarshmallowKeyValueCodec,
+    SharedKey,
+)
 
 
 class BaseExplorePermalinkCommand(BaseCommand, ABC):
     resource: KeyValueResource = KeyValueResource.EXPLORE_PERMALINK
-    codec = JsonKeyValueCodec()
+    codec = MarshmallowKeyValueCodec(ExplorePermalinkSchema())
 
     @property
     def salt(self) -> str:
diff --git a/superset/explore/permalink/commands/create.py b/superset/explore/permalink/commands/create.py
index 21c0f4e42f..90e64f6df7 100644
--- a/superset/explore/permalink/commands/create.py
+++ b/superset/explore/permalink/commands/create.py
@@ -23,6 +23,7 @@ from superset.explore.permalink.commands.base import BaseExplorePermalinkCommand
 from superset.explore.permalink.exceptions import ExplorePermalinkCreateFailedError
 from superset.explore.utils import check_access as check_chart_access
 from superset.key_value.commands.create import CreateKeyValueCommand
+from superset.key_value.exceptions import KeyValueCodecEncodeException
 from superset.key_value.utils import encode_permalink_key
 from superset.utils.core import DatasourceType
 
@@ -58,6 +59,8 @@ class CreateExplorePermalinkCommand(BaseExplorePermalinkCommand):
             if key.id is None:
                 raise ExplorePermalinkCreateFailedError("Unexpected missing key id")
             return encode_permalink_key(key=key.id, salt=self.salt)
+        except KeyValueCodecEncodeException as ex:
+            raise ExplorePermalinkCreateFailedError(str(ex)) from ex
         except SQLAlchemyError as ex:
             logger.exception("Error running create command")
             raise ExplorePermalinkCreateFailedError() from ex
diff --git a/superset/explore/permalink/commands/get.py b/superset/explore/permalink/commands/get.py
index 4823117ece..1aa093b380 100644
--- a/superset/explore/permalink/commands/get.py
+++ b/superset/explore/permalink/commands/get.py
@@ -25,7 +25,11 @@ from superset.explore.permalink.exceptions import ExplorePermalinkGetFailedError
 from superset.explore.permalink.types import ExplorePermalinkValue
 from superset.explore.utils import check_access as check_chart_access
 from superset.key_value.commands.get import GetKeyValueCommand
-from superset.key_value.exceptions import KeyValueGetFailedError, KeyValueParseKeyError
+from superset.key_value.exceptions import (
+    KeyValueCodecDecodeException,
+    KeyValueGetFailedError,
+    KeyValueParseKeyError,
+)
 from superset.key_value.utils import decode_permalink_id
 from superset.utils.core import DatasourceType
 
@@ -59,6 +63,7 @@ class GetExplorePermalinkCommand(BaseExplorePermalinkCommand):
             return None
         except (
             DatasetNotFoundError,
+            KeyValueCodecDecodeException,
             KeyValueGetFailedError,
             KeyValueParseKeyError,
         ) as ex:
diff --git a/superset/explore/permalink/schemas.py b/superset/explore/permalink/schemas.py
index e1f9d069b8..8b1ae129e8 100644
--- a/superset/explore/permalink/schemas.py
+++ b/superset/explore/permalink/schemas.py
@@ -17,7 +17,7 @@
 from marshmallow import fields, Schema
 
 
-class ExplorePermalinkPostSchema(Schema):
+class ExplorePermalinkStateSchema(Schema):
     formData = fields.Dict(
         required=True,
         allow_none=False,
@@ -37,3 +37,27 @@ class ExplorePermalinkPostSchema(Schema):
         allow_none=True,
         description="URL Parameters",
     )
+
+
+class ExplorePermalinkSchema(Schema):
+    chartId = fields.Integer(
+        required=False,
+        allow_none=True,
+        metadata={"description": "The id of the chart"},
+    )
+    datasourceType = fields.String(
+        required=True,
+        allow_none=False,
+        metadata={"description": "The type of the datasource"},
+    )
+    datasourceId = fields.Integer(
+        required=False,
+        allow_none=True,
+        metadata={"description": "The id of the datasource"},
+    )
+    datasource = fields.String(
+        required=False,
+        allow_none=True,
+        metadata={"description": "The fully qualified datasource reference"},
+    )
+    state = fields.Nested(ExplorePermalinkStateSchema())
diff --git a/superset/key_value/types.py b/superset/key_value/types.py
index 07d06414f6..fb9c31899f 100644
--- a/superset/key_value/types.py
+++ b/superset/key_value/types.py
@@ -24,6 +24,13 @@ from enum import Enum
 from typing import Any, Optional, TypedDict
 from uuid import UUID
 
+from marshmallow import Schema, ValidationError
+
+from superset.key_value.exceptions import (
+    KeyValueCodecDecodeException,
+    KeyValueCodecEncodeException,
+)
+
 
 @dataclass
 class Key:
@@ -61,10 +68,16 @@ class KeyValueCodec(ABC):
 
 class JsonKeyValueCodec(KeyValueCodec):
     def encode(self, value: dict[Any, Any]) -> bytes:
-        return bytes(json.dumps(value), encoding="utf-8")
+        try:
+            return bytes(json.dumps(value), encoding="utf-8")
+        except TypeError as ex:
+            raise KeyValueCodecEncodeException(str(ex)) from ex
 
     def decode(self, value: bytes) -> dict[Any, Any]:
-        return json.loads(value)
+        try:
+            return json.loads(value)
+        except TypeError as ex:
+            raise KeyValueCodecDecodeException(str(ex)) from ex
 
 
 class PickleKeyValueCodec(KeyValueCodec):
@@ -73,3 +86,22 @@ class PickleKeyValueCodec(KeyValueCodec):
 
     def decode(self, value: bytes) -> dict[Any, Any]:
         return pickle.loads(value)
+
+
+class MarshmallowKeyValueCodec(JsonKeyValueCodec):
+    def __init__(self, schema: Schema):
+        self.schema = schema
+
+    def encode(self, value: dict[Any, Any]) -> bytes:
+        try:
+            obj = self.schema.dump(value)
+            return super().encode(obj)
+        except ValidationError as ex:
+            raise KeyValueCodecEncodeException(message=str(ex)) from ex
+
+    def decode(self, value: bytes) -> dict[Any, Any]:
+        try:
+            obj = super().decode(value)
+            return self.schema.load(obj)
+        except ValidationError as ex:
+            raise KeyValueCodecEncodeException(message=str(ex)) from ex
diff --git a/tests/integration_tests/explore/permalink/api_tests.py b/tests/integration_tests/explore/permalink/api_tests.py
index 4c6a3c12dd..3a07bd977a 100644
--- a/tests/integration_tests/explore/permalink/api_tests.py
+++ b/tests/integration_tests/explore/permalink/api_tests.py
@@ -22,8 +22,9 @@ import pytest
 from sqlalchemy.orm import Session
 
 from superset import db
+from superset.explore.permalink.schemas import ExplorePermalinkSchema
 from superset.key_value.models import KeyValueEntry
-from superset.key_value.types import JsonKeyValueCodec, KeyValueResource
+from superset.key_value.types import KeyValueResource, MarshmallowKeyValueCodec
 from superset.key_value.utils import decode_permalink_id, encode_permalink_key
 from superset.models.slice import Slice
 from superset.utils.core import DatasourceType
@@ -94,14 +95,17 @@ def test_get_missing_chart(
     chart_id = 1234
     entry = KeyValueEntry(
         resource=KeyValueResource.EXPLORE_PERMALINK,
-        value=JsonKeyValueCodec().encode(
+        value=MarshmallowKeyValueCodec(ExplorePermalinkSchema()).encode(
             {
                 "chartId": chart_id,
                 "datasourceId": chart.datasource.id,
-                "datasourceType": DatasourceType.TABLE,
-                "formData": {
-                    "slice_id": chart_id,
-                    "datasource": f"{chart.datasource.id}__{chart.datasource.type}",
+                "datasourceType": DatasourceType.TABLE.value,
+                "state": {
+                    "urlParams": [["foo", "bar"]],
+                    "formData": {
+                        "slice_id": chart_id,
+                        "datasource": f"{chart.datasource.id}__{chart.datasource.type}",
+                    },
                 },
             }
         ),
diff --git a/tests/unit_tests/key_value/codec_test.py b/tests/unit_tests/key_value/codec_test.py
new file mode 100644
index 0000000000..1442a3a95a
--- /dev/null
+++ b/tests/unit_tests/key_value/codec_test.py
@@ -0,0 +1,122 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from contextlib import nullcontext
+from typing import Any
+
+import pytest
+from marshmallow import Schema
+
+from superset.dashboards.permalink.schemas import DashboardPermalinkSchema
+from superset.key_value.exceptions import KeyValueCodecEncodeException
+from superset.key_value.types import (
+    JsonKeyValueCodec,
+    MarshmallowKeyValueCodec,
+    PickleKeyValueCodec,
+)
+
+
+@pytest.mark.parametrize(
+    "input_,expected_result",
+    [
+        (
+            {"foo": "bar"},
+            {"foo": "bar"},
+        ),
+        (
+            {"foo": (1, 2, 3)},
+            {"foo": [1, 2, 3]},
+        ),
+        (
+            {1, 2, 3},
+            KeyValueCodecEncodeException(),
+        ),
+        (
+            object(),
+            KeyValueCodecEncodeException(),
+        ),
+    ],
+)
+def test_json_codec(input_: Any, expected_result: Any):
+    cm = (
+        pytest.raises(type(expected_result))
+        if isinstance(expected_result, Exception)
+        else nullcontext()
+    )
+    with cm:
+        codec = JsonKeyValueCodec()
+        encoded_value = codec.encode(input_)
+        assert expected_result == codec.decode(encoded_value)
+
+
+@pytest.mark.parametrize(
+    "schema,input_,expected_result",
+    [
+        (
+            DashboardPermalinkSchema(),
+            {
+                "dashboardId": "1",
+                "state": {
+                    "urlParams": [["foo", "bar"], ["foo", "baz"]],
+                },
+            },
+            {
+                "dashboardId": "1",
+                "state": {
+                    "urlParams": [("foo", "bar"), ("foo", "baz")],
+                },
+            },
+        ),
+        (
+            DashboardPermalinkSchema(),
+            {"foo": "bar"},
+            KeyValueCodecEncodeException(),
+        ),
+    ],
+)
+def test_marshmallow_codec(schema: Schema, input_: Any, expected_result: Any):
+    cm = (
+        pytest.raises(type(expected_result))
+        if isinstance(expected_result, Exception)
+        else nullcontext()
+    )
+    with cm:
+        codec = MarshmallowKeyValueCodec(schema)
+        encoded_value = codec.encode(input_)
+        assert expected_result == codec.decode(encoded_value)
+
+
+@pytest.mark.parametrize(
+    "input_,expected_result",
+    [
+        (
+            {1, 2, 3},
+            {1, 2, 3},
+        ),
+        (
+            {"foo": 1, "bar": {1: (1, 2, 3)}, "baz": {1, 2, 3}},
+            {
+                "foo": 1,
+                "bar": {1: (1, 2, 3)},
+                "baz": {1, 2, 3},
+            },
+        ),
+    ],
+)
+def test_pickle_codec(input_: Any, expected_result: Any):
+    codec = PickleKeyValueCodec()
+    encoded_value = codec.encode(input_)
+    assert expected_result == codec.decode(encoded_value)


[superset] 03/03: chore(key-value): use json serialization for main resources (#23888)

Posted by el...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

elizabeth pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/superset.git

commit 893097b68cb9b6dfd10399aeb6234d3f4e266a66
Author: Ville Brofeldt <33...@users.noreply.github.com>
AuthorDate: Thu May 4 08:04:05 2023 +0300

    chore(key-value): use json serialization for main resources (#23888)
---
 superset/extensions/metastore_cache.py              |  1 +
 ...c2a5681ddfd_convert_key_value_entries_to_json.py | 21 +++++++++++++++++++++
 .../explore/permalink/api_tests.py                  |  8 ++++++++
 3 files changed, 30 insertions(+)

diff --git a/superset/extensions/metastore_cache.py b/superset/extensions/metastore_cache.py
index 0dbd42b2b4..4e6f0067dd 100644
--- a/superset/extensions/metastore_cache.py
+++ b/superset/extensions/metastore_cache.py
@@ -31,6 +31,7 @@ from superset.key_value.types import (
 from superset.key_value.utils import get_uuid_namespace
 
 RESOURCE = KeyValueResource.METASTORE_CACHE
+CODEC = PickleKeyValueCodec()
 
 logger = logging.getLogger(__name__)
 
diff --git a/superset/migrations/versions/2023-05-01_12-03_9c2a5681ddfd_convert_key_value_entries_to_json.py b/superset/migrations/versions/2023-05-01_12-03_9c2a5681ddfd_convert_key_value_entries_to_json.py
index 6e55f3ddc9..ea285d0808 100644
--- a/superset/migrations/versions/2023-05-01_12-03_9c2a5681ddfd_convert_key_value_entries_to_json.py
+++ b/superset/migrations/versions/2023-05-01_12-03_9c2a5681ddfd_convert_key_value_entries_to_json.py
@@ -17,14 +17,22 @@
 """convert key-value entries to json
 
 Revision ID: 9c2a5681ddfd
+<<<<<<< HEAD
 Revises: f3c2d8ec8595
+=======
+Revises: 7e67aecbf3f1
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
 Create Date: 2023-05-01 12:03:17.079862
 
 """
 
 # revision identifiers, used by Alembic.
 revision = "9c2a5681ddfd"
+<<<<<<< HEAD
 down_revision = "f3c2d8ec8595"
+=======
+down_revision = "7e67aecbf3f1"
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
 
 import io
 import json
@@ -45,10 +53,14 @@ RESOURCES_TO_MIGRATE = ("app", "dashboard_permalink", "explore_permalink")
 
 class RestrictedUnpickler(pickle.Unpickler):
     def find_class(self, module, name):
+<<<<<<< HEAD
         if not (module == "superset.utils.core" and name == "DatasourceType"):
             raise pickle.UnpicklingError(f"Unpickling of {module}.{name} is forbidden")
 
         return super().find_class(module, name)
+=======
+        raise pickle.UnpicklingError(f"Unpickling of {module}.{name} is forbidden")
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
 
 
 class KeyValueEntry(Base):
@@ -61,12 +73,16 @@ class KeyValueEntry(Base):
 def upgrade():
     bind = op.get_bind()
     session: Session = db.Session(bind=bind)
+<<<<<<< HEAD
     truncated_count = 0
+=======
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
     for entry in paginated_update(
         session.query(KeyValueEntry).filter(
             KeyValueEntry.resource.in_(RESOURCES_TO_MIGRATE)
         )
     ):
+<<<<<<< HEAD
         try:
             value = RestrictedUnpickler(io.BytesIO(entry.value)).load() or {}
         except pickle.UnpicklingError as ex:
@@ -83,6 +99,11 @@ def upgrade():
     if truncated_count:
         print(f"Replaced {truncated_count} corrupted values with an empty value")
 
+=======
+        value = RestrictedUnpickler(io.BytesIO(entry.value)).load() or {}
+        entry.value = bytes(json.dumps(value), encoding="utf-8")
+
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
 
 def downgrade():
     bind = op.get_bind()
diff --git a/tests/integration_tests/explore/permalink/api_tests.py b/tests/integration_tests/explore/permalink/api_tests.py
index 3a07bd977a..c7b631239e 100644
--- a/tests/integration_tests/explore/permalink/api_tests.py
+++ b/tests/integration_tests/explore/permalink/api_tests.py
@@ -24,7 +24,11 @@ from sqlalchemy.orm import Session
 from superset import db
 from superset.explore.permalink.schemas import ExplorePermalinkSchema
 from superset.key_value.models import KeyValueEntry
+<<<<<<< HEAD
 from superset.key_value.types import KeyValueResource, MarshmallowKeyValueCodec
+=======
+from superset.key_value.types import JsonKeyValueCodec, KeyValueResource
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
 from superset.key_value.utils import decode_permalink_id, encode_permalink_key
 from superset.models.slice import Slice
 from superset.utils.core import DatasourceType
@@ -95,7 +99,11 @@ def test_get_missing_chart(
     chart_id = 1234
     entry = KeyValueEntry(
         resource=KeyValueResource.EXPLORE_PERMALINK,
+<<<<<<< HEAD
         value=MarshmallowKeyValueCodec(ExplorePermalinkSchema()).encode(
+=======
+        value=JsonKeyValueCodec().encode(
+>>>>>>> f1fa1a733... chore(key-value): use json serialization for main resources (#23888)
             {
                 "chartId": chart_id,
                 "datasourceId": chart.datasource.id,


[superset] 02/03: fix: dashboard ownership check (#24249)

Posted by el...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

elizabeth pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/superset.git

commit f6b5d4e84d66303f4e2fcf9092e750a1a148eac3
Author: Beto Dealmeida <ro...@dealmeida.net>
AuthorDate: Tue May 30 12:18:20 2023 -0700

    fix: dashboard ownership check (#24249)
    
    (cherry picked from commit 7e70f19b6829eb7226cb53da9eeb1aff83b18dbe)
---
 superset-frontend/src/dashboard/util/permissionUtils.ts | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/superset-frontend/src/dashboard/util/permissionUtils.ts b/superset-frontend/src/dashboard/util/permissionUtils.ts
index 3ea63976bf..6a218349b2 100644
--- a/superset-frontend/src/dashboard/util/permissionUtils.ts
+++ b/superset-frontend/src/dashboard/util/permissionUtils.ts
@@ -42,7 +42,7 @@ const isUserDashboardOwner = (
   user: UserWithPermissionsAndRoles | UndefinedUser,
 ) =>
   isUserWithPermissionsAndRoles(user) &&
-  dashboard.owners.some(owner => owner.username === user.username);
+  dashboard.owners.some(owner => owner.id === user.userId);
 
 export const canUserEditDashboard = (
   dashboard: Dashboard,