You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@superset.apache.org by GitBox <gi...@apache.org> on 2022/05/11 16:37:59 UTC

[GitHub] [superset] hughhhh opened a new pull request, #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

hughhhh opened a new pull request, #20030:
URL: https://github.com/apache/superset/pull/20030

   <!---
   Please write the PR title following the conventions at https://www.conventionalcommits.org/en/v1.0.0/
   Example:
   fix(dashboard): load charts correctly
   -->
   
   ### SUMMARY
   <!--- Describe the change below, including rationale and design decisions -->
   Building a data access object for Datasource to make it easier to query a source based upon id and type. With sip-68 work we'll be allowing users to power charts with different data object such as (queries, savedqueries, sl_datasets, and sl_tables) this work will make it easier to do quick look up of these objects with out having to write and ORM query.
   
   ### BEFORE/AFTER SCREENSHOTS OR ANIMATED GIF
   <!--- Skip this if not applicable -->
   
   ### TESTING INSTRUCTIONS
   <!--- Required! What steps can be taken to manually verify the changes? -->
   
   ### ADDITIONAL INFORMATION
   <!--- Check any relevant boxes with "x" -->
   <!--- HINT: Include "Fixes #nnn" if you are fixing an existing issue -->
   - [ ] Has associated issue:
   - [ ] Required feature flags:
   - [ ] Changes UI
   - [ ] Includes DB Migration (follow approval process in [SIP-59](https://github.com/apache/superset/issues/13351))
     - [ ] Migration is atomic, supports rollback & is backwards-compatible
     - [ ] Confirm DB migration upgrade and downgrade tested
     - [ ] Runtime estimates and downtime expectations provided
   - [ ] Introduces new feature or API
   - [ ] Removes existing feature or API
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870848216


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .options(
+                    subqueryload(datasource_class.columns),
+                    subqueryload(datasource_class.metrics),
+                )
+                .filter_by(id=datasource_id)
+                .one()
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_name(
+        cls,
+        session: Session,
+        database: Database,
+        datasource_name: str,
+        schema: Optional[str] = None,
+    ) -> List[Datasource]:
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):

Review Comment:
   why only sqlatable here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] hughhhh commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
hughhhh commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870860659


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   Yea i'll move the `Datasource` reference next to the enum



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870847446


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .options(
+                    subqueryload(datasource_class.columns),
+                    subqueryload(datasource_class.metrics),

Review Comment:
   we weren't planning to add a metrics column for queries or saved queries, but have this as a calculated column. I'm not sure if this will work here. You may have to confine this lookup to whether the class has a metrics column or not. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] betodealmeida commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
betodealmeida commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870826933


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test

Review Comment:
   Don't leave TODOs for other people! :)
   ```suggestion
           # TODO(hughhhh): add unit test
   ```
   
   Or remove it altogether.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] pkdotson commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
pkdotson commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870549624


##########
tests/unit_tests/dao/datasource_test.py:
##########
@@ -0,0 +1,116 @@
+# 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.
+
+import pytest
+from sqlalchemy.orm.session import Session
+
+
+def create_test_data(session: Session) -> None:
+    from superset.columns.models import Column
+    from superset.connectors.sqla.models import SqlaTable, TableColumn
+    from superset.models.core import Database
+    from superset.models.sql_lab import Query, SavedQuery
+
+    engine = session.get_bind()
+    SqlaTable.metadata.create_all(engine)  # pylint: disable=no-member
+
+    db = Database(database_name="my_database", sqlalchemy_uri="sqlite://")
+
+    columns = [
+        TableColumn(column_name="a", type="INTEGER"),
+    ]
+
+    sqla_table = SqlaTable(
+        table_name="my_sqla_table",
+        columns=columns,
+        metrics=[],
+        database=db,
+    )
+
+    query_obj = Query(
+        client_id="foo",
+        database=db,
+        tab_name="test_tab",
+        sql_editor_id="test_editor_id",
+        sql="select * from bar",
+        select_sql="select * from bar",
+        executed_sql="select * from bar",
+        limit=100,
+        select_as_cta=False,
+        rows=100,
+        error_message="none",
+        results_key="abc",
+    )
+
+    saved_query = SavedQuery(database=db, sql="select * from foo")
+
+    session.add(saved_query)
+    session.add(query_obj)
+    session.add(db)
+    session.add(sqla_table)
+    session.flush()
+
+
+def test_get_datasource_sqlatable(app_context: None, session: Session) -> None:
+    from superset.connectors.sqla.models import SqlaTable
+    from superset.dao.datasource.dao import DatasourceDAO
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="table", datasource_id=1, session=session
+    )
+
+    assert 1 == result.id
+    assert "my_sqla_table" == result.table_name
+    assert isinstance(result, SqlaTable)
+
+
+def test_get_datasource_query(app_context: None, session: Session) -> None:
+    from superset.dao.datasource.dao import DatasourceDAO
+    from superset.models.sql_lab import Query
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="query", datasource_id=1, session=session
+    )
+
+    assert result.id == 1
+    assert isinstance(result, Query)
+
+
+def test_get_datasource_saved_query(app_context: None, session: Session) -> None:
+    from superset.dao.datasource.dao import DatasourceDAO
+    from superset.models.sql_lab import SavedQuery
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="saved_query", datasource_id=1, session=session
+    )
+
+    assert result.id == 1
+    assert isinstance(result, SavedQuery)
+
+
+def test_get_datasource_sl_table(app_context: None, session: Session) -> None:
+    pass

Review Comment:
   Is there a reason these are just pass?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] betodealmeida commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
betodealmeida commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r871821798


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   Yeah, if you need to add `Any` to make `mypy` happy it's because something is wrong. Adding `Any` never fixes anything, it only makes the type checker happy. In this case it defeats the purpose of type checking, because all objects are of type `Any`.
   
   Was this fixed by using `Type`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] betodealmeida commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
betodealmeida commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r871821798


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   Yeah, if you need to add `Any` to make `mypy` pass it's because something is wrong. Adding `Any` never fixes anything, it only makes the type checker happy. In this case it defeats the purpose of type checking, because all objects are of type `Any`.
   
   Was this fixed by using `Type`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870844630


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   do you think we should keep the `Datasource` type and `DatasourceType` enum in the same file so that if one changes, we can be sure to change the other?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] codecov[bot] commented on pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #20030:
URL: https://github.com/apache/superset/pull/20030#issuecomment-1124008568

   # [Codecov](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#20030](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1dc9c74) into [master](https://codecov.io/gh/apache/superset/commit/3a379af1e7f67c9d420725a29cdfd6c5e983606b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3a379af) will **decrease** coverage by `11.96%`.
   > The diff coverage is `54.16%`.
   
   ```diff
   @@             Coverage Diff             @@
   ##           master   #20030       +/-   ##
   ===========================================
   - Coverage   66.33%   54.36%   -11.97%     
   ===========================================
     Files        1713     1714        +1     
     Lines       64083    64131       +48     
     Branches     6734     6734               
   ===========================================
   - Hits        42509    34868     -7641     
   - Misses      19863    27552     +7689     
     Partials     1711     1711               
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hive | `53.66% <ø> (ø)` | |
   | mysql | `?` | |
   | postgres | `?` | |
   | presto | `53.51% <ø> (ø)` | |
   | python | `57.69% <54.16%> (-24.84%)` | :arrow_down: |
   | sqlite | `?` | |
   | unit | `48.72% <54.16%> (+0.06%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [superset/dao/datasource/dao.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvZGFvL2RhdGFzb3VyY2UvZGFvLnB5) | `54.16% <54.16%> (ø)` | |
   | [superset/utils/dashboard\_import\_export.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvdXRpbHMvZGFzaGJvYXJkX2ltcG9ydF9leHBvcnQucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [superset/key\_value/commands/upsert.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQva2V5X3ZhbHVlL2NvbW1hbmRzL3Vwc2VydC5weQ==) | `0.00% <0.00%> (-89.59%)` | :arrow_down: |
   | [superset/key\_value/commands/update.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQva2V5X3ZhbHVlL2NvbW1hbmRzL3VwZGF0ZS5weQ==) | `0.00% <0.00%> (-89.37%)` | :arrow_down: |
   | [superset/key\_value/commands/delete.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQva2V5X3ZhbHVlL2NvbW1hbmRzL2RlbGV0ZS5weQ==) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [superset/key\_value/commands/delete\_expired.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQva2V5X3ZhbHVlL2NvbW1hbmRzL2RlbGV0ZV9leHBpcmVkLnB5) | `0.00% <0.00%> (-80.77%)` | :arrow_down: |
   | [superset/dashboards/commands/importers/v0.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvZGFzaGJvYXJkcy9jb21tYW5kcy9pbXBvcnRlcnMvdjAucHk=) | `15.62% <0.00%> (-76.25%)` | :arrow_down: |
   | [superset/databases/commands/test\_connection.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvZGF0YWJhc2VzL2NvbW1hbmRzL3Rlc3RfY29ubmVjdGlvbi5weQ==) | `31.42% <0.00%> (-68.58%)` | :arrow_down: |
   | [superset/datasets/commands/update.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvZGF0YXNldHMvY29tbWFuZHMvdXBkYXRlLnB5) | `25.88% <0.00%> (-68.24%)` | :arrow_down: |
   | [superset/datasets/commands/create.py](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3VwZXJzZXQvZGF0YXNldHMvY29tbWFuZHMvY3JlYXRlLnB5) | `30.18% <0.00%> (-67.93%)` | :arrow_down: |
   | ... and [268 more](https://codecov.io/gh/apache/superset/pull/20030/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3a379af...1dc9c74](https://codecov.io/gh/apache/superset/pull/20030?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870850418


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources

Review Comment:
   I think we can just limit this to just SqlaTable since no charts or dashboards will be able to be saved with any other datasources



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870850877


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()

Review Comment:
   same thing here re what Beto said above. This could be a missing Table or Query. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870846544


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:

Review Comment:
   I feel like this should just return `datasets` since for now this is the only datasource that can be attached to a saved chart. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870847446


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .options(
+                    subqueryload(datasource_class.columns),
+                    subqueryload(datasource_class.metrics),

Review Comment:
   we weren't planning to add a metrics column for queries or saved queries, but have this as a calculated column. I'm not sure if this will work here. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] betodealmeida commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
betodealmeida commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870826245


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session

Review Comment:
   Small nit, but it's nice to standardize: in your other methods the `session` is the first argument, in this one it's the last. I'd move it to first argument here, for consistency.



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,

Review Comment:
   A similar small nit here: in general `schema` comes after `database` in our functions/methods (since they are hierarchical), I would swap the order for consistency.



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   ```suggestion
   Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery]
   ```
   
   We don't want `Datasource` to be of type `Any` — on the contrary, it's really nice that the type clearly defines what constitutes a datasource.



##########
tests/unit_tests/dao/datasource_test.py:
##########
@@ -0,0 +1,181 @@
+# 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.
+
+import pytest
+from sqlalchemy.orm.session import Session
+
+from superset.utils.core import DatasourceType
+
+
+def create_test_data(session: Session) -> None:

Review Comment:
   You can also rewrite this as a fixture:
   
   ```python
   @pytest.fixture
   def session_with_data(session: Session) -> Iterator[Session]:
       ...  # code of `create_test_data` here
       yield session
   ```
   
   And then in your tests:
   
   ```python
   def test_get_datasource_sqlatable(app_context: None, session_with_data: Session) -> None:
       ...
   ```
   
   But there's no need, a function like this also works fine.



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()

Review Comment:
   We should create a custom exception here (`DatasourceTypeNotSupportedError`, for example).



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .options(
+                    subqueryload(datasource_class.columns),
+                    subqueryload(datasource_class.metrics),
+                )
+                .filter_by(id=datasource_id)
+                .one()
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_name(
+        cls,
+        session: Session,
+        database: Database,
+        datasource_name: str,
+        schema: Optional[str] = None,
+    ) -> List[Datasource]:
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):

Review Comment:
   Ditto here.



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):

Review Comment:
   Nit, if you invert the if you can remove an indentation level:
   
   ```python
   if not isinstance(datasource_class, SqlaTable):
       return []
   
   return (
       session.query(...)
       ...
   )
   ```



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test

Review Comment:
   Don't leavel TODOs for other people! :)
   ```suggestion
           # TODO(hughhhh): add unit test
   ```
   
   Or remove it altogether.



##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):

Review Comment:
   Same here, if you invert the if the code is a bit easier to read:
   
   ```python
   if not isinstance(datasource_class, SqlaTable):
       return None
   
   return (...)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] hughhhh merged pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
hughhhh merged PR #20030:
URL: https://github.com/apache/superset/pull/20030


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] hughhhh commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
hughhhh commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870581775


##########
tests/unit_tests/dao/datasource_test.py:
##########
@@ -0,0 +1,116 @@
+# 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.
+
+import pytest
+from sqlalchemy.orm.session import Session
+
+
+def create_test_data(session: Session) -> None:
+    from superset.columns.models import Column
+    from superset.connectors.sqla.models import SqlaTable, TableColumn
+    from superset.models.core import Database
+    from superset.models.sql_lab import Query, SavedQuery
+
+    engine = session.get_bind()
+    SqlaTable.metadata.create_all(engine)  # pylint: disable=no-member
+
+    db = Database(database_name="my_database", sqlalchemy_uri="sqlite://")
+
+    columns = [
+        TableColumn(column_name="a", type="INTEGER"),
+    ]
+
+    sqla_table = SqlaTable(
+        table_name="my_sqla_table",
+        columns=columns,
+        metrics=[],
+        database=db,
+    )
+
+    query_obj = Query(
+        client_id="foo",
+        database=db,
+        tab_name="test_tab",
+        sql_editor_id="test_editor_id",
+        sql="select * from bar",
+        select_sql="select * from bar",
+        executed_sql="select * from bar",
+        limit=100,
+        select_as_cta=False,
+        rows=100,
+        error_message="none",
+        results_key="abc",
+    )
+
+    saved_query = SavedQuery(database=db, sql="select * from foo")
+
+    session.add(saved_query)
+    session.add(query_obj)
+    session.add(db)
+    session.add(sqla_table)
+    session.flush()
+
+
+def test_get_datasource_sqlatable(app_context: None, session: Session) -> None:
+    from superset.connectors.sqla.models import SqlaTable
+    from superset.dao.datasource.dao import DatasourceDAO
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="table", datasource_id=1, session=session
+    )
+
+    assert 1 == result.id
+    assert "my_sqla_table" == result.table_name
+    assert isinstance(result, SqlaTable)
+
+
+def test_get_datasource_query(app_context: None, session: Session) -> None:
+    from superset.dao.datasource.dao import DatasourceDAO
+    from superset.models.sql_lab import Query
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="query", datasource_id=1, session=session
+    )
+
+    assert result.id == 1
+    assert isinstance(result, Query)
+
+
+def test_get_datasource_saved_query(app_context: None, session: Session) -> None:
+    from superset.dao.datasource.dao import DatasourceDAO
+    from superset.models.sql_lab import SavedQuery
+
+    create_test_data(session)
+
+    result = DatasourceDAO.get_datasource(
+        datasource_type="saved_query", datasource_id=1, session=session
+    )
+
+    assert result.id == 1
+    assert isinstance(result, SavedQuery)
+
+
+def test_get_datasource_sl_table(app_context: None, session: Session) -> None:
+    pass

Review Comment:
   I'm currently working on these test now and will push them up soon



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870848216


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:
+        datasources: List[Datasource] = []
+        for source_class in DatasourceDAO.sources.values():
+            qry = session.query(source_class)
+            if isinstance(source_class, SqlaTable):
+                qry = source_class.default_query(qry)
+            datasources.extend(qry.all())
+        return datasources
+
+    @classmethod
+    def get_datasource_by_name(  # pylint: disable=too-many-arguments
+        cls,
+        session: Session,
+        datasource_type: DatasourceType,
+        datasource_name: str,
+        schema: str,
+        database_name: str,
+    ) -> Optional[Datasource]:
+        datasource_class = DatasourceDAO.sources[datasource_type]
+        if isinstance(datasource_class, SqlaTable):
+            return datasource_class.get_datasource_by_name(
+                session, datasource_name, schema, database_name
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_permissions(  # pylint: disable=invalid-name
+        cls,
+        session: Session,
+        database: Database,
+        permissions: Set[str],
+        schema_perms: Set[str],
+    ) -> List[Datasource]:
+        # TODO(bogdan): add unit test
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .filter_by(database_id=database.id)
+                .filter(
+                    or_(
+                        datasource_class.perm.in_(permissions),
+                        datasource_class.schema_perm.in_(schema_perms),
+                    )
+                )
+                .all()
+            )
+        return []
+
+    @classmethod
+    def get_eager_datasource(
+        cls, session: Session, datasource_type: str, datasource_id: int
+    ) -> Optional[Datasource]:
+        """Returns datasource with columns and metrics."""
+        datasource_class = DatasourceDAO.sources[DatasourceType[datasource_type]]
+        if isinstance(datasource_class, SqlaTable):
+            return (
+                session.query(datasource_class)
+                .options(
+                    subqueryload(datasource_class.columns),
+                    subqueryload(datasource_class.metrics),
+                )
+                .filter_by(id=datasource_id)
+                .one()
+            )
+        return None
+
+    @classmethod
+    def query_datasources_by_name(
+        cls,
+        session: Session,
+        database: Database,
+        datasource_name: str,
+        schema: Optional[str] = None,
+    ) -> List[Datasource]:
+        datasource_class = DatasourceDAO.sources[DatasourceType[database.type]]
+        if isinstance(datasource_class, SqlaTable):

Review Comment:
   why only sqlatable on these? Is the plan to add in the rest when we need them?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] eschutho commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
eschutho commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870846544


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]
+
+
+class DatasourceDAO(BaseDAO):
+
+    sources: Dict[DatasourceType, Datasource] = {
+        DatasourceType.SQLATABLE: SqlaTable,
+        DatasourceType.QUERY: Query,
+        DatasourceType.SAVEDQUERY: SavedQuery,
+        DatasourceType.DATASET: Dataset,
+        DatasourceType.TABLE: Table,
+    }
+
+    @classmethod
+    def get_datasource(
+        cls, datasource_type: DatasourceType, datasource_id: int, session: Session
+    ) -> Datasource:
+        if datasource_type not in cls.sources:
+            raise DatasetNotFoundError()
+
+        datasource = (
+            session.query(cls.sources[datasource_type])
+            .filter_by(id=datasource_id)
+            .one_or_none()
+        )
+
+        if not datasource:
+            raise DatasetNotFoundError()
+
+        return datasource
+
+    @classmethod
+    def get_all_datasources(cls, session: Session) -> List[Datasource]:

Review Comment:
   I feel like this should just return `datasets` since for now this is the only datasource that can be attached to a saved chart. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org


[GitHub] [superset] hughhhh commented on a diff in pull request #20030: feat(sip-68): Add DatasourceDAO class to manage querying different datasources easier

Posted by GitBox <gi...@apache.org>.
hughhhh commented on code in PR #20030:
URL: https://github.com/apache/superset/pull/20030#discussion_r870860323


##########
superset/dao/datasource/dao.py:
##########
@@ -0,0 +1,146 @@
+# 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 typing import Any, Dict, List, Optional, Set, Union
+
+from flask_babel import _
+from sqlalchemy import or_
+from sqlalchemy.orm import Session, subqueryload
+from sqlalchemy.orm.exc import NoResultFound
+
+from superset.connectors.sqla.models import SqlaTable
+from superset.dao.base import BaseDAO
+from superset.datasets.commands.exceptions import DatasetNotFoundError
+from superset.datasets.models import Dataset
+from superset.models.core import Database
+from superset.models.sql_lab import Query, SavedQuery
+from superset.tables.models import Table
+from superset.utils.core import DatasourceType
+
+Datasource = Union[Dataset, SqlaTable, Table, Query, SavedQuery, Any]

Review Comment:
   @betodealmeida this is a hack to allow the types to play nice with mypy, the good thing is it still does some type checking. I was going to look into this more, but literally spent most of the day looking into it and had no luck. I can revisit this if you think it's a mandatory thing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@superset.apache.org
For additional commands, e-mail: notifications-help@superset.apache.org