You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@superset.apache.org by be...@apache.org on 2023/08/08 00:37:43 UTC

[superset] 02/02: More docs

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

beto pushed a commit to branch db-diagnostics
in repository https://gitbox.apache.org/repos/asf/superset.git

commit 26da99444e6536ff37367786896325f326732028
Author: Beto Dealmeida <ro...@dealmeida.net>
AuthorDate: Mon Aug 7 17:36:54 2023 -0700

    More docs
---
 superset/db_engine_specs/README.md | 408 +++++++++++++++++++++++++++++++++++++
 superset/db_engine_specs/lib.py    |   2 +-
 2 files changed, 409 insertions(+), 1 deletion(-)

diff --git a/superset/db_engine_specs/README.md b/superset/db_engine_specs/README.md
index 577c8afd88..5d01920b9d 100644
--- a/superset/db_engine_specs/README.md
+++ b/superset/db_engine_specs/README.md
@@ -322,3 +322,411 @@ SQLAlchemy recommends escaping colons to prevent them from being interpreted as
 This works for most databases except Athena. The `allows_escaped_colons` attribute specifies if the database supports the escape colon.
 
 ## Basic features
+
+These are features that all DB engine specs should support, as the name suggests. They provide a much better user experience for the user.
+
+### Time grains
+
+The most basic feature that DB engine specs need to support is defining time grain expressions. These are dialect-specific SQL expressions that are used to compute metrics on a given time grain when building charts. For example, when computing the metric `COUNT(*)` on a daily basis, Superset will generate the following query:
+
+```sql
+SELECT
+  <DB engine spec expression for TimeGrain.DAY>,
+  COUNT(*)
+...
+GROUP BY
+  <DB engine spec expression for TimeGrain.DAY>
+```
+
+For some databases with support for `DATE_TRUNC` or `TIME_FLOOR` this is easy. Here's how Apache Druid computes 15 minute aggregations:
+
+```sql
+TIME_FLOOR(CAST({col} AS TIMESTAMP), 'PT15M')
+```
+
+Where `{col}` is the time column being aggregated — the expression is actually a Jinja2 template. Druid uses the ISO standard for durations, with `PT15M` representing 15 minutes.
+
+On the other and, here's the same for SQLite:
+
+```sql
+DATETIME(
+  STRFTIME(
+    '%Y-%m-%dT%H:%M:00',
+    {col}
+  ),
+  printf(
+    '-%d minutes',
+    CAST(strftime('%M', {col}) AS INT) % 15
+  )
+)
+```
+
+The SQLite version has to truncate the column down to the minute, and then subtract a number of minutes equals to the modulo 15.
+
+Time grain expressions are defined in the `_time_grain_expressions` class attribute, which maps from a `superset.constants.TimeGrain` to the SQL expression. The dictionary has a special key `None`, that should map to the column directly, for when no time grain is specified.
+
+Note that it's possible to add new time grains via configuration. For example, if you want to add a "2 seconds" time grain to your installation you can add it to `TIME_GRAIN_ADDONS`, and implement it in `TIME_GRAIN_ADDON_EXPRESSIONS`:
+
+```python
+# superset_config.py
+TIME_GRAIN_ADDONS = {"PT2S": "2 second"}
+
+TIME_GRAIN_ADDON_EXPRESSIONS = {
+    "clickhouse": {
+        "PT2S": "toDateTime(intDiv(toUInt32(toDateTime({col})), 2)*2)",
+    }
+}
+```
+
+### Column type mapping
+
+Column type mapping, defined in the `column_type_mappings` class attribute, is just a way of mapping type names from the database to types Superset understand. The default values in `BaseEngineSpec` are sane:
+
+```python
+_default_column_type_mappings: tuple[ColumnTypeMapping, ...] = (
+    (
+        re.compile(r"^string", re.IGNORECASE),
+        types.String(),
+        GenericDataType.STRING,
+    ),
+    (
+        re.compile(r"^float", re.IGNORECASE),
+        types.Float(),
+        GenericDataType.NUMERIC,
+    ),
+    (
+        re.compile(r"^date", re.IGNORECASE),
+        types.Date(),
+        GenericDataType.TEMPORAL,
+    ),
+    (
+        re.compile(r"^bool(ean)?", re.IGNORECASE),
+        types.Boolean(),
+        GenericDataType.BOOLEAN,
+    ),
+    ...
+)
+```
+
+But you might want to implement more specific types in the DB engine spec, or complex types. For example, for MSSQL we have:
+
+```python
+from sqlalchemy.dialects.mssql.base import SMALLDATETIME
+
+class MssqlEngineSpec(BaseEngineSpec):
+    ...
+    column_type_mappings = (
+        (
+            re.compile(r"^smalldatetime.*", re.IGNORECASE),
+            SMALLDATETIME(),
+            GenericDataType.TEMPORAL,
+        ),
+    )
+```
+
+### Function names
+
+DB engine specs should implement a class method called `get_function_names` that retuns a list of strings, representing all the function names that the database supports. This is used for autocomplete in SQL Lab.
+
+### Masked encrypted extra
+
+Superset does a good job in keeping credentials secure. When you add a database with a password, for example:
+
+```
+postgresql://admin:password123@db.example.org:5432/db
+```
+
+The password is sent over the network only when the database is created. When you edit the database later, Superset will return this as the SQLAlchemy URI:
+
+```
+postgresql://admin:XXXXXXXXXX@db.example.org:5432/db
+```
+
+The password will be masked in the API response; it's not just masked in the browser UI. This is done in order to avoid sending the password unnecessarily over the network. Also, if a non-admin user has access to the API response, they won't be able to know the database password.
+
+When the database is edited, the Superset backend is smart enough to replace the masked password with the actual password, unless the password has changed. That is, if you change the database in the URI from `db` to `db2` the SQLAlchemy URI will be stored in the backend as:
+
+```
+postgresql://admin:password123@db.example.org:5432/db2
+```
+
+The password is not the only piece of information where security is critical. For many databases (like BigQuery), sensitive information is stored in the credentials JSON payload. For example:
+
+```json
+{
+  "type": "service_account",
+  "project_id": "dbt-tutorial-347100",
+  "private_key_id": "4bc71f06990c864a590fad8b94be6a5904fc171f",
+  "private_key": "<SENSITIVE INFORMATION>",
+  "client_email": "dbt-user-278@dbt-tutorial-347100.iam.gserviceaccount.com",
+  "client_id": "115666988796889519425",
+  "auth_uri": "https://accounts.google.com/o/oauth2/auth",
+  "token_uri": "https://oauth2.googleapis.com/token",
+  "auth_provider_x509_cert_url": "https://www.googleapis.com/oauth2/v1/certs",
+  "client_x509_cert_url": "https://www.googleapis.com/robot/v1/metadata/x509/dbt-user-278%40dbt-tutorial-347100.iam.gserviceaccount.com"
+}
+```
+
+Similarly to password, we don't want to send `private_key` to the client when a database is edited; the Superset API should never return its actual contents. Instead, Superset should return a masked value, and users should be able to edit the JSON without having to type in the `private_key` on every edit.
+
+To do this, DB engine specs and implement 2 methods, `mask_encrypted_extra` and `unmask_encrypted_extra`. They have these names because the credentials are stored in an encrypted column called `encrypted_extra`. Here's how these methods look like for BigQuery:
+
+```python
+from superset.constants import PASSWORD_MASK
+
+
+class BigQueryEngineSpec(BaseEngineSpec):
+
+    @classmethod
+    def mask_encrypted_extra(cls, encrypted_extra: str | None) -> str | None:
+        if encrypted_extra is None:
+            return encrypted_extra
+
+        try:
+            config = json.loads(encrypted_extra)
+        except (json.JSONDecodeError, TypeError):
+            return encrypted_extra
+
+        try:
+            config["credentials_info"]["private_key"] = PASSWORD_MASK
+        except KeyError:
+            pass
+
+        return json.dumps(config)
+
+    @classmethod
+    def unmask_encrypted_extra(
+        cls,
+        old: str | None,
+        new: str | None
+    ) -> str | None:
+        if old is None or new is None:
+            return new
+
+        try:
+            old_config = json.loads(old)
+            new_config = json.loads(new)
+        except (TypeError, json.JSONDecodeError):
+            return new
+
+        if "credentials_info" not in new_config:
+            return new
+
+        if "private_key" not in new_config["credentials_info"]:
+            return new
+
+        if new_config["credentials_info"]["private_key"] == PASSWORD_MASK:
+            new_config["credentials_info"]["private_key"] = old_config[
+                "credentials_info"
+            ]["private_key"]
+
+        return json.dumps(new_config)
+```
+
+This way, when a user edits an existing BigQuery connection, the `private_key` is shown as `XXXXXXXXXX`. Everything else in the JSON is still displayed, and the user can change any of the fields without having to provide the private key.
+
+Note that while this is a basic feature that should be implemented for security reasons, it only makes sense in DB engine specs that use `encrypted_extra` to store connection information.
+
+## Nice to have features
+
+The next set of features are nice to have. They don't apply to all databases, and are not strictly needed for security or usability.
+
+### User impersonation
+
+In general there's no user-level granularity when accessing a database in Superset. A single database connection is shared by all users who have access to that database. There are many use cases when this is not desirable, and some databases implement mechanisms in which they can **impersonate users**, potentially reducing the scope of permissions available to run the query.
+
+For example, the Google Sheets DB engine spec implements this via the `get_url_for_impersonation` class method:
+
+```python
+class GSheetsEngineSpec(ShillelaghEngineSpec):
+
+    @classmethod
+    def get_url_for_impersonation(
+        cls,
+        url: URL,
+        impersonate_user: bool,
+        username: str | None,
+    ) -> URL:
+        if impersonate_user and username is not None:
+            user = security_manager.find_user(username=username)
+            if user and user.email:
+                url = url.update_query_dict({"subject": user.email})
+
+        return url
+```
+
+The method `get_url_for_impersonation` updates the SQLAlchemy URI before every query. In this particular case, it will fetch the user's email and add it to the `subject` query argument. The driver will then lower the permissions to match that given user. This allows the connection to be configured with a service account that has access to all the spreadsheets, while giving users access to only the spreadsheets they own are have been shared with them (or with their organization — Google w [...]
+
+Alternatively, it's also possible to impersonate users by implemeneting the `update_impersonation_config`. This is a class method which modifies `connect_args` in place. You can use either method, and ideally they [should be consolidated in a single one](https://github.com/apache/superset/issues/24910).
+
+### File upload
+
+When a DB engine spec supports file upload it declares so via the `supports_file_upload` class attribute. The base class implementation is very generic and should work for any database that has support for `CREATE TABLE`. It leverages Pandas and the [`df_to_sql`](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.to_sql.html) method.
+
+For some databases the `df_to_sql` classmethod needs to be implemented. For example, for BigQuery the DB engine spec implements a custom method that uses the [`to_gbq`](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.to_gbq.html) method.
+
+### Extra table metadata
+
+DB engine specs can return additional metadata associated with a table. This is done via the `extra_table_metadata` class method. Trino uses this to return information about the latest partition, for example, and Bigquery returns clustering information. This information is then surfaced in the SQL Lab UI, when browsing tables in the metadata explorer (on the left panel).
+
+### DB API exception mapping
+
+Different DB API 2.0 drivers implement different exceptions, even if they have the same name. The `get_dbapi_exception_mapping` class method returns a dictionary mapping these custom exceptions to Superset exceptions, so that Superset can return more specific errors when an exception is raised by the underlying driver.
+
+For example, for ClickHouse we have:
+
+```python
+from urllib3.exceptions import NewConnectionError
+
+from superset.db_engine_specs.exceptions import SupersetDBAPIDatabaseError
+
+
+class ClickHouseEngineSpec(ClickHouseBaseEngineSpec):
+
+    @classmethod
+    def get_dbapi_exception_mapping(cls) -> dict[type[Exception], type[Exception]]:
+        return {NewConnectionError: SupersetDBAPIDatabaseError}
+```
+
+This way, if the ClickHouse driver raises a `NewConnectionError` it would get wrapped in a `SupersetDBAPIDatabaseError`.
+
+### Custom errors
+
+Queries can fail in many different ways. For example, in SQLite:
+
+```sql
+sqlite> CREATE TABLE a (b INT);
+sqlite> SELECT c FROM a;
+Error: no such column: c
+sqlite>
+```
+
+When a query fails, Superset will return the message, "Error: no such column: c", to the user as a generic error.
+
+Since ideally we want to return specific and actionable error messages, DB engine specs can implement methods that map error messages to more specific errors. For example, the SQLite DB engine specs defines:
+
+```python
+COLUMN_DOES_NOT_EXIST_REGEX = re.compile("no such column: (?P<column_name>.+)")
+
+
+class SqliteEngineSpec(BaseEngineSpec):
+
+    custom_errors: dict[Pattern[str], tuple[str, SupersetErrorType, dict[str, Any]]] = 
+        COLUMN_DOES_NOT_EXIST_REGEX: (
+            __('We can\'t seem to resolve the column "%(column_name)s"'),
+            SupersetErrorType.COLUMN_DOES_NOT_EXIST_ERROR,
+            {},
+        ),
+    }
+```
+
+This way, when a user selects a column that doesn't exist Superset can return a more informative error.
+
+### Dynamic schema
+
+In SQL Lab it's possible to select a database, and then a schema in that database. Ideally, when running a query in SQL Lab, any unqualified table names (eg, `table`, instead of `schema.table`) should be in the selected schema. For example, if the user select `dev` as the schema and then runs the following query:
+
+```sql
+SELECT * FROM my_table
+```
+
+The table `my_table` should live in the `dev` schema. In order to do that, it's necessary to modify the SQLAlchemy URI before running the query. Since different databases have different ways of doing that, this functionality is implemented via the `adjust_engine_params` class method. The method receives the SQLAlchemy URI and `connect_args`, as well as the schema in which the query should run. It then returns a potentially modified URI and `connect_args` to ensure that the query runs in  [...]
+
+When a DB engine specs implements `adjust_engine_params` it should have the class attribute `supports_dynamic_schema` set to true. This is critical for security, since **it allows Superset to know to which schema any unqualified table names belong to**. For example, in the query above, if the database supports dynamic schema, Superset would check to see if the user running the query has access to `dev.my_table`. On the other hand, if the database doesn't support dynamic schema, Superset  [...]
+
+Implementing this method is also important for usability. When the method is not implemented selecting the schema in SQL Lab has no effect on the schema in which the query runs, resulting in a confusing results when using unqualified table names.
+
+### Catalog
+
+In general, databases support a hierarchy of concepts of one-to-many concepts:
+
+1. Database
+2. Catalog
+3. Namespace
+4. Table
+5. Column
+
+These concepts have different names depending on the database. For example, Postgres uses the following terminology:
+
+1. Cluster (database)
+2. Database (catalog)
+3. Schema (namespace)
+4. Table
+5. Column
+
+BigQuery, on the other hand:
+
+1. Bigquery (database)
+2. Project (catalog)
+3. Schema (namespace)
+4. Table
+5. Column
+
+Hive and Trino:
+
+1. Database
+2. Catalog
+3. Schema
+4. Table
+5. Column
+
+If the database supports catalogs, then the DB engine spec should have the `supports_catalog` class attribute set to true.
+
+### Dynamic catalog
+
+Superset has no support for multiple catalogs. A given SQLAlchemy URI connects to a single catalog, and it's impossible to browse other catalogs, or change the catalog. This means that datasets can only be added for the main catalog of the database. For example, with this Postgres SQLAlchemy URI:
+
+```
+postgresql://admin:password123@db.example.org:5432/db
+```
+
+Here, datasets can only be added to the `db` catalog (which Postgres calls a "database").
+
+One confusing problem is that many databases allow querying across catalogs in SQL Lab. For example, with BigQuery one can write:
+
+```sql
+SELECT * FROM project.schema.table
+```
+
+This means that **even though the database is configured for a given catalog (project), users can query other projects**. This is a common workaround for creating datasets in catalogs other than the catalog configured in the database: just create a virtual dataset.
+
+Ideally we would want users to be able to choose the catalog when using SQL Lab and when creating datasets. In order to do that, DB engine specs need to implement a method that rewrites the SQLAlchemy URI depending on the desired catalog. This method already exists, and is the same method used for dynamic schemas, `adjust_engine_params`, but currently there are no UI affordances for choosing a catalog.
+
+Before the UI is implemented Superset still needs to implement support for catalogs in its security manager. But in the meantime, it's possible for DB engine spec developers to support dynamic catalogs, by setting `supports_dynamic_catalog` to true and implementing `adjust_engine_params` to handle a catalog.
+
+### SSH tunneling
+
+Superset can connect to databases via an SSH tunnel. For databases where this doesn't make sense (eg, SQLite or BigQuery) the DB engine spec should have `disable_ssh_tunneling` set to true.
+
+### Query cancelation
+
+XXX
+
+### Get metrics on dataset creation
+
+When a physical dataset is first created, the `get_metrics` class method is called on the table. The base implementation returns the `COUNT(*)` metric, but DB engine specs can override `get_metrics` to return other metrics. This method is useful for semantic layers that contain their own metrics definitions; when Superset connect to them it can automatically create those metrics when a dataset is added.
+
+This feature is still experimental, and ideally there would be a mechanism for calling it periodically or when a dataset is explored, in order to sync new metric definitions to the dataset.
+
+### `WHERE` on latest partition
+
+XXX
+
+## Advanced features
+
+### Expand complex types
+
+XXX
+
+### Query cost estimation
+
+XXX
+
+### SQL validation
+
+XXX
+
+## Testing DB engine specs
+
+XXX
+
diff --git a/superset/db_engine_specs/lib.py b/superset/db_engine_specs/lib.py
index 93db779c6e..50ded4644b 100644
--- a/superset/db_engine_specs/lib.py
+++ b/superset/db_engine_specs/lib.py
@@ -136,7 +136,7 @@ def diagnose(spec: type[BaseEngineSpec]) -> dict[str, Any]:
             "file_upload": spec.supports_file_upload,
             "extra_table_metadata": has_custom_method(spec, "extra_table_metadata"),
             "dbapi_exception_mapping": has_custom_method(
-                spec, "get_dbapi_exception_mappin"
+                spec, "get_dbapi_exception_mapping"
             ),
             "custom_errors": (
                 has_custom_method(spec, "extract_errors")