diff --git a/CHANGES.rst b/CHANGES.rst index abb84d1c..8a2cfdf9 100644 --- a/CHANGES.rst +++ b/CHANGES.rst @@ -7,6 +7,9 @@ Kartothek 4.0.2 (2021-04-xx) ============================ * Fix a bug in ``MetaPartition._reconstruct_index_columns`` that would raise an ``IndexError`` when loading few columns of a dataset with many primary indices. +* Add :meth:`~kartothek.io.eager.copy_dataset` to copy and optionally rename datasets within one store or between stores (eager only) +* Add renaming option to :meth:`~kartothek.io.eager_cube.copy_cube` +* moved testing code from `~kartothek.io.testing`to `tests` Kartothek 4.0.1 (2021-04-13) @@ -14,7 +17,6 @@ Kartothek 4.0.1 (2021-04-13) * Fixed dataset corruption after updates when table names other than "table" are used (#445). - Kartothek 4.0.0 (2021-03-17) ============================ @@ -476,7 +478,7 @@ Version 3.1.0 (2019-07-10) - Ensure binary column names are read as type ``str``: - Ensure dataframe columns are of type ``str`` in :func:`~kartothek.core.common_metadata.empty_dataframe_from_schema` - - Testing: create :func:`~kartothek.io.testing.read.test_binary_column_metadata` which checks column names stored as + - Testing: create `~kartothek.io.testing.read.test_binary_column_metadata` which checks column names stored as ``bytes`` objects are read as type ``str`` - fix issue where it was possible to add an index to an existing dataset by using update functions and partition indices diff --git a/kartothek/core/dataset.py b/kartothek/core/dataset.py index 15db87e0..5298c6c1 100644 --- a/kartothek/core/dataset.py +++ b/kartothek/core/dataset.py @@ -907,6 +907,47 @@ def from_dataset(dataset): ds_builder.partitions = dataset.partitions return ds_builder + def modify_uuid(self, target_uuid: str): + """ + Modify the dataset uuid and depending metadata: + - paths to partitioning files + - path to index files + + Parameters + ---------- + target_uuid: str + Modified dataset UUID. + + Returns + ------- + DatasetMetadataBuilder + modified builder object + """ + + # modify file names in partition metadata + modified_partitions = {} + for p_key, p in self.partitions.items(): + pdict = p.to_dict() + for table_key, table_file in pdict["files"].items(): + if table_file.startswith(f"{self.uuid}/"): + pdict["files"][table_key] = table_file.replace( + self.uuid, target_uuid, 1 + ) + modified_partitions[p_key] = Partition.from_dict(p_key, pdict) + + self.partitions = modified_partitions + + for i_key, i in self.indices.items(): + if ( + isinstance(i, ExplicitSecondaryIndex) + and i.index_storage_key is not None + ): + i.index_storage_key = i.index_storage_key.replace( + self.uuid, target_uuid, 1 + ) + self.uuid = target_uuid + return self + def add_partition(self, name, partition): """ Add an (embedded) Partition. diff --git a/kartothek/io/eager.py b/kartothek/io/eager.py index 6ac0f531..a412ca0f 100644 --- a/kartothek/io/eager.py +++ b/kartothek/io/eager.py @@ -15,6 +15,8 @@ from kartothek.core.naming import ( DEFAULT_METADATA_STORAGE_FORMAT, DEFAULT_METADATA_VERSION, + METADATA_BASE_SUFFIX, + METADATA_FORMAT_JSON, PARQUET_FILE_SUFFIX, get_partition_file_prefix, ) @@ -44,6 +46,8 @@ ) from kartothek.io_components.write import raise_if_dataset_exists from kartothek.serialization import DataFrameSerializer +from kartothek.utils.ktk_adapters import get_dataset_keys +from kartothek.utils.store import copy_rename_keys __all__ = ( "delete_dataset", @@ -56,6 +60,7 @@ "update_dataset_from_dataframes", "build_dataset_indices", "garbage_collect_dataset", + "copy_dataset", ) @@ -738,3 +743,62 @@ def garbage_collect_dataset(dataset_uuid=None, store=None, factory=None): # Given that `nested_files` is a generator with a single element, just # return the output of `delete_files` on that element. return delete_files(next(nested_files), store_factory=ds_factory.store_factory) + + +def copy_dataset( + source_dataset_uuid: str, + store: KeyValueStore, + target_dataset_uuid: Optional[str] = None, + target_store: Optional[KeyValueStore] = None, +): + """ + Copies and optionally renames a dataset, either from one store to another or + within one store. + + Parameters + ---------- + source_dataset_uuid: + UUID of source dataset + store: + Source store + target_dataset_uuid: + UUID of target dataset. May be the same as src_dataset_uuid, if store + and tgt_store are different. If empty, src_dataset_uuid is used + target_store: + Target Store. May be the same as store, if src_dataset_uuid and + target_dataset_uuid are different. If empty, value from parameter store is + used + """ + if target_dataset_uuid is None: + target_dataset_uuid = source_dataset_uuid + if target_store is None: + target_store = store + + if (source_dataset_uuid == target_dataset_uuid) & (store == target_store): + raise ValueError( + "Cannot copy to a dataset with the same UUID within the same store!" + ) + + ds_factory_source = _ensure_factory( + dataset_uuid=source_dataset_uuid, store=store, factory=None, + ) + + # Create a dict of {source key: target key} entries + keys = get_dataset_keys(ds_factory_source.dataset_metadata) + mapped_keys = { + source_key: source_key.replace(source_dataset_uuid, target_dataset_uuid) + for source_key in keys + } + + # Create a dict of metadata which has to be changed. This is only the + # .by-dataset-metadata.json file + md_transformed = { + f"{source_dataset_uuid}{METADATA_BASE_SUFFIX}{METADATA_FORMAT_JSON}": DatasetMetadataBuilder.from_dataset( + ds_factory_source.dataset_metadata + ) + .modify_uuid(target_dataset_uuid) + .to_json()[1] + } + + # Copy the keys from one store to another + copy_rename_keys(mapped_keys, store, target_store, md_transformed) diff --git a/kartothek/io/eager_cube.py b/kartothek/io/eager_cube.py index 34a7747e..01fa71be 100644 --- a/kartothek/io/eager_cube.py +++ b/kartothek/io/eager_cube.py @@ -1,8 +1,9 @@ """ Eager IO aka "everything is done locally and immediately". """ +import logging from collections import defaultdict -from typing import Any, Dict, Iterable, List, Optional, Sequence, Union +from typing import Any, Callable, Dict, Iterable, List, Optional, Sequence, Union import pandas as pd import simplekv @@ -15,18 +16,21 @@ KTK_CUBE_DF_SERIALIZER, KTK_CUBE_METADATA_STORAGE_FORMAT, KTK_CUBE_METADATA_VERSION, + KTK_CUBE_UUID_SEPARATOR, ) from kartothek.core.cube.cube import Cube from kartothek.core.dataset import DatasetMetadata from kartothek.core.typing import StoreFactory from kartothek.io.eager import ( + copy_dataset, + delete_dataset, store_dataframes_as_dataset, update_dataset_from_dataframes, ) from kartothek.io_components.cube.append import check_existing_datasets from kartothek.io_components.cube.cleanup import get_keys_to_clean from kartothek.io_components.cube.common import assert_stores_different -from kartothek.io_components.cube.copy import get_copy_keys +from kartothek.io_components.cube.copy import get_copy_keys, get_datasets_to_copy from kartothek.io_components.cube.query import load_group, plan_query, quick_concat from kartothek.io_components.cube.remove import ( prepare_metapartitions_for_removal_action, @@ -53,6 +57,8 @@ from kartothek.utils.pandas import concat_dataframes from kartothek.utils.store import copy_keys +logger = logging.getLogger() + __all__ = ( "append_to_cube", "build_cube", @@ -419,10 +425,57 @@ def delete_cube(cube, store, datasets=None): store.delete(k) -def copy_cube(cube, src_store, tgt_store, overwrite=False, datasets=None): +def _transform_uuid( + src_uuid: str, + cube_prefix: str, + renamed_cube_prefix: Optional[str], + renamed_datasets: Optional[Dict[str, str]], +): + """ + Transform a uuid from ++ to + ++ + :param src_uuid: + Uuid to transform + :param cube_prefix: + Cube prefix before renaming + :param renamed_cube: + Optional new cube prefix + :param renamed_datasets: + Optional dict of {old dataset name: new dataset name} entries to rename datasets + """ + tgt_uuid = src_uuid + if renamed_cube_prefix: + tgt_uuid = src_uuid.replace( + f"{cube_prefix}{KTK_CUBE_UUID_SEPARATOR}", + f"{renamed_cube_prefix}{KTK_CUBE_UUID_SEPARATOR}", + ) + + if renamed_datasets: + for ds_old, ds_new in renamed_datasets.items(): + if f"{KTK_CUBE_UUID_SEPARATOR}{ds_old}" in tgt_uuid: + tgt_uuid = tgt_uuid.replace( + f"{KTK_CUBE_UUID_SEPARATOR}{ds_old}", + f"{KTK_CUBE_UUID_SEPARATOR}{ds_new}", + ) + return tgt_uuid + + +def copy_cube( + cube: Cube, + src_store: Union[KeyValueStore, Callable[[], KeyValueStore]], + tgt_store: Union[KeyValueStore, Callable[[], KeyValueStore]], + overwrite: bool = False, + datasets: Union[None, Iterable[str], Dict[str, DatasetMetadata]] = None, + renamed_cube_prefix: Optional[str] = None, + renamed_datasets: Optional[Dict[str, str]] = None, +): """ Copy cube from one store to another. + .. warning:: + A failing copy operation can not be rolled back if the `overwrite` flag is enabled + and might leave the overwritten dataset in an inconsistent state. + Parameters ---------- cube: Cube @@ -433,9 +486,16 @@ def copy_cube(cube, src_store, tgt_store, overwrite=False, datasets=None): Target KV store. overwrite: bool If possibly existing datasets in the target store should be overwritten. - datasets: Union[None, Iterable[str], Dict[str, kartothek.core.dataset.DatasetMetadata]] + datasets: Union[None, Iterable[str], Dict[str, DatasetMetadata]] Datasets to copy, must all be part of the cube. May be either the result of :func:`~kartothek.api.discover.discover_datasets`, a list of Ktk_cube dataset ID or ``None`` (in which case entire cube will be copied). + renamed_cube_prefix: Optional[str] + Optional new cube prefix. If specified, the cube will be renamed while copying. + renamed_datasets: Optional[Dict[str, str]] + Optional dict with {old dataset name: new dataset name} entries. If provided, + the datasets will be renamed accordingly during copying. When the parameter + datasets is specified, the datasets to rename must be a subset of the datasets + to copy. """ if callable(src_store): src_store = src_store() @@ -445,14 +505,59 @@ def copy_cube(cube, src_store, tgt_store, overwrite=False, datasets=None): src_store, tgt_store, cube.ktk_dataset_uuid(cube.seed_dataset) ) - keys = get_copy_keys( - cube=cube, - src_store=src_store, - tgt_store=tgt_store, - overwrite=overwrite, - datasets=datasets, - ) - copy_keys(keys, src_store, tgt_store) + if (renamed_cube_prefix is None) and (renamed_datasets is None): + # If we don't rename the datasets, we can perform an optimized copy operation. + keys = get_copy_keys( + cube=cube, + src_store=src_store, + tgt_store=tgt_store, + overwrite=overwrite, + datasets=datasets, + ) + copy_keys(keys, src_store, tgt_store) + else: + datasets_to_copy = get_datasets_to_copy( + cube=cube, + src_store=src_store, + tgt_store=tgt_store, + overwrite=overwrite, + datasets=datasets, + ) + copied = [] # type: List[str] + for src_ds_name, src_ds_meta in datasets_to_copy.items(): + tgt_ds_uuid = _transform_uuid( + src_uuid=src_ds_meta.uuid, + cube_prefix=cube.uuid_prefix, + renamed_cube_prefix=renamed_cube_prefix, + renamed_datasets=renamed_datasets, + ) + try: + copy_dataset( + source_dataset_uuid=src_ds_meta.uuid, + store=src_store, + target_dataset_uuid=tgt_ds_uuid, + target_store=tgt_store, + ) + except Exception as e: + if overwrite: + # We can't roll back safely if the target dataset has been partially overwritten. + raise RuntimeError(e) + else: + logger.exception( + f"Copying datasets from source dataset {src_ds_meta.uuid} to target {tgt_ds_uuid} failed." + ) + + # Roll back the unsuccesful copy operation by deleting all datasets that have been copied so far from the target store. + for ds_uuid in copied: + delete_dataset(dataset_uuid=ds_uuid, store=tgt_store) + + logger.info( + "Deleted the copied datasets {} from target store.".format( + ", ".join(copied) + ) + ) + else: + copied.append(tgt_ds_uuid) def collect_stats(cube, store, datasets=None): diff --git a/kartothek/io/testing/append_cube.py b/kartothek/io/testing/append_cube.py deleted file mode 100644 index d4023443..00000000 --- a/kartothek/io/testing/append_cube.py +++ /dev/null @@ -1,459 +0,0 @@ -import pandas as pd -import pytest - -from kartothek.core.cube.constants import ( - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, -) -from kartothek.core.cube.cube import Cube -from kartothek.core.dataset import DatasetMetadata -from kartothek.io.eager_cube import build_cube -from kartothek.io.testing.utils import assert_num_row_groups -from kartothek.serialization._parquet import ParquetSerializer - -__all__ = ( - "existing_cube", - "test_append_partitions", - "test_append_partitions_no_ts", - "test_compression_is_compatible_on_append_cube", - "test_fails_incompatible_dtypes", - "test_fails_missing_column", - "test_fails_unknown_dataset", - "test_indices", - "test_metadata", - "test_rowgroups_are_applied_when_df_serializer_is_passed_to_append_cube", - "test_single_rowgroup_when_df_serializer_is_not_passed_to_append_cube", -) - - -@pytest.fixture -def existing_cube(function_store): - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [10, 11, 12, 13], - } - ) - df_enrich = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v2": [10, 11, 12, 13], - "i2": [10, 11, 12, 13], - } - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - metadata={"source": {"a": 10, "b": 11}, "enrich": {"a": 20, "b": 21}}, - ) - return cube - - -def test_append_partitions(driver, function_store, existing_cube): - partitions_source_1 = set( - DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("source"), function_store() - ).partitions.keys() - ) - partitions_enrich_1 = set( - DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("enrich"), function_store() - ).partitions.keys() - ) - - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [20, 21, 22, 23], - "i1": [20, 21, 22, 23], - } - ) - - result = driver( - data={"source": df_source}, cube=existing_cube, store=function_store - ) - - assert set(result.keys()) == {"source"} - - ds_source = result["source"] - ds_enrich = DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("enrich"), function_store() - ) - - partitions_source_2 = set(ds_source.partitions.keys()) - partitions_enrich_2 = set(ds_enrich.partitions.keys()) - - assert len(partitions_source_2) > len(partitions_source_1) - assert partitions_source_1.issubset(partitions_source_2) - - assert partitions_enrich_2 == partitions_enrich_1 - - -@pytest.mark.parametrize("chunk_size_build", [None, 2]) -@pytest.mark.parametrize("chunk_size_append", [None, 2]) -def test_rowgroups_are_applied_when_df_serializer_is_passed_to_append_cube( - driver, function_store, chunk_size_build, chunk_size_append -): - """ - Test that the dataset is split into row groups depending on the chunk size - - Partitions build with ``chunk_size=None`` should keep a single row group after the append. Partitions that are newly created with - ``chunk_size>0`` should be split into row groups accordingly. - """ - - # Build cube - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size_build), - ) - - # Append to cube - df_append = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], - ) - result = driver( - data={"seed": df_append}, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size_append), - ) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 2, 1: 2, 2: 1, 3: 3} - part_chunk_size = { - 0: chunk_size_build, - 1: chunk_size_build, - 2: chunk_size_append, - 3: chunk_size_append, - } - - assert len(dataset.partitions) == 4 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_single_rowgroup_when_df_serializer_is_not_passed_to_append_cube( - driver, function_store -): - """ - Test that the dataset has a single row group as default path - """ - - # Build cube - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, cube=cube, store=function_store, - ) - - # Append to cube - df_append = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], - ) - result = driver(data={"seed": df_append}, cube=cube, store=function_store,) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 2, 1: 2, 2: 1, 3: 3} - part_chunk_size = {0: None, 1: None, 2: None, 3: None} - - assert len(dataset.partitions) == 4 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_compression_is_compatible_on_append_cube(driver, function_store): - """ - Test that partitons written with different compression algorithms are compatible - - The compression algorithms are not parametrized because their availability depends - on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts - of the code. A fully parametrized test would also increase runtime and test complexity - unnecessarily. - """ - # Build cube - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="SNAPPY"), - ) - - # Append to cube - df_append = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], - ) - result = driver( - data={"seed": df_append}, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="GZIP"), - ) - dataset = result["seed"].load_all_indices(function_store()) - - assert len(dataset.partitions) == 4 - - -def test_append_partitions_no_ts(driver, function_store): - df_source1 = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [10, 11, 12, 13], - } - ) - df_enrich1 = pd.DataFrame( - {"x": [0, 1, 2, 3], "v2": [10, 11, 12, 13], "i2": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data={"source": df_source1, "enrich": df_enrich1}, - cube=cube, - store=function_store, - metadata={"source": {"a": 10, "b": 11}, "enrich": {"a": 20, "b": 21}}, - partition_on={"enrich": []}, - ) - - partitions_source_1 = set( - DatasetMetadata.load_from_store( - cube.ktk_dataset_uuid("source"), function_store() - ).partitions.keys() - ) - partitions_enrich_1 = set( - DatasetMetadata.load_from_store( - cube.ktk_dataset_uuid("enrich"), function_store() - ).partitions.keys() - ) - - df_source2 = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [20, 21, 22, 23], - "i1": [20, 21, 22, 23], - } - ) - df_enrich2 = pd.DataFrame( - {"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23], "i2": [20, 21, 22, 23]} - ) - - result = driver( - data={"source": df_source2, "enrich": df_enrich2}, - cube=cube, - store=function_store, - ) - - assert set(result.keys()) == {"source", "enrich"} - - ds_source = result["source"] - ds_enrich = result["enrich"] - - partitions_source_2 = set(ds_source.partitions.keys()) - partitions_enrich_2 = set(ds_enrich.partitions.keys()) - - assert len(partitions_source_2) > len(partitions_source_1) - assert partitions_source_1.issubset(partitions_source_2) - - assert len(partitions_enrich_2) > len(partitions_enrich_1) - assert partitions_enrich_1.issubset(partitions_enrich_2) - - -def test_indices(driver, function_store, existing_cube): - idx1_1 = set( - DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("source"), function_store() - ) - .load_all_indices(function_store()) - .indices["i1"] - .index_dct.keys() - ) - idx2_1 = set( - DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("enrich"), function_store() - ) - .load_all_indices(function_store()) - .indices["i2"] - .index_dct.keys() - ) - - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [20, 21, 22, 23], - "i1": [20, 21, 22, 23], - } - ) - - result = driver( - data={"source": df_source}, cube=existing_cube, store=function_store - ) - - assert set(result.keys()) == {"source"} - - ds_source = result["source"] - ds_enrich = DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("enrich"), function_store() - ) - - idx1_2 = set( - ds_source.load_all_indices(function_store()).indices["i1"].index_dct.keys() - ) - idx2_2 = set( - ds_enrich.load_all_indices(function_store()).indices["i2"].index_dct.keys() - ) - - assert idx1_1.issubset(idx1_2) - assert len(idx1_1) < len(idx1_2) - - assert idx2_1 == idx2_2 - - -def test_fails_incompatible_dtypes(driver, function_store, existing_cube): - """ - Should also cross check w/ seed dataset. - """ - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10.0, 11.0, 12.0, 13.0], - "i1": [10, 11, 12, 13], - } - ) - - with pytest.raises(ValueError, match="Schema violation"): - driver(data={"source": df_source}, cube=existing_cube, store=function_store) - - -def test_fails_missing_column(driver, function_store, existing_cube): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "i1": [10, 11, 12, 13]} - ) - - with pytest.raises(ValueError, match="Schema violation"): - driver(data={"source": df_source}, cube=existing_cube, store=function_store) - - -def test_fails_unknown_dataset(driver, function_store, existing_cube): - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [10, 11, 12, 13], - } - ) - df_zoo = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v3": [10, 11, 12, 13], - "i3": [10, 11, 12, 13], - } - ) - - keys_pre = set(function_store().keys()) - - with pytest.raises(ValueError, match="Unknown / non-existing datasets: zoo"): - driver( - data={"source": df_source, "zoo": df_zoo}, - cube=existing_cube, - store=function_store, - ) - - keys_post = set(function_store().keys()) - assert keys_pre == keys_post - - -def test_metadata(driver, function_store, existing_cube): - """ - Test auto- and user-generated metadata. - """ - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [20, 21, 22, 23], - "i1": [20, 21, 22, 23], - } - ) - - result = driver( - data={"source": df_source}, - cube=existing_cube, - store=function_store, - metadata={"source": {"a": 12, "c": 13}}, - ) - - assert set(result.keys()) == {"source"} - - ds_source = result["source"] - assert set(ds_source.metadata.keys()) == { - "a", - "b", - "c", - "creation_time", - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, - } - assert ds_source.metadata["a"] == 12 - assert ds_source.metadata["b"] == 11 - assert ds_source.metadata["c"] == 13 - assert ds_source.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( - existing_cube.dimension_columns - ) - assert ds_source.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is True - assert ds_source.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( - existing_cube.partition_columns - ) - assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] - - ds_enrich = DatasetMetadata.load_from_store( - existing_cube.ktk_dataset_uuid("enrich"), function_store() - ) - assert set(ds_enrich.metadata.keys()) == { - "a", - "b", - "creation_time", - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, - } - assert ds_enrich.metadata["a"] == 20 - assert ds_enrich.metadata["b"] == 21 - assert ds_enrich.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( - existing_cube.dimension_columns - ) - assert ds_enrich.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is False - assert ds_enrich.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( - existing_cube.partition_columns - ) - assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] diff --git a/kartothek/io/testing/build_cube.py b/kartothek/io/testing/build_cube.py deleted file mode 100644 index e8b8b620..00000000 --- a/kartothek/io/testing/build_cube.py +++ /dev/null @@ -1,1456 +0,0 @@ -import numpy as np -import pandas as pd -import pandas.testing as pdt -import pyarrow as pa -import pytest -from pandas.arrays import SparseArray - -from kartothek.core.cube.constants import ( - KTK_CUBE_DF_SERIALIZER, - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, -) -from kartothek.core.cube.cube import Cube -from kartothek.core.dataset import DatasetMetadata -from kartothek.core.index import ExplicitSecondaryIndex, PartitionIndex -from kartothek.io.testing.utils import assert_num_row_groups -from kartothek.io_components.cube.write import MultiTableCommitAborted -from kartothek.io_components.metapartition import SINGLE_TABLE -from kartothek.serialization._parquet import ParquetSerializer - -__all__ = ( - "test_accept_projected_duplicates", - "test_distinct_branches", - "test_do_not_modify_df", - "test_empty_df", - "test_fail_all_empty", - "test_fail_duplicates_global", - "test_fail_duplicates_local", - "test_fail_no_store_factory", - "test_fail_nondistinc_payload", - "test_fail_not_a_df", - "test_fail_partial_build", - "test_fail_partial_overwrite", - "test_fail_partition_on_1", - "test_fail_partition_on_3", - "test_fail_partition_on_4", - "test_fail_partition_on_nondistinc_payload", - "test_fail_sparse", - "test_fail_wrong_dataset_ids", - "test_fail_wrong_types", - "test_fails_duplicate_columns", - "test_fails_metadata_nested_wrong_type", - "test_fails_metadata_unknown_id", - "test_fails_metadata_wrong_type", - "test_fails_missing_dimension_columns", - "test_fails_missing_partition_columns", - "test_fails_missing_seed", - "test_fails_no_dimension_columns", - "test_fails_null_dimension", - "test_fails_null_index", - "test_fails_null_partition", - "test_fails_projected_duplicates", - "test_indices", - "test_metadata", - "test_nones", - "test_overwrite", - "test_overwrite_rollback_ktk_cube", - "test_parquet", - "test_partition_on_enrich_extra", - "test_partition_on_enrich_none", - "test_partition_on_index_column", - "test_projected_data", - "test_regression_pseudo_duplicates", - "test_rowgroups_are_applied_when_df_serializer_is_passed_to_build_cube", - "test_simple_seed_only", - "test_simple_two_datasets", - "test_single_rowgroup_when_df_serializer_is_not_passed_to_build_cube", - "test_split", -) - - -def test_simple_seed_only(driver, function_store): - """ - Simple integration test w/ a seed dataset only. This is the most simple way to create a cube. - """ - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - result = driver(data=df, cube=cube, store=function_store) - - assert set(result.keys()) == {cube.seed_dataset} - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert ds.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert len(ds.partitions) == 2 - - assert set(ds.indices.keys()) == {"p", "x"} - assert isinstance(ds.indices["p"], PartitionIndex) - assert isinstance(ds.indices["x"], ExplicitSecondaryIndex) - - assert ds.table_name == SINGLE_TABLE - - -def test_simple_two_datasets(driver, function_store): - """ - Simple intergration test w/ 2 datasets. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 2 - - assert set(ds_source.indices.keys()) == {"p", "x"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - - assert set(ds_enrich.indices.keys()) == {"p"} - assert isinstance(ds_enrich.indices["p"], PartitionIndex) - - assert ds_source.table_name == SINGLE_TABLE - assert ds_enrich.table_name == SINGLE_TABLE - - -def test_indices(driver, function_store): - """ - Test that index structures are created correctly. - """ - df_source = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [100, 101, 102, 103], - } - ) - df_enrich = pd.DataFrame( - { - "x": [0, 1, 4, 5], - "p": [0, 0, 2, 2], - "v2": [20, 21, 22, 23], - "i2": [200, 201, 202, 203], - } - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2"], - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert set(ds_source.indices.keys()) == {"p", "x", "i1"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - assert isinstance(ds_source.indices["i1"], ExplicitSecondaryIndex) - - assert set(ds_enrich.indices.keys()) == {"p", "i2"} - assert isinstance(ds_enrich.indices["p"], PartitionIndex) - assert isinstance(ds_enrich.indices["i2"], ExplicitSecondaryIndex) - - -def test_dimension_index_suppression(driver, function_store): - """ - Test that suppress_index_on works as expected - """ - df_source = pd.DataFrame( - { - "x": [0, 0, 1, 1], - "y": [10, 11, 12, 13], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [100, 101, 102, 103], - } - ) - cube = Cube( - dimension_columns=["x", "y"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2"], - suppress_index_on=["x"], - ) - result = driver(data={"source": df_source}, cube=cube, store=function_store) - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - - assert set(ds_source.indices.keys()) == {"p", "i1", "y"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["i1"], ExplicitSecondaryIndex) - assert isinstance(ds_source.indices["y"], ExplicitSecondaryIndex) - - -def test_do_not_modify_df(driver, function_store): - """ - Functions should not modify their inputs. - """ - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - df_backup = df.copy() - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - driver(data=df, cube=cube, store=function_store) - - pdt.assert_frame_equal(df, df_backup) - - -@pytest.mark.filterwarnings("ignore::UnicodeWarning") -def test_parquet(driver, function_store): - """ - Ensure the parquet files we generate are properly normalized. - """ - df = pd.DataFrame( - data={ - "x": [10, 1, 1, 0, 0], - "y": [10, 0, 1, 1, 0], - "p": [0, 1, 1, 1, 1], - "föö".encode("utf8"): [100, 10, 11, 12, 13], - "v": np.nan, - }, - index=[0, 1000, 1001, 1002, 1003], - columns=["x", "y", "p", "föö".encode("utf8"), "v"], - ) - - cube = Cube( - dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" - ) - result = driver(data=df, cube=cube, store=function_store) - - assert set(result.keys()) == {cube.seed_dataset} - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert len(ds.partitions) == 2 - for p in (0, 1): - part_key = ds.indices["p"].index_dct[p][0] - part = ds.partitions[part_key] - key = part.files[SINGLE_TABLE] - - df_actual = KTK_CUBE_DF_SERIALIZER.restore_dataframe(function_store(), key) - df_expected = ( - df.loc[df["p"] == p] - .sort_values(["x", "y"]) - .reset_index(drop=True) - .drop(columns=["p"]) - .rename(columns={"föö".encode("utf8"): "föö"}) - ) - - pdt.assert_frame_equal(df_actual.reset_index(drop=True), df_expected) - - -@pytest.mark.parametrize("chunk_size", [None, 2]) -def test_rowgroups_are_applied_when_df_serializer_is_passed_to_build_cube( - driver, function_store, chunk_size -): - """ - Test that the dataset is split into row groups depending on the chunk size - """ - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"],) - - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - result = driver( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size), - ) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 3} - part_chunk_size = {0: chunk_size, 1: chunk_size} - - assert len(dataset.partitions) == 2 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_single_rowgroup_when_df_serializer_is_not_passed_to_build_cube( - driver, function_store -): - """ - Test that the dataset has a single row group as default path - """ - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"],) - - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - result = driver(data=df, cube=cube, store=function_store,) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 3} - part_chunk_size = {0: None, 1: None} - - assert len(dataset.partitions) == 2 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_fail_sparse(driver, driver_name, function_store): - """ - Ensure that sparse dataframes are rejected. - """ - df = pd.DataFrame( - data={ - "x": SparseArray([10, 1, 1, 0, 0]), - "y": SparseArray([10, 0, 1, 1, 0]), - "p": SparseArray([0, 1, 1, 1, 1]), - "v": SparseArray([np.nan] * 5), - } - ) - - cube = Cube( - dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" - ) - with pytest.raises(TypeError, match="Sparse data is not supported."): - driver(data=df, cube=cube, store=function_store) - - -def test_metadata(driver, function_store): - """ - Test auto- and user-generated metadata. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 4, 5], "p": [0, 0, 2, 2], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - metadata={"enrich": {"foo": 1}}, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset] - assert set(ds_source.metadata.keys()) == { - "creation_time", - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, - } - assert ds_source.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( - cube.dimension_columns - ) - assert ds_source.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is True - assert ds_source.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( - cube.partition_columns - ) - assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] - - ds_enrich = result["enrich"] - assert set(ds_enrich.metadata.keys()) == { - "creation_time", - KTK_CUBE_METADATA_DIMENSION_COLUMNS, - KTK_CUBE_METADATA_KEY_IS_SEED, - KTK_CUBE_METADATA_PARTITION_COLUMNS, - KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, - "foo", - } - assert ds_enrich.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( - cube.dimension_columns - ) - assert ds_enrich.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is False - assert ds_enrich.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( - cube.partition_columns - ) - assert ds_enrich.metadata["foo"] == 1 - assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] - - -def test_fails_metadata_wrong_type(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises( - TypeError, match="Provided metadata should be a dict but is int" - ): - driver(data={"source": df_source}, cube=cube, store=function_store, metadata=1) - - -def test_fails_metadata_unknown_id(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises( - ValueError, - match="Provided metadata for otherwise unspecified ktk_cube_dataset_ids: bar, foo", - ): - driver( - data={"source": df_source}, - cube=cube, - store=function_store, - metadata={"source": {}, "foo": {}, "bar": {}}, - ) - - -def test_fails_metadata_nested_wrong_type(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises( - TypeError, - match="Provided metadata for dataset source should be a dict but is int", - ): - driver( - data={"source": df_source}, - cube=cube, - store=function_store, - metadata={"source": 1}, - ) - - -def test_fails_missing_seed(driver, function_store): - """ - A cube must contain its seed dataset, check this constraint as early as possible. - """ - df = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v": [10, 11]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - with pytest.raises(ValueError) as exc: - driver(data={"foo": df}, cube=cube, store=function_store) - assert 'Seed data ("seed") is missing.' in str(exc.value) - assert list(function_store().keys()) == [] - - -def test_fails_missing_dimension_columns(driver, function_store): - """ - Ensure that we catch missing dimension columns early. - """ - df_source = pd.DataFrame({"x": [0, 1], "p": 0}) - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(ValueError) as exc: - driver(data=df_source, cube=cube, store=function_store) - assert 'Missing dimension columns in seed data "source": y, z' in str(exc.value) - assert list(function_store().keys()) == [] - - -def test_fails_no_dimension_columns(driver, function_store): - """ - Ensure that we catch missing dimension columns early. - """ - df_source = pd.DataFrame({"x": [0, 1], "y": [0, 1], "z": [0, 1], "p": 0}) - df_enrich = pd.DataFrame({"p": [0], "v1": 0}) - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(ValueError) as exc: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - assert ( - 'Dataset "enrich" must have at least 1 of the following dimension columns: x, y' - in str(exc.value) - ) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_fails_duplicate_columns(driver, function_store, driver_name): - """ - Catch weird pandas behavior. - """ - if driver_name == "dask_dataframe": - pytest.skip("already detected by dask.dataframe") - - df = pd.DataFrame( - {"x": [0, 1], "p": 0, "a": 1, "b": 2}, columns=["x", "p", "a", "b"] - ).rename(columns={"b": "a"}) - assert len(df.columns) == 4 - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - assert 'Duplicate columns found in dataset "seed": x, p, a, a' in str(exc.value) - assert list(function_store().keys()) == [] - - -def test_fails_missing_partition_columns(driver, function_store): - """ - Just make the Kartothek error nicer. - """ - df = pd.DataFrame({"x": [0, 1], "p": 0}) - cube = Cube( - dimension_columns=["x"], partition_columns=["p", "q", "r"], uuid_prefix="cube" - ) - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - assert 'Missing partition columns in dataset "seed": q, r' in str(exc.value) - assert list(function_store().keys()) == [] - - -def test_overwrite(driver, function_store): - """ - Test overwrite behavior aka call the build function if the cube already exists. - """ - df1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v": [10, 11]}) - df2 = pd.DataFrame({"x": [2, 3], "p": [1, 1], "v": [12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - driver(data=df1, cube=cube, store=function_store) - - # implicit overwrite fails - keys = set(function_store().keys()) - with pytest.raises(RuntimeError) as exc: - driver(data=df1, cube=cube, store=function_store) - assert "already exists and overwrite is not permitted" in str(exc.value) - assert set(function_store().keys()) == keys - - # explicit overwrite works - result = driver(data=df2, cube=cube, store=function_store, overwrite=True) - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert len(ds.partitions) == 1 - - assert set(ds.indices["p"].index_dct.keys()) == {1} - - -def test_split(driver, function_store): - """ - Imagine the user already splits the data. - """ - df_source1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v1": [10, 11]}) - df_source2 = pd.DataFrame({"x": [2, 3], "p": [1, 1], "v1": [12, 13]}) - df_enrich = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v2": [20, 21]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data=[{"source": df_source1, "enrich": df_enrich}, df_source2], - cube=cube, - store=function_store, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 1 - - -@pytest.mark.parametrize("empty_first", [False, True]) -def test_empty_df(driver, function_store, empty_first): - """ - Might happen during DB queries. - """ - df1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v1": [10, 11]}) - df2 = df1.loc[[]] - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data=[df2, df1] if empty_first else [df1, df2], cube=cube, store=function_store - ) - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert len(ds.partitions) == 1 - assert ( - len(list(function_store().keys())) == 4 - ) # DS metadata, "x" index, common metadata, 1 partition - - -def test_fail_duplicates_local(driver, driver_name, function_store): - """ - Might happen during DB queries. - """ - if driver_name == "dask_dataframe": - pytest.xfail(reason="Cannot guarantee duplicates for DDF") - df = pd.DataFrame( - { - "x": [0, 0], - "y": ["a", "a"], - "z": [pd.Timestamp("2017"), pd.Timestamp("2017")], - "p": [0, 0], - } - ) - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - msg = """ -Found duplicate cells by [p, x, y, z] in dataset "source", example: - -Keys: -p 0 -x 0 -y a -z 2017-01-01 00:00:00 - -Identical Payload: -n/a - -Non-Idential Payload: -n/a -""".strip() - assert msg in str(exc.value) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_accept_projected_duplicates(driver, function_store): - """ - Otherwise partitioning does not work w/ projected data. - """ - df_seed = pd.DataFrame({"x": [0, 1, 0, 1], "y": [0, 0, 1, 1], "p": [0, 0, 1, 1]}) - df_enrich = pd.DataFrame({"x": [0, 1, 0, 1], "p": [0, 0, 1, 1]}) - cube = Cube( - dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" - ) - driver(data={"seed": df_seed, "enrich": df_enrich}, cube=cube, store=function_store) - - -@pytest.mark.xfail( - strict=True, reason="Cannot be checked with current index structures." -) -def test_fail_duplicates_global(driver_name, driver, function_store): - """ - Might happen due to bugs. - """ - if driver_name == "eager": - pytest.skip(reason="Problem does not occur in eager mode.") - - df1 = pd.DataFrame({"x": [0], "y": ["a"], "z": [pd.Timestamp("2017")], "p": [0]}) - df2 = pd.DataFrame({"x": [0], "y": ["a"], "z": [pd.Timestamp("2017")], "p": [1]}) - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(ValueError): - driver(data=[df1, df2], cube=cube, store=function_store) - - -def test_regression_pseudo_duplicates(driver, function_store): - """ - Might happen due to bugs. - """ - df = pd.DataFrame({"x": [0, 0, 2, 3], "y": [0, 1, 2, 2], "p": [0, 1, 0, 1]}) - cube = Cube( - dimension_columns=["x", "y"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - driver(data=df, cube=cube, store=function_store) - - -def test_fail_wrong_types(driver, function_store): - """ - Might catch nasty pandas and other type bugs. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0.0, 1.0, 2.0, 3.0], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - - cause = exc_info.value.__cause__ - assert isinstance(cause, ValueError) - assert 'Found incompatible entries for column "x"' in str(cause) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_distinct_branches(driver, function_store): - """ - Just check this actually works. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data=[{"source": df_source}, {"enrich": df_enrich}], - cube=cube, - store=function_store, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 2 - - -def test_fail_nondistinc_payload(driver, function_store): - """ - This would lead to problems during the query phase. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - cause = exc_info.value.__cause__ - assert isinstance(cause, ValueError) - assert "Found columns present in multiple datasets" in str(cause) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_fail_partial_overwrite(driver, function_store): - """ - Either overwrite all or no datasets. - """ - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - - df_source1 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich1 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - driver( - data={"source": df_source1, "enrich": df_enrich1}, - cube=cube, - store=function_store, - ) - - keys = set(function_store().keys()) - df_source2 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} - ) - with pytest.raises(ValueError) as exc: - driver( - data={"source": df_source2}, cube=cube, store=function_store, overwrite=True - ) - assert ( - str(exc.value) - == "Following datasets exists but are not overwritten (partial overwrite), this is not allowed: enrich" - ) - assert set(function_store().keys()) == keys - - -def test_fail_partial_build(driver, function_store): - """ - Either overwrite all or no datasets. - """ - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - - df_source1 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich1 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - driver( - data={"source": df_source1, "enrich": df_enrich1}, - cube=cube, - store=function_store, - ) - - # delete everything that belongs to the seed dataset - to_delete = { - k - for k in function_store().keys() - if k.startswith(cube.ktk_dataset_uuid(cube.seed_dataset)) - } - for k in to_delete: - function_store().delete(k) - - keys = set(function_store().keys()) - df_source2 = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} - ) - with pytest.raises(ValueError) as exc: - driver(data={"source": df_source2}, cube=cube, store=function_store) - assert ( - str(exc.value) - == "Following datasets exists but are not overwritten (partial overwrite), this is not allowed: enrich" - ) - assert set(function_store().keys()) == keys - - -def test_fails_projected_duplicates(driver, driver_name, function_store): - """ - Test if duplicate check also works w/ projected data. (was a regression) - """ - if driver_name == "dask_dataframe": - pytest.xfail(reason="Cannot guarantee duplicates for DDF") - df_source = pd.DataFrame( - { - "x": [0, 1, 0, 1], - "y": [0, 0, 1, 1], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - } - ) - df_enrich = pd.DataFrame( - {"y": [0, 0, 1, 1], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23], "v3": 42} - ) - cube = Cube( - dimension_columns=["x", "y"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(ValueError) as exc: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - msg = """ -Found duplicate cells by [p, y] in dataset "enrich", example: - -Keys: -p 0 -y 0 - -Identical Payload: -v3 42 - -Non-Idential Payload: - v2 -0 20 -1 21 -""".strip() - assert msg in str(exc.value) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_projected_data(driver, function_store): - """ - Projected dataset (useful for de-duplication). - """ - df_source = pd.DataFrame( - { - "x": [0, 1, 0, 1], - "y": [0, 0, 1, 1], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - } - ) - df_enrich = pd.DataFrame({"y": [0, 1], "p": [0, 1], "v2": [20, 21]}) - cube = Cube( - dimension_columns=["x", "y"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 2 - - -def test_fails_null_dimension(driver, function_store): - """ - Since we do not allow NULL values in queries, it should be banned from dimension columns in the first place. - """ - df = pd.DataFrame( - {"x": [0, 1, 2, np.nan], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - - assert 'Found NULL-values in dimension column "x" of dataset "seed"' in str(exc) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) - - -def test_fails_null_partition(driver, function_store): - """ - Since we do not allow NULL values in queries, it should be banned from partition columns in the first place. - """ - df = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, np.nan], "v": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - assert 'Found NULL-values in partition column "p" of dataset "seed"' in str( - exc.value - ) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) - - -def test_fails_null_index(driver, function_store): - """ - Since we do not allow NULL values in queries, it should be banned from index columns in the first place. - """ - df = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v": [10, 11, 12, 13], - "i1": [0, 1, 2, np.nan], - } - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - index_columns=["i1"], - ) - with pytest.raises(ValueError) as exc: - driver(data=df, cube=cube, store=function_store) - assert 'Found NULL-values in index column "i1"' in str(exc.value) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) - - -@pytest.mark.xfail(reason="different") -def test_fail_all_empty(driver, driver_name, function_store): - """ - Might happen due to DB-based filters. - """ - df = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} - ).loc[[]] - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver(data=df, cube=cube, store=function_store) - exc = exc_info.value.__cause__ - assert isinstance(exc, ValueError) - assert "Cannot write empty datasets" in str(exc) - - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_overwrite_rollback_ktk_cube(driver, function_store): - """ - Checks that require a rollback (like overlapping columns) should recover the former state correctly. - """ - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2", "i3", "i4"], - ) - - df_source1 = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "i1": [10, 11, 12, 13], - } - ) - df_enrich1 = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "i2": [20, 21, 22, 23], - "v2": [20, 21, 22, 23], - } - ) - driver( - data={"source": df_source1, "enrich": df_enrich1}, - cube=cube, - store=function_store, - ) - - df_source2 = pd.DataFrame( - { - "x": [10, 11], - "p": [10, 10], - "v1": [10.0, 11.0], # also use another dtype here (was int) - "i3": [10, 11], - } - ) - df_enrich2 = pd.DataFrame( - {"x": [10, 11], "p": [10, 10], "v1": [20, 21], "i4": [20, 21]} - ) - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver( - data={"source": df_source2, "enrich": df_enrich2}, - cube=cube, - store=function_store, - overwrite=True, - ) - cause = exc_info.value.__cause__ - assert isinstance(cause, ValueError) - assert str(cause).startswith("Found columns present in multiple datasets:") - - ds_source = DatasetMetadata.load_from_store( - uuid=cube.ktk_dataset_uuid("source"), store=function_store() - ).load_all_indices(function_store()) - ds_enrich = DatasetMetadata.load_from_store( - uuid=cube.ktk_dataset_uuid("enrich"), store=function_store() - ).load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 2 - - assert set(ds_source.indices.keys()) == {"p", "x", "i1"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - assert set(ds_source.indices["x"].index_dct.keys()) == {0, 1, 2, 3} - assert set(ds_source.indices["i1"].index_dct.keys()) == {10, 11, 12, 13} - - assert set(ds_enrich.indices.keys()) == {"p", "i2"} - assert isinstance(ds_enrich.indices["p"], PartitionIndex) - assert set(ds_enrich.indices["i2"].index_dct.keys()) == {20, 21, 22, 23} - - assert ds_source.schema.field("v1").type == pa.int64() - - -@pytest.mark.parametrize("none_first", [False, True]) -def test_nones(driver, function_store, none_first, driver_name): - """ - Test what happens if user passes None to ktk_cube. - """ - if driver_name == "dask_dataframe": - pytest.skip("user cannot create None-partitions with dask.dataframe") - - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - result = driver( - data=[None, df] if none_first else [df, None], cube=cube, store=function_store - ) - - assert set(result.keys()) == {cube.seed_dataset} - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert ds.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert len(ds.partitions) == 2 - - assert set(ds.indices.keys()) == {"p", "x"} - assert isinstance(ds.indices["p"], PartitionIndex) - assert isinstance(ds.indices["x"], ExplicitSecondaryIndex) - - assert ds.table_name == SINGLE_TABLE - - -def test_fail_not_a_df(driver, function_store): - """ - Pass some weird objects in. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(TypeError) as exc: - driver( - data={"source": df_source, "enrich": pd.Series(range(10))}, - cube=cube, - store=function_store, - ) - assert ( - 'Provided DataFrame is not a pandas.DataFrame or None, but is a "Series"' - in str(exc.value) - ) - - -def test_fail_wrong_dataset_ids(driver, function_store, skip_eager, driver_name): - if driver_name == "dask_dataframe": - pytest.skip("not an interface for dask.dataframe") - - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - - with pytest.raises(ValueError) as exc: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - ktk_cube_dataset_ids=["source", "extra"], - ) - - assert ( - 'Ktk_cube Dataset ID "enrich" is present during pipeline execution but was not ' - "specified in ktk_cube_dataset_ids (extra, source)." in str(exc.value) - ) - - -def test_fail_no_store_factory(driver, function_store, skip_eager): - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - store = function_store() - with pytest.raises(TypeError) as exc: - driver(data=df, cube=cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_partition_on_1(driver, function_store): - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube( - dimension_columns=["x"], partition_columns=["p", "q"], uuid_prefix="cube" - ) - - with pytest.raises(ValueError) as exc_info: - driver( - data=df, - cube=cube, - store=function_store, - partition_on={cube.seed_dataset: ["x", "p"]}, - ) - - cause = exc_info.value # .__cause__ - assert isinstance(cause, ValueError) - assert ( - "Seed dataset seed must have the following, fixed partition-on attribute: p, q" - in str(cause) - ) - - assert set(function_store().keys()) == set() - - -def test_fail_partition_on_3(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - - with pytest.raises( - ValueError, - match="partition-on attribute of dataset enrich contains duplicates: p, p", - ): - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": ["p", "p"]}, - ) - - assert set(function_store().keys()) == set() - - -def test_fail_partition_on_4(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - - with pytest.raises( - ValueError, match="Unspecified but provided partition columns in enrich: p" - ): - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": []}, - ) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) - - -def test_partition_on_enrich_none(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": []}, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 1 - - assert set(ds_source.indices.keys()) == {"p", "x"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - - assert set(ds_enrich.indices.keys()) == set() - - assert ds_source.table_name == SINGLE_TABLE - assert ds_enrich.table_name == SINGLE_TABLE - - -def test_partition_on_enrich_extra(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": ["p", "x"]}, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 4 - - assert set(ds_source.indices.keys()) == {"p", "x"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - - assert set(ds_enrich.indices.keys()) == {"p", "x"} - assert isinstance(ds_enrich.indices["p"], PartitionIndex) - assert isinstance(ds_enrich.indices["x"], PartitionIndex) - - assert ds_source.table_name == SINGLE_TABLE - assert ds_enrich.table_name == SINGLE_TABLE - - -def test_partition_on_index_column(driver, function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "i": [0, 0, 1, 2], "v2": [20, 21, 22, 23]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - index_columns=["i"], - uuid_prefix="cube", - seed_dataset="source", - ) - result = driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": ["i"]}, - ) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - - ds_source = result[cube.seed_dataset].load_all_indices(function_store()) - ds_enrich = result["enrich"].load_all_indices(function_store()) - - assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) - assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") - - assert len(ds_source.partitions) == 2 - assert len(ds_enrich.partitions) == 3 - - assert set(ds_source.indices.keys()) == {"p", "x"} - assert isinstance(ds_source.indices["p"], PartitionIndex) - assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) - - assert set(ds_enrich.indices.keys()) == {"i"} - assert isinstance(ds_enrich.indices["i"], PartitionIndex) - - assert ds_source.table_name == SINGLE_TABLE - assert ds_enrich.table_name == SINGLE_TABLE - - -def test_fail_partition_on_nondistinc_payload(driver, function_store): - """ - This would lead to problems during the query phase. - """ - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v1": [20, 21, 22, 23]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - ) - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver( - data={"source": df_source, "enrich": df_enrich}, - cube=cube, - store=function_store, - partition_on={"enrich": ["v1"]}, - ) - cause = exc_info.value.__cause__ - assert isinstance(cause, ValueError) - assert "Found columns present in multiple datasets" in str(cause) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) - assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) diff --git a/kartothek/io/testing/cleanup_cube.py b/kartothek/io/testing/cleanup_cube.py deleted file mode 100644 index 6c97ff4e..00000000 --- a/kartothek/io/testing/cleanup_cube.py +++ /dev/null @@ -1,229 +0,0 @@ -# -*- coding: utf-8 -*- -import pandas as pd -import pytest - -from kartothek.core.cube.constants import KTK_CUBE_UUID_SEPARATOR -from kartothek.core.cube.cube import Cube -from kartothek.io.eager_cube import build_cube, copy_cube - -__all__ = ( - "test_additional_files", - "test_delete_by_correct_uuid", - "test_fail_blocksize_negative", - "test_fail_blocksize_wrong_type", - "test_fail_blocksize_zero", - "test_fails_no_store_factory", - "test_missing_cube_files", - "test_missing_metadata", - "test_missing_seed_dataset", - "test_noop", - "test_overwrite_check_with_copy", -) - - -def test_delete_by_correct_uuid(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube_foo = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="foo") - build_cube( - data={cube_foo.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube_foo, - store=function_store, - ) - - cube_foo_bar = Cube( - dimension_columns=["x"], partition_columns=["p"], uuid_prefix="foo_bar" - ) - build_cube( - data={cube_foo_bar.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube_foo_bar, - store=function_store, - ) - store = function_store() - foo_bar_keys = {k for k in store.keys() if "foo_bar" in k} - store.delete("foo++seed.by-dataset-metadata.json") - store.delete("foo++enrich.by-dataset-metadata.json") - - driver(cube=cube_foo, store=function_store) - assert foo_bar_keys == set(store.keys()) - - -def test_missing_seed_dataset(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - store = function_store() - seed_keys = {k for k in store.keys() if "cube++seed" in k and "/" in k} - enrich_keys = {k for k in store.keys() if "cube++enrich" in k} - - for k in seed_keys: - store.delete(k) - - driver(cube=cube, store=function_store) - - assert enrich_keys == set(store.keys()) - - -def test_missing_cube_files(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - store = function_store() - enrich_keys = {k for k in store.keys() if "cube++enrich" in k and "/" in k} - for k in enrich_keys: - store.delete(k) - - driver(cube=cube, store=function_store) - - assert "cube++enrich.by-dataset-metadata.json" not in store.keys() - - -def test_missing_metadata(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - store = function_store() - enrich_keys = {k for k in store.keys() if "cube++enrich" in k} - - store.delete("cube++enrich.by-dataset-metadata.json") - - driver(cube=cube, store=function_store) - - assert not enrich_keys.intersection(store.keys()) - - -def test_noop(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - - keys = set(function_store().keys()) - - driver(cube=cube, store=function_store) - - assert set(function_store().keys()) == keys - - -def test_overwrite_check_with_copy(driver, function_store, function_store2): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - # build twice - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - overwrite=True, - ) - - # copy to another store to detect keys - copy_cube(cube=cube, src_store=function_store, tgt_store=function_store2) - keys = set(function_store2().keys()) - - assert set(function_store().keys()) != keys - driver(cube=cube, store=function_store) - assert set(function_store().keys()) == keys - - -def test_additional_files(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube(data=df_seed, cube=cube, store=function_store) - - key_in_ds = cube.ktk_dataset_uuid(cube.seed_dataset) + "/foo" - key_with_ds_prefix = cube.ktk_dataset_uuid(cube.seed_dataset) + ".foo" - key_with_cube_prefix = cube.uuid_prefix + ".foo" - key_with_cube_prefix_separator = cube.uuid_prefix + KTK_CUBE_UUID_SEPARATOR + ".foo" - - function_store().put(key_in_ds, b"") - function_store().put(key_with_ds_prefix, b"") - function_store().put(key_with_cube_prefix, b"") - function_store().put(key_with_cube_prefix_separator, b"") - - driver(cube=cube, store=function_store) - assert key_in_ds not in set(function_store().keys()) - assert key_with_ds_prefix not in set(function_store().keys()) - assert key_with_cube_prefix in set(function_store().keys()) - assert key_with_cube_prefix_separator not in set(function_store().keys()) - - -def test_fails_no_store_factory(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - store = function_store() - with pytest.raises(TypeError) as exc: - driver(cube=cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(TypeError, match="blocksize must be an integer but is str"): - driver(cube=cube, store=function_store, blocksize="foo") - - -def test_fail_blocksize_negative(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): - driver(cube=cube, store=function_store, blocksize=-1) - - -def test_fail_blocksize_zero(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): - driver(cube=cube, store=function_store, blocksize=0) diff --git a/kartothek/io/testing/copy_cube.py b/kartothek/io/testing/copy_cube.py deleted file mode 100644 index b813f99c..00000000 --- a/kartothek/io/testing/copy_cube.py +++ /dev/null @@ -1,388 +0,0 @@ -# -*- coding: utf-8 -*- -import pandas as pd -import pytest - -from kartothek.api.discover import discover_datasets_unchecked -from kartothek.core.cube.cube import Cube -from kartothek.io.eager_cube import build_cube -from kartothek.utils.ktk_adapters import get_dataset_keys - -__all__ = ( - "assert_same_keys", - "built_cube", - "cube", - "df_enrich", - "df_seed", - "simple_cube_1", - "simple_cube_2", - "test_fail_blocksize_negative", - "test_fail_blocksize_wrong_type", - "test_fail_blocksize_zero", - "test_fail_no_src_cube", - "test_fail_no_src_cube_dataset", - "test_fail_no_store_factory_src", - "test_fail_no_store_factory_tgt", - "test_fail_stores_identical_overwrite_false", - "test_ignore_other", - "test_invalid_partial_copy", - "test_invalid_partial_copy1", - "test_invalid_partial_copy2", - "test_overwrite_fail", - "test_overwrite_ok", - "test_partial_copy_dataset_dict", - "test_partial_copy_dataset_list", - "test_read_only_source", - "test_simple", -) - - -@pytest.fixture -def cube(): - return Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - -@pytest.fixture -def df_seed(): - return pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) - - -@pytest.fixture -def df_enrich(): - return pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) - - -@pytest.fixture -def built_cube(df_seed, df_enrich, cube, function_store): - return build_cube( - data={cube.seed_dataset: df_seed.copy(), "enrich": df_enrich.copy()}, - cube=cube, - store=function_store, - ) - - -@pytest.fixture -def simple_cube_1(function_store, built_cube): - return set(function_store().keys()) - - -@pytest.fixture -def simple_cube_2(df_seed, df_enrich, cube, function_store2): - build_cube(data={cube.seed_dataset: df_seed}, cube=cube, store=function_store2) - return set(function_store2().keys()) - - -def assert_same_keys(store1, store2, keys): - k1 = set(store1().keys()) - k2 = set(store2().keys()) - assert keys.issubset(k1) - assert keys.issubset(k2) - - for k in sorted(keys): - b1 = store1().get(k) - b2 = store1().get(k) - assert b1 == b2 - - -def test_simple(driver, function_store, function_store2, cube, simple_cube_1): - driver(cube=cube, src_store=function_store, tgt_store=function_store2) - assert_same_keys(function_store, function_store2, simple_cube_1) - - -def test_overwrite_fail( - driver, function_store, function_store2, cube, simple_cube_1, simple_cube_2 -): - assert simple_cube_1 != simple_cube_2 - - data_backup = {k: function_store2().get(k) for k in simple_cube_2} - - with pytest.raises(RuntimeError) as exc: - driver(cube=cube, src_store=function_store, tgt_store=function_store2) - assert ( - str(exc.value) - == 'Dataset "cube++seed" exists in target store but overwrite was set to False' - ) - - # check everything kept untouched - assert set(function_store2().keys()) == simple_cube_2 - for k in sorted(simple_cube_2): - assert function_store2().get(k) == data_backup[k] - - -def test_overwrite_ok( - driver, function_store, function_store2, cube, simple_cube_1, simple_cube_2 -): - driver( - cube=cube, src_store=function_store, tgt_store=function_store2, overwrite=True - ) - assert_same_keys(function_store, function_store2, simple_cube_1) - - -@pytest.mark.parametrize("overwrite", [False, True]) -def test_fail_stores_identical_overwrite_false( - driver, function_store, cube, built_cube, overwrite -): - with pytest.raises(ValueError) as exc: - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store, - overwrite=overwrite, - ) - assert str(exc.value) == "Stores are identical but should not be." - - -def test_ignore_other(driver, function_store, function_store2): - dfs = [] - cubes = [] - for i in range(3): - dfs.append( - pd.DataFrame( - { - "x{}".format(i): [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v{}".format(i): [10, 11, 12, 13], - } - ) - ) - - cubes.append( - Cube( - dimension_columns=["x{}".format(i)], - partition_columns=["p"], - uuid_prefix="cube{}".format(i), - ) - ) - - build_cube(data=dfs[0], cube=cubes[0], store=function_store) - build_cube(data=dfs[1], cube=cubes[1], store=function_store) - build_cube(data=dfs[2], cube=cubes[2], store=function_store2) - - keys_in_1 = set(function_store().keys()) - keys_in_2 = set(function_store2().keys()) - data_backup1 = {k: function_store().get(k) for k in keys_in_1} - data_backup2 = {k: function_store2().get(k) for k in keys_in_2} - - driver(cube=cubes[1], src_store=function_store, tgt_store=function_store2) - - # store 1 is untouched - assert set(function_store().keys()) == keys_in_1 - for k in sorted(keys_in_1): - assert function_store().get(k) == data_backup1[k] - - # store 2 is partly untouched - for k in sorted(keys_in_2): - assert function_store2().get(k) == data_backup2[k] - - # test new keys - keys_new = set(function_store2().keys()) - keys_in_2 - assert_same_keys(function_store, function_store2, keys_new) - - -def test_invalid_partial_copy1( - df_seed, df_enrich, cube, function_store, function_store2, simple_cube_2, driver -): - # build a cube that would be incompatible w/ simple_cube_2 - df_seed = df_seed.copy() - df_enrich = df_enrich.copy() - - df_seed["x"] = df_seed["x"].astype(str) - df_enrich["x"] = df_enrich["x"].astype(str) - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - - keys = set(function_store().keys()) - - # now copy simple_cube_2 over existing cube. - # this only copies the seed table since simple_cube_2 does not have an enrich table. - # it should fail because X is incompatible - with pytest.raises(ValueError) as exc: - driver( - cube=cube, - src_store=function_store2, - tgt_store=function_store, - overwrite=True, - ) - assert 'Found incompatible entries for column "x"' in str(exc.value) - assert keys == set(function_store().keys()) - - -def test_invalid_partial_copy2( - df_seed, df_enrich, cube, function_store, function_store2, simple_cube_1, driver -): - # build a cube that would be incompatible w/ simple_cube_1 - df_seed = df_seed.copy() - df_enrich = df_enrich.copy() - - df_seed["x"] = df_seed["x"].astype(str) - df_enrich["x"] = df_enrich["x"].astype(str) - build_cube( - data={cube.seed_dataset: df_seed, "enrich2": df_enrich}, - cube=cube, - store=function_store2, - ) - - keys = set(function_store2().keys()) - - # now copy simple_cube_1 over existing cube. - # this only copies the seed and enrich table since simple_cube_1 does not have an enrich2 table. - # it should fail because X is incompatible. - with pytest.raises(ValueError) as exc: - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - overwrite=True, - ) - assert "Found columns present in multiple datasets" in str(exc.value) - assert keys == set(function_store2().keys()) - - -def test_partial_copy_dataset_list( - driver, function_store, function_store2, cube, built_cube -): - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - datasets=["seed", "enrich"], - ) - all_datasets = discover_datasets_unchecked( - uuid_prefix=cube.uuid_prefix, - store=function_store, - filter_ktk_cube_dataset_ids=["seed", "enrich"], - ) - copied_ds_keys = set() - copied_ds_keys |= get_dataset_keys(all_datasets["seed"]) - copied_ds_keys |= get_dataset_keys(all_datasets["enrich"]) - tgt_store_keys = set(function_store2().keys()) - assert copied_ds_keys == tgt_store_keys - - -def test_partial_copy_dataset_dict( - driver, function_store, function_store2, cube, built_cube -): - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - datasets={"seed": built_cube["seed"], "enrich": built_cube["enrich"]}, - ) - all_datasets = discover_datasets_unchecked( - uuid_prefix=cube.uuid_prefix, - store=function_store, - filter_ktk_cube_dataset_ids=["seed", "enrich"], - ) - copied_ds_keys = set() - copied_ds_keys |= get_dataset_keys(all_datasets["seed"]) - copied_ds_keys |= get_dataset_keys(all_datasets["enrich"]) - tgt_store_keys = set(function_store2().keys()) - assert copied_ds_keys == tgt_store_keys - - -def test_invalid_partial_copy( - driver, df_seed, df_enrich, function_store, function_store2, cube, built_cube -): - # build a cube that would be incompatible with cube in function_store - df_seed = df_seed.copy() - df_enrich = df_enrich.copy() - df_seed["x"] = df_seed["x"].astype(str) - df_enrich["x"] = df_enrich["x"].astype(str) - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store2, - ) - tgt_store_key_before = set(function_store2().keys()) - with pytest.raises(ValueError) as exc: - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - overwrite=True, - datasets=["enrich"], - ) - assert 'Found incompatible entries for column "x"' in str(exc.value) - assert tgt_store_key_before == set(function_store2().keys()) - - -def test_fail_no_store_factory_src( - driver, function_store, function_store2, cube, skip_eager -): - store = function_store() - with pytest.raises(TypeError) as exc: - driver(cube=cube, src_store=store, tgt_store=function_store2, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_no_store_factory_tgt( - driver, function_store, function_store2, cube, skip_eager -): - store = function_store2() - with pytest.raises(TypeError) as exc: - driver(cube=cube, src_store=function_store, tgt_store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_no_src_cube(cube, function_store, function_store2, driver): - with pytest.raises(RuntimeError) as exc: - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - overwrite=False, - ) - assert "not found" in str(exc.value) - - -def test_fail_no_src_cube_dataset( - cube, built_cube, function_store, function_store2, driver -): - with pytest.raises(RuntimeError) as exc: - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - overwrite=False, - datasets=["non_existing"], - ) - assert "non_existing" in str(exc.value) - - -def test_read_only_source( - driver, function_store_ro, function_store2, cube, simple_cube_1 -): - driver(cube=cube, src_store=function_store_ro, tgt_store=function_store2) - assert_same_keys(function_store_ro, function_store2, simple_cube_1) - - -def test_fail_blocksize_wrong_type( - driver, function_store, function_store2, cube, simple_cube_1, skip_eager -): - with pytest.raises(TypeError, match="blocksize must be an integer but is str"): - driver( - cube=cube, - src_store=function_store, - tgt_store=function_store2, - blocksize="foo", - ) - - -def test_fail_blocksize_negative( - driver, function_store, function_store2, cube, simple_cube_1, skip_eager -): - with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): - driver( - cube=cube, src_store=function_store, tgt_store=function_store2, blocksize=-1 - ) - - -def test_fail_blocksize_zero( - driver, function_store, function_store2, cube, simple_cube_1, skip_eager -): - with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): - driver( - cube=cube, src_store=function_store, tgt_store=function_store2, blocksize=0 - ) diff --git a/kartothek/io/testing/delete_cube.py b/kartothek/io/testing/delete_cube.py deleted file mode 100644 index b26010ac..00000000 --- a/kartothek/io/testing/delete_cube.py +++ /dev/null @@ -1,178 +0,0 @@ -# -*- coding: utf-8 -*- -import pandas as pd -import pytest - -from kartothek.api.discover import discover_datasets_unchecked -from kartothek.core.cube.cube import Cube -from kartothek.io.eager_cube import build_cube -from kartothek.utils.ktk_adapters import get_dataset_keys - -__all__ = ( - "test_delete_twice", - "test_fail_blocksize_negative", - "test_fail_blocksize_wrong_type", - "test_fail_blocksize_zero", - "test_fail_no_store_factory", - "test_keep_garbage_due_to_no_listing", - "test_keep_other", - "test_partial_delete", - "test_simple", -) - - -def test_simple(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - driver(cube=cube, store=function_store) - - assert set(function_store().keys()) == set() - - -def test_keep_other(driver, function_store): - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube1 = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube1") - cube2 = cube1.copy(uuid_prefix="cube2") - - build_cube(data=df, cube=cube1, store=function_store) - keys = set(function_store().keys()) - - build_cube(data=df, cube=cube2, store=function_store) - - driver(cube=cube2, store=function_store) - - assert set(function_store().keys()) == keys - - -def test_keep_garbage_due_to_no_listing(driver, function_store): - df1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - df2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [2, 2, 3, 3], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - # test build DF1 to see which keys are created - build_cube(data=df1, cube=cube, store=function_store) - keys1 = set(function_store().keys()) - - # wipe - for k in list(function_store().keys()): - function_store().delete(k) - - # test build DF2 to see which keys are created - build_cube(data=df2, cube=cube, store=function_store) - keys2 = set(function_store().keys()) - - # wipe again - for k in list(function_store().keys()): - function_store().delete(k) - - # some keys are obviosly present everytime (like central metadata and - # common metadata) - keys_common = keys1 & keys2 - - # build DF1 and overwrite w/ DF2 - build_cube(data=df1, cube=cube, store=function_store) - keys3 = set(function_store().keys()) - - build_cube(data=df2, cube=cube, store=function_store, overwrite=True) - - # now some keys if DF1 must be leftovers/gargabe that cannot be deleted w/o listing the entire store (which would - # be too expensive) - gargabe = keys3 - keys_common - assert len(gargabe) > 0 - - driver(cube=cube, store=function_store) - - assert set(function_store().keys()) == gargabe - - -def test_delete_twice(driver, function_store): - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube(data=df, cube=cube, store=function_store) - driver(cube=cube, store=function_store) - driver(cube=cube, store=function_store) - - assert set(function_store().keys()) == set() - - -def test_partial_delete(driver, function_store): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} - ) - df_1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "a": [20, 21, 22, 23]}) - df_2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "b": [20, 21, 22, 23]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - datasets = build_cube( - data={cube.seed_dataset: df_seed, "enrich-1": df_1, "enrich-2": df_2}, - cube=cube, - store=function_store, - ) - enrich_1_keys = get_dataset_keys( - discover_datasets_unchecked( - uuid_prefix=cube.uuid_prefix, - store=function_store, - filter_ktk_cube_dataset_ids=["enrich-1"], - )["enrich-1"] - ) - enrich_2_keys = get_dataset_keys( - discover_datasets_unchecked( - uuid_prefix=cube.uuid_prefix, - store=function_store, - filter_ktk_cube_dataset_ids=["enrich-2"], - )["enrich-2"] - ) - all_keys = set(function_store().keys()) - driver(cube=cube, store=function_store, datasets=["enrich-1"]) - assert set(function_store().keys()) == all_keys - enrich_1_keys - - driver(cube=cube, store=function_store, datasets={"enrich-2": datasets["enrich-2"]}) - assert set(function_store().keys()) == all_keys - enrich_1_keys - enrich_2_keys - - -def test_fail_no_store_factory(driver, function_store, skip_eager): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - store = function_store() - with pytest.raises(TypeError) as exc: - driver(cube=cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(TypeError, match="blocksize must be an integer but is str"): - driver(cube=cube, store=function_store, blocksize="foo") - - -def test_fail_blocksize_negative(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): - driver(cube=cube, store=function_store, blocksize=-1) - - -def test_fail_blocksize_zero(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): - driver(cube=cube, store=function_store, blocksize=0) diff --git a/kartothek/io/testing/extend_cube.py b/kartothek/io/testing/extend_cube.py deleted file mode 100644 index 866999fd..00000000 --- a/kartothek/io/testing/extend_cube.py +++ /dev/null @@ -1,473 +0,0 @@ -import pandas as pd -import pytest - -from kartothek.core.cube.cube import Cube -from kartothek.core.dataset import DatasetMetadata -from kartothek.core.index import ExplicitSecondaryIndex, PartitionIndex -from kartothek.io.eager_cube import build_cube -from kartothek.io.testing.utils import assert_num_row_groups -from kartothek.io_components.cube.write import MultiTableCommitAborted -from kartothek.io_components.metapartition import SINGLE_TABLE -from kartothek.serialization._parquet import ParquetSerializer - -__all__ = ( - "existing_cube", - "test_compression_is_compatible_on_extend_cube", - "test_fail_all_empty", - "test_fail_no_store_factory", - "test_fail_not_a_df", - "test_fail_wrong_dataset_ids", - "test_fails_incompatible_dtypes", - "test_fails_metadata_nested_wrong_type", - "test_fails_metadata_unknown_id", - "test_fails_metadata_wrong_type", - "test_fails_overlapping_payload_enrich", - "test_fails_overlapping_payload_overwrite", - "test_fails_overlapping_payload_partial", - "test_fails_overlapping_payload_seed", - "test_fails_seed_dataset", - "test_overwrite_move_columns", - "test_overwrite_single", - "test_rowgroups_are_applied_when_df_serializer_is_passed_to_extend_cube", - "test_simple", - "test_single_rowgroup_when_df_serializer_is_not_passed_to_extend_cube", -) - - -@pytest.fixture -def existing_cube(function_store): - df_source = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store - ) - return cube - - -def test_simple(driver, function_store, existing_cube): - """ - Simple integration test w/ single extra dataset. - """ - df = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v3": [10, 11, 12, 13], - "i3": [100, 101, 102, 103], - } - ) - result = driver(data={"extra": df}, cube=existing_cube, store=function_store) - - assert set(result.keys()) == {"extra"} - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert ds.uuid == existing_cube.ktk_dataset_uuid("extra") - assert len(ds.partitions) == 2 - - assert set(ds.indices.keys()) == {"p", "i3"} - assert isinstance(ds.indices["p"], PartitionIndex) - assert isinstance(ds.indices["i3"], ExplicitSecondaryIndex) - - assert ds.table_name == SINGLE_TABLE - - -@pytest.mark.parametrize("chunk_size", [None, 2]) -def test_rowgroups_are_applied_when_df_serializer_is_passed_to_extend_cube( - driver, function_store, existing_cube, chunk_size -): - """ - Test that the dataset is split into row groups depending on the chunk size - """ - df_extra = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], - ) - result = driver( - data={"extra": df_extra}, - cube=existing_cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size), - ) - dataset = result["extra"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 3} - part_chunk_size = {0: chunk_size, 1: chunk_size} - - assert len(dataset.partitions) == 2 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_single_rowgroup_when_df_serializer_is_not_passed_to_extend_cube( - driver, function_store, existing_cube -): - """ - Test that the dataset has a single row group as default path - """ - df_extra = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], - ) - result = driver(data={"extra": df_extra}, cube=existing_cube, store=function_store,) - dataset = result["extra"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 3} - part_chunk_size = {0: None, 1: None} - - assert len(dataset.partitions) == 2 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_compression_is_compatible_on_extend_cube(driver, function_store): - """ - Test that partitons written with different compression algorithms are compatible - - The compression algorithms are not parametrized because their availability depends - on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts - of the code. A fully parametrized test would also increase runtime and test complexity - unnecessarily. - """ - # Build cube - df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="SNAPPY"), - ) - - df_extra = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], - ) - result = driver( - data={"extra": df_extra}, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="GZIP"), - ) - dataset = result["extra"].load_all_indices(function_store()) - - assert len(dataset.partitions) == 2 - - -def test_fails_incompatible_dtypes(driver, function_store, existing_cube): - """ - Should also cross check w/ seed dataset. - """ - df = pd.DataFrame( - { - "x": [0.0, 1.0, 2.0, 3.0], - "p": [0, 0, 1, 1], - "v3": [10, 11, 12, 13], - "i3": [100, 101, 102, 103], - } - ) - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver(data={"extra": df}, cube=existing_cube, store=function_store) - cause = exc_info.value.__cause__ - assert isinstance(cause, ValueError) - assert 'Found incompatible entries for column "x"' in str(cause) - assert not DatasetMetadata.exists( - existing_cube.ktk_dataset_uuid("extra"), function_store() - ) - - -def test_fails_seed_dataset(driver, function_store, existing_cube): - """ - Users cannot overwrite seed dataset since it is used for consisteny checks. - """ - pre_keys = set(function_store().keys()) - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) - with pytest.raises(ValueError) as exc: - driver( - data={existing_cube.seed_dataset: df}, - cube=existing_cube, - store=function_store, - ) - assert 'Seed data ("source") cannot be written during extension.' in str(exc.value) - - post_keys = set(function_store().keys()) - assert pre_keys == post_keys - - -def test_fails_overlapping_payload_seed(driver, function_store, existing_cube): - """ - Forbidden by spec, results in problems during query. - """ - pre_keys = set(function_store().keys()) - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) - with pytest.raises(ValueError) as exc: - driver(data={"extra": df}, cube=existing_cube, store=function_store) - assert 'Payload written in "extra" is already present in cube: v1' in str(exc.value) - assert not DatasetMetadata.exists( - existing_cube.ktk_dataset_uuid("extra"), function_store() - ) - - post_keys = set(function_store().keys()) - assert pre_keys == post_keys - - -def test_fails_overlapping_payload_enrich(driver, function_store, existing_cube): - """ - Forbidden by spec, results in problems during query. - """ - pre_keys = set(function_store().keys()) - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) - with pytest.raises(ValueError) as exc: - driver(data={"extra": df}, cube=existing_cube, store=function_store) - assert 'Payload written in "extra" is already present in cube: v2' in str(exc.value) - assert not DatasetMetadata.exists( - existing_cube.ktk_dataset_uuid("extra"), function_store() - ) - - post_keys = set(function_store().keys()) - assert pre_keys == post_keys - - -def test_fails_overlapping_payload_partial(driver, function_store, existing_cube): - """ - Forbidden by spec, results in problems during query. - """ - pre_keys = set(function_store().keys()) - df1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) - df2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]}) - with pytest.raises(ValueError) as exc: - driver( - data={"extra1": df1, "extra2": df2}, - cube=existing_cube, - store=function_store, - ) - assert 'Payload written in "extra1" is already present in cube: v2' in str( - exc.value - ) - - assert not DatasetMetadata.exists( - existing_cube.ktk_dataset_uuid("extra1"), function_store() - ) - # extra2 might exist, depending on the compute graph - - # extra2 keys might be present, only look that extra1 is absent - post_keys = set(function_store().keys()) - extra_keys = post_keys - pre_keys - extra1_keys = {k for k in extra_keys if "extra1" in k} - assert extra1_keys == set() - - -def test_fails_overlapping_payload_overwrite(driver, function_store, existing_cube): - """ - Forbidden by spec, results in problems during query. - """ - pre_keys = set(function_store().keys()) - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) - with pytest.raises(ValueError) as exc: - driver( - data={"enrich": df}, - cube=existing_cube, - store=function_store, - overwrite=True, - ) - assert 'Payload written in "enrich" is already present in cube: v1' in str( - exc.value - ) - - post_keys = set(function_store().keys()) - assert pre_keys == post_keys - - -def test_overwrite_single(driver, function_store, existing_cube): - """ - Simple overwrite of the enrich dataset. - """ - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]}) - - # does not work w/o explicit flag - keys = set(function_store().keys()) - with pytest.raises(RuntimeError) as exc: - driver(data={"enrich": df}, cube=existing_cube, store=function_store) - assert "already exists" in str(exc.value) - assert set(function_store().keys()) == keys - - # but works with flag - result = driver( - data={"enrich": df}, cube=existing_cube, store=function_store, overwrite=True - ) - - assert set(result.keys()) == {"enrich"} - - ds = list(result.values())[0] - ds = ds.load_all_indices(function_store()) - - assert ds.uuid == existing_cube.ktk_dataset_uuid("enrich") - assert len(ds.partitions) == 2 - - -def test_overwrite_move_columns(driver, function_store, existing_cube): - """ - Move columns v1 and i1 from enrich to extra. - """ - df_enrich = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v3": [10, 11, 12, 13], - "i3": [100, 101, 102, 103], - } - ) - df_extra = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v2": [10, 11, 12, 13], - "i2": [100, 101, 102, 103], - } - ) - - result = driver( - data={"enrich": df_enrich, "extra": df_extra}, - cube=existing_cube, - store=function_store, - overwrite=True, - ) - - assert set(result.keys()) == {"enrich", "extra"} - - ds_enrich = result["enrich"].load_all_indices(function_store()) - ds_extra = result["extra"].load_all_indices(function_store()) - - assert set(ds_enrich.indices.keys()) == {"p", "i3"} - assert isinstance(ds_enrich.indices["p"], PartitionIndex) - assert isinstance(ds_enrich.indices["i3"], ExplicitSecondaryIndex) - - assert set(ds_extra.indices.keys()) == {"p", "i2"} - assert isinstance(ds_extra.indices["p"], PartitionIndex) - assert isinstance(ds_extra.indices["i2"], ExplicitSecondaryIndex) - - -def test_fail_all_empty(driver, function_store, existing_cube): - """ - Might happen due to DB-based filters. - """ - df = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} - ).loc[[]] - - with pytest.raises(MultiTableCommitAborted) as exc_info: - driver(data={"extra": df}, cube=existing_cube, store=function_store) - exc = exc_info.value.__cause__ - assert isinstance(exc, ValueError) - assert "Cannot write empty datasets: extra" in str(exc) - assert not DatasetMetadata.exists( - existing_cube.ktk_dataset_uuid("extra"), function_store() - ) - - -def test_fail_not_a_df(driver, function_store, existing_cube): - """ - Pass some weird objects in. - """ - with pytest.raises(TypeError) as exc: - driver( - data={"extra": pd.Series(range(10))}, - cube=existing_cube, - store=function_store, - ) - assert ( - 'Provided DataFrame is not a pandas.DataFrame or None, but is a "Series"' - in str(exc.value) - ) - - -def test_fail_wrong_dataset_ids( - driver, function_store, existing_cube, skip_eager, driver_name -): - if driver_name == "dask_dataframe": - pytest.skip("not an interface for dask.dataframe") - - df_extra = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} - ) - with pytest.raises(ValueError) as exc: - driver( - data={"extra": df_extra}, - cube=existing_cube, - store=function_store, - ktk_cube_dataset_ids=["other"], - ) - - assert ( - 'Ktk_cube Dataset ID "extra" is present during pipeline execution ' - "but was not specified in ktk_cube_dataset_ids (other)." in str(exc.value) - ) - - -def test_fail_no_store_factory(driver, function_store, existing_cube, skip_eager): - df = pd.DataFrame( - { - "x": [0, 1, 2, 3], - "p": [0, 0, 1, 1], - "v3": [10, 11, 12, 13], - "i3": [100, 101, 102, 103], - } - ) - store = function_store() - with pytest.raises(TypeError) as exc: - driver(data={"extra": df}, cube=existing_cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fails_metadata_wrong_type(driver, function_store, existing_cube): - df_extra = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} - ) - with pytest.raises( - TypeError, match="Provided metadata should be a dict but is int" - ): - driver( - data={"extra": df_extra}, - cube=existing_cube, - store=function_store, - metadata=1, - ) - - -def test_fails_metadata_unknown_id(driver, function_store, existing_cube): - df_extra = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} - ) - with pytest.raises( - ValueError, - match="Provided metadata for otherwise unspecified ktk_cube_dataset_ids: bar, foo", - ): - driver( - data={"extra": df_extra}, - cube=existing_cube, - store=function_store, - metadata={"extra": {}, "foo": {}, "bar": {}}, - ) - - -def test_fails_metadata_nested_wrong_type(driver, function_store, existing_cube): - df_extra = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} - ) - with pytest.raises( - TypeError, - match="Provided metadata for dataset extra should be a dict but is int", - ): - driver( - data={"extra": df_extra}, - cube=existing_cube, - store=function_store, - metadata={"extra": 1}, - ) diff --git a/kartothek/io/testing/query_cube.py b/kartothek/io/testing/query_cube.py deleted file mode 100644 index 5a04e645..00000000 --- a/kartothek/io/testing/query_cube.py +++ /dev/null @@ -1,1437 +0,0 @@ -from datetime import timedelta -from functools import partial -from itertools import permutations - -import dask.bag as db -import numpy as np -import pandas as pd -import pandas.testing as pdt -import pytest -from hypothesis import given, settings -from hypothesis import strategies as st - -from kartothek.core.cube.conditions import ( - C, - Conjunction, - EqualityCondition, - GreaterEqualCondition, - GreaterThanCondition, - InequalityCondition, - InIntervalCondition, - IsInCondition, - LessEqualCondition, - LessThanCondition, -) -from kartothek.core.cube.cube import Cube -from kartothek.io.dask.bag_cube import build_cube_from_bag -from kartothek.io.eager import build_dataset_indices -from kartothek.io.eager_cube import append_to_cube, build_cube, remove_partitions - -__all__ = ( - "apply_condition_unsafe", - "data_no_part", - "fullrange_cube", - "fullrange_data", - "fullrange_df", - "massive_partitions_cube", - "massive_partitions_data", - "massive_partitions_df", - "multipartition_cube", - "multipartition_df", - "no_part_cube", - "no_part_df", - "other_part_cube", - "sparse_outer_cube", - "sparse_outer_data", - "sparse_outer_df", - "sparse_outer_opt_cube", - "sparse_outer_opt_df", - "test_complete", - "test_condition", - "test_condition_on_null", - "test_cube", - "test_delayed_index_build_correction_restriction", - "test_delayed_index_build_partition_by", - "test_df", - "test_fail_blocksize_negative", - "test_fail_blocksize_wrong_type", - "test_fail_blocksize_zero", - "test_fail_empty_dimension_columns", - "test_fail_missing_condition_columns", - "test_fail_missing_dimension_columns", - "test_fail_missing_partition_by", - "test_fail_missing_payload_columns", - "test_fail_no_store_factory", - "test_fail_projection", - "test_fail_unindexed_partition_by", - "test_fail_unstable_dimension_columns", - "test_fail_unstable_partition_by", - "test_filter_select", - "test_hypothesis", - "test_overlay_tricky", - "test_partition_by", - "test_projection", - "test_select", - "test_simple_roundtrip", - "test_sort", - "test_stresstest_index_select_row", - "test_wrong_condition_type", - "testset", - "updated_cube", - "updated_df", -) - - -@pytest.fixture(scope="module") -def fullrange_data(): - return { - "seed": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": np.arange(16), - "i1": np.arange(16), - } - ), - "enrich_dense": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v2": np.arange(16), - "i2": np.arange(16), - } - ), - "enrich_sparse": pd.DataFrame( - { - "y": [0, 1, 2, 3, 0, 1, 2, 3], - "z": 0, - "p": [0, 0, 1, 1, 0, 0, 1, 1], - "q": [0, 0, 0, 0, 1, 1, 1, 1], - "v3": np.arange(8), - "i3": np.arange(8), - } - ), - } - - -@pytest.fixture(scope="module") -def fullrange_cube(module_store, fullrange_data): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="fullrange_cube", - index_columns=["i1", "i2", "i3"], - ) - build_cube(data=fullrange_data, store=module_store, cube=cube) - return cube - - -@pytest.fixture(scope="module") -def multipartition_cube(module_store, fullrange_data, fullrange_cube): - def _gen(part): - result = {} - for dataset_id, df in fullrange_data.items(): - df = df.copy() - df["z"] = part - result[dataset_id] = df - return result - - cube = fullrange_cube.copy(uuid_prefix="multipartition_cube") - build_cube_from_bag( - data=db.from_sequence([0, 1], partition_size=1).map(_gen), - store=module_store, - cube=cube, - ktk_cube_dataset_ids=["seed", "enrich_dense", "enrich_sparse"], - ).compute() - return cube - - -@pytest.fixture(scope="module") -def sparse_outer_data(): - return { - "seed": pd.DataFrame( - { - "x": [0, 1, 0], - "y": [0, 0, 1], - "z": 0, - "p": [0, 1, 2], - "q": 0, - "v1": [0, 3, 7], - "i1": [0, 3, 7], - } - ), - "enrich_dense": pd.DataFrame( - { - "x": [0, 0], - "y": [0, 1], - "z": 0, - "p": [0, 2], - "q": 0, - "v2": [0, 7], - "i2": [0, 7], - } - ), - "enrich_sparse": pd.DataFrame( - {"y": [0, 0], "z": 0, "p": [0, 1], "q": 0, "v3": [0, 3], "i3": [0, 3]} - ), - } - - -@pytest.fixture(scope="module") -def sparse_outer_cube(module_store, sparse_outer_data): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="sparse_outer_cube", - index_columns=["i1", "i2", "i3"], - ) - build_cube(data=sparse_outer_data, store=module_store, cube=cube) - return cube - - -@pytest.fixture(scope="module") -def sparse_outer_opt_cube( - module_store, - sparse_outer_data, - sparse_outer_cube, - sparse_outer_df, - sparse_outer_opt_df, -): - data = {} - for dataset_id in sparse_outer_data.keys(): - df = sparse_outer_data[dataset_id].copy() - - for col in sparse_outer_opt_df.columns: - if col in df.columns: - dtype = sparse_outer_opt_df[col].dtype - - if dtype == np.float64: - dtype = np.int64 - elif dtype == np.float32: - dtype = np.int32 - elif dtype == np.float16: - dtype = np.int16 - - df[col] = df[col].astype(dtype) - - data[dataset_id] = df - - cube = sparse_outer_cube.copy(uuid_prefix="sparse_outer_opt_cube") - build_cube(data=data, store=module_store, cube=cube) - return cube - - -@pytest.fixture(scope="module") -def massive_partitions_data(): - n = 17 - return { - "seed": pd.DataFrame( - { - "x": np.arange(n), - "y": np.arange(n), - "z": np.arange(n), - "p": np.arange(n), - "q": np.arange(n), - "v1": np.arange(n), - "i1": np.arange(n), - } - ), - "enrich_1": pd.DataFrame( - { - "x": np.arange(n), - "y": np.arange(n), - "z": np.arange(n), - "p": np.arange(n), - "q": np.arange(n), - "v2": np.arange(n), - "i2": np.arange(n), - } - ), - "enrich_2": pd.DataFrame( - { - "y": np.arange(n), - "z": np.arange(n), - "p": np.arange(n), - "q": np.arange(n), - "v3": np.arange(n), - "i3": np.arange(n), - } - ), - } - - -@pytest.fixture(scope="module") -def massive_partitions_cube(module_store, massive_partitions_data): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="massive_partitions_cube", - index_columns=["i1", "i2", "i3"], - ) - build_cube(data=massive_partitions_data, store=module_store, cube=cube) - return cube - - -@pytest.fixture(scope="module") -def fullrange_df(): - return ( - pd.DataFrame( - data={ - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": np.arange(16), - "v2": np.arange(16), - "v3": [0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7], - "i1": np.arange(16), - "i2": np.arange(16), - "i3": [0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7], - }, - columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], - ) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture(scope="module") -def multipartition_df(fullrange_df): - dfs = [] - for z in (0, 1): - df = fullrange_df.copy() - df["z"] = z - dfs.append(df) - return ( - pd.concat(dfs, ignore_index=True) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture(scope="module") -def sparse_outer_df(): - return ( - pd.DataFrame( - data={ - "x": [0, 1, 0], - "y": [0, 0, 1], - "z": 0, - "p": [0, 1, 2], - "q": 0, - "v1": [0, 3, 7], - "v2": [0, np.nan, 7], - "v3": [0, 3, np.nan], - "i1": [0, 3, 7], - "i2": [0, np.nan, 7], - "i3": [0, 3, np.nan], - }, - columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], - ) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture(scope="module") -def sparse_outer_opt_df(sparse_outer_df): - df = sparse_outer_df.copy() - df["x"] = df["x"].astype(np.int16) - df["y"] = df["y"].astype(np.int32) - df["z"] = df["z"].astype(np.int8) - - df["v1"] = df["v1"].astype(np.int8) - df["i1"] = df["i1"].astype(np.int8) - - return df - - -@pytest.fixture(scope="module") -def massive_partitions_df(): - n = 17 - return ( - pd.DataFrame( - data={ - "x": np.arange(n), - "y": np.arange(n), - "z": np.arange(n), - "p": np.arange(n), - "q": np.arange(n), - "v1": np.arange(n), - "v2": np.arange(n), - "v3": np.arange(n), - "i1": np.arange(n), - "i2": np.arange(n), - "i3": np.arange(n), - }, - columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], - ) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture(scope="module") -def updated_cube(module_store, fullrange_data): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="updated_cube", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data={ - cube.seed_dataset: pd.DataFrame( - { - "x": [0, 0, 1, 1, 2, 2], - "y": [0, 1, 0, 1, 0, 1], - "z": 0, - "p": [0, 0, 1, 1, 2, 2], - "q": 0, - "v1": np.arange(6), - "i1": np.arange(6), - } - ), - "enrich": pd.DataFrame( - { - "x": [0, 0, 1, 1, 2, 2], - "y": [0, 1, 0, 1, 0, 1], - "z": 0, - "p": [0, 0, 1, 1, 2, 2], - "q": 0, - "v2": np.arange(6), - "i2": np.arange(6), - } - ), - "extra": pd.DataFrame( - { - "y": [0, 1, 0, 1, 0, 1], - "z": 0, - "p": [0, 0, 1, 1, 2, 2], - "q": 0, - "v3": np.arange(6), - "i3": np.arange(6), - } - ), - }, - store=module_store, - cube=cube, - ) - remove_partitions( - cube=cube, - store=module_store, - ktk_cube_dataset_ids=["enrich"], - conditions=C("p") >= 1, - ) - append_to_cube( - data={ - "enrich": pd.DataFrame( - { - "x": [1, 1], - "y": [0, 1], - "z": 0, - "p": [1, 1], - "q": 0, - "v2": [7, 8], - "i2": [7, 8], - } - ) - }, - store=module_store, - cube=cube, - ) - return cube - - -@pytest.fixture(scope="module") -def updated_df(): - return ( - pd.DataFrame( - data={ - "x": [0, 0, 1, 1, 2, 2], - "y": [0, 1, 0, 1, 0, 1], - "z": 0, - "p": [0, 0, 1, 1, 2, 2], - "q": 0, - "v1": np.arange(6), - "v2": [0, 1, 7, 8, np.nan, np.nan], - "v3": np.arange(6), - "i1": np.arange(6), - "i2": [0, 1, 7, 8, np.nan, np.nan], - "i3": np.arange(6), - }, - columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], - ) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture(scope="module") -def data_no_part(): - return { - "seed": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": np.arange(16), - "i1": np.arange(16), - } - ), - "enrich_dense": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "v2": np.arange(16), - "i2": np.arange(16), - } - ), - "enrich_sparse": pd.DataFrame( - {"y": [0, 1, 2, 3], "z": 0, "v3": np.arange(4), "i3": np.arange(4)} - ), - } - - -@pytest.fixture(scope="module") -def no_part_cube(module_store, data_no_part): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="data_no_part", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data=data_no_part, - store=module_store, - cube=cube, - partition_on={"enrich_dense": [], "enrich_sparse": []}, - ) - return cube - - -@pytest.fixture(scope="module") -def other_part_cube(module_store, data_no_part): - cube = Cube( - dimension_columns=["x", "y", "z"], - partition_columns=["p", "q"], - uuid_prefix="other_part_cube", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data=data_no_part, - store=module_store, - cube=cube, - partition_on={"enrich_dense": ["i2"], "enrich_sparse": ["i3"]}, - ) - return cube - - -@pytest.fixture(scope="module") -def no_part_df(): - return ( - pd.DataFrame( - data={ - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "z": 0, - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": np.arange(16), - "v2": np.arange(16), - "v3": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "i1": np.arange(16), - "i2": np.arange(16), - "i3": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - }, - columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], - ) - .sort_values(["x", "y", "z", "p", "q"]) - .reset_index(drop=True) - ) - - -@pytest.fixture( - params=[ - "fullrange", - "multipartition", - "sparse_outer", - "sparse_outer_opt", - "massive_partitions", - "updated", - "no_part", - "other_part", - ], - scope="module", -) -def testset(request): - return request.param - - -@pytest.fixture(scope="module") -def test_cube( - testset, - fullrange_cube, - multipartition_cube, - sparse_outer_cube, - sparse_outer_opt_cube, - massive_partitions_cube, - updated_cube, - no_part_cube, - other_part_cube, -): - if testset == "fullrange": - return fullrange_cube - elif testset == "multipartition": - return multipartition_cube - elif testset == "sparse_outer": - return sparse_outer_cube - elif testset == "sparse_outer_opt": - return sparse_outer_opt_cube - elif testset == "massive_partitions": - return massive_partitions_cube - elif testset == "updated": - return updated_cube - elif testset == "no_part": - return no_part_cube - elif testset == "other_part": - return other_part_cube - else: - raise ValueError("Unknown param {}".format(testset)) - - -@pytest.fixture(scope="module") -def test_df( - testset, - fullrange_df, - multipartition_df, - sparse_outer_df, - sparse_outer_opt_df, - massive_partitions_df, - updated_df, - no_part_df, -): - if testset == "fullrange": - return fullrange_df - elif testset == "multipartition": - return multipartition_df - elif testset == "sparse_outer": - return sparse_outer_df - elif testset == "sparse_outer_opt": - return sparse_outer_opt_df - elif testset == "massive_partitions": - return massive_partitions_df - elif testset == "updated": - return updated_df - elif testset in ("no_part", "other_part"): - return no_part_df - else: - raise ValueError("Unknown param {}".format(testset)) - - -def test_simple_roundtrip(driver, function_store, function_store_rwro): - df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube(data=df, cube=cube, store=function_store) - result = driver(cube=cube, store=function_store_rwro) - assert len(result) == 1 - df_actual = result[0] - df_expected = df.reindex(columns=["p", "v", "x"]) - pdt.assert_frame_equal(df_actual, df_expected) - - -def test_complete(driver, module_store, test_cube, test_df): - result = driver(cube=test_cube, store=module_store) - assert len(result) == 1 - df_actual = result[0] - pdt.assert_frame_equal(df_actual, test_df) - - -def apply_condition_unsafe(df, cond): - # For the sparse_outer testset, the test_df has the wrong datatype because we cannot encode missing integer data in - # pandas. - # - # The condition will not be applicable to the DF because the DF has floats while conditions have ints. We fix that - # by modifying the the condition. - # - # In case there is no missing data because of the right conditions, kartothek will return integer data. - # assert_frame_equal will then complain about this. So in case there is no missing data, let's recover the correct - # dtype here. - - if not isinstance(cond, Conjunction): - cond = Conjunction(cond) - - float_cols = {col for col in df.columns if df[col].dtype == float} - - # convert int to float conditions - cond2 = Conjunction([]) - for col, conj in cond.split_by_column().items(): - if col in float_cols: - parts = [] - for part in conj.conditions: - if isinstance(part, IsInCondition): - part = IsInCondition( - column=part.column, value=tuple((float(v) for v in part.value)) - ) - elif isinstance(part, InIntervalCondition): - part = InIntervalCondition( - column=part.column, - start=float(part.start), - stop=float(part.stop), - ) - else: - part = part.__class__(column=part.column, value=float(part.value)) - parts.append(part) - conj = Conjunction(parts) - cond2 &= conj - - # apply conditions - df = cond2.filter_df(df).reset_index(drop=True) - - # convert float columns to int columns - for col in df.columns: - if df[col].notnull().all(): - dtype = df[col].dtype - if dtype == np.float64: - dtype = np.int64 - elif dtype == np.float32: - dtype = np.int32 - elif dtype == np.float16: - dtype = np.int16 - - df[col] = df[col].astype(dtype) - - return df - - -@pytest.mark.parametrize( - "cond", - [ - C("v1") >= 7, - C("v1") >= 10000, - C("v2") >= 7, - C("v3") >= 3, - C("i1") >= 7, - C("i1") >= 10000, - C("i2") >= 7, - C("i2") != 0, - C("i3") >= 3, - C("p") >= 1, - C("q") >= 1, - C("x") >= 1, - C("y") >= 1, - (C("x") == 3) & (C("y") == 3), - (C("i1") > 0) & (C("i2") > 0), - Conjunction([]), - ], -) -def test_condition(driver, module_store, test_cube, test_df, cond): - result = driver(cube=test_cube, store=module_store, conditions=cond) - - df_expected = apply_condition_unsafe(test_df, cond) - - if df_expected.empty: - assert len(result) == 0 - else: - assert len(result) == 1 - df_actual = result[0] - pdt.assert_frame_equal(df_actual, df_expected) - - -@pytest.mark.parametrize("payload_columns", [["v1", "v2"], ["v2", "v3"], ["v3"]]) -def test_select(driver, module_store, test_cube, test_df, payload_columns): - result = driver(cube=test_cube, store=module_store, payload_columns=payload_columns) - assert len(result) == 1 - df_actual = result[0] - df_expected = test_df.loc[ - :, sorted(set(payload_columns) | {"x", "y", "z", "p", "q"}) - ] - pdt.assert_frame_equal(df_actual, df_expected) - - -def test_filter_select(driver, module_store, test_cube, test_df): - result = driver( - cube=test_cube, - store=module_store, - payload_columns=["v1", "v2"], - conditions=(C("i3") >= 3), # completely unrelated to the payload - ) - assert len(result) == 1 - df_actual = result[0] - df_expected = test_df.loc[ - test_df["i3"] >= 3, ["p", "q", "v1", "v2", "x", "y", "z"] - ].reset_index(drop=True) - pdt.assert_frame_equal(df_actual, df_expected) - - -@pytest.mark.parametrize( - "partition_by", - [["i1"], ["i2"], ["i3"], ["x"], ["y"], ["p"], ["q"], ["i1", "i2"], ["x", "y"]], -) -def test_partition_by(driver, module_store, test_cube, test_df, partition_by): - dfs_actual = driver(cube=test_cube, store=module_store, partition_by=partition_by) - - dfs_expected = [ - df_g.reset_index(drop=True) - for g, df_g in test_df.groupby(partition_by, sort=True) - ] - for df_expected in dfs_expected: - for col in df_expected.columns: - if df_expected[col].dtype == float: - try: - df_expected[col] = df_expected[col].astype(int) - except Exception: - pass - - assert len(dfs_actual) == len(dfs_expected) - for df_actual, df_expected in zip(dfs_actual, dfs_expected): - pdt.assert_frame_equal(df_actual, df_expected) - - -@pytest.mark.parametrize("dimension_columns", list(permutations(["x", "y", "z"]))) -def test_sort(driver, module_store, test_cube, test_df, dimension_columns): - result = driver( - cube=test_cube, store=module_store, dimension_columns=dimension_columns - ) - assert len(result) == 1 - df_actual = result[0] - df_expected = test_df.sort_values( - list(dimension_columns) + list(test_cube.partition_columns) - ).reset_index(drop=True) - pdt.assert_frame_equal(df_actual, df_expected) - - -@pytest.mark.parametrize("payload_columns", [["y", "z"], ["y", "z", "v3"]]) -def test_projection(driver, module_store, test_cube, test_df, payload_columns): - result = driver( - cube=test_cube, - store=module_store, - dimension_columns=["y", "z"], - payload_columns=payload_columns, - ) - assert len(result) == 1 - df_actual = result[0] - df_expected = ( - test_df.loc[:, sorted(set(payload_columns) | {"y", "z", "p", "q"})] - .drop_duplicates() - .sort_values(["y", "z", "p", "q"]) - .reset_index(drop=True) - ) - pdt.assert_frame_equal(df_actual, df_expected) - - -def test_stresstest_index_select_row(driver, function_store): - n_indices = 100 - n_rows = 1000 - - data = {"x": np.arange(n_rows), "p": 0} - for i in range(n_indices): - data["i{}".format(i)] = np.arange(n_rows) - df = pd.DataFrame(data) - - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - index_columns=["i{}".format(i) for i in range(n_indices)], - ) - - build_cube(data=df, cube=cube, store=function_store) - - conditions = Conjunction([(C("i{}".format(i)) == 0) for i in range(n_indices)]) - - result = driver( - cube=cube, - store=function_store, - conditions=conditions, - payload_columns=["p", "x"], - ) - assert len(result) == 1 - df_actual = result[0] - df_expected = df.loc[df["x"] == 0].reindex(columns=["p", "x"]) - pdt.assert_frame_equal(df_actual, df_expected) - - -def test_fail_missing_dimension_columns(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver(cube=test_cube, store=module_store, dimension_columns=["x", "a", "b"]) - assert ( - "Following dimension columns were requested but are missing from the cube: a, b" - in str(exc.value) - ) - - -def test_fail_empty_dimension_columns(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver(cube=test_cube, store=module_store, dimension_columns=[]) - assert "Dimension columns cannot be empty." in str(exc.value) - - -def test_fail_missing_partition_by(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver(cube=test_cube, store=module_store, partition_by=["foo"]) - assert ( - "Following partition-by columns were requested but are missing from the cube: foo" - in str(exc.value) - ) - - -def test_fail_unindexed_partition_by(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver(cube=test_cube, store=module_store, partition_by=["v1", "v2"]) - assert ( - "Following partition-by columns are not indexed and cannot be used: v1, v2" - in str(exc.value) - ) - - -def test_fail_missing_condition_columns(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver( - cube=test_cube, - store=module_store, - conditions=(C("foo") == 1) & (C("bar") == 2), - ) - assert ( - "Following condition columns are required but are missing from the cube: bar, foo" - in str(exc.value) - ) - - -def test_fail_missing_payload_columns(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver(cube=test_cube, store=module_store, payload_columns=["foo", "bar"]) - assert "Cannot find the following requested payload columns: bar, foo" in str( - exc.value - ) - - -def test_fail_projection(driver, module_store, test_cube, test_df): - with pytest.raises(ValueError) as exc: - driver( - cube=test_cube, - store=module_store, - dimension_columns=["y", "z"], - payload_columns=["v1"], - ) - assert ( - 'Cannot project dataset "seed" with dimensionality [x, y, z] to [y, z] ' - "while keeping the following payload intact: v1" in str(exc.value) - ) - - -def test_fail_unstable_dimension_columns(driver, module_store, test_cube, test_df): - with pytest.raises(TypeError) as exc: - driver(cube=test_cube, store=module_store, dimension_columns={"x", "y"}) - assert "which has type set has an unstable iteration order" in str(exc.value) - - -def test_fail_unstable_partition_by(driver, module_store, test_cube, test_df): - with pytest.raises(TypeError) as exc: - driver(cube=test_cube, store=module_store, partition_by={"x", "y"}) - assert "which has type set has an unstable iteration order" in str(exc.value) - - -def test_wrong_condition_type(driver, function_store, driver_name): - types = { - "int": pd.Series([-1], dtype=np.int64), - "uint": pd.Series([1], dtype=np.uint64), - "float": pd.Series([1.3], dtype=np.float64), - "bool": pd.Series([True], dtype=np.bool_), - "str": pd.Series(["foo"], dtype=object), - } - cube = Cube( - dimension_columns=["d_{}".format(t) for t in sorted(types.keys())], - partition_columns=["p_{}".format(t) for t in sorted(types.keys())], - uuid_prefix="typed_cube", - index_columns=["i_{}".format(t) for t in sorted(types.keys())], - ) - data = { - "seed": pd.DataFrame( - { - "{}_{}".format(prefix, t): types[t] - for t in sorted(types.keys()) - for prefix in ["d", "p", "v1"] - } - ), - "enrich": pd.DataFrame( - { - "{}_{}".format(prefix, t): types[t] - for t in sorted(types.keys()) - for prefix in ["d", "p", "i", "v2"] - } - ), - } - build_cube(data=data, store=function_store, cube=cube) - - df = pd.DataFrame( - { - "{}_{}".format(prefix, t): types[t] - for t in sorted(types.keys()) - for prefix in ["d", "p", "i", "v1", "v2"] - } - ) - - for col in df.columns: - t1 = col.split("_")[1] - - for t2 in sorted(types.keys()): - cond = C(col) == types[t2].values[0] - - if t1 == t2: - result = driver(cube=cube, store=function_store, conditions=cond) - assert len(result) == 1 - df_actual = result[0] - df_expected = cond.filter_df(df).reset_index(drop=True) - pdt.assert_frame_equal(df_actual, df_expected, check_like=True) - else: - with pytest.raises(TypeError) as exc: - driver(cube=cube, store=function_store, conditions=cond) - assert "has wrong type" in str(exc.value) - - -def test_condition_on_null(driver, function_store): - df = pd.DataFrame( - { - "x": pd.Series([0, 1, 2], dtype=np.int64), - "p": pd.Series([0, 0, 1], dtype=np.int64), - "v_f1": pd.Series([0, np.nan, 2], dtype=np.float64), - "v_f2": pd.Series([0, 1, np.nan], dtype=np.float64), - "v_f3": pd.Series([np.nan, np.nan, np.nan], dtype=np.float64), - "v_s1": pd.Series(["a", None, "c"], dtype=object), - "v_s2": pd.Series(["a", "b", None], dtype=object), - "v_s3": pd.Series([None, None, None], dtype=object), - } - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="nulled_cube", - index_columns=[], - ) - build_cube(data=df, store=function_store, cube=cube) - - for col in df.columns: - # only iterate over the value columns (not the dimension / partition column): - if not col.startswith("v"): - continue - - # col_type will be either 'f' for float or 's' for string; see column - # names above - col_type = col.split("_")[1][0] - if col_type == "f": - value = 1.2 - elif col_type == "s": - value = "foo" - else: - raise RuntimeError("unknown type") - - cond = C(col) == value - - df_expected = cond.filter_df(df).reset_index(drop=True) - - result = driver(cube=cube, store=function_store, conditions=cond) - - if df_expected.empty: - assert len(result) == 0 - else: - assert len(result) == 1 - df_actual = result[0] - pdt.assert_frame_equal(df_actual, df_expected, check_like=True) - - -def test_fail_no_store_factory(driver, module_store, test_cube, skip_eager): - store = module_store() - with pytest.raises(TypeError) as exc: - driver(cube=test_cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_delayed_index_build_partition_by(driver, function_store): - df_seed = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}) - df_extend = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [0, 0, 0, 1]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="delayed_index_cube", - index_columns=[], - ) - build_cube( - data={"seed": df_seed, "extend": df_extend}, store=function_store, cube=cube - ) - - build_dataset_indices( - store=function_store, - dataset_uuid=cube.ktk_dataset_uuid("extend"), - columns=["v"], - ) - - results = driver(cube=cube, store=function_store, partition_by=["v"]) - assert len(results) == 2 - - df_result1 = pd.DataFrame( - data={"x": [0, 1, 2], "p": [0, 0, 1], "v": [0, 0, 0]}, columns=["p", "v", "x"] - ) - df_result2 = pd.DataFrame( - data={"x": [3], "p": [1], "v": [1]}, columns=["p", "v", "x"] - ) - pdt.assert_frame_equal(results[0], df_result1) - pdt.assert_frame_equal(results[1], df_result2) - - -def test_fail_blocksize_wrong_type( - driver, module_store, test_cube, skip_eager, driver_name -): - if driver_name == "dask_dataframe": - pytest.skip("not relevant for dask.dataframe") - - with pytest.raises(TypeError, match="blocksize must be an integer but is str"): - driver(cube=test_cube, store=module_store, blocksize="foo") - - -def test_fail_blocksize_negative( - driver, module_store, test_cube, skip_eager, driver_name -): - if driver_name == "dask_dataframe": - pytest.skip("not relevant for dask.dataframe") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): - driver(cube=test_cube, store=module_store, blocksize=-1) - - -def test_fail_blocksize_zero(driver, module_store, test_cube, skip_eager, driver_name): - if driver_name == "dask_dataframe": - pytest.skip("not relevant for dask.dataframe") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): - driver(cube=test_cube, store=module_store, blocksize=0) - - -def test_delayed_index_build_correction_restriction(driver, function_store): - """ - Ensure that adding extra indices for dimension columns does not mark other datasets as restrictive. - """ - df_seed = pd.DataFrame({"x": [0, 1, 2, 3, 4, 5], "p": [0, 0, 1, 1, 2, 2]}) - df_extend = pd.DataFrame({"x": [0, 1, 2], "p": [0, 0, 1], "v": [0, 1, 2]}) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="delayed_index_cube", - index_columns=[], - ) - build_cube( - data={"seed": df_seed, "extend": df_extend}, store=function_store, cube=cube - ) - - build_dataset_indices( - store=function_store, - dataset_uuid=cube.ktk_dataset_uuid("extend"), - columns=["x"], - ) - - results = driver(cube=cube, store=function_store, conditions=C("x") >= 0) - assert len(results) == 1 - - df_actual = results[0] - df_expected = pd.DataFrame( - { - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v": [0, 1, 2, np.nan, np.nan, np.nan], - }, - columns=["p", "v", "x"], - ) - pdt.assert_frame_equal(df_actual, df_expected) - - -time_travel_stages_ops_df = [ - ( - partial( - build_cube, - data={ - "source": pd.DataFrame( - { - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v1": [0, 1, 2, 3, 4, 5], - "i1": [0, 1, 2, 3, 4, 5], - } - ), - "enrich": pd.DataFrame( - { - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v2": [0, 1, 2, 3, 4, 5], - "i2": [0, 1, 2, 3, 4, 5], - } - ), - }, - ), - pd.DataFrame( - data={ - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v1": [0, 1, 2, 3, 4, 5], - "i1": [0, 1, 2, 3, 4, 5], - "v2": [0, 1, 2, 3, 4, 5], - "i2": [0, 1, 2, 3, 4, 5], - }, - columns=["i1", "i2", "p", "v1", "v2", "x"], - ), - ), - ( - partial( - remove_partitions, ktk_cube_dataset_ids=["enrich"], conditions=C("p") > 0 - ), - pd.DataFrame( - data={ - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v1": [0, 1, 2, 3, 4, 5], - "i1": [0, 1, 2, 3, 4, 5], - "v2": [0, 1, np.nan, np.nan, np.nan, np.nan], - "i2": [0, 1, np.nan, np.nan, np.nan, np.nan], - }, - columns=["i1", "i2", "p", "v1", "v2", "x"], - ), - ), - ( - partial( - append_to_cube, - data={"enrich": pd.DataFrame({"x": [2], "p": [1], "v2": [20], "i2": [20]})}, - ), - pd.DataFrame( - data={ - "x": [0, 1, 2, 3, 4, 5], - "p": [0, 0, 1, 1, 2, 2], - "v1": [0, 1, 2, 3, 4, 5], - "i1": [0, 1, 2, 3, 4, 5], - "v2": [0, 1, 20, np.nan, np.nan, np.nan], - "i2": [0, 1, 20, np.nan, np.nan, np.nan], - }, - columns=["i1", "i2", "p", "v1", "v2", "x"], - ), - ), - ( - partial( - append_to_cube, - data={ - "source": pd.DataFrame( - { - "x": [4, 5, 6, 7], - "p": [2, 2, 3, 3], - "v1": [40, 50, 60, 70], - "i1": [40, 50, 60, 70], - } - ) - }, - ), - pd.DataFrame( - data={ - "x": [0, 1, 2, 3, 4, 5, 6, 7], - "p": [0, 0, 1, 1, 2, 2, 3, 3], - "v1": [0, 1, 2, 3, 40, 50, 60, 70], - "i1": [0, 1, 2, 3, 40, 50, 60, 70], - "v2": [0, 1, 20, np.nan, np.nan, np.nan, np.nan, np.nan], - "i2": [0, 1, 20, np.nan, np.nan, np.nan, np.nan, np.nan], - }, - columns=["i1", "i2", "p", "v1", "v2", "x"], - ), - ), -] - - -def test_overlay_tricky(driver, function_store): - cube = Cube( - dimension_columns=["x", "y"], - partition_columns=["p", "q"], - uuid_prefix="time_travel_cube_tricky", - seed_dataset="source", - ) - - build_cube( - data={ - cube.seed_dataset: pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": 1, - } - ), - "no_part": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "v2": 1, - } - ), - "q": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v3": 1, - } - ), - "a": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "a": [0, 1, 0, 0, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1, 0, 1], - "v4": 1, - } - ), - }, - cube=cube, - store=function_store, - partition_on={"no_part": [], "q": ["q"], "a": ["a"]}, - ) - append_to_cube( - data={ - cube.seed_dataset: pd.DataFrame( - { - "x": [0, 1, 0, 1, 2, 3, 2, 3], - "y": [2, 2, 3, 3, 0, 0, 1, 1], - "p": [1, 1, 1, 1, 0, 0, 0, 0], - "q": [0, 0, 0, 0, 1, 1, 1, 1], - "v1": 2, - } - ), - "no_part": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "v2": 2, - } - ), - "q": pd.DataFrame( - { - "x": [0, 1, 0, 1, 0, 1, 0, 1], - "y": [0, 0, 1, 1, 2, 2, 3, 3], - "q": [0, 0, 0, 0, 0, 0, 0, 0], - "v3": 2, - } - ), - "a": pd.DataFrame( - { - "x": [1, 0, 1, 2, 3, 2, 3, 3], - "y": [0, 2, 2, 1, 1, 2, 2, 3], - "a": [1, 1, 1, 1, 1, 1, 1, 1], - "v4": 2, - } - ), - }, - cube=cube, - store=function_store, - ) - - df_expected = ( - pd.DataFrame( - data={ - "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], - "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], - "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], - "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], - "v1": [1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1], - "v2": [2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], - "v3": [2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1], - "v4": [1, 2, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 2, 2, 1, 2], - "a": [0, 1, 0, 0, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1, 0, 1], - }, - columns=["a", "p", "q", "v1", "v2", "v3", "v4", "x", "y"], - ) - .sort_values(["x", "y", "p", "q"]) - .reset_index(drop=True) - ) - - result = driver(cube=cube, store=function_store) - assert len(result) == 1 - df_actual = result[0] - pdt.assert_frame_equal(df_actual, df_expected) - - -cond_types_simple = [ - EqualityCondition, - LessEqualCondition, - LessThanCondition, - GreaterEqualCondition, - GreaterThanCondition, - InequalityCondition, -] - -cond_types_all = cond_types_simple + [IsInCondition, InIntervalCondition] # type:ignore - - -def _tuple_to_condition(t): - col, cond_type, v1, v2, vset = t - if issubclass(cond_type, tuple(cond_types_simple)): - return cond_type(col, v1) - elif cond_type == IsInCondition: - return cond_type(col, vset) - elif cond_type == InIntervalCondition: - return cond_type(col, v1, v2) - raise ValueError("Unknown condition type {}".format(cond_type)) - - -st_columns = st.sampled_from( - ["x", "y", "z", "p", "q", "i1", "i2", "i3", "v1", "v2", "v3"] -) -st_values = st.integers(min_value=-1, max_value=17) -st_cond_types = st.sampled_from(cond_types_all) -st_conditions = st.tuples( - st_columns, st_cond_types, st_values, st_values, st.sets(st_values) -).map(_tuple_to_condition) - - -@given( - conditions=st.lists(st_conditions).map(Conjunction), - dimension_columns=st.permutations(["x", "y", "z"]), - payload_columns=st.sets(st_columns), -) -@settings(deadline=timedelta(seconds=5)) -def test_hypothesis( - driver, - driver_name, - module_store, - test_cube, - test_df, - dimension_columns, - payload_columns, - conditions, -): - if driver_name != "eager": - pytest.skip("only eager is fast enough") - - result = driver( - cube=test_cube, - store=module_store, - dimension_columns=dimension_columns, - payload_columns=payload_columns, - conditions=conditions, - ) - - df_expected = ( - apply_condition_unsafe(test_df, conditions) - .sort_values(dimension_columns + list(test_cube.partition_columns)) - .loc[:, sorted({"x", "y", "z", "p", "q"} | payload_columns)] - .reset_index(drop=True) - ) - - if df_expected.empty: - assert len(result) == 0 - else: - assert len(result) == 1 - df_actual = result[0] - pdt.assert_frame_equal(df_actual, df_expected) diff --git a/kartothek/io/testing/stats_cube.py b/kartothek/io/testing/stats_cube.py deleted file mode 100644 index 8933b0ea..00000000 --- a/kartothek/io/testing/stats_cube.py +++ /dev/null @@ -1,97 +0,0 @@ -# -*- coding: utf-8 -*- -import dask.bag as db -import pandas as pd -import pytest - -from kartothek.core.cube.cube import Cube -from kartothek.io.dask.bag_cube import build_cube_from_bag -from kartothek.io.eager_cube import build_cube - -__all__ = ( - "test_fail_blocksize_negative", - "test_fail_blocksize_wrong_type", - "test_fail_blocksize_zero", - "test_fail_no_store_factory", - "test_multifile", - "test_simple", -) - - -def test_simple(driver, function_store, function_store_rwro): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - df_enrich = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "foo": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube( - data={cube.seed_dataset: df_seed, "enrich": df_enrich}, - cube=cube, - store=function_store, - ) - result = driver(cube=cube, store=function_store_rwro) - - assert set(result.keys()) == {cube.seed_dataset, "enrich"} - stats_seed = result[cube.seed_dataset] - - assert stats_seed["partitions"] == 2 - assert stats_seed["files"] == 2 - assert stats_seed["rows"] == 4 - assert stats_seed["blobsize"] > 0 - - stats_enrich = result["enrich"] - assert stats_enrich["partitions"] == stats_seed["partitions"] - assert stats_enrich["files"] == stats_seed["files"] - assert stats_enrich["rows"] == stats_seed["rows"] - assert stats_enrich["blobsize"] != stats_seed["blobsize"] - - -def test_multifile(driver, function_store): - dfs = [pd.DataFrame({"x": [i], "p": [0], "v1": [10]}) for i in range(2)] - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube_from_bag( - data=db.from_sequence(dfs, partition_size=1), cube=cube, store=function_store - ).compute() - - result = driver(cube=cube, store=function_store) - - assert set(result.keys()) == {cube.seed_dataset} - stats_seed = result[cube.seed_dataset] - assert stats_seed["partitions"] == 1 - assert stats_seed["files"] == 2 - assert stats_seed["rows"] == 2 - assert stats_seed["blobsize"] > 0 - - -def test_fail_no_store_factory(driver, function_store, skip_eager): - df_seed = pd.DataFrame( - {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} - ) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - build_cube(data=df_seed, cube=cube, store=function_store) - store = function_store() - with pytest.raises(TypeError) as exc: - driver(cube=cube, store=store, no_run=True) - assert str(exc.value) == "store must be a factory but is HFilesystemStore" - - -def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(TypeError, match="blocksize must be an integer but is str"): - driver(cube=cube, store=function_store, blocksize="foo") - - -def test_fail_blocksize_negative(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): - driver(cube=cube, store=function_store, blocksize=-1) - - -def test_fail_blocksize_zero(driver, function_store, skip_eager): - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") - - with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): - driver(cube=cube, store=function_store, blocksize=0) diff --git a/kartothek/io/testing/update_cube.py b/kartothek/io/testing/update_cube.py deleted file mode 100644 index ff1bff1a..00000000 --- a/kartothek/io/testing/update_cube.py +++ /dev/null @@ -1,367 +0,0 @@ -from typing import Tuple - -import numpy as np -import pandas as pd -import pytest - -from kartothek.core.cube.conditions import C -from kartothek.core.cube.cube import Cube -from kartothek.core.dataset import DatasetMetadata -from kartothek.io.eager import read_table -from kartothek.io.eager_cube import build_cube, extend_cube, query_cube -from kartothek.io.testing.utils import assert_num_row_groups -from kartothek.serialization._parquet import ParquetSerializer - - -def _write_cube(function_store) -> Tuple[pd.DataFrame, Cube]: - """ - Write a cube with dimension column "x" and partition column "p" - - returns the 'source' and 'enrich' dataframes and the cube specification. - """ - df_source = pd.DataFrame( - { - "i1": [10, 11, 12, 13], - "p": [0, 0, 1, 1], - "v1": [10, 11, 12, 13], - "x": [0, 1, 2, 3], - } - ) - cube = Cube( - dimension_columns=["x"], - partition_columns=["p"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["i1", "i2", "i3"], - ) - build_cube( - data={"source": df_source}, - cube=cube, - store=function_store, - metadata={"source": {"meta_at_create": "data"}}, - ) - return df_source, cube - - -def _extend_cube(cube, function_store) -> pd.DataFrame: - # extend the existing cube by a dataset 'ex' with columns a = x + 1000 - df = pd.DataFrame({"a": [1000, 1001], "p": [0, 1], "x": [0, 2]}) - extend_cube({"ex": df}, cube, function_store) - return df - - -@pytest.mark.parametrize( - "remove_partitions,new_partitions", - [ - # only append: - ([], [4, 5]), - # do nothing: - ([], []), - # partial overwrite with new data for p=0 - ([0], [0, 1, 4]), - # explicitly remove p=0 without overwriting it - ([0], [1, 4]), - # overwrite all: - ([0, 1], [0, 1]), - ], -) -def test_update_partitions(driver, function_store, remove_partitions, new_partitions): - df_source, cube = _write_cube(function_store) - - df_source_new = pd.DataFrame( - { - "i1": range(200, 200 + len(new_partitions)), - "p": np.array(new_partitions, np.int64), - "v1": range(300, 300 + len(new_partitions)), - "x": range(100, 100 + len(new_partitions)), - } - ) - - # what should remain of the old data: - df_source_of_old = df_source.loc[~df_source["p"].isin(set(remove_partitions))] - df_source_expected_after = pd.concat( - [df_source_of_old, df_source_new], sort=False, ignore_index=True - ) - - remove_conditions = C("p").isin(remove_partitions) - - result = driver( - data={"source": df_source_new}, - remove_conditions=remove_conditions, - cube=cube, - store=function_store, - ktk_cube_dataset_ids={"source"}, - metadata={"source": {"some_new_meta": 42}}, - ) - - assert set(result.keys()) == {"source"} - - dm_source_after = DatasetMetadata.load_from_store( - cube.ktk_dataset_uuid("source"), function_store(), load_all_indices=True - ) - - assert "some_new_meta" in dm_source_after.metadata - assert "meta_at_create" in dm_source_after.metadata - - # check values for "p" are as expected: - expected_p_source = (set(df_source["p"].unique()) - set(remove_partitions)) | set( - new_partitions - ) - assert set(dm_source_after.indices["p"].index_dct) == expected_p_source - - df_read = query_cube(cube, function_store)[0] - - assert set(df_read.columns) == set(df_source_expected_after.columns) - - for df in (df_read, df_source_expected_after): - df.sort_values("x", inplace=True) - df.reset_index(drop=True, inplace=True) - - pd.testing.assert_frame_equal(df_read, df_source_expected_after) - - -@pytest.mark.parametrize("chunk_size_build", [None, 2]) -@pytest.mark.parametrize("chunk_size_update", [None, 2]) -def test_rowgroups_are_applied_when_df_serializer_is_passed_to_update_cube( - driver, function_store, chunk_size_build, chunk_size_update -): - """ - Test that the dataset is split into row groups depending on the chunk size - - Partitions build with ``chunk_size=None`` should keep a single row group if they - are not touched by the update. Partitions that are newly created or replaced with - ``chunk_size>0`` should be split into row groups accordingly. - """ - # Build cube - df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size_build), - ) - - # Update cube - replace p=1 and append p=2 partitions - df_update = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], - ) - result = driver( - data={"seed": df_update}, - remove_conditions=(C("p") == 1), # Remove p=1 partition - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(chunk_size=chunk_size_update), - ) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 2, 2: 2} - part_chunk_size = { - 0: chunk_size_build, - 1: chunk_size_update, - 2: chunk_size_update, - } - - assert len(dataset.partitions) == 3 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_single_rowgroup_when_df_serializer_is_not_passed_to_update_cube( - driver, function_store -): - """ - Test that the dataset has a single row group as default path - """ - # Build cube - df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, cube=cube, store=function_store, - ) - - # Update cube - replace p=1 and append p=2 partitions - df_update = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], - ) - result = driver( - data={"seed": df_update}, - remove_conditions=(C("p") == 1), # Remove p=1 partition - cube=cube, - store=function_store, - ) - dataset = result["seed"].load_all_indices(function_store()) - - part_num_rows = {0: 1, 1: 2, 2: 2} - part_chunk_size = {0: None, 1: None, 2: None} - - assert len(dataset.partitions) == 3 - assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) - - -def test_compression_is_compatible_on_update_cube(driver, function_store): - """ - Test that partitons written with different compression algorithms are compatible - - The compression algorithms are not parametrized because their availability depends - on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts - of the code. A fully parametrized test would also increase runtime and test complexity - unnecessarily. - """ - # Build cube - df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) - cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") - build_cube( - data=df, - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="SNAPPY"), - ) - - # Update cube - replace p=1 and append p=2 partitions - df_update = pd.DataFrame( - data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], - ) - result = driver( - data={"seed": df_update}, - remove_conditions=(C("p") == 1), # Remove p=1 partition - cube=cube, - store=function_store, - df_serializer=ParquetSerializer(compression="GZIP"), - ) - dataset = result["seed"].load_all_indices(function_store()) - - assert len(dataset.partitions) == 3 - - -@pytest.mark.parametrize( - "ktk_cube_dataset_ids", [{"source", "ex"}, {"source"}, {"ex"}, set()] -) -def test_update_respects_ktk_cube_dataset_ids( - driver, function_store, ktk_cube_dataset_ids -): - df_source, cube = _write_cube(function_store) - df_ex = _extend_cube(cube, function_store) - - remove_conditions = C("p") == 0 - - # This implicitly also tests that `data={}` behaves as expected and still deletes partitions - # as requested via ktk_cube_dataset_ids and remove_conditions - result = driver( - data={}, - remove_conditions=remove_conditions, - cube=cube, - store=function_store, - ktk_cube_dataset_ids=ktk_cube_dataset_ids, - ) - assert set(result) == ktk_cube_dataset_ids - df_read = query_cube(cube, function_store)[0] - - # expected result: df_source left joined with df_ex; choosing the subset of p!=0 from each - # that is in `ktk_cube_dataset_ids`: - if "source" in ktk_cube_dataset_ids: - df_source = df_source.loc[df_source["p"] != 0] - if "ex" in ktk_cube_dataset_ids: - df_ex = df_ex.loc[df_ex["p"] != 0] - df_expected = df_source.merge(df_ex[["x", "a"]], how="left", on="x") - df_expected = df_expected[sorted(df_expected.columns)] - pd.testing.assert_frame_equal(df_read, df_expected) - - # test "ex" separately, because the test above based on the *left* merge does not tell us much about - # "ex" in case the partitions were removed from "source" - df_ex_read = read_table(cube.ktk_dataset_uuid("ex"), function_store) - if "ex" in ktk_cube_dataset_ids: - assert set(df_ex_read["p"]) == {1} - else: - assert set(df_ex_read["p"]) == {0, 1} - - -def test_cube_update_secondary_indices_subset(function_store, driver): - - cube1 = Cube( - dimension_columns=["A"], - partition_columns=["P"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["indexed"], - ) - df_1 = pd.DataFrame({"A": range(10), "P": 1, "indexed": 1, "not-indexed": 1}) - build_cube( - data={"source": df_1}, - cube=cube1, - store=function_store, - metadata={"source": {"meta_at_create": "data"}}, - ) - - cube2 = Cube( - dimension_columns=["A"], - partition_columns=["P"], - uuid_prefix="cube", - seed_dataset="source", - ) - df_2 = pd.DataFrame({"A": range(10, 20), "P": 1, "indexed": 2, "not-indexed": 1}) - driver( - data={"source": df_2}, cube=cube2, store=function_store, remove_conditions=None - ) - - dataset_uuid = cube2.ktk_dataset_uuid(cube2.seed_dataset) - dm = DatasetMetadata.load_from_store( - dataset_uuid, function_store(), load_all_indices=True - ) - obs_values = dm.indices["indexed"].observed_values() - - assert sorted(obs_values) == [1, 2] - - cube2 = Cube( - dimension_columns=["A"], - partition_columns=["P"], - uuid_prefix="cube", - seed_dataset="source", - index_columns=["not-indexed"], - ) - with pytest.raises( - ValueError, - match='ExplicitSecondaryIndex or PartitionIndex "not-indexed" is missing in dataset', - ): - driver( - data={"source": df_2}, - cube=cube2, - store=function_store, - remove_conditions=None, - ) - - -def test_cube_blacklist_dimension_index(function_store, driver): - - cube1 = Cube( - dimension_columns=["A", "B"], - partition_columns=["P"], - uuid_prefix="cube", - seed_dataset="source", - ) - df_1 = pd.DataFrame({"A": range(10), "P": 1, "B": 1, "payload": ""}) - build_cube( - data={"source": df_1}, - cube=cube1, - store=function_store, - metadata={"source": {"meta_at_create": "data"}}, - ) - - cube2 = Cube( - dimension_columns=["A", "B"], - partition_columns=["P"], - uuid_prefix="cube", - seed_dataset="source", - suppress_index_on=["B"], - ) - df_2 = pd.DataFrame({"A": range(10), "P": 1, "B": 2, "payload": ""}) - driver( - data={"source": df_2}, cube=cube2, store=function_store, remove_conditions=None - ) - - dataset_uuid = cube2.ktk_dataset_uuid(cube2.seed_dataset) - dm = DatasetMetadata.load_from_store( - dataset_uuid, function_store(), load_all_indices=True - ) - obs_values = dm.indices["B"].observed_values() - - assert sorted(obs_values) == [1, 2] diff --git a/kartothek/io/testing/utils.py b/kartothek/io/testing/utils.py deleted file mode 100644 index fb5c4f0f..00000000 --- a/kartothek/io/testing/utils.py +++ /dev/null @@ -1,41 +0,0 @@ -import math - -import numpy as np -import pandas as pd -from pyarrow.parquet import ParquetFile - -from kartothek.io.eager import store_dataframes_as_dataset - - -def create_dataset(dataset_uuid, store_factory, metadata_version): - df = pd.DataFrame( - {"P": np.arange(0, 10), "L": np.arange(0, 10), "TARGET": np.arange(10, 20)} - ) - - df_list = [df.copy(deep=True), df.copy(deep=True)] - - return store_dataframes_as_dataset( - dfs=df_list, - store=store_factory, - dataset_uuid=dataset_uuid, - metadata_version=metadata_version, - secondary_indices="P", - ) - - -def assert_num_row_groups(store, dataset, part_num_rows, part_chunk_size): - """ - Assert that the row groups of each partition match the expectation based on the - number of rows and the chunk size - """ - # Iterate over the partitions of each index value - for index, partitions in dataset.indices["p"].index_dct.items(): - for part_key in partitions: - key = dataset.partitions[part_key].files["table"] - parquet_file = ParquetFile(store.open(key)) - if part_chunk_size[index] is None: - assert parquet_file.num_row_groups == 1 - else: - assert parquet_file.num_row_groups == math.ceil( - part_num_rows[index] / part_chunk_size[index] - ) diff --git a/kartothek/io_components/cube/copy.py b/kartothek/io_components/cube/copy.py index decc21a4..f8e47d97 100644 --- a/kartothek/io_components/cube/copy.py +++ b/kartothek/io_components/cube/copy.py @@ -1,39 +1,46 @@ from __future__ import absolute_import from copy import copy +from typing import Callable, Dict, Iterable, Optional, Union + +from simplekv import KeyValueStore from kartothek.api.discover import check_datasets, discover_datasets_unchecked +from kartothek.core.cube.cube import Cube +from kartothek.core.dataset import DatasetMetadata from kartothek.utils.ktk_adapters import get_dataset_keys __all__ = ("get_copy_keys",) -def get_copy_keys(cube, src_store, tgt_store, overwrite, datasets=None): +def get_datasets_to_copy( + cube: Cube, + src_store: Union[Callable[[], KeyValueStore], KeyValueStore], + tgt_store: Union[Callable[[], KeyValueStore], KeyValueStore], + overwrite: bool, + datasets: Optional[Union[Iterable[str], Dict[str, DatasetMetadata]]] = None, +): """ - Get and check keys that should be copied from one store to another. + Determine all dataset names of a given cube that should be copied and apply addtional consistency checks. + Copying only a specific set of datasets is possible by providing a list of dataset names via the parameter `datasets`. Parameters ---------- - cube: kartothek.core.cube.cube.Cube + cube: Cube specification. - src_store: Union[Callable[[], simplekv.KeyValueStore], simplekv.KeyValueStore] + src_store: Source KV store. - tgt_store: Union[Callable[[], simplekv.KeyValueStore], simplekv.KeyValueStore] + tgt_store: Target KV store. - overwrite: bool + overwrite: If possibly existing datasets in the target store should be overwritten. - datasets: Union[None, Iterable[str], Dict[str, kartothek.core.dataset.DatasetMetadata]] + datasets: Datasets to copy, must all be part of the cube. May be either the result of :func:`~kartothek.api.discover.discover_datasets`, an iterable of Ktk_cube dataset ID or ``None`` (in which case entire cube will be copied). - Returns - ------- - keys: Set[str] - Set of keys to copy. - - Raises - ------ - RuntimeError: In case the copy would not pass successfully or if there is no cube in ``src_store``. + Return + ----- + All datasets that should be copied. """ if not isinstance(datasets, dict): new_datasets = discover_datasets_unchecked( @@ -71,6 +78,49 @@ def get_copy_keys(cube, src_store, tgt_store, overwrite, datasets=None): all_datasets.update(new_datasets) check_datasets(all_datasets, cube) + return new_datasets + + +def get_copy_keys( + cube: Cube, + src_store: Union[Callable[[], KeyValueStore], KeyValueStore], + tgt_store: Union[Callable[[], KeyValueStore], KeyValueStore], + overwrite: bool, + datasets: Optional[Union[Iterable[str], Dict[str, DatasetMetadata]]] = None, +): + """ + Get and check keys that should be copied from one store to another. + + Parameters + ---------- + cube: + Cube specification. + src_store: + Source KV store. + tgt_store: + Target KV store. + overwrite: + If possibly existing datasets in the target store should be overwritten. + datasets: + Datasets to copy, must all be part of the cube. May be either the result of :func:`~kartothek.api.discover.discover_datasets`, an + iterable of Ktk_cube dataset ID or ``None`` (in which case entire cube will be copied). + + Returns + ------- + keys: Set[str] + Set of keys to copy. + + Raises + ------ + RuntimeError: In case the copy would not pass successfully or if there is no cube in ``src_store``. + """ + new_datasets = get_datasets_to_copy( + cube=cube, + src_store=src_store, + tgt_store=tgt_store, + overwrite=overwrite, + datasets=datasets, + ) keys = set() for ktk_cube_dataset_id in sorted(new_datasets.keys()): diff --git a/kartothek/utils/store.py b/kartothek/utils/store.py index bae3474a..28648fb4 100644 --- a/kartothek/utils/store.py +++ b/kartothek/utils/store.py @@ -3,8 +3,10 @@ """ import logging import time +from typing import Callable, Dict, Iterable, Union from urllib.parse import quote +from simplekv import KeyValueStore from simplekv.contrib import VALID_KEY_RE_EXTENDED try: @@ -48,7 +50,6 @@ class _ResourceNotFoundError: # type: ignore _logger = logging.getLogger(__name__) - # Specialized implementation for azure-storage-blob < 12, using BlockBlobService (`bbs`): @@ -72,33 +73,38 @@ def _azure_bbs_content_md5(block_blob_service, container, key, accept_missing=Fa raise KeyError(key) -def _copy_azure_bbs(keys, src_store, tgt_store): +def _copy_azure_bbs(key_mappings, src_store, tgt_store): src_container = src_store.container tgt_container = tgt_store.container src_bbs = src_store.block_blob_service tgt_bbs = tgt_store.block_blob_service cprops = {} - for k in keys: + for src_key, tgt_key in key_mappings.items(): + source_md5 = _azure_bbs_content_md5( - src_bbs, src_container, k, accept_missing=False + src_bbs, src_container, src_key, accept_missing=False ) if source_md5 is None: - _logger.debug("Missing hash for {}".format(k)) + _logger.debug("Missing hash for {}".format(src_key)) else: tgt_md5 = _azure_bbs_content_md5( - tgt_bbs, tgt_container, k, accept_missing=True + tgt_bbs, tgt_container, tgt_key, accept_missing=True ) if source_md5 == tgt_md5: - _logger.debug("Omitting copy to {} (checksum match)".format(k)) + _logger.debug( + "Omitting copy from {} to {} (checksum match)".format( + src_key, tgt_key + ) + ) continue copy_source = src_bbs.make_blob_url( - src_container, quote(k), sas_token=src_bbs.sas_token + src_container, quote(src_key), sas_token=src_bbs.sas_token ) - cprops[k] = tgt_bbs.copy_blob(tgt_container, k, copy_source) + cprops[tgt_key] = tgt_bbs.copy_blob(tgt_container, tgt_key, copy_source) for k, cprop in cprops.items(): while True: @@ -140,27 +146,44 @@ def _azure_cc_content_md5(cc, key, accept_missing=False): raise KeyError(key) -def _copy_azure_cc(keys, src_store, tgt_store): +def _copy_azure_cc(key_mappings, src_store, tgt_store): + """ + Copies a list of items from one Azure store to another. + + Parameters + ---------- + key_mappings: Dict[str, str] + Mapping of source key names to target key names. May be equal if a key will + not be renamed. + src_store: KeyValueStore + Source KV store + tgt_store: KeyValueStore + Target KV store + """ src_cc = src_store.blob_container_client tgt_cc = tgt_store.blob_container_client copy_ids = {} - for k in keys: - source_md5 = _azure_cc_content_md5(src_cc, k, accept_missing=False) + for src_key, tgt_key in key_mappings.items(): + source_md5 = _azure_cc_content_md5(src_cc, src_key, accept_missing=False) if source_md5 is None: - _logger.debug("Missing hash for {}".format(k)) + _logger.debug("Missing hash for {}".format(src_key)) else: - tgt_md5 = _azure_cc_content_md5(tgt_cc, k, accept_missing=True) + tgt_md5 = _azure_cc_content_md5(tgt_cc, tgt_key, accept_missing=True) if source_md5 == tgt_md5: - _logger.debug("Omitting copy to {} (checksum match)".format(k)) + _logger.debug( + "Omitting copy from {} to {} (checksum match)".format( + src_key, tgt_key + ) + ) continue - copy_source = src_cc.get_blob_client(k).url - copy_ids[k] = tgt_cc.get_blob_client(k).start_copy_from_url(copy_source)[ - "copy_id" - ] + copy_source = src_cc.get_blob_client(src_key).url + copy_ids[tgt_key] = tgt_cc.get_blob_client(tgt_key).start_copy_from_url( + copy_source + )["copy_id"] for k, copy_id in copy_ids.items(): while True: @@ -181,19 +204,77 @@ def _copy_azure_cc(keys, src_store, tgt_store): ) -def _copy_naive(keys, src_store, tgt_store): - for k in keys: - tgt_store.put(k, src_store.get(k)) +def _copy_naive( + key_mappings: Dict[str, str], + src_store: KeyValueStore, + tgt_store: KeyValueStore, + mapped_metadata: Dict[str, bytes] = None, +): + """ + Copies a list of items from one KV store to another. + + Parameters + ---------- + key_mappings: Dict[str, str] + Mapping of source key names to target key names. May be equal if a key will + not be renamed. + src_store: simplekv.KeyValueStore + Source KV store + tgt_store: simplekv.KeyValueStore + Target KV store + mapped_metadata: Dict[str, bytes] + Mapping containing {key: modified metadata} values which will be written + directly instead of being copied + """ + for src_key, tgt_key in key_mappings.items(): + if (mapped_metadata is not None) and (src_key in mapped_metadata): + item = mapped_metadata.get(src_key) + else: + item = src_store.get(src_key) + tgt_store.put(tgt_key, item) + + +def copy_rename_keys( + key_mappings: Dict[str, str], + src_store: KeyValueStore, + tgt_store: KeyValueStore, + mapped_metadata: Dict[str, bytes], +): + """ + Copy keys between to stores or within one store, and rename them. + + Parameters + ---------- + key_mappings: Dict[str, str] + Dict with {old key: new key} mappings to rename keys during copying + src_store: simplekv.KeyValueStore + Source KV store. + tgt_store: simplekv.KeyValueStore + Target KV store. + mapped_metadata: Dict[str, bytes] + Dict with {source key: modified data} entries; the objects corresponding to + the keys will not be copied. Instead, the modified data will directly be put + to the target store. + """ + for k in key_mappings.keys(): + if (k is None) or (not VALID_KEY_RE_EXTENDED.match(k)) or (k == "/"): + raise ValueError("Illegal key: {}".format(k)) + _logger.debug("copy_rename_keys: Use naive slow-path.") + _copy_naive(key_mappings, src_store, tgt_store, mapped_metadata) -def copy_keys(keys, src_store, tgt_store): +def copy_keys( + keys: Iterable[str], + src_store: Union[KeyValueStore, Callable[[], KeyValueStore]], + tgt_store: Union[KeyValueStore, Callable[[], KeyValueStore]], +): """ - Copy keys from one store the another. + Copy keys between two stores or within one store. Parameters ---------- keys: Iterable[str] - Keys to copy. + Set of keys to copy without renaming; src_store: Union[simplekv.KeyValueStore, Callable[[], simplekv.KeyValueStore]] Source KV store. tgt_store: Union[simplekv.KeyValueStore, Callable[[], simplekv.KeyValueStore]] @@ -204,7 +285,9 @@ def copy_keys(keys, src_store, tgt_store): if callable(tgt_store): tgt_store = tgt_store() - keys = sorted(keys) + # otherwise, create a identity mapping dict which does not change the key + key_mappings = {src_key: src_key for src_key in keys} + for k in keys: if (k is None) or (not VALID_KEY_RE_EXTENDED.match(k)) or (k == "/"): raise ValueError("Illegal key: {}".format(k)) @@ -213,12 +296,12 @@ def copy_keys(keys, src_store, tgt_store): _logger.debug( "Azure stores based on BlockBlobStorage class detected, use fast-path." ) - _copy_azure_bbs(keys, src_store, tgt_store) + _copy_azure_bbs(key_mappings, src_store, tgt_store) elif _has_azure_cc(src_store) and _has_azure_cc(tgt_store): _logger.debug( "Azure stores based on ContainerClient class detected, use fast-path." ) - _copy_azure_cc(keys, src_store, tgt_store) + _copy_azure_cc(key_mappings, src_store, tgt_store) else: _logger.debug("Use naive slow-path.") - _copy_naive(keys, src_store, tgt_store) + _copy_naive(key_mappings, src_store, tgt_store) diff --git a/tests/conftest.py b/tests/conftest.py index be0d9c19..20a65db5 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -539,3 +539,18 @@ def evaluation_dataset(meta_partitions_evaluation_files_only, store_session): dataset_uuid="evaluation_uuid", store=store_session, ) + + +@pytest.fixture(params=[True, False], ids=["use_categoricals", "no_categoricals"]) +def use_categoricals(request): + return request.param + + +@pytest.fixture(params=[True, False], ids=["use_factory", "no_factory"]) +def use_dataset_factory(request, dates_as_object): + return request.param + + +@pytest.fixture(params=[True, False], ids=["dates_as_object", "datest_as_datetime"]) +def dates_as_object(request): + return request.param diff --git a/tests/core/test_builder.py b/tests/core/test_builder.py index 0c511458..767bb66c 100644 --- a/tests/core/test_builder.py +++ b/tests/core/test_builder.py @@ -69,6 +69,48 @@ def test_builder_to_dataset(metadata_version, frozen_time): assert result == expected_from_dict +def test_builder_modify_uuid_embedded_index(metadata_version, frozen_time): + expected = { + "dataset_uuid": "uuid_new", + "dataset_metadata_version": metadata_version, + "partitions": {"part_2": {"files": {"core": "uuid_new/core/part_2.parquet"}}}, + "metadata": {"key": "value", "creation_time": TIME_TO_FREEZE_ISO}, + "indices": {"col1": {"a": ["part1"], "b": ["part2"]}}, + } + + builder = DatasetMetadataBuilder("uuid", metadata_version=metadata_version) + part_2 = Partition("part_2", {"core": "uuid/core/part_2.parquet"}) + builder.add_partition("part_2", part_2) + builder.add_metadata("key", "value") + builder.add_embedded_index( + "col1", ExplicitSecondaryIndex("col1", {"a": ["part1"], "b": ["part2"]}) + ) + builder.modify_uuid("uuid_new") + result = builder.to_dataset() + expected_from_dict = DatasetMetadata.from_dict(expected) + assert result == expected_from_dict + + +def test_builder_modify_uuid_external_index(metadata_version, frozen_time): + expected = { + "dataset_uuid": "uuid_new", + "dataset_metadata_version": metadata_version, + "partitions": {"part_2": {"files": {"core": "uuid_new/core/part_2.parquet"}}}, + "metadata": {"key": "value", "creation_time": TIME_TO_FREEZE_ISO}, + "indices": {"col1": "uuid_new.col1.by-dataset-index.parquet"}, + } + + builder = DatasetMetadataBuilder("uuid", metadata_version=metadata_version) + part_2 = Partition("part_2", {"core": "uuid/core/part_2.parquet"}) + builder.add_partition("part_2", part_2) + builder.add_metadata("key", "value") + builder.add_external_index("col1") + builder.modify_uuid("uuid_new") + result = builder.to_dataset() + expected_from_dict = DatasetMetadata.from_dict(expected) + assert result == expected_from_dict + + def test_builder_full(metadata_version, frozen_time): expected = { "dataset_uuid": "uuid", diff --git a/kartothek/io/testing/__init__.py b/tests/io/common/__init__.py similarity index 100% rename from kartothek/io/testing/__init__.py rename to tests/io/common/__init__.py diff --git a/tests/io/common/conftest.py b/tests/io/common/conftest.py new file mode 100644 index 00000000..bf5c51d2 --- /dev/null +++ b/tests/io/common/conftest.py @@ -0,0 +1,443 @@ +import pickle +from functools import partial + +import dask +import dask.bag as db +import dask.dataframe as dd +import numpy as np +import pandas as pd +import pytest +from storefact import get_store_from_url + +from kartothek.io.dask.bag import ( + build_dataset_indices__bag, + read_dataset_as_dataframe_bag, + read_dataset_as_metapartitions_bag, + store_bag_as_dataset, +) +from kartothek.io.dask.dataframe import read_dataset_as_ddf, update_dataset_from_ddf +from kartothek.io.dask.delayed import ( + delete_dataset__delayed, + garbage_collect_dataset__delayed, + read_dataset_as_delayed, + store_delayed_as_dataset, + update_dataset_from_delayed, +) +from kartothek.io.eager import ( + build_dataset_indices, + delete_dataset, + garbage_collect_dataset, + read_dataset_as_dataframes, + read_table, + store_dataframes_as_dataset, + update_dataset_from_dataframes, +) +from kartothek.io.iter import ( + read_dataset_as_dataframes__iterator, + store_dataframes_as_dataset__iter, + update_dataset_from_dataframes__iter, +) +from kartothek.io_components.metapartition import SINGLE_TABLE + + +class NoPickle: + def __getstate__(self): + raise RuntimeError("do NOT pickle this object!") + + +def mark_nopickle(obj): + setattr(obj, "_nopickle", NoPickle()) + + +def no_pickle_factory(url): + return partial(no_pickle_store, url) + + +def no_pickle_store(url): + store = get_store_from_url(url) + mark_nopickle(store) + return store + + +@pytest.fixture(params=["URL", "KeyValue", "Callable"]) +def store_input_types(request, tmpdir): + url = f"hfs://{tmpdir}" + + if request.param == "URL": + return url + elif request.param == "KeyValue": + return get_store_from_url(url) + elif request.param == "Callable": + return no_pickle_factory(url) + else: + raise RuntimeError(f"Encountered unknown store type {type(request.param)}") + + +@pytest.fixture(params=["eager", "iter", "dask.bag", "dask.delayed", "dask.dataframe"]) +def backend_identifier(request): + return request.param + + +@pytest.fixture(params=["dataframe", "table"]) +def output_type(request, backend_identifier): + if (backend_identifier in ["iter", "dask.bag", "dask.delayed"]) and ( + request.param == "table" + ): + pytest.skip() + if (backend_identifier == "dask.dataframe") and (request.param == "dataframe"): + pytest.skip() + return request.param + + +@pytest.fixture(scope="session") +def dataset_dispatch_by_uuid(): + import uuid + + return uuid.uuid1().hex + + +@pytest.fixture(scope="session") +def dataset_dispatch_by( + metadata_version, store_session_factory, dataset_dispatch_by_uuid +): + cluster1 = pd.DataFrame( + {"A": [1, 1], "B": [10, 10], "C": [1, 2], "Content": ["cluster1", "cluster1"]} + ) + cluster2 = pd.DataFrame( + {"A": [1, 1], "B": [10, 10], "C": [2, 3], "Content": ["cluster2", "cluster2"]} + ) + cluster3 = pd.DataFrame({"A": [1], "B": [20], "C": [1], "Content": ["cluster3"]}) + cluster4 = pd.DataFrame( + {"A": [2, 2], "B": [10, 10], "C": [1, 2], "Content": ["cluster4", "cluster4"]} + ) + clusters = [cluster1, cluster2, cluster3, cluster4] + + store_dataframes_as_dataset__iter( + df_generator=clusters, + store=store_session_factory, + dataset_uuid=dataset_dispatch_by_uuid, + metadata_version=metadata_version, + partition_on=["A", "B"], + secondary_indices=["C"], + ) + return pd.concat(clusters).sort_values(["A", "B", "C"]).reset_index(drop=True) + + +def _read_table(*args, **kwargs): + kwargs.pop("dispatch_by", None) + res = read_table(*args, **kwargs) + + if len(res): + # Array split conserves dtypes + return np.array_split(res, len(res)) + else: + return [res] + + +def _load_dataframes_iter(output_type, *args, **kwargs): + if output_type == "dataframe": + func = read_dataset_as_dataframes__iterator + else: + raise ValueError("Unknown output type {}".format(output_type)) + return list(func(*args, **kwargs)) + + +# FIXME: handle removal of metparittion function properly. +# FIXME: consolidate read_Dataset_as_dataframes (replaced by iter) +def _read_dataset_eager(output_type, *args, **kwargs): + if output_type == "table": + return _read_table + elif output_type == "dataframe": + return read_dataset_as_dataframes + else: + raise NotImplementedError() + + +def _load_dataframes_bag(output_type, *args, **kwargs): + if output_type == "dataframe": + func = read_dataset_as_dataframe_bag + elif output_type == "metapartition": + func = read_dataset_as_metapartitions_bag + tasks = func(*args, **kwargs) + + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + + result = tasks.compute() + return result + + +def _load_dataframes_delayed(output_type, *args, **kwargs): + if "tables" in kwargs: + param_tables = kwargs.pop("tables") + kwargs["table"] = param_tables + func = partial(read_dataset_as_delayed) + tasks = func(*args, **kwargs) + + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + + result = [task.compute() for task in tasks] + return result + + +def _read_as_ddf( + dataset_uuid, + store, + factory=None, + categoricals=None, + tables=None, + dataset_has_index=False, + **kwargs, +): + table = tables or SINGLE_TABLE + + ddf = read_dataset_as_ddf( + dataset_uuid=dataset_uuid, + store=store, + factory=factory, + categoricals=categoricals, + table=table, + **kwargs, + ) + if categoricals: + assert ddf._meta.dtypes["P"] == pd.api.types.CategoricalDtype( + categories=["__UNKNOWN_CATEGORIES__"], ordered=False + ) + if dataset_has_index: + assert ddf._meta.dtypes["L"] == pd.api.types.CategoricalDtype( + categories=[1, 2], ordered=False + ) + else: + assert ddf._meta.dtypes["L"] == pd.api.types.CategoricalDtype( + categories=["__UNKNOWN_CATEGORIES__"], ordered=False + ) + + s = pickle.dumps(ddf, pickle.HIGHEST_PROTOCOL) + ddf = pickle.loads(s) + + ddf = ddf.compute().reset_index(drop=True) + + def extract_dataframe(ix): + df = ddf.iloc[[ix]].copy() + for col in df.columns: + if pd.api.types.is_categorical_dtype(df[col]): + df[col] = df[col].cat.remove_unused_categories() + return df.reset_index(drop=True) + + return [extract_dataframe(ix) for ix in ddf.index] + + +@pytest.fixture() +def bound_load_dataframes(output_type, backend_identifier): + if backend_identifier == "eager": + return _read_dataset_eager(output_type) + elif backend_identifier == "iter": + return partial(_load_dataframes_iter, output_type) + elif backend_identifier == "dask.bag": + return partial(_load_dataframes_bag, output_type) + elif backend_identifier == "dask.delayed": + return partial(_load_dataframes_delayed, output_type) + elif backend_identifier == "dask.dataframe": + return _read_as_ddf + else: + raise NotImplementedError + + +def store_dataframes_eager(dfs, **kwargs): + # Positional arguments in function but `None` is acceptable input + for kw in ("dataset_uuid", "store"): + if kw not in kwargs: + kwargs[kw] = None + + return store_dataframes_as_dataset(dfs=dfs, **kwargs) + + +def _store_dataframes_iter(df_list, *args, **kwargs): + df_generator = (x for x in df_list) + return store_dataframes_as_dataset__iter(df_generator, *args, **kwargs) + + +def _store_dataframes_dask_bag(df_list, *args, **kwargs): + bag = store_bag_as_dataset(db.from_sequence(df_list), *args, **kwargs) + s = pickle.dumps(bag, pickle.HIGHEST_PROTOCOL) + bag = pickle.loads(s) + return bag.compute() + + +def _store_dataframes_dask_delayed(df_list, *args, **kwargs): + tasks = store_delayed_as_dataset(df_list, *args, **kwargs) + + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + + return tasks.compute() + + +@pytest.fixture() +def bound_store_dataframes(backend_identifier): + if backend_identifier == "eager": + return store_dataframes_eager + elif backend_identifier == "iter": + return _store_dataframes_iter + elif backend_identifier == "dask.bag": + return _store_dataframes_dask_bag + elif backend_identifier == "dask.delayed": + return _store_dataframes_dask_delayed + elif backend_identifier == "dask.dataframe": + # not implemented for dask.dataframe + pytest.skip() + else: + raise NotImplementedError + + +def _update_dataset_iter(df_list, *args, **kwargs): + if isinstance(df_list, pd.DataFrame): + df_list = [df_list] + df_generator = (x for x in df_list) + return update_dataset_from_dataframes__iter(df_generator, *args, **kwargs) + + +def _update_dataset_delayed(partitions, *args, **kwargs): + if not isinstance(partitions, list): + partitions = [partitions] + tasks = update_dataset_from_delayed(partitions, *args, **kwargs) + + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + + return tasks.compute() + + +def _id(part): + if isinstance(part, pd.DataFrame): + return part + else: + return part[0] + + +def update_dataset_dataframe(partitions, *args, **kwargs): + # TODO: Simplify once parse_input_to_metapartition is removed / obsolete + + if isinstance(partitions, pd.DataFrame): + partitions = dd.from_pandas(partitions, npartitions=1) + elif partitions is not None: + delayed_partitions = [dask.delayed(_id)(part) for part in partitions] + partitions = dd.from_delayed(delayed_partitions) + else: + partitions = None + + # Replace `table_name` with `table` keyword argument to enable shared test code + # via `bound_update_dataset` fixture + if "table_name" in kwargs: + kwargs["table"] = kwargs["table_name"] + del kwargs["table_name"] + + ddf = update_dataset_from_ddf(partitions, *args, **kwargs) + + s = pickle.dumps(ddf, pickle.HIGHEST_PROTOCOL) + ddf = pickle.loads(s) + + return ddf.compute() + + +def _return_none(): + return None + + +@pytest.fixture() +def bound_update_dataset(backend_identifier): + if backend_identifier == "eager": + return update_dataset_from_dataframes + elif backend_identifier == "iter": + return _update_dataset_iter + elif backend_identifier == "dask.bag": + # no tests impleme ted for update and dask.bag + pytest.skip() + elif backend_identifier == "dask.delayed": + return _update_dataset_delayed + elif backend_identifier == "dask.dataframe": + return update_dataset_dataframe + else: + raise NotImplementedError + + +def _delete_delayed(*args, **kwargs): + tasks = delete_dataset__delayed(*args, **kwargs) + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + dask.compute(tasks) + + +@pytest.fixture() +def bound_delete_dataset(backend_identifier): + if backend_identifier == "eager": + return delete_dataset + elif backend_identifier == "iter": + # no tests implemented for delete and iter + pytest.skip() + elif backend_identifier == "dask.bag": + # no tests implemented for update and dask.bag + pytest.skip() + elif backend_identifier == "dask.delayed": + return _delete_delayed + elif backend_identifier == "dask.dataframe": + # no tests implemented for update and dask.dataframe + pytest.skip() + else: + raise NotImplementedError + + +def _build_indices_bag(*args, **kwargs): + bag = build_dataset_indices__bag(*args, **kwargs) + + # pickle roundtrip to ensure we don't need the inefficient cloudpickle fallback + s = pickle.dumps(bag, pickle.HIGHEST_PROTOCOL) + bag = pickle.loads(s) + + bag.compute() + + +@pytest.fixture() +def bound_build_dataset_indices(backend_identifier): + if backend_identifier == "eager": + return build_dataset_indices + elif backend_identifier == "iter": + # no tests implemented for index and iter + pytest.skip() + elif backend_identifier == "dask.bag": + return _build_indices_bag + elif backend_identifier == "dask.delayed": + # no tests implemented for index and dask.delayed + pytest.skip() + elif backend_identifier == "dask.dataframe": + # no tests implemented for index and dask.dataframe + pytest.skip() + else: + raise NotImplementedError + + +def _run_garbage_collect_delayed(*args, **kwargs): + tasks = garbage_collect_dataset__delayed(*args, **kwargs) + s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) + tasks = pickle.loads(s) + dask.compute(tasks) + + +@pytest.fixture() +def garbage_collect_callable(backend_identifier): + if backend_identifier == "eager": + return garbage_collect_dataset + elif backend_identifier == "iter": + # no tests implemented for gc and iter + pytest.skip() + elif backend_identifier == "dask.bag": + # no tests implemented for gc and dask.bag + pytest.skip() + elif backend_identifier == "dask.delayed": + return _run_garbage_collect_delayed + elif backend_identifier == "dask.dataframe": + # no tests implemented for gc and dask.dataframe + pytest.skip() + else: + raise NotImplementedError diff --git a/kartothek/io/testing/delete.py b/tests/io/common/test_delete.py similarity index 100% rename from kartothek/io/testing/delete.py rename to tests/io/common/test_delete.py diff --git a/kartothek/io/testing/gc.py b/tests/io/common/test_gc.py similarity index 100% rename from kartothek/io/testing/gc.py rename to tests/io/common/test_gc.py diff --git a/kartothek/io/testing/index.py b/tests/io/common/test_index.py similarity index 99% rename from kartothek/io/testing/index.py rename to tests/io/common/test_index.py index 73e0e310..c2feeb4a 100644 --- a/kartothek/io/testing/index.py +++ b/tests/io/common/test_index.py @@ -1,4 +1,3 @@ -# -*- coding: utf-8 -*- import numpy as np import pandas as pd import pytest diff --git a/kartothek/io/testing/read.py b/tests/io/common/test_read.py similarity index 80% rename from kartothek/io/testing/read.py rename to tests/io/common/test_read.py index 073fc06b..52a45d69 100644 --- a/kartothek/io/testing/read.py +++ b/tests/io/common/test_read.py @@ -1,218 +1,25 @@ """ -This module is a collection of tests which should be implemented by all kartothek -**read** backends. The tests are not subject to the semantic versioning scheme and may change with minor or even patch releases. - -To use the tests of this module, add the following import statement to your test module and ensure that the following fixtures are available in your test environment. - -``` -from kartothek.io.testing.read import * # noqa -``` - -Fixtures required to be implemented: - -* ``output_type`` - One of {`dataframe`, `metpartition`, `table`} to define the outptu type of the returned result. -* ``bound_load_dataframes`` - A callable which will retrieve the partitions in the format specified by ``output_type``. The callable should accept all keyword arguments expected for a kartothek reader. - -Source test data - -* ``dataset`` - A fixture generating test data (TODO: Expose this as a testing function) -* ``store_factory`` - A function scoped store factory -* ``store_session_factory`` - A session scoped store factory - -Feature toggles (optional): - -The following fixtures should be present (see tests.read.conftest) -* ``use_categoricals`` - Whether or not the call retrievs categorical data. -* ``dates_as_object`` - Whether or not the call retrievs date columns as objects. - +This test suite all reading tests which were formerly contained in the +kartothek.io.testing package. +All tests in this test program will be executed for different backends: +eager, iter, dask.[bag, dataframe, delayed] +and for other combinations of fixtures. This will be controlled in conftest.py. + +Since certain combinations of fixtures will not make sense, these tests will be skipped. +This results in a relatively large number of tests which will be skipped in total. """ import datetime -from functools import partial from itertools import permutations import pandas as pd import pandas.testing as pdt import pytest -from storefact import get_store_from_url from kartothek.io.eager import store_dataframes_as_dataset -from kartothek.io.iter import store_dataframes_as_dataset__iter from kartothek.io_components.metapartition import SINGLE_TABLE, MetaPartition -@pytest.fixture(params=[True, False], ids=["use_categoricals", "no_categoricals"]) -def use_categoricals(request): - return request.param - - -@pytest.fixture(params=[True, False], ids=["dates_as_object", "datest_as_datetime"]) -def dates_as_object(request): - return request.param - - -@pytest.fixture(params=[True, False], ids=["use_factory", "no_factory"]) -def use_dataset_factory(request, dates_as_object): - return request.param - - -class NoPickle: - def __getstate__(self): - raise RuntimeError("do NOT pickle this object!") - - -def mark_nopickle(obj): - setattr(obj, "_nopickle", NoPickle()) - - -def no_pickle_store(url): - store = get_store_from_url(url) - mark_nopickle(store) - return store - - -def no_pickle_factory(url): - - return partial(no_pickle_store, url) - - -@pytest.fixture(params=["URL", "KeyValue", "Callable"]) -def store_input_types(request, tmpdir): - url = f"hfs://{tmpdir}" - - if request.param == "URL": - return url - elif request.param == "KeyValue": - return get_store_from_url(url) - elif request.param == "Callable": - return no_pickle_factory(url) - else: - raise RuntimeError(f"Encountered unknown store type {type(request.param)}") - - -def test_store_input_types(store_input_types, bound_load_dataframes): - from kartothek.io.eager import store_dataframes_as_dataset - from kartothek.serialization.testing import get_dataframe_not_nested - - dataset_uuid = "dataset_uuid" - df = get_dataframe_not_nested(10) - - store_dataframes_as_dataset( - dfs=[df], - dataset_uuid=dataset_uuid, - store=store_input_types, - partition_on=[df.columns[0]], - secondary_indices=[df.columns[1]], - ) - - # Use predicates to trigger partition pruning with indices - predicates = [ - [ - (df.columns[0], "==", df.loc[0, df.columns[0]]), - (df.columns[1], "==", df.loc[0, df.columns[1]]), - ] - ] - - result = bound_load_dataframes( - dataset_uuid=dataset_uuid, - store=store_input_types, - predicates=predicates, - dates_as_object=True, - ) - - if isinstance(result, list): - result = result[0] - - if isinstance(result, MetaPartition): - result = result.data - - if isinstance(result, dict): - result = result[SINGLE_TABLE] - - pdt.assert_frame_equal(result, df.head(1), check_dtype=False) - - -def _perform_read_test( - dataset_uuid, - store_factory, - execute_read_callable, - use_categoricals, - output_type, - dates_as_object, - read_kwargs=None, - ds_factory=None, -): - if not read_kwargs: - read_kwargs = {} - if use_categoricals: - # dataset_with_index has an index on L but not on P - categoricals = ["P", "L"] - else: - categoricals = None - - result = execute_read_callable( - dataset_uuid=dataset_uuid, - store=store_factory, - factory=ds_factory, - categoricals=categoricals, - dates_as_object=dates_as_object, - **read_kwargs, - ) - - assert len(result) == 2 - - if output_type == "metapartition": - for res in result: - assert isinstance(res, MetaPartition) - result = [mp.data for mp in result] - - def sort_by(obj): - return obj[SINGLE_TABLE].P.iloc[0] - - elif output_type == "table": - assert isinstance(result[0], pd.DataFrame) - assert "P" in result[0] - - def sort_by(obj): - return obj.P.iloc[0] - - else: - assert isinstance(result[0], pd.DataFrame) - assert "P" in result[0] - - def sort_by(obj): - return obj.P.iloc[0] - - result = sorted(result, key=sort_by) - - expected_df_core_1 = pd.DataFrame( - {"P": [1], "L": [1], "TARGET": [1], "DATE": [datetime.date(2010, 1, 1)]} - ) - expected_df_core_2 = pd.DataFrame( - {"P": [2], "L": [2], "TARGET": [2], "DATE": [datetime.date(2009, 12, 31)]} - ) - expected_dfs = [ - expected_df_core_1, - expected_df_core_2, - ] - - for res, expected_df_core in zip(result, expected_dfs): - if not dates_as_object: - expected_df_core["DATE"] = pd.to_datetime(expected_df_core["DATE"]) - if use_categoricals: - expected_df_core = expected_df_core.astype( - {"P": "category", "L": "category"} - ) - - pdt.assert_frame_equal( - res.reset_index(drop=True), - expected_df_core.reset_index(drop=True), - check_dtype=False, - check_like=True, - check_categorical=False, - ) - - @pytest.mark.parametrize( "predicates", [ @@ -291,11 +98,6 @@ def test_read_dataset_as_dataframes_predicate_empty( assert len(result) == 0 -def _gen_partition(b_c): - b, c = b_c - return pd.DataFrame({"a": [1], "b": [b], "c": c}) - - def test_read_dataset_as_dataframes_dispatch_by_empty( store_session_factory, dataset_dispatch_by, @@ -345,40 +147,6 @@ def test_read_dataset_as_dataframes_dispatch_by_single_col( unique_a.add(unique_dispatch[0]) -@pytest.fixture(scope="session") -def dataset_dispatch_by_uuid(): - import uuid - - return uuid.uuid1().hex - - -@pytest.fixture(scope="session") -def dataset_dispatch_by( - metadata_version, store_session_factory, dataset_dispatch_by_uuid -): - cluster1 = pd.DataFrame( - {"A": [1, 1], "B": [10, 10], "C": [1, 2], "Content": ["cluster1", "cluster1"]} - ) - cluster2 = pd.DataFrame( - {"A": [1, 1], "B": [10, 10], "C": [2, 3], "Content": ["cluster2", "cluster2"]} - ) - cluster3 = pd.DataFrame({"A": [1], "B": [20], "C": [1], "Content": ["cluster3"]}) - cluster4 = pd.DataFrame( - {"A": [2, 2], "B": [10, 10], "C": [1, 2], "Content": ["cluster4", "cluster4"]} - ) - clusters = [cluster1, cluster2, cluster3, cluster4] - - store_dataframes_as_dataset__iter( - df_generator=clusters, - store=store_session_factory, - dataset_uuid=dataset_dispatch_by_uuid, - metadata_version=metadata_version, - partition_on=["A", "B"], - secondary_indices=["C"], - ) - return pd.concat(clusters).sort_values(["A", "B", "C"]).reset_index(drop=True) - - def test_read_dataset_as_dataframes_dispatch_by_multi_col( store_session_factory, bound_load_dataframes, @@ -444,6 +212,87 @@ def test_read_dispatch_by_with_predicates( assert len(dispatched) == expected_dispatches, dispatched +def _perform_read_test( + dataset_uuid, + store_factory, + execute_read_callable, + use_categoricals, + output_type, + dates_as_object, + read_kwargs=None, + ds_factory=None, +): + if not read_kwargs: + read_kwargs = {} + if use_categoricals: + # dataset_with_index has an index on L but not on P + categoricals = ["P", "L"] + else: + categoricals = None + + result = execute_read_callable( + dataset_uuid=dataset_uuid, + store=store_factory, + factory=ds_factory, + categoricals=categoricals, + dates_as_object=dates_as_object, + **read_kwargs, + ) + + assert len(result) == 2 + + if output_type == "metapartition": + for res in result: + assert isinstance(res, MetaPartition) + result = [mp.data for mp in result] + + def sort_by(obj): + return obj[SINGLE_TABLE].P.iloc[0] + + elif output_type == "table": + assert isinstance(result[0], pd.DataFrame) + assert "P" in result[0] + + def sort_by(obj): + return obj.P.iloc[0] + + else: + assert isinstance(result[0], pd.DataFrame) + assert "P" in result[0] + + def sort_by(obj): + return obj.P.iloc[0] + + result = sorted(result, key=sort_by) + + expected_df_core_1 = pd.DataFrame( + {"P": [1], "L": [1], "TARGET": [1], "DATE": [datetime.date(2010, 1, 1)]} + ) + expected_df_core_2 = pd.DataFrame( + {"P": [2], "L": [2], "TARGET": [2], "DATE": [datetime.date(2009, 12, 31)]} + ) + expected_dfs = [ + expected_df_core_1, + expected_df_core_2, + ] + + for res, expected_df_core in zip(result, expected_dfs): + if not dates_as_object: + expected_df_core["DATE"] = pd.to_datetime(expected_df_core["DATE"]) + if use_categoricals: + expected_df_core = expected_df_core.astype( + {"P": "category", "L": "category"} + ) + + pdt.assert_frame_equal( + res.reset_index(drop=True), + expected_df_core.reset_index(drop=True), + check_dtype=False, + check_like=True, + check_categorical=False, + ) + + def test_read_dataset_as_dataframes( dataset, store_session_factory, @@ -474,6 +323,48 @@ def test_read_dataset_as_dataframes( ) +def test_store_input_types(store_input_types, bound_load_dataframes): + from kartothek.io.eager import store_dataframes_as_dataset + from kartothek.serialization.testing import get_dataframe_not_nested + + dataset_uuid = "dataset_uuid" + df = get_dataframe_not_nested(10) + + store_dataframes_as_dataset( + dfs=[df], + dataset_uuid=dataset_uuid, + store=store_input_types, + partition_on=[df.columns[0]], + secondary_indices=[df.columns[1]], + ) + + # Use predicates to trigger partition pruning with indices + predicates = [ + [ + (df.columns[0], "==", df.loc[0, df.columns[0]]), + (df.columns[1], "==", df.loc[0, df.columns[1]]), + ] + ] + + result = bound_load_dataframes( + dataset_uuid=dataset_uuid, + store=store_input_types, + predicates=predicates, + dates_as_object=True, + ) + + if isinstance(result, list): + result = result[0] + + if isinstance(result, MetaPartition): + result = result.data + + if isinstance(result, dict): + result = result[SINGLE_TABLE] + + pdt.assert_frame_equal(result, df.head(1), check_dtype=False) + + def test_read_dataset_as_dataframes_columns_projection( store_factory, bound_load_dataframes, metadata_version ): diff --git a/kartothek/io/testing/update.py b/tests/io/common/test_update.py similarity index 99% rename from kartothek/io/testing/update.py rename to tests/io/common/test_update.py index ea9bcd68..b7867313 100644 --- a/kartothek/io/testing/update.py +++ b/tests/io/common/test_update.py @@ -1,4 +1,4 @@ -# -*- coding: utf-8 -*- +# coding: utf-8 -*- # pylint: disable=E1101 diff --git a/kartothek/io/testing/write.py b/tests/io/common/test_write.py similarity index 95% rename from kartothek/io/testing/write.py rename to tests/io/common/test_write.py index 1995a945..63a25583 100644 --- a/kartothek/io/testing/write.py +++ b/tests/io/common/test_write.py @@ -1,15 +1,9 @@ -# -*- coding: utf-8 -*- -# pylint: disable=E1101 - - from collections import OrderedDict -from functools import partial import numpy as np import pandas as pd import pandas.testing as pdt import pytest -from storefact import get_store_from_url from kartothek.core.dataset import DatasetMetadata from kartothek.core.uuid import gen_uuid @@ -18,40 +12,6 @@ from kartothek.serialization import DataFrameSerializer -class NoPickle: - def __getstate__(self): - raise RuntimeError("do NOT pickle this object!") - - -def mark_nopickle(obj): - setattr(obj, "_nopickle", NoPickle()) - - -def no_pickle_store(url): - store = get_store_from_url(url) - mark_nopickle(store) - return store - - -def no_pickle_factory(url): - - return partial(no_pickle_store, url) - - -@pytest.fixture(params=["URL", "KeyValue", "Callable"]) -def store_input_types(request, tmpdir): - url = f"hfs://{tmpdir}" - - if request.param == "URL": - return url - elif request.param == "KeyValue": - return get_store_from_url(url) - elif request.param == "Callable": - return no_pickle_factory(url) - else: - raise RuntimeError(f"Encountered unknown store type {type(request.param)}") - - def test_store_input_types(store_input_types, bound_store_dataframes): from kartothek.serialization.testing import get_dataframe_not_nested diff --git a/tests/io/common/utils.py b/tests/io/common/utils.py new file mode 100644 index 00000000..71c877d4 --- /dev/null +++ b/tests/io/common/utils.py @@ -0,0 +1,20 @@ +import numpy as np +import pandas as pd + +from kartothek.io.eager import store_dataframes_as_dataset + + +def create_dataset(dataset_uuid, store_factory, metadata_version): + df = pd.DataFrame( + {"P": np.arange(0, 10), "L": np.arange(0, 10), "TARGET": np.arange(10, 20)} + ) + + df_list = [df.copy(deep=True), df.copy(deep=True)] + + return store_dataframes_as_dataset( + dfs=df_list, + store=store_factory, + dataset_uuid=dataset_uuid, + metadata_version=metadata_version, + secondary_indices="P", + ) diff --git a/tests/io/cube/test_append.py b/tests/io/cube/test_append.py index ab94e80f..102612b6 100644 --- a/tests/io/cube/test_append.py +++ b/tests/io/cube/test_append.py @@ -1,10 +1,21 @@ +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_write, wrap_ddf_write +from kartothek.core.cube.constants import ( + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, +) +from kartothek.core.cube.cube import Cube +from kartothek.core.dataset import DatasetMetadata from kartothek.io.dask.bag_cube import append_to_cube_from_bag from kartothek.io.dask.dataframe_cube import append_to_cube_from_dataframe -from kartothek.io.eager_cube import append_to_cube -from kartothek.io.testing.append_cube import * # noqa +from kartothek.io.eager_cube import append_to_cube, build_cube +from kartothek.serialization._parquet import ParquetSerializer + +from .utils import assert_num_row_groups @pytest.fixture @@ -19,3 +30,434 @@ def driver(driver_name): return append_to_cube else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +@pytest.fixture +def existing_cube(function_store): + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [10, 11, 12, 13], + } + ) + df_enrich = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v2": [10, 11, 12, 13], + "i2": [10, 11, 12, 13], + } + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + metadata={"source": {"a": 10, "b": 11}, "enrich": {"a": 20, "b": 21}}, + ) + return cube + + +def test_append_partitions(driver, function_store, existing_cube): + partitions_source_1 = set( + DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("source"), function_store() + ).partitions.keys() + ) + partitions_enrich_1 = set( + DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("enrich"), function_store() + ).partitions.keys() + ) + + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [20, 21, 22, 23], + "i1": [20, 21, 22, 23], + } + ) + + result = driver( + data={"source": df_source}, cube=existing_cube, store=function_store + ) + + assert set(result.keys()) == {"source"} + + ds_source = result["source"] + ds_enrich = DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("enrich"), function_store() + ) + + partitions_source_2 = set(ds_source.partitions.keys()) + partitions_enrich_2 = set(ds_enrich.partitions.keys()) + + assert len(partitions_source_2) > len(partitions_source_1) + assert partitions_source_1.issubset(partitions_source_2) + + assert partitions_enrich_2 == partitions_enrich_1 + + +@pytest.mark.parametrize("chunk_size_build", [None, 2]) +@pytest.mark.parametrize("chunk_size_append", [None, 2]) +def test_rowgroups_are_applied_when_df_serializer_is_passed_to_append_cube( + driver, function_store, chunk_size_build, chunk_size_append +): + """ + Test that the dataset is split into row groups depending on the chunk size + + Partitions build with ``chunk_size=None`` should keep a single row group after the append. Partitions that are newly created with + ``chunk_size>0`` should be split into row groups accordingly. + """ + + # Build cube + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size_build), + ) + + # Append to cube + df_append = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], + ) + result = driver( + data={"seed": df_append}, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size_append), + ) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 2, 1: 2, 2: 1, 3: 3} + part_chunk_size = { + 0: chunk_size_build, + 1: chunk_size_build, + 2: chunk_size_append, + 3: chunk_size_append, + } + + assert len(dataset.partitions) == 4 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_single_rowgroup_when_df_serializer_is_not_passed_to_append_cube( + driver, function_store +): + """ + Test that the dataset has a single row group as default path + """ + + # Build cube + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, cube=cube, store=function_store, + ) + + # Append to cube + df_append = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], + ) + result = driver(data={"seed": df_append}, cube=cube, store=function_store,) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 2, 1: 2, 2: 1, 3: 3} + part_chunk_size = {0: None, 1: None, 2: None, 3: None} + + assert len(dataset.partitions) == 4 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_compression_is_compatible_on_append_cube(driver, function_store): + """ + Test that partitons written with different compression algorithms are compatible + + The compression algorithms are not parametrized because their availability depends + on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts + of the code. A fully parametrized test would also increase runtime and test complexity + unnecessarily. + """ + # Build cube + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="SNAPPY"), + ) + + # Append to cube + df_append = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [2, 3, 3, 3]}, columns=["x", "p"], + ) + result = driver( + data={"seed": df_append}, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="GZIP"), + ) + dataset = result["seed"].load_all_indices(function_store()) + + assert len(dataset.partitions) == 4 + + +def test_append_partitions_no_ts(driver, function_store): + df_source1 = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [10, 11, 12, 13], + } + ) + df_enrich1 = pd.DataFrame( + {"x": [0, 1, 2, 3], "v2": [10, 11, 12, 13], "i2": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data={"source": df_source1, "enrich": df_enrich1}, + cube=cube, + store=function_store, + metadata={"source": {"a": 10, "b": 11}, "enrich": {"a": 20, "b": 21}}, + partition_on={"enrich": []}, + ) + + partitions_source_1 = set( + DatasetMetadata.load_from_store( + cube.ktk_dataset_uuid("source"), function_store() + ).partitions.keys() + ) + partitions_enrich_1 = set( + DatasetMetadata.load_from_store( + cube.ktk_dataset_uuid("enrich"), function_store() + ).partitions.keys() + ) + + df_source2 = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [20, 21, 22, 23], + "i1": [20, 21, 22, 23], + } + ) + df_enrich2 = pd.DataFrame( + {"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23], "i2": [20, 21, 22, 23]} + ) + + result = driver( + data={"source": df_source2, "enrich": df_enrich2}, + cube=cube, + store=function_store, + ) + + assert set(result.keys()) == {"source", "enrich"} + + ds_source = result["source"] + ds_enrich = result["enrich"] + + partitions_source_2 = set(ds_source.partitions.keys()) + partitions_enrich_2 = set(ds_enrich.partitions.keys()) + + assert len(partitions_source_2) > len(partitions_source_1) + assert partitions_source_1.issubset(partitions_source_2) + + assert len(partitions_enrich_2) > len(partitions_enrich_1) + assert partitions_enrich_1.issubset(partitions_enrich_2) + + +def test_indices(driver, function_store, existing_cube): + idx1_1 = set( + DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("source"), function_store() + ) + .load_all_indices(function_store()) + .indices["i1"] + .index_dct.keys() + ) + idx2_1 = set( + DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("enrich"), function_store() + ) + .load_all_indices(function_store()) + .indices["i2"] + .index_dct.keys() + ) + + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [20, 21, 22, 23], + "i1": [20, 21, 22, 23], + } + ) + + result = driver( + data={"source": df_source}, cube=existing_cube, store=function_store + ) + + assert set(result.keys()) == {"source"} + + ds_source = result["source"] + ds_enrich = DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("enrich"), function_store() + ) + + idx1_2 = set( + ds_source.load_all_indices(function_store()).indices["i1"].index_dct.keys() + ) + idx2_2 = set( + ds_enrich.load_all_indices(function_store()).indices["i2"].index_dct.keys() + ) + + assert idx1_1.issubset(idx1_2) + assert len(idx1_1) < len(idx1_2) + + assert idx2_1 == idx2_2 + + +def test_fails_incompatible_dtypes(driver, function_store, existing_cube): + """ + Should also cross check w/ seed dataset. + """ + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10.0, 11.0, 12.0, 13.0], + "i1": [10, 11, 12, 13], + } + ) + + with pytest.raises(ValueError, match="Schema violation"): + driver(data={"source": df_source}, cube=existing_cube, store=function_store) + + +def test_fails_missing_column(driver, function_store, existing_cube): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "i1": [10, 11, 12, 13]} + ) + + with pytest.raises(ValueError, match="Schema violation"): + driver(data={"source": df_source}, cube=existing_cube, store=function_store) + + +def test_fails_unknown_dataset(driver, function_store, existing_cube): + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [10, 11, 12, 13], + } + ) + df_zoo = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v3": [10, 11, 12, 13], + "i3": [10, 11, 12, 13], + } + ) + + keys_pre = set(function_store().keys()) + + with pytest.raises(ValueError, match="Unknown / non-existing datasets: zoo"): + driver( + data={"source": df_source, "zoo": df_zoo}, + cube=existing_cube, + store=function_store, + ) + + keys_post = set(function_store().keys()) + assert keys_pre == keys_post + + +def test_metadata(driver, function_store, existing_cube): + """ + Test auto- and user-generated metadata. + """ + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [20, 21, 22, 23], + "i1": [20, 21, 22, 23], + } + ) + + result = driver( + data={"source": df_source}, + cube=existing_cube, + store=function_store, + metadata={"source": {"a": 12, "c": 13}}, + ) + + assert set(result.keys()) == {"source"} + + ds_source = result["source"] + assert set(ds_source.metadata.keys()) == { + "a", + "b", + "c", + "creation_time", + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, + } + assert ds_source.metadata["a"] == 12 + assert ds_source.metadata["b"] == 11 + assert ds_source.metadata["c"] == 13 + assert ds_source.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( + existing_cube.dimension_columns + ) + assert ds_source.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is True + assert ds_source.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( + existing_cube.partition_columns + ) + assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] + + ds_enrich = DatasetMetadata.load_from_store( + existing_cube.ktk_dataset_uuid("enrich"), function_store() + ) + assert set(ds_enrich.metadata.keys()) == { + "a", + "b", + "creation_time", + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, + } + assert ds_enrich.metadata["a"] == 20 + assert ds_enrich.metadata["b"] == 21 + assert ds_enrich.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( + existing_cube.dimension_columns + ) + assert ds_enrich.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is False + assert ds_enrich.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( + existing_cube.partition_columns + ) + assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] diff --git a/tests/io/cube/test_build.py b/tests/io/cube/test_build.py index a62a2938..dfdeaa92 100644 --- a/tests/io/cube/test_build.py +++ b/tests/io/cube/test_build.py @@ -3,15 +3,32 @@ import dask import dask.bag as db import dask.core +import numpy as np import pandas as pd +import pandas.testing as pdt +import pyarrow as pa import pytest +from pandas.arrays import SparseArray from tests.io.cube.utils import wrap_bag_write, wrap_ddf_write +from kartothek.core.cube.constants import ( + KTK_CUBE_DF_SERIALIZER, + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, +) from kartothek.core.cube.cube import Cube +from kartothek.core.dataset import DatasetMetadata +from kartothek.core.index import ExplicitSecondaryIndex, PartitionIndex from kartothek.io.dask.bag_cube import build_cube_from_bag from kartothek.io.dask.dataframe_cube import build_cube_from_dataframe from kartothek.io.eager_cube import build_cube -from kartothek.io.testing.build_cube import * # noqa +from kartothek.io_components.cube.write import MultiTableCommitAborted +from kartothek.io_components.metapartition import SINGLE_TABLE +from kartothek.serialization._parquet import ParquetSerializer + +from .utils import assert_num_row_groups @pytest.fixture @@ -158,3 +175,1388 @@ def test_function_executed_once(driver, function_store, driver_name, skip_eager) raise ValueError("Missing implementation for driver: {}".format(driver_name)) assert len(function_store().keys(prefix="counter.")) == 2 + + +def test_simple_seed_only(driver, function_store): + """ + Simple integration test w/ a seed dataset only. This is the most simple way to create a cube. + """ + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + result = driver(data=df, cube=cube, store=function_store) + + assert set(result.keys()) == {cube.seed_dataset} + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert ds.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert len(ds.partitions) == 2 + + assert set(ds.indices.keys()) == {"p", "x"} + assert isinstance(ds.indices["p"], PartitionIndex) + assert isinstance(ds.indices["x"], ExplicitSecondaryIndex) + + assert ds.table_name == SINGLE_TABLE + + +def test_simple_two_datasets(driver, function_store): + """ + Simple intergration test w/ 2 datasets. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 2 + + assert set(ds_source.indices.keys()) == {"p", "x"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + + assert set(ds_enrich.indices.keys()) == {"p"} + assert isinstance(ds_enrich.indices["p"], PartitionIndex) + + assert ds_source.table_name == SINGLE_TABLE + assert ds_enrich.table_name == SINGLE_TABLE + + +def test_indices(driver, function_store): + """ + Test that index structures are created correctly. + """ + df_source = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [100, 101, 102, 103], + } + ) + df_enrich = pd.DataFrame( + { + "x": [0, 1, 4, 5], + "p": [0, 0, 2, 2], + "v2": [20, 21, 22, 23], + "i2": [200, 201, 202, 203], + } + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2"], + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert set(ds_source.indices.keys()) == {"p", "x", "i1"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + assert isinstance(ds_source.indices["i1"], ExplicitSecondaryIndex) + + assert set(ds_enrich.indices.keys()) == {"p", "i2"} + assert isinstance(ds_enrich.indices["p"], PartitionIndex) + assert isinstance(ds_enrich.indices["i2"], ExplicitSecondaryIndex) + + +def test_dimension_index_suppression(driver, function_store): + """ + Test that suppress_index_on works as expected + """ + df_source = pd.DataFrame( + { + "x": [0, 0, 1, 1], + "y": [10, 11, 12, 13], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [100, 101, 102, 103], + } + ) + cube = Cube( + dimension_columns=["x", "y"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2"], + suppress_index_on=["x"], + ) + result = driver(data={"source": df_source}, cube=cube, store=function_store) + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + + assert set(ds_source.indices.keys()) == {"p", "i1", "y"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["i1"], ExplicitSecondaryIndex) + assert isinstance(ds_source.indices["y"], ExplicitSecondaryIndex) + + +def test_do_not_modify_df(driver, function_store): + """ + Functions should not modify their inputs. + """ + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + df_backup = df.copy() + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + driver(data=df, cube=cube, store=function_store) + + pdt.assert_frame_equal(df, df_backup) + + +@pytest.mark.filterwarnings("ignore::UnicodeWarning") +def test_parquet(driver, function_store): + """ + Ensure the parquet files we generate are properly normalized. + """ + df = pd.DataFrame( + data={ + "x": [10, 1, 1, 0, 0], + "y": [10, 0, 1, 1, 0], + "p": [0, 1, 1, 1, 1], + "föö".encode("utf8"): [100, 10, 11, 12, 13], + "v": np.nan, + }, + index=[0, 1000, 1001, 1002, 1003], + columns=["x", "y", "p", "föö".encode("utf8"), "v"], + ) + + cube = Cube( + dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" + ) + result = driver(data=df, cube=cube, store=function_store) + + assert set(result.keys()) == {cube.seed_dataset} + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert len(ds.partitions) == 2 + for p in (0, 1): + part_key = ds.indices["p"].index_dct[p][0] + part = ds.partitions[part_key] + key = part.files[SINGLE_TABLE] + + df_actual = KTK_CUBE_DF_SERIALIZER.restore_dataframe(function_store(), key) + df_expected = ( + df.loc[df["p"] == p] + .sort_values(["x", "y"]) + .reset_index(drop=True) + .drop(columns=["p"]) + .rename(columns={"föö".encode("utf8"): "föö"}) + ) + + pdt.assert_frame_equal(df_actual.reset_index(drop=True), df_expected) + + +@pytest.mark.parametrize("chunk_size", [None, 2]) +def test_rowgroups_are_applied_when_df_serializer_is_passed_to_build_cube( + driver, function_store, chunk_size +): + """ + Test that the dataset is split into row groups depending on the chunk size + """ + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"],) + + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + result = driver( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size), + ) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 3} + part_chunk_size = {0: chunk_size, 1: chunk_size} + + assert len(dataset.partitions) == 2 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_single_rowgroup_when_df_serializer_is_not_passed_to_build_cube( + driver, function_store +): + """ + Test that the dataset has a single row group as default path + """ + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"],) + + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + result = driver(data=df, cube=cube, store=function_store,) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 3} + part_chunk_size = {0: None, 1: None} + + assert len(dataset.partitions) == 2 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_fail_sparse(driver, driver_name, function_store): + """ + Ensure that sparse dataframes are rejected. + """ + df = pd.DataFrame( + data={ + "x": SparseArray([10, 1, 1, 0, 0]), + "y": SparseArray([10, 0, 1, 1, 0]), + "p": SparseArray([0, 1, 1, 1, 1]), + "v": SparseArray([np.nan] * 5), + } + ) + + cube = Cube( + dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" + ) + with pytest.raises(TypeError, match="Sparse data is not supported."): + driver(data=df, cube=cube, store=function_store) + + +def test_metadata(driver, function_store): + """ + Test auto- and user-generated metadata. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 4, 5], "p": [0, 0, 2, 2], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + metadata={"enrich": {"foo": 1}}, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset] + assert set(ds_source.metadata.keys()) == { + "creation_time", + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, + } + assert ds_source.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( + cube.dimension_columns + ) + assert ds_source.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is True + assert ds_source.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( + cube.partition_columns + ) + assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] + + ds_enrich = result["enrich"] + assert set(ds_enrich.metadata.keys()) == { + "creation_time", + KTK_CUBE_METADATA_DIMENSION_COLUMNS, + KTK_CUBE_METADATA_KEY_IS_SEED, + KTK_CUBE_METADATA_PARTITION_COLUMNS, + KTK_CUBE_METADATA_SUPPRESS_INDEX_ON, + "foo", + } + assert ds_enrich.metadata[KTK_CUBE_METADATA_DIMENSION_COLUMNS] == list( + cube.dimension_columns + ) + assert ds_enrich.metadata[KTK_CUBE_METADATA_KEY_IS_SEED] is False + assert ds_enrich.metadata[KTK_CUBE_METADATA_PARTITION_COLUMNS] == list( + cube.partition_columns + ) + assert ds_enrich.metadata["foo"] == 1 + assert ds_source.metadata[KTK_CUBE_METADATA_SUPPRESS_INDEX_ON] == [] + + +def test_fails_metadata_wrong_type(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises( + TypeError, match="Provided metadata should be a dict but is int" + ): + driver(data={"source": df_source}, cube=cube, store=function_store, metadata=1) + + +def test_fails_metadata_unknown_id(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises( + ValueError, + match="Provided metadata for otherwise unspecified ktk_cube_dataset_ids: bar, foo", + ): + driver( + data={"source": df_source}, + cube=cube, + store=function_store, + metadata={"source": {}, "foo": {}, "bar": {}}, + ) + + +def test_fails_metadata_nested_wrong_type(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises( + TypeError, + match="Provided metadata for dataset source should be a dict but is int", + ): + driver( + data={"source": df_source}, + cube=cube, + store=function_store, + metadata={"source": 1}, + ) + + +def test_fails_missing_seed(driver, function_store): + """ + A cube must contain its seed dataset, check this constraint as early as possible. + """ + df = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v": [10, 11]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + with pytest.raises(ValueError) as exc: + driver(data={"foo": df}, cube=cube, store=function_store) + assert 'Seed data ("seed") is missing.' in str(exc.value) + assert list(function_store().keys()) == [] + + +def test_fails_missing_dimension_columns(driver, function_store): + """ + Ensure that we catch missing dimension columns early. + """ + df_source = pd.DataFrame({"x": [0, 1], "p": 0}) + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(ValueError) as exc: + driver(data=df_source, cube=cube, store=function_store) + assert 'Missing dimension columns in seed data "source": y, z' in str(exc.value) + assert list(function_store().keys()) == [] + + +def test_fails_no_dimension_columns(driver, function_store): + """ + Ensure that we catch missing dimension columns early. + """ + df_source = pd.DataFrame({"x": [0, 1], "y": [0, 1], "z": [0, 1], "p": 0}) + df_enrich = pd.DataFrame({"p": [0], "v1": 0}) + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(ValueError) as exc: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + assert ( + 'Dataset "enrich" must have at least 1 of the following dimension columns: x, y' + in str(exc.value) + ) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_fails_duplicate_columns(driver, function_store, driver_name): + """ + Catch weird pandas behavior. + """ + if driver_name == "dask_dataframe": + pytest.skip("already detected by dask.dataframe") + + df = pd.DataFrame( + {"x": [0, 1], "p": 0, "a": 1, "b": 2}, columns=["x", "p", "a", "b"] + ).rename(columns={"b": "a"}) + assert len(df.columns) == 4 + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + assert 'Duplicate columns found in dataset "seed": x, p, a, a' in str(exc.value) + assert list(function_store().keys()) == [] + + +def test_fails_missing_partition_columns(driver, function_store): + """ + Just make the Kartothek error nicer. + """ + df = pd.DataFrame({"x": [0, 1], "p": 0}) + cube = Cube( + dimension_columns=["x"], partition_columns=["p", "q", "r"], uuid_prefix="cube" + ) + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + assert 'Missing partition columns in dataset "seed": q, r' in str(exc.value) + assert list(function_store().keys()) == [] + + +def test_overwrite(driver, function_store): + """ + Test overwrite behavior aka call the build function if the cube already exists. + """ + df1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v": [10, 11]}) + df2 = pd.DataFrame({"x": [2, 3], "p": [1, 1], "v": [12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + driver(data=df1, cube=cube, store=function_store) + + # implicit overwrite fails + keys = set(function_store().keys()) + with pytest.raises(RuntimeError) as exc: + driver(data=df1, cube=cube, store=function_store) + assert "already exists and overwrite is not permitted" in str(exc.value) + assert set(function_store().keys()) == keys + + # explicit overwrite works + result = driver(data=df2, cube=cube, store=function_store, overwrite=True) + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert len(ds.partitions) == 1 + + assert set(ds.indices["p"].index_dct.keys()) == {1} + + +def test_split(driver, function_store): + """ + Imagine the user already splits the data. + """ + df_source1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v1": [10, 11]}) + df_source2 = pd.DataFrame({"x": [2, 3], "p": [1, 1], "v1": [12, 13]}) + df_enrich = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v2": [20, 21]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data=[{"source": df_source1, "enrich": df_enrich}, df_source2], + cube=cube, + store=function_store, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 1 + + +@pytest.mark.parametrize("empty_first", [False, True]) +def test_empty_df(driver, function_store, empty_first): + """ + Might happen during DB queries. + """ + df1 = pd.DataFrame({"x": [0, 1], "p": [0, 0], "v1": [10, 11]}) + df2 = df1.loc[[]] + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data=[df2, df1] if empty_first else [df1, df2], cube=cube, store=function_store + ) + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert len(ds.partitions) == 1 + assert ( + len(list(function_store().keys())) == 4 + ) # DS metadata, "x" index, common metadata, 1 partition + + +def test_fail_duplicates_local(driver, driver_name, function_store): + """ + Might happen during DB queries. + """ + if driver_name == "dask_dataframe": + pytest.xfail(reason="Cannot guarantee duplicates for DDF") + df = pd.DataFrame( + { + "x": [0, 0], + "y": ["a", "a"], + "z": [pd.Timestamp("2017"), pd.Timestamp("2017")], + "p": [0, 0], + } + ) + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + msg = """ +Found duplicate cells by [p, x, y, z] in dataset "source", example: + +Keys: +p 0 +x 0 +y a +z 2017-01-01 00:00:00 + +Identical Payload: +n/a + +Non-Idential Payload: +n/a +""".strip() + assert msg in str(exc.value) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_accept_projected_duplicates(driver, function_store): + """ + Otherwise partitioning does not work w/ projected data. + """ + df_seed = pd.DataFrame({"x": [0, 1, 0, 1], "y": [0, 0, 1, 1], "p": [0, 0, 1, 1]}) + df_enrich = pd.DataFrame({"x": [0, 1, 0, 1], "p": [0, 0, 1, 1]}) + cube = Cube( + dimension_columns=["x", "y"], partition_columns=["p"], uuid_prefix="cube" + ) + driver(data={"seed": df_seed, "enrich": df_enrich}, cube=cube, store=function_store) + + +@pytest.mark.xfail( + strict=True, reason="Cannot be checked with current index structures." +) +def test_fail_duplicates_global(driver_name, driver, function_store): + """ + Might happen due to bugs. + """ + if driver_name == "eager": + pytest.skip(reason="Problem does not occur in eager mode.") + + df1 = pd.DataFrame({"x": [0], "y": ["a"], "z": [pd.Timestamp("2017")], "p": [0]}) + df2 = pd.DataFrame({"x": [0], "y": ["a"], "z": [pd.Timestamp("2017")], "p": [1]}) + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(ValueError): + driver(data=[df1, df2], cube=cube, store=function_store) + + +def test_regression_pseudo_duplicates(driver, function_store): + """ + Might happen due to bugs. + """ + df = pd.DataFrame({"x": [0, 0, 2, 3], "y": [0, 1, 2, 2], "p": [0, 1, 0, 1]}) + cube = Cube( + dimension_columns=["x", "y"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + driver(data=df, cube=cube, store=function_store) + + +def test_fail_wrong_types(driver, function_store): + """ + Might catch nasty pandas and other type bugs. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0.0, 1.0, 2.0, 3.0], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + + cause = exc_info.value.__cause__ + assert isinstance(cause, ValueError) + assert 'Found incompatible entries for column "x"' in str(cause) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_distinct_branches(driver, function_store): + """ + Just check this actually works. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data=[{"source": df_source}, {"enrich": df_enrich}], + cube=cube, + store=function_store, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 2 + + +def test_fail_nondistinc_payload(driver, function_store): + """ + This would lead to problems during the query phase. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + cause = exc_info.value.__cause__ + assert isinstance(cause, ValueError) + assert "Found columns present in multiple datasets" in str(cause) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_fail_partial_overwrite(driver, function_store): + """ + Either overwrite all or no datasets. + """ + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + + df_source1 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich1 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + driver( + data={"source": df_source1, "enrich": df_enrich1}, + cube=cube, + store=function_store, + ) + + keys = set(function_store().keys()) + df_source2 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} + ) + with pytest.raises(ValueError) as exc: + driver( + data={"source": df_source2}, cube=cube, store=function_store, overwrite=True + ) + assert ( + str(exc.value) + == "Following datasets exists but are not overwritten (partial overwrite), this is not allowed: enrich" + ) + assert set(function_store().keys()) == keys + + +def test_fail_partial_build(driver, function_store): + """ + Either overwrite all or no datasets. + """ + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + + df_source1 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich1 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + driver( + data={"source": df_source1, "enrich": df_enrich1}, + cube=cube, + store=function_store, + ) + + # delete everything that belongs to the seed dataset + to_delete = { + k + for k in function_store().keys() + if k.startswith(cube.ktk_dataset_uuid(cube.seed_dataset)) + } + for k in to_delete: + function_store().delete(k) + + keys = set(function_store().keys()) + df_source2 = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} + ) + with pytest.raises(ValueError) as exc: + driver(data={"source": df_source2}, cube=cube, store=function_store) + assert ( + str(exc.value) + == "Following datasets exists but are not overwritten (partial overwrite), this is not allowed: enrich" + ) + assert set(function_store().keys()) == keys + + +def test_fails_projected_duplicates(driver, driver_name, function_store): + """ + Test if duplicate check also works w/ projected data. (was a regression) + """ + if driver_name == "dask_dataframe": + pytest.xfail(reason="Cannot guarantee duplicates for DDF") + df_source = pd.DataFrame( + { + "x": [0, 1, 0, 1], + "y": [0, 0, 1, 1], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + } + ) + df_enrich = pd.DataFrame( + {"y": [0, 0, 1, 1], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23], "v3": 42} + ) + cube = Cube( + dimension_columns=["x", "y"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(ValueError) as exc: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + msg = """ +Found duplicate cells by [p, y] in dataset "enrich", example: + +Keys: +p 0 +y 0 + +Identical Payload: +v3 42 + +Non-Idential Payload: + v2 +0 20 +1 21 +""".strip() + assert msg in str(exc.value) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_projected_data(driver, function_store): + """ + Projected dataset (useful for de-duplication). + """ + df_source = pd.DataFrame( + { + "x": [0, 1, 0, 1], + "y": [0, 0, 1, 1], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + } + ) + df_enrich = pd.DataFrame({"y": [0, 1], "p": [0, 1], "v2": [20, 21]}) + cube = Cube( + dimension_columns=["x", "y"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 2 + + +def test_fails_null_dimension(driver, function_store): + """ + Since we do not allow NULL values in queries, it should be banned from dimension columns in the first place. + """ + df = pd.DataFrame( + {"x": [0, 1, 2, np.nan], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + + assert 'Found NULL-values in dimension column "x" of dataset "seed"' in str(exc) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) + + +def test_fails_null_partition(driver, function_store): + """ + Since we do not allow NULL values in queries, it should be banned from partition columns in the first place. + """ + df = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, np.nan], "v": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + assert 'Found NULL-values in partition column "p" of dataset "seed"' in str( + exc.value + ) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) + + +def test_fails_null_index(driver, function_store): + """ + Since we do not allow NULL values in queries, it should be banned from index columns in the first place. + """ + df = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v": [10, 11, 12, 13], + "i1": [0, 1, 2, np.nan], + } + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + index_columns=["i1"], + ) + with pytest.raises(ValueError) as exc: + driver(data=df, cube=cube, store=function_store) + assert 'Found NULL-values in index column "i1"' in str(exc.value) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("seed"), function_store()) + + +@pytest.mark.xfail(reason="different") +def test_fail_all_empty(driver, driver_name, function_store): + """ + Might happen due to DB-based filters. + """ + df = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} + ).loc[[]] + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver(data=df, cube=cube, store=function_store) + exc = exc_info.value.__cause__ + assert isinstance(exc, ValueError) + assert "Cannot write empty datasets" in str(exc) + + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_overwrite_rollback_ktk_cube(driver, function_store): + """ + Checks that require a rollback (like overlapping columns) should recover the former state correctly. + """ + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2", "i3", "i4"], + ) + + df_source1 = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "i1": [10, 11, 12, 13], + } + ) + df_enrich1 = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "i2": [20, 21, 22, 23], + "v2": [20, 21, 22, 23], + } + ) + driver( + data={"source": df_source1, "enrich": df_enrich1}, + cube=cube, + store=function_store, + ) + + df_source2 = pd.DataFrame( + { + "x": [10, 11], + "p": [10, 10], + "v1": [10.0, 11.0], # also use another dtype here (was int) + "i3": [10, 11], + } + ) + df_enrich2 = pd.DataFrame( + {"x": [10, 11], "p": [10, 10], "v1": [20, 21], "i4": [20, 21]} + ) + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver( + data={"source": df_source2, "enrich": df_enrich2}, + cube=cube, + store=function_store, + overwrite=True, + ) + cause = exc_info.value.__cause__ + assert isinstance(cause, ValueError) + assert str(cause).startswith("Found columns present in multiple datasets:") + + ds_source = DatasetMetadata.load_from_store( + uuid=cube.ktk_dataset_uuid("source"), store=function_store() + ).load_all_indices(function_store()) + ds_enrich = DatasetMetadata.load_from_store( + uuid=cube.ktk_dataset_uuid("enrich"), store=function_store() + ).load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 2 + + assert set(ds_source.indices.keys()) == {"p", "x", "i1"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + assert set(ds_source.indices["x"].index_dct.keys()) == {0, 1, 2, 3} + assert set(ds_source.indices["i1"].index_dct.keys()) == {10, 11, 12, 13} + + assert set(ds_enrich.indices.keys()) == {"p", "i2"} + assert isinstance(ds_enrich.indices["p"], PartitionIndex) + assert set(ds_enrich.indices["i2"].index_dct.keys()) == {20, 21, 22, 23} + + assert ds_source.schema.field("v1").type == pa.int64() + + +@pytest.mark.parametrize("none_first", [False, True]) +def test_nones(driver, function_store, none_first, driver_name): + """ + Test what happens if user passes None to ktk_cube. + """ + if driver_name == "dask_dataframe": + pytest.skip("user cannot create None-partitions with dask.dataframe") + + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + result = driver( + data=[None, df] if none_first else [df, None], cube=cube, store=function_store + ) + + assert set(result.keys()) == {cube.seed_dataset} + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert ds.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert len(ds.partitions) == 2 + + assert set(ds.indices.keys()) == {"p", "x"} + assert isinstance(ds.indices["p"], PartitionIndex) + assert isinstance(ds.indices["x"], ExplicitSecondaryIndex) + + assert ds.table_name == SINGLE_TABLE + + +def test_fail_not_a_df(driver, function_store): + """ + Pass some weird objects in. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(TypeError) as exc: + driver( + data={"source": df_source, "enrich": pd.Series(range(10))}, + cube=cube, + store=function_store, + ) + assert ( + 'Provided DataFrame is not a pandas.DataFrame or None, but is a "Series"' + in str(exc.value) + ) + + +def test_fail_wrong_dataset_ids(driver, function_store, skip_eager, driver_name): + if driver_name == "dask_dataframe": + pytest.skip("not an interface for dask.dataframe") + + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + + with pytest.raises(ValueError) as exc: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + ktk_cube_dataset_ids=["source", "extra"], + ) + + assert ( + 'Ktk_cube Dataset ID "enrich" is present during pipeline execution but was not ' + "specified in ktk_cube_dataset_ids (extra, source)." in str(exc.value) + ) + + +def test_fail_no_store_factory(driver, function_store, skip_eager): + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + store = function_store() + with pytest.raises(TypeError) as exc: + driver(data=df, cube=cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_partition_on_1(driver, function_store): + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube( + dimension_columns=["x"], partition_columns=["p", "q"], uuid_prefix="cube" + ) + + with pytest.raises(ValueError) as exc_info: + driver( + data=df, + cube=cube, + store=function_store, + partition_on={cube.seed_dataset: ["x", "p"]}, + ) + + cause = exc_info.value # .__cause__ + assert isinstance(cause, ValueError) + assert ( + "Seed dataset seed must have the following, fixed partition-on attribute: p, q" + in str(cause) + ) + + assert set(function_store().keys()) == set() + + +def test_fail_partition_on_3(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + + with pytest.raises( + ValueError, + match="partition-on attribute of dataset enrich contains duplicates: p, p", + ): + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": ["p", "p"]}, + ) + + assert set(function_store().keys()) == set() + + +def test_fail_partition_on_4(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + + with pytest.raises( + ValueError, match="Unspecified but provided partition columns in enrich: p" + ): + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": []}, + ) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) + + +def test_partition_on_enrich_none(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v2": [20, 21, 22, 23]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": []}, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 1 + + assert set(ds_source.indices.keys()) == {"p", "x"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + + assert set(ds_enrich.indices.keys()) == set() + + assert ds_source.table_name == SINGLE_TABLE + assert ds_enrich.table_name == SINGLE_TABLE + + +def test_partition_on_enrich_extra(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": ["p", "x"]}, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 4 + + assert set(ds_source.indices.keys()) == {"p", "x"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + + assert set(ds_enrich.indices.keys()) == {"p", "x"} + assert isinstance(ds_enrich.indices["p"], PartitionIndex) + assert isinstance(ds_enrich.indices["x"], PartitionIndex) + + assert ds_source.table_name == SINGLE_TABLE + assert ds_enrich.table_name == SINGLE_TABLE + + +def test_partition_on_index_column(driver, function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "i": [0, 0, 1, 2], "v2": [20, 21, 22, 23]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + index_columns=["i"], + uuid_prefix="cube", + seed_dataset="source", + ) + result = driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": ["i"]}, + ) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + + ds_source = result[cube.seed_dataset].load_all_indices(function_store()) + ds_enrich = result["enrich"].load_all_indices(function_store()) + + assert ds_source.uuid == cube.ktk_dataset_uuid(cube.seed_dataset) + assert ds_enrich.uuid == cube.ktk_dataset_uuid("enrich") + + assert len(ds_source.partitions) == 2 + assert len(ds_enrich.partitions) == 3 + + assert set(ds_source.indices.keys()) == {"p", "x"} + assert isinstance(ds_source.indices["p"], PartitionIndex) + assert isinstance(ds_source.indices["x"], ExplicitSecondaryIndex) + + assert set(ds_enrich.indices.keys()) == {"i"} + assert isinstance(ds_enrich.indices["i"], PartitionIndex) + + assert ds_source.table_name == SINGLE_TABLE + assert ds_enrich.table_name == SINGLE_TABLE + + +def test_fail_partition_on_nondistinc_payload(driver, function_store): + """ + This would lead to problems during the query phase. + """ + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame({"x": [0, 1, 2, 3], "v1": [20, 21, 22, 23]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + ) + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver( + data={"source": df_source, "enrich": df_enrich}, + cube=cube, + store=function_store, + partition_on={"enrich": ["v1"]}, + ) + cause = exc_info.value.__cause__ + assert isinstance(cause, ValueError) + assert "Found columns present in multiple datasets" in str(cause) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("source"), function_store()) + assert not DatasetMetadata.exists(cube.ktk_dataset_uuid("enrich"), function_store()) diff --git a/tests/io/cube/test_cleanup.py b/tests/io/cube/test_cleanup.py index 67ce89ed..2d27e002 100644 --- a/tests/io/cube/test_cleanup.py +++ b/tests/io/cube/test_cleanup.py @@ -1,11 +1,14 @@ # -*- coding: utf-8 -*- +# from kartothek.io.testing.cleanup_cube import * # noqa +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_delete +from kartothek.core.cube.constants import KTK_CUBE_UUID_SEPARATOR +from kartothek.core.cube.cube import Cube from kartothek.io.dask.bag_cube import cleanup_cube_bag -from kartothek.io.eager_cube import cleanup_cube -from kartothek.io.testing.cleanup_cube import * # noqa +from kartothek.io.eager_cube import build_cube, cleanup_cube, copy_cube @pytest.fixture @@ -20,3 +23,211 @@ def driver(driver_name): return cleanup_cube else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +def test_delete_by_correct_uuid(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube_foo = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="foo") + build_cube( + data={cube_foo.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube_foo, + store=function_store, + ) + + cube_foo_bar = Cube( + dimension_columns=["x"], partition_columns=["p"], uuid_prefix="foo_bar" + ) + build_cube( + data={cube_foo_bar.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube_foo_bar, + store=function_store, + ) + store = function_store() + foo_bar_keys = {k for k in store.keys() if "foo_bar" in k} + store.delete("foo++seed.by-dataset-metadata.json") + store.delete("foo++enrich.by-dataset-metadata.json") + + driver(cube=cube_foo, store=function_store) + assert foo_bar_keys == set(store.keys()) + + +def test_missing_seed_dataset(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + store = function_store() + seed_keys = {k for k in store.keys() if "cube++seed" in k and "/" in k} + enrich_keys = {k for k in store.keys() if "cube++enrich" in k} + + for k in seed_keys: + store.delete(k) + + driver(cube=cube, store=function_store) + + assert enrich_keys == set(store.keys()) + + +def test_missing_cube_files(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + store = function_store() + enrich_keys = {k for k in store.keys() if "cube++enrich" in k and "/" in k} + for k in enrich_keys: + store.delete(k) + + driver(cube=cube, store=function_store) + + assert "cube++enrich.by-dataset-metadata.json" not in store.keys() + + +def test_missing_metadata(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + store = function_store() + enrich_keys = {k for k in store.keys() if "cube++enrich" in k} + + store.delete("cube++enrich.by-dataset-metadata.json") + + driver(cube=cube, store=function_store) + + assert not enrich_keys.intersection(store.keys()) + + +def test_noop(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + + keys = set(function_store().keys()) + + driver(cube=cube, store=function_store) + + assert set(function_store().keys()) == keys + + +def test_overwrite_check_with_copy(driver, function_store, function_store2): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + # build twice + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + overwrite=True, + ) + + # copy to another store to detect keys + copy_cube(cube=cube, src_store=function_store, tgt_store=function_store2) + keys = set(function_store2().keys()) + + assert set(function_store().keys()) != keys + driver(cube=cube, store=function_store) + assert set(function_store().keys()) == keys + + +def test_additional_files(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube(data=df_seed, cube=cube, store=function_store) + + key_in_ds = cube.ktk_dataset_uuid(cube.seed_dataset) + "/foo" + key_with_ds_prefix = cube.ktk_dataset_uuid(cube.seed_dataset) + ".foo" + key_with_cube_prefix = cube.uuid_prefix + ".foo" + key_with_cube_prefix_separator = cube.uuid_prefix + KTK_CUBE_UUID_SEPARATOR + ".foo" + + function_store().put(key_in_ds, b"") + function_store().put(key_with_ds_prefix, b"") + function_store().put(key_with_cube_prefix, b"") + function_store().put(key_with_cube_prefix_separator, b"") + + driver(cube=cube, store=function_store) + assert key_in_ds not in set(function_store().keys()) + assert key_with_ds_prefix not in set(function_store().keys()) + assert key_with_cube_prefix in set(function_store().keys()) + assert key_with_cube_prefix_separator not in set(function_store().keys()) + + +def test_fails_no_store_factory(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + store = function_store() + with pytest.raises(TypeError) as exc: + driver(cube=cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(TypeError, match="blocksize must be an integer but is str"): + driver(cube=cube, store=function_store, blocksize="foo") + + +def test_fail_blocksize_negative(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): + driver(cube=cube, store=function_store, blocksize=-1) + + +def test_fail_blocksize_zero(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): + driver(cube=cube, store=function_store, blocksize=0) diff --git a/tests/io/cube/test_copy.py b/tests/io/cube/test_copy.py index 5c2f2fcf..5c60c954 100644 --- a/tests/io/cube/test_copy.py +++ b/tests/io/cube/test_copy.py @@ -1,10 +1,14 @@ # -*- coding: utf-8 -*- +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_copy +from kartothek.api.discover import discover_datasets_unchecked +from kartothek.core.cube.cube import Cube from kartothek.io.dask.bag_cube import copy_cube_bag -from kartothek.io.eager_cube import copy_cube -from kartothek.io.testing.copy_cube import * # noqa +from kartothek.io.eager import copy_dataset +from kartothek.io.eager_cube import build_cube, copy_cube, query_cube +from kartothek.utils.ktk_adapters import get_dataset_keys @pytest.fixture @@ -19,3 +23,560 @@ def driver(driver_name): return copy_cube else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +@pytest.fixture +def cube(): + return Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + +@pytest.fixture +def simple_cube_1(function_store, built_cube): + return set(function_store().keys()) + + +@pytest.fixture +def simple_cube_2(df_seed, df_enrich, cube, function_store2): + build_cube(data={cube.seed_dataset: df_seed}, cube=cube, store=function_store2) + return set(function_store2().keys()) + + +@pytest.fixture +def built_cube(df_seed, df_enrich, cube, function_store): + return build_cube( + data={cube.seed_dataset: df_seed.copy(), "enrich": df_enrich.copy()}, + cube=cube, + store=function_store, + ) + + +@pytest.fixture +def df_seed(): + return pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) + + +@pytest.fixture +def df_enrich(): + return pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) + + +def assert_same_keys(store1, store2, keys): + k1 = set(store1().keys()) + k2 = set(store2().keys()) + assert keys.issubset(k1) + assert keys.issubset(k2) + + for k in sorted(keys): + b1 = store1().get(k) + b2 = store1().get(k) + assert b1 == b2 + + +def assert_target_cube_readable(tgt_cube_uuid, tgt_store, df_seed, df_enrich): + tgt_cube = Cube( + dimension_columns=["x"], partition_columns=["p"], uuid_prefix=tgt_cube_uuid + ) + tgt_cube_res = query_cube(cube=tgt_cube, store=tgt_store)[0] + assert tgt_cube_res is not None + assert tgt_cube_res[["x", "p", "v1"]].equals(df_seed) + assert tgt_cube_res[["x", "p", "v2"]].equals(df_enrich) + + +def test_simple(driver, function_store, function_store2, cube, simple_cube_1): + driver(cube=cube, src_store=function_store, tgt_store=function_store2) + assert_same_keys(function_store, function_store2, simple_cube_1) + + +def test_simple_copy_cube_rename_dataset( + driver, function_store, function_store2, cube, simple_cube_1, df_seed, df_enrich +): + """ + Rename a dataset while copying, but leave the cube name as is + """ + + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + + ds_name_old = "enrich" + ds_name_new = "augmented" + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + renamed_datasets={ds_name_old: ds_name_new}, + ) + + tgt_keys = function_store2().keys() + for src_key in sorted(simple_cube_1): + tgt_key = src_key.replace(ds_name_old, ds_name_new) + assert tgt_key in tgt_keys + src_blob = function_store().get(src_key) + tgt_blob = function_store2().get(tgt_key) + if tgt_key.endswith("by-dataset-metadata.json"): + src_blob_mod = ( + src_blob.decode("utf-8") + .replace(ds_name_old, ds_name_new) + .encode("utf-8") + ) + assert src_blob_mod == tgt_blob + else: + assert src_blob == tgt_blob + + assert_target_cube_readable("cube", function_store2, df_seed, df_enrich) + + +def test_simple_copy_cube_rename_cube_prefix( + driver, function_store, function_store2, cube, simple_cube_1, df_seed, df_enrich +): + """ + Rename a cube while copying, but leave the dataset names as they are + """ + old_cube_prefix = "cube" + new_cube_prefix = "my_target_cube" + + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + renamed_cube_prefix=new_cube_prefix, + ) + + tgt_keys = function_store2().keys() + for src_key in sorted(simple_cube_1): + tgt_key = src_key.replace(f"{old_cube_prefix}++", f"{new_cube_prefix}++") + assert tgt_key in tgt_keys + + src_blob = function_store().get(src_key) + tgt_blob = function_store2().get(tgt_key) + if tgt_key.endswith("by-dataset-metadata.json"): + src_blob_mod = ( + src_blob.decode("utf-8") + .replace(f"{old_cube_prefix}++", f"{new_cube_prefix}++") + .encode("utf-8") + ) + assert src_blob_mod == tgt_blob + else: + assert src_blob == tgt_blob + + assert_target_cube_readable(new_cube_prefix, function_store2, df_seed, df_enrich) + + +def test_simple_copy_cube_rename_cube_prefix_and_dataset( + driver, function_store, function_store2, cube, simple_cube_1, df_seed, df_enrich +): + """ + Rename a cube and a dataset while copying + """ + old_cube_prefix = "cube" + new_cube_prefix = "my_target_cube" + ds_name_old = "enrich" + ds_name_new = "augmented" + + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + renamed_cube_prefix=new_cube_prefix, + renamed_datasets={ds_name_old: ds_name_new}, + ) + + tgt_keys = function_store2().keys() + for src_key in sorted(simple_cube_1): + tgt_key = src_key.replace( + f"{old_cube_prefix}++", f"{new_cube_prefix}++" + ).replace(f"++{ds_name_old}", f"++{ds_name_new}") + assert tgt_key in tgt_keys + + src_blob = function_store().get(src_key) + tgt_blob = function_store2().get(tgt_key) + + if tgt_key.endswith("by-dataset-metadata.json"): + src_blob_mod = ( + src_blob.decode("utf-8") + .replace(f"{old_cube_prefix}++", f"{new_cube_prefix}++") + .replace(f"++{ds_name_old}", f"++{ds_name_new}") + .encode("utf-8") + ) + assert src_blob_mod == tgt_blob + else: + assert src_blob == tgt_blob + + assert_target_cube_readable(new_cube_prefix, function_store2, df_seed, df_enrich) + + +def test_simple_rename_cube_same_stores( + driver, function_store, cube, simple_cube_1, df_seed, df_enrich +): + new_cube_prefix = "my_target_cube" + ds_name_old = "enrich" + ds_name_new = "augmented" + + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + + with pytest.raises(ValueError): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store, + renamed_cube_prefix=new_cube_prefix, + renamed_datasets={ds_name_old: ds_name_new}, + ) + + +def test_copy_fail_overwrite_true( + driver, mocker, cube, simple_cube_1, function_store, function_store2 +): + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + with pytest.raises(RuntimeError): + with mocker.patch( + "kartothek.io.eager_cube.copy_dataset", + side_effect=ValueError("Copying cube failed horribly."), + ): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + renamed_cube_prefix="new_cube", + overwrite=True, + ) + + +def test_copy_fail_overwrite_false( + driver, mocker, cube, simple_cube_1, function_store, function_store2 +): + # NB: only implemented for eager copying so far + if "copy_cube" not in str(driver): + pytest.skip() + + def side_effect(*args, **kwargs): + if side_effect.counter == 0: + side_effect.counter += 1 + return copy_dataset(*args, **kwargs) + else: + raise ValueError("Something unexpected happened during cube copy.") + + side_effect.counter = 0 + + with mocker.patch("kartothek.io.eager_cube.copy_dataset", side_effect=side_effect): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + renamed_cube_prefix="new_cube", + overwrite=False, + ) + + assert len(function_store2().keys()) == 0 + + +def test_overwrite_fail( + driver, function_store, function_store2, cube, simple_cube_1, simple_cube_2 +): + assert simple_cube_1 != simple_cube_2 + + data_backup = {k: function_store2().get(k) for k in simple_cube_2} + + with pytest.raises(RuntimeError) as exc: + driver(cube=cube, src_store=function_store, tgt_store=function_store2) + assert ( + str(exc.value) + == 'Dataset "cube++seed" exists in target store but overwrite was set to False' + ) + + # check everything kept untouched + assert set(function_store2().keys()) == simple_cube_2 + for k in sorted(simple_cube_2): + assert function_store2().get(k) == data_backup[k] + + +def test_overwrite_ok( + driver, function_store, function_store2, cube, simple_cube_1, simple_cube_2 +): + driver( + cube=cube, src_store=function_store, tgt_store=function_store2, overwrite=True + ) + assert_same_keys(function_store, function_store2, simple_cube_1) + + +@pytest.mark.parametrize("overwrite", [False, True]) +def test_fail_stores_identical_overwrite_false( + driver, function_store, cube, built_cube, overwrite +): + with pytest.raises(ValueError) as exc: + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store, + overwrite=overwrite, + ) + assert str(exc.value) == "Stores are identical but should not be." + + +def test_ignore_other(driver, function_store, function_store2): + dfs = [] + cubes = [] + for i in range(3): + dfs.append( + pd.DataFrame( + { + "x{}".format(i): [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v{}".format(i): [10, 11, 12, 13], + } + ) + ) + + cubes.append( + Cube( + dimension_columns=["x{}".format(i)], + partition_columns=["p"], + uuid_prefix="cube{}".format(i), + ) + ) + + build_cube(data=dfs[0], cube=cubes[0], store=function_store) + build_cube(data=dfs[1], cube=cubes[1], store=function_store) + build_cube(data=dfs[2], cube=cubes[2], store=function_store2) + + keys_in_1 = set(function_store().keys()) + keys_in_2 = set(function_store2().keys()) + data_backup1 = {k: function_store().get(k) for k in keys_in_1} + data_backup2 = {k: function_store2().get(k) for k in keys_in_2} + + driver(cube=cubes[1], src_store=function_store, tgt_store=function_store2) + + # store 1 is untouched + assert set(function_store().keys()) == keys_in_1 + for k in sorted(keys_in_1): + assert function_store().get(k) == data_backup1[k] + + # store 2 is partly untouched + for k in sorted(keys_in_2): + assert function_store2().get(k) == data_backup2[k] + + # test new keys + keys_new = set(function_store2().keys()) - keys_in_2 + assert_same_keys(function_store, function_store2, keys_new) + + +def test_invalid_partial_copy1( + df_seed, df_enrich, cube, function_store, function_store2, simple_cube_2, driver +): + # build a cube that would be incompatible w/ simple_cube_2 + df_seed = df_seed.copy() + df_enrich = df_enrich.copy() + + df_seed["x"] = df_seed["x"].astype(str) + df_enrich["x"] = df_enrich["x"].astype(str) + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + + keys = set(function_store().keys()) + + # now copy simple_cube_2 over existing cube. + # this only copies the seed table since simple_cube_2 does not have an enrich table. + # it should fail because X is incompatible + with pytest.raises(ValueError) as exc: + driver( + cube=cube, + src_store=function_store2, + tgt_store=function_store, + overwrite=True, + ) + assert 'Found incompatible entries for column "x"' in str(exc.value) + assert keys == set(function_store().keys()) + + +def test_invalid_partial_copy2( + df_seed, df_enrich, cube, function_store, function_store2, simple_cube_1, driver +): + # build a cube that would be incompatible w/ simple_cube_1 + df_seed = df_seed.copy() + df_enrich = df_enrich.copy() + + df_seed["x"] = df_seed["x"].astype(str) + df_enrich["x"] = df_enrich["x"].astype(str) + build_cube( + data={cube.seed_dataset: df_seed, "enrich2": df_enrich}, + cube=cube, + store=function_store2, + ) + + keys = set(function_store2().keys()) + + # now copy simple_cube_1 over existing cube. + # this only copies the seed and enrich table since simple_cube_1 does not have an enrich2 table. + # it should fail because X is incompatible. + with pytest.raises(ValueError) as exc: + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + overwrite=True, + ) + assert "Found columns present in multiple datasets" in str(exc.value) + assert keys == set(function_store2().keys()) + + +def test_partial_copy_dataset_list( + driver, function_store, function_store2, cube, built_cube +): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + datasets=["seed", "enrich"], + ) + all_datasets = discover_datasets_unchecked( + uuid_prefix=cube.uuid_prefix, + store=function_store, + filter_ktk_cube_dataset_ids=["seed", "enrich"], + ) + copied_ds_keys = set() + copied_ds_keys |= get_dataset_keys(all_datasets["seed"]) + copied_ds_keys |= get_dataset_keys(all_datasets["enrich"]) + tgt_store_keys = set(function_store2().keys()) + assert copied_ds_keys == tgt_store_keys + + +def test_partial_copy_dataset_dict( + driver, function_store, function_store2, cube, built_cube +): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + datasets={"seed": built_cube["seed"], "enrich": built_cube["enrich"]}, + ) + all_datasets = discover_datasets_unchecked( + uuid_prefix=cube.uuid_prefix, + store=function_store, + filter_ktk_cube_dataset_ids=["seed", "enrich"], + ) + copied_ds_keys = set() + copied_ds_keys |= get_dataset_keys(all_datasets["seed"]) + copied_ds_keys |= get_dataset_keys(all_datasets["enrich"]) + tgt_store_keys = set(function_store2().keys()) + assert copied_ds_keys == tgt_store_keys + + +def test_invalid_partial_copy( + driver, df_seed, df_enrich, function_store, function_store2, cube, built_cube +): + # build a cube that would be incompatible with cube in function_store + df_seed = df_seed.copy() + df_enrich = df_enrich.copy() + df_seed["x"] = df_seed["x"].astype(str) + df_enrich["x"] = df_enrich["x"].astype(str) + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store2, + ) + tgt_store_key_before = set(function_store2().keys()) + with pytest.raises(ValueError) as exc: + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + overwrite=True, + datasets=["enrich"], + ) + assert 'Found incompatible entries for column "x"' in str(exc.value) + assert tgt_store_key_before == set(function_store2().keys()) + + +def test_fail_no_store_factory_src( + driver, function_store, function_store2, cube, skip_eager +): + store = function_store() + with pytest.raises(TypeError) as exc: + driver(cube=cube, src_store=store, tgt_store=function_store2, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_no_store_factory_tgt( + driver, function_store, function_store2, cube, skip_eager +): + store = function_store2() + with pytest.raises(TypeError) as exc: + driver(cube=cube, src_store=function_store, tgt_store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_no_src_cube(cube, function_store, function_store2, driver): + with pytest.raises(RuntimeError) as exc: + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + overwrite=False, + ) + assert "not found" in str(exc.value) + + +def test_fail_no_src_cube_dataset( + cube, built_cube, function_store, function_store2, driver +): + with pytest.raises(RuntimeError) as exc: + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + overwrite=False, + datasets=["non_existing"], + ) + assert "non_existing" in str(exc.value) + + +def test_read_only_source( + driver, function_store_ro, function_store2, cube, simple_cube_1 +): + driver(cube=cube, src_store=function_store_ro, tgt_store=function_store2) + assert_same_keys(function_store_ro, function_store2, simple_cube_1) + + +def test_fail_blocksize_wrong_type( + driver, function_store, function_store2, cube, simple_cube_1, skip_eager +): + with pytest.raises(TypeError, match="blocksize must be an integer but is str"): + driver( + cube=cube, + src_store=function_store, + tgt_store=function_store2, + blocksize="foo", + ) + + +def test_fail_blocksize_negative( + driver, function_store, function_store2, cube, simple_cube_1, skip_eager +): + with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): + driver( + cube=cube, src_store=function_store, tgt_store=function_store2, blocksize=-1 + ) + + +def test_fail_blocksize_zero( + driver, function_store, function_store2, cube, simple_cube_1, skip_eager +): + with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): + driver( + cube=cube, src_store=function_store, tgt_store=function_store2, blocksize=0 + ) diff --git a/tests/io/cube/test_delete.py b/tests/io/cube/test_delete.py index 558366a7..227905bd 100644 --- a/tests/io/cube/test_delete.py +++ b/tests/io/cube/test_delete.py @@ -1,9 +1,13 @@ +# from kartothek.io.testing.delete_cube import * # noqa +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_delete +from kartothek.api.discover import discover_datasets_unchecked +from kartothek.core.cube.cube import Cube from kartothek.io.dask.bag_cube import delete_cube_bag -from kartothek.io.eager_cube import delete_cube -from kartothek.io.testing.delete_cube import * # noqa +from kartothek.io.eager_cube import build_cube, delete_cube +from kartothek.utils.ktk_adapters import get_dataset_keys @pytest.fixture @@ -18,3 +22,161 @@ def driver(driver_name): return delete_cube else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +def test_simple(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + driver(cube=cube, store=function_store) + + assert set(function_store().keys()) == set() + + +def test_keep_other(driver, function_store): + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube1 = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube1") + cube2 = cube1.copy(uuid_prefix="cube2") + + build_cube(data=df, cube=cube1, store=function_store) + keys = set(function_store().keys()) + + build_cube(data=df, cube=cube2, store=function_store) + + driver(cube=cube2, store=function_store) + + assert set(function_store().keys()) == keys + + +def test_keep_garbage_due_to_no_listing(driver, function_store): + df1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + df2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [2, 2, 3, 3], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + # test build DF1 to see which keys are created + build_cube(data=df1, cube=cube, store=function_store) + keys1 = set(function_store().keys()) + + # wipe + for k in list(function_store().keys()): + function_store().delete(k) + + # test build DF2 to see which keys are created + build_cube(data=df2, cube=cube, store=function_store) + keys2 = set(function_store().keys()) + + # wipe again + for k in list(function_store().keys()): + function_store().delete(k) + + # some keys are obviosly present everytime (like central metadata and + # common metadata) + keys_common = keys1 & keys2 + + # build DF1 and overwrite w/ DF2 + build_cube(data=df1, cube=cube, store=function_store) + keys3 = set(function_store().keys()) + + build_cube(data=df2, cube=cube, store=function_store, overwrite=True) + + # now some keys if DF1 must be leftovers/gargabe that cannot be deleted w/o listing the entire store (which would + # be too expensive) + gargabe = keys3 - keys_common + assert len(gargabe) > 0 + + driver(cube=cube, store=function_store) + + assert set(function_store().keys()) == gargabe + + +def test_delete_twice(driver, function_store): + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube(data=df, cube=cube, store=function_store) + driver(cube=cube, store=function_store) + driver(cube=cube, store=function_store) + + assert set(function_store().keys()) == set() + + +def test_partial_delete(driver, function_store): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} + ) + df_1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "a": [20, 21, 22, 23]}) + df_2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "b": [20, 21, 22, 23]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + datasets = build_cube( + data={cube.seed_dataset: df_seed, "enrich-1": df_1, "enrich-2": df_2}, + cube=cube, + store=function_store, + ) + enrich_1_keys = get_dataset_keys( + discover_datasets_unchecked( + uuid_prefix=cube.uuid_prefix, + store=function_store, + filter_ktk_cube_dataset_ids=["enrich-1"], + )["enrich-1"] + ) + enrich_2_keys = get_dataset_keys( + discover_datasets_unchecked( + uuid_prefix=cube.uuid_prefix, + store=function_store, + filter_ktk_cube_dataset_ids=["enrich-2"], + )["enrich-2"] + ) + all_keys = set(function_store().keys()) + driver(cube=cube, store=function_store, datasets=["enrich-1"]) + assert set(function_store().keys()) == all_keys - enrich_1_keys + + driver(cube=cube, store=function_store, datasets={"enrich-2": datasets["enrich-2"]}) + assert set(function_store().keys()) == all_keys - enrich_1_keys - enrich_2_keys + + +def test_fail_no_store_factory(driver, function_store, skip_eager): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + store = function_store() + with pytest.raises(TypeError) as exc: + driver(cube=cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(TypeError, match="blocksize must be an integer but is str"): + driver(cube=cube, store=function_store, blocksize="foo") + + +def test_fail_blocksize_negative(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): + driver(cube=cube, store=function_store, blocksize=-1) + + +def test_fail_blocksize_zero(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): + driver(cube=cube, store=function_store, blocksize=0) diff --git a/tests/io/cube/test_extend.py b/tests/io/cube/test_extend.py index 7ff863ca..419ce522 100644 --- a/tests/io/cube/test_extend.py +++ b/tests/io/cube/test_extend.py @@ -8,10 +8,16 @@ from tests.io.cube.utils import wrap_bag_write, wrap_ddf_write from kartothek.core.cube.cube import Cube +from kartothek.core.dataset import DatasetMetadata +from kartothek.core.index import ExplicitSecondaryIndex, PartitionIndex from kartothek.io.dask.bag_cube import extend_cube_from_bag from kartothek.io.dask.dataframe_cube import extend_cube_from_dataframe -from kartothek.io.eager_cube import extend_cube -from kartothek.io.testing.extend_cube import * # noqa +from kartothek.io.eager_cube import build_cube, extend_cube +from kartothek.io_components.cube.write import MultiTableCommitAborted +from kartothek.io_components.metapartition import SINGLE_TABLE +from kartothek.serialization._parquet import ParquetSerializer + +from .utils import assert_num_row_groups @pytest.fixture @@ -110,3 +116,442 @@ def test_function_executed_once(driver, function_store, driver_name, existing_cu raise ValueError("Missing implementation for driver: {}".format(driver_name)) assert len(function_store().keys(prefix="counter.")) == 2 + + +@pytest.fixture +def existing_cube(function_store): + df_source = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]} + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data={"source": df_source, "enrich": df_enrich}, cube=cube, store=function_store + ) + return cube + + +def test_simple(driver, function_store, existing_cube): + """ + Simple integration test w/ single extra dataset. + """ + df = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v3": [10, 11, 12, 13], + "i3": [100, 101, 102, 103], + } + ) + result = driver(data={"extra": df}, cube=existing_cube, store=function_store) + + assert set(result.keys()) == {"extra"} + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert ds.uuid == existing_cube.ktk_dataset_uuid("extra") + assert len(ds.partitions) == 2 + + assert set(ds.indices.keys()) == {"p", "i3"} + assert isinstance(ds.indices["p"], PartitionIndex) + assert isinstance(ds.indices["i3"], ExplicitSecondaryIndex) + + assert ds.table_name == SINGLE_TABLE + + +@pytest.mark.parametrize("chunk_size", [None, 2]) +def test_rowgroups_are_applied_when_df_serializer_is_passed_to_extend_cube( + driver, function_store, existing_cube, chunk_size +): + """ + Test that the dataset is split into row groups depending on the chunk size + """ + df_extra = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], + ) + result = driver( + data={"extra": df_extra}, + cube=existing_cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size), + ) + dataset = result["extra"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 3} + part_chunk_size = {0: chunk_size, 1: chunk_size} + + assert len(dataset.partitions) == 2 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_single_rowgroup_when_df_serializer_is_not_passed_to_extend_cube( + driver, function_store, existing_cube +): + """ + Test that the dataset has a single row group as default path + """ + df_extra = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], + ) + result = driver(data={"extra": df_extra}, cube=existing_cube, store=function_store,) + dataset = result["extra"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 3} + part_chunk_size = {0: None, 1: None} + + assert len(dataset.partitions) == 2 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_compression_is_compatible_on_extend_cube(driver, function_store): + """ + Test that partitons written with different compression algorithms are compatible + + The compression algorithms are not parametrized because their availability depends + on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts + of the code. A fully parametrized test would also increase runtime and test complexity + unnecessarily. + """ + # Build cube + df = pd.DataFrame(data={"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="SNAPPY"), + ) + + df_extra = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [0, 1, 1, 1]}, columns=["x", "p"], + ) + result = driver( + data={"extra": df_extra}, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="GZIP"), + ) + dataset = result["extra"].load_all_indices(function_store()) + + assert len(dataset.partitions) == 2 + + +def test_fails_incompatible_dtypes(driver, function_store, existing_cube): + """ + Should also cross check w/ seed dataset. + """ + df = pd.DataFrame( + { + "x": [0.0, 1.0, 2.0, 3.0], + "p": [0, 0, 1, 1], + "v3": [10, 11, 12, 13], + "i3": [100, 101, 102, 103], + } + ) + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver(data={"extra": df}, cube=existing_cube, store=function_store) + cause = exc_info.value.__cause__ + assert isinstance(cause, ValueError) + assert 'Found incompatible entries for column "x"' in str(cause) + assert not DatasetMetadata.exists( + existing_cube.ktk_dataset_uuid("extra"), function_store() + ) + + +def test_fails_seed_dataset(driver, function_store, existing_cube): + """ + Users cannot overwrite seed dataset since it is used for consisteny checks. + """ + pre_keys = set(function_store().keys()) + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) + with pytest.raises(ValueError) as exc: + driver( + data={existing_cube.seed_dataset: df}, + cube=existing_cube, + store=function_store, + ) + assert 'Seed data ("source") cannot be written during extension.' in str(exc.value) + + post_keys = set(function_store().keys()) + assert pre_keys == post_keys + + +def test_fails_overlapping_payload_seed(driver, function_store, existing_cube): + """ + Forbidden by spec, results in problems during query. + """ + pre_keys = set(function_store().keys()) + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) + with pytest.raises(ValueError) as exc: + driver(data={"extra": df}, cube=existing_cube, store=function_store) + assert 'Payload written in "extra" is already present in cube: v1' in str(exc.value) + assert not DatasetMetadata.exists( + existing_cube.ktk_dataset_uuid("extra"), function_store() + ) + + post_keys = set(function_store().keys()) + assert pre_keys == post_keys + + +def test_fails_overlapping_payload_enrich(driver, function_store, existing_cube): + """ + Forbidden by spec, results in problems during query. + """ + pre_keys = set(function_store().keys()) + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) + with pytest.raises(ValueError) as exc: + driver(data={"extra": df}, cube=existing_cube, store=function_store) + assert 'Payload written in "extra" is already present in cube: v2' in str(exc.value) + assert not DatasetMetadata.exists( + existing_cube.ktk_dataset_uuid("extra"), function_store() + ) + + post_keys = set(function_store().keys()) + assert pre_keys == post_keys + + +def test_fails_overlapping_payload_partial(driver, function_store, existing_cube): + """ + Forbidden by spec, results in problems during query. + """ + pre_keys = set(function_store().keys()) + df1 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [10, 11, 12, 13]}) + df2 = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]}) + with pytest.raises(ValueError) as exc: + driver( + data={"extra1": df1, "extra2": df2}, + cube=existing_cube, + store=function_store, + ) + assert 'Payload written in "extra1" is already present in cube: v2' in str( + exc.value + ) + + assert not DatasetMetadata.exists( + existing_cube.ktk_dataset_uuid("extra1"), function_store() + ) + # extra2 might exist, depending on the compute graph + + # extra2 keys might be present, only look that extra1 is absent + post_keys = set(function_store().keys()) + extra_keys = post_keys - pre_keys + extra1_keys = {k for k in extra_keys if "extra1" in k} + assert extra1_keys == set() + + +def test_fails_overlapping_payload_overwrite(driver, function_store, existing_cube): + """ + Forbidden by spec, results in problems during query. + """ + pre_keys = set(function_store().keys()) + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]}) + with pytest.raises(ValueError) as exc: + driver( + data={"enrich": df}, + cube=existing_cube, + store=function_store, + overwrite=True, + ) + assert 'Payload written in "enrich" is already present in cube: v1' in str( + exc.value + ) + + post_keys = set(function_store().keys()) + assert pre_keys == post_keys + + +def test_overwrite_single(driver, function_store, existing_cube): + """ + Simple overwrite of the enrich dataset. + """ + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]}) + + # does not work w/o explicit flag + keys = set(function_store().keys()) + with pytest.raises(RuntimeError) as exc: + driver(data={"enrich": df}, cube=existing_cube, store=function_store) + assert "already exists" in str(exc.value) + assert set(function_store().keys()) == keys + + # but works with flag + result = driver( + data={"enrich": df}, cube=existing_cube, store=function_store, overwrite=True + ) + + assert set(result.keys()) == {"enrich"} + + ds = list(result.values())[0] + ds = ds.load_all_indices(function_store()) + + assert ds.uuid == existing_cube.ktk_dataset_uuid("enrich") + assert len(ds.partitions) == 2 + + +def test_overwrite_move_columns(driver, function_store, existing_cube): + """ + Move columns v1 and i1 from enrich to extra. + """ + df_enrich = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v3": [10, 11, 12, 13], + "i3": [100, 101, 102, 103], + } + ) + df_extra = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v2": [10, 11, 12, 13], + "i2": [100, 101, 102, 103], + } + ) + + result = driver( + data={"enrich": df_enrich, "extra": df_extra}, + cube=existing_cube, + store=function_store, + overwrite=True, + ) + + assert set(result.keys()) == {"enrich", "extra"} + + ds_enrich = result["enrich"].load_all_indices(function_store()) + ds_extra = result["extra"].load_all_indices(function_store()) + + assert set(ds_enrich.indices.keys()) == {"p", "i3"} + assert isinstance(ds_enrich.indices["p"], PartitionIndex) + assert isinstance(ds_enrich.indices["i3"], ExplicitSecondaryIndex) + + assert set(ds_extra.indices.keys()) == {"p", "i2"} + assert isinstance(ds_extra.indices["p"], PartitionIndex) + assert isinstance(ds_extra.indices["i2"], ExplicitSecondaryIndex) + + +def test_fail_all_empty(driver, function_store, existing_cube): + """ + Might happen due to DB-based filters. + """ + df = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]} + ).loc[[]] + + with pytest.raises(MultiTableCommitAborted) as exc_info: + driver(data={"extra": df}, cube=existing_cube, store=function_store) + exc = exc_info.value.__cause__ + assert isinstance(exc, ValueError) + assert "Cannot write empty datasets: extra" in str(exc) + assert not DatasetMetadata.exists( + existing_cube.ktk_dataset_uuid("extra"), function_store() + ) + + +def test_fail_not_a_df(driver, function_store, existing_cube): + """ + Pass some weird objects in. + """ + with pytest.raises(TypeError) as exc: + driver( + data={"extra": pd.Series(range(10))}, + cube=existing_cube, + store=function_store, + ) + assert ( + 'Provided DataFrame is not a pandas.DataFrame or None, but is a "Series"' + in str(exc.value) + ) + + +def test_fail_wrong_dataset_ids( + driver, function_store, existing_cube, skip_eager, driver_name +): + if driver_name == "dask_dataframe": + pytest.skip("not an interface for dask.dataframe") + + df_extra = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v2": [20, 21, 22, 23]} + ) + with pytest.raises(ValueError) as exc: + driver( + data={"extra": df_extra}, + cube=existing_cube, + store=function_store, + ktk_cube_dataset_ids=["other"], + ) + + assert ( + 'Ktk_cube Dataset ID "extra" is present during pipeline execution ' + "but was not specified in ktk_cube_dataset_ids (other)." in str(exc.value) + ) + + +def test_fail_no_store_factory(driver, function_store, existing_cube, skip_eager): + df = pd.DataFrame( + { + "x": [0, 1, 2, 3], + "p": [0, 0, 1, 1], + "v3": [10, 11, 12, 13], + "i3": [100, 101, 102, 103], + } + ) + store = function_store() + with pytest.raises(TypeError) as exc: + driver(data={"extra": df}, cube=existing_cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fails_metadata_wrong_type(driver, function_store, existing_cube): + df_extra = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} + ) + with pytest.raises( + TypeError, match="Provided metadata should be a dict but is int" + ): + driver( + data={"extra": df_extra}, + cube=existing_cube, + store=function_store, + metadata=1, + ) + + +def test_fails_metadata_unknown_id(driver, function_store, existing_cube): + df_extra = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} + ) + with pytest.raises( + ValueError, + match="Provided metadata for otherwise unspecified ktk_cube_dataset_ids: bar, foo", + ): + driver( + data={"extra": df_extra}, + cube=existing_cube, + store=function_store, + metadata={"extra": {}, "foo": {}, "bar": {}}, + ) + + +def test_fails_metadata_nested_wrong_type(driver, function_store, existing_cube): + df_extra = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v3": [10, 11, 12, 13]} + ) + with pytest.raises( + TypeError, + match="Provided metadata for dataset extra should be a dict but is int", + ): + driver( + data={"extra": df_extra}, + cube=existing_cube, + store=function_store, + metadata={"extra": 1}, + ) diff --git a/tests/io/cube/test_query.py b/tests/io/cube/test_query.py index 512ee649..3025d5f2 100644 --- a/tests/io/cube/test_query.py +++ b/tests/io/cube/test_query.py @@ -1,10 +1,38 @@ +from datetime import timedelta +from functools import partial +from itertools import permutations + +import dask.bag as db +import numpy as np +import pandas as pd +import pandas.testing as pdt import pytest +from hypothesis import given, settings +from hypothesis import strategies as st from tests.io.cube.utils import wrap_bag_read, wrap_ddf_read -from kartothek.io.dask.bag_cube import query_cube_bag +from kartothek.core.cube.conditions import ( + C, + Conjunction, + EqualityCondition, + GreaterEqualCondition, + GreaterThanCondition, + InequalityCondition, + InIntervalCondition, + IsInCondition, + LessEqualCondition, + LessThanCondition, +) +from kartothek.core.cube.cube import Cube +from kartothek.io.dask.bag_cube import build_cube_from_bag, query_cube_bag from kartothek.io.dask.dataframe_cube import query_cube_dataframe -from kartothek.io.eager_cube import query_cube -from kartothek.io.testing.query_cube import * # noqa +from kartothek.io.eager import build_dataset_indices +from kartothek.io.eager_cube import ( + append_to_cube, + build_cube, + query_cube, + remove_partitions, +) @pytest.fixture(scope="session") @@ -19,3 +47,1358 @@ def driver(driver_name): return query_cube else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +@pytest.fixture(scope="module") +def fullrange_data(): + return { + "seed": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": np.arange(16), + "i1": np.arange(16), + } + ), + "enrich_dense": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v2": np.arange(16), + "i2": np.arange(16), + } + ), + "enrich_sparse": pd.DataFrame( + { + "y": [0, 1, 2, 3, 0, 1, 2, 3], + "z": 0, + "p": [0, 0, 1, 1, 0, 0, 1, 1], + "q": [0, 0, 0, 0, 1, 1, 1, 1], + "v3": np.arange(8), + "i3": np.arange(8), + } + ), + } + + +@pytest.fixture(scope="module") +def fullrange_cube(module_store, fullrange_data): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="fullrange_cube", + index_columns=["i1", "i2", "i3"], + ) + build_cube(data=fullrange_data, store=module_store, cube=cube) + return cube + + +@pytest.fixture(scope="module") +def multipartition_cube(module_store, fullrange_data, fullrange_cube): + def _gen(part): + result = {} + for dataset_id, df in fullrange_data.items(): + df = df.copy() + df["z"] = part + result[dataset_id] = df + return result + + cube = fullrange_cube.copy(uuid_prefix="multipartition_cube") + build_cube_from_bag( + data=db.from_sequence([0, 1], partition_size=1).map(_gen), + store=module_store, + cube=cube, + ktk_cube_dataset_ids=["seed", "enrich_dense", "enrich_sparse"], + ).compute() + return cube + + +@pytest.fixture(scope="module") +def sparse_outer_data(): + return { + "seed": pd.DataFrame( + { + "x": [0, 1, 0], + "y": [0, 0, 1], + "z": 0, + "p": [0, 1, 2], + "q": 0, + "v1": [0, 3, 7], + "i1": [0, 3, 7], + } + ), + "enrich_dense": pd.DataFrame( + { + "x": [0, 0], + "y": [0, 1], + "z": 0, + "p": [0, 2], + "q": 0, + "v2": [0, 7], + "i2": [0, 7], + } + ), + "enrich_sparse": pd.DataFrame( + {"y": [0, 0], "z": 0, "p": [0, 1], "q": 0, "v3": [0, 3], "i3": [0, 3]} + ), + } + + +@pytest.fixture(scope="module") +def sparse_outer_cube(module_store, sparse_outer_data): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="sparse_outer_cube", + index_columns=["i1", "i2", "i3"], + ) + build_cube(data=sparse_outer_data, store=module_store, cube=cube) + return cube + + +@pytest.fixture(scope="module") +def sparse_outer_opt_cube( + module_store, + sparse_outer_data, + sparse_outer_cube, + sparse_outer_df, + sparse_outer_opt_df, +): + data = {} + for dataset_id in sparse_outer_data.keys(): + df = sparse_outer_data[dataset_id].copy() + + for col in sparse_outer_opt_df.columns: + if col in df.columns: + dtype = sparse_outer_opt_df[col].dtype + + if dtype == np.float64: + dtype = np.int64 + elif dtype == np.float32: + dtype = np.int32 + elif dtype == np.float16: + dtype = np.int16 + + df[col] = df[col].astype(dtype) + + data[dataset_id] = df + + cube = sparse_outer_cube.copy(uuid_prefix="sparse_outer_opt_cube") + build_cube(data=data, store=module_store, cube=cube) + return cube + + +@pytest.fixture(scope="module") +def massive_partitions_data(): + n = 17 + return { + "seed": pd.DataFrame( + { + "x": np.arange(n), + "y": np.arange(n), + "z": np.arange(n), + "p": np.arange(n), + "q": np.arange(n), + "v1": np.arange(n), + "i1": np.arange(n), + } + ), + "enrich_1": pd.DataFrame( + { + "x": np.arange(n), + "y": np.arange(n), + "z": np.arange(n), + "p": np.arange(n), + "q": np.arange(n), + "v2": np.arange(n), + "i2": np.arange(n), + } + ), + "enrich_2": pd.DataFrame( + { + "y": np.arange(n), + "z": np.arange(n), + "p": np.arange(n), + "q": np.arange(n), + "v3": np.arange(n), + "i3": np.arange(n), + } + ), + } + + +@pytest.fixture(scope="module") +def massive_partitions_cube(module_store, massive_partitions_data): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="massive_partitions_cube", + index_columns=["i1", "i2", "i3"], + ) + build_cube(data=massive_partitions_data, store=module_store, cube=cube) + return cube + + +@pytest.fixture(scope="module") +def fullrange_df(): + return ( + pd.DataFrame( + data={ + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": np.arange(16), + "v2": np.arange(16), + "v3": [0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7], + "i1": np.arange(16), + "i2": np.arange(16), + "i3": [0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7], + }, + columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], + ) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture(scope="module") +def multipartition_df(fullrange_df): + dfs = [] + for z in (0, 1): + df = fullrange_df.copy() + df["z"] = z + dfs.append(df) + return ( + pd.concat(dfs, ignore_index=True) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture(scope="module") +def sparse_outer_df(): + return ( + pd.DataFrame( + data={ + "x": [0, 1, 0], + "y": [0, 0, 1], + "z": 0, + "p": [0, 1, 2], + "q": 0, + "v1": [0, 3, 7], + "v2": [0, np.nan, 7], + "v3": [0, 3, np.nan], + "i1": [0, 3, 7], + "i2": [0, np.nan, 7], + "i3": [0, 3, np.nan], + }, + columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], + ) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture(scope="module") +def sparse_outer_opt_df(sparse_outer_df): + df = sparse_outer_df.copy() + df["x"] = df["x"].astype(np.int16) + df["y"] = df["y"].astype(np.int32) + df["z"] = df["z"].astype(np.int8) + + df["v1"] = df["v1"].astype(np.int8) + df["i1"] = df["i1"].astype(np.int8) + + return df + + +@pytest.fixture(scope="module") +def massive_partitions_df(): + n = 17 + return ( + pd.DataFrame( + data={ + "x": np.arange(n), + "y": np.arange(n), + "z": np.arange(n), + "p": np.arange(n), + "q": np.arange(n), + "v1": np.arange(n), + "v2": np.arange(n), + "v3": np.arange(n), + "i1": np.arange(n), + "i2": np.arange(n), + "i3": np.arange(n), + }, + columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], + ) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture(scope="module") +def updated_cube(module_store, fullrange_data): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="updated_cube", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data={ + cube.seed_dataset: pd.DataFrame( + { + "x": [0, 0, 1, 1, 2, 2], + "y": [0, 1, 0, 1, 0, 1], + "z": 0, + "p": [0, 0, 1, 1, 2, 2], + "q": 0, + "v1": np.arange(6), + "i1": np.arange(6), + } + ), + "enrich": pd.DataFrame( + { + "x": [0, 0, 1, 1, 2, 2], + "y": [0, 1, 0, 1, 0, 1], + "z": 0, + "p": [0, 0, 1, 1, 2, 2], + "q": 0, + "v2": np.arange(6), + "i2": np.arange(6), + } + ), + "extra": pd.DataFrame( + { + "y": [0, 1, 0, 1, 0, 1], + "z": 0, + "p": [0, 0, 1, 1, 2, 2], + "q": 0, + "v3": np.arange(6), + "i3": np.arange(6), + } + ), + }, + store=module_store, + cube=cube, + ) + remove_partitions( + cube=cube, + store=module_store, + ktk_cube_dataset_ids=["enrich"], + conditions=C("p") >= 1, + ) + append_to_cube( + data={ + "enrich": pd.DataFrame( + { + "x": [1, 1], + "y": [0, 1], + "z": 0, + "p": [1, 1], + "q": 0, + "v2": [7, 8], + "i2": [7, 8], + } + ) + }, + store=module_store, + cube=cube, + ) + return cube + + +@pytest.fixture(scope="module") +def updated_df(): + return ( + pd.DataFrame( + data={ + "x": [0, 0, 1, 1, 2, 2], + "y": [0, 1, 0, 1, 0, 1], + "z": 0, + "p": [0, 0, 1, 1, 2, 2], + "q": 0, + "v1": np.arange(6), + "v2": [0, 1, 7, 8, np.nan, np.nan], + "v3": np.arange(6), + "i1": np.arange(6), + "i2": [0, 1, 7, 8, np.nan, np.nan], + "i3": np.arange(6), + }, + columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], + ) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture(scope="module") +def data_no_part(): + return { + "seed": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": np.arange(16), + "i1": np.arange(16), + } + ), + "enrich_dense": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "v2": np.arange(16), + "i2": np.arange(16), + } + ), + "enrich_sparse": pd.DataFrame( + {"y": [0, 1, 2, 3], "z": 0, "v3": np.arange(4), "i3": np.arange(4)} + ), + } + + +@pytest.fixture(scope="module") +def no_part_cube(module_store, data_no_part): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="data_no_part", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data=data_no_part, + store=module_store, + cube=cube, + partition_on={"enrich_dense": [], "enrich_sparse": []}, + ) + return cube + + +@pytest.fixture(scope="module") +def other_part_cube(module_store, data_no_part): + cube = Cube( + dimension_columns=["x", "y", "z"], + partition_columns=["p", "q"], + uuid_prefix="other_part_cube", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data=data_no_part, + store=module_store, + cube=cube, + partition_on={"enrich_dense": ["i2"], "enrich_sparse": ["i3"]}, + ) + return cube + + +@pytest.fixture(scope="module") +def no_part_df(): + return ( + pd.DataFrame( + data={ + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "z": 0, + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": np.arange(16), + "v2": np.arange(16), + "v3": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "i1": np.arange(16), + "i2": np.arange(16), + "i3": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + }, + columns=["i1", "i2", "i3", "p", "q", "v1", "v2", "v3", "x", "y", "z"], + ) + .sort_values(["x", "y", "z", "p", "q"]) + .reset_index(drop=True) + ) + + +@pytest.fixture( + params=[ + "fullrange", + "multipartition", + "sparse_outer", + "sparse_outer_opt", + "massive_partitions", + "updated", + "no_part", + "other_part", + ], + scope="module", +) +def testset(request): + return request.param + + +@pytest.fixture(scope="module") +def test_cube( + testset, + fullrange_cube, + multipartition_cube, + sparse_outer_cube, + sparse_outer_opt_cube, + massive_partitions_cube, + updated_cube, + no_part_cube, + other_part_cube, +): + if testset == "fullrange": + return fullrange_cube + elif testset == "multipartition": + return multipartition_cube + elif testset == "sparse_outer": + return sparse_outer_cube + elif testset == "sparse_outer_opt": + return sparse_outer_opt_cube + elif testset == "massive_partitions": + return massive_partitions_cube + elif testset == "updated": + return updated_cube + elif testset == "no_part": + return no_part_cube + elif testset == "other_part": + return other_part_cube + else: + raise ValueError("Unknown param {}".format(testset)) + + +@pytest.fixture(scope="module") +def test_df( + testset, + fullrange_df, + multipartition_df, + sparse_outer_df, + sparse_outer_opt_df, + massive_partitions_df, + updated_df, + no_part_df, +): + if testset == "fullrange": + return fullrange_df + elif testset == "multipartition": + return multipartition_df + elif testset == "sparse_outer": + return sparse_outer_df + elif testset == "sparse_outer_opt": + return sparse_outer_opt_df + elif testset == "massive_partitions": + return massive_partitions_df + elif testset == "updated": + return updated_df + elif testset in ("no_part", "other_part"): + return no_part_df + else: + raise ValueError("Unknown param {}".format(testset)) + + +def test_simple_roundtrip(driver, function_store, function_store_rwro): + df = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [10, 11, 12, 13]}) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube(data=df, cube=cube, store=function_store) + result = driver(cube=cube, store=function_store_rwro) + assert len(result) == 1 + df_actual = result[0] + df_expected = df.reindex(columns=["p", "v", "x"]) + pdt.assert_frame_equal(df_actual, df_expected) + + +def test_complete(driver, module_store, test_cube, test_df): + result = driver(cube=test_cube, store=module_store) + assert len(result) == 1 + df_actual = result[0] + pdt.assert_frame_equal(df_actual, test_df) + + +def apply_condition_unsafe(df, cond): + # For the sparse_outer testset, the test_df has the wrong datatype because we cannot encode missing integer data in + # pandas. + # + # The condition will not be applicable to the DF because the DF has floats while conditions have ints. We fix that + # by modifying the the condition. + # + # In case there is no missing data because of the right conditions, kartothek will return integer data. + # assert_frame_equal will then complain about this. So in case there is no missing data, let's recover the correct + # dtype here. + + if not isinstance(cond, Conjunction): + cond = Conjunction(cond) + + float_cols = {col for col in df.columns if df[col].dtype == float} + + # convert int to float conditions + cond2 = Conjunction([]) + for col, conj in cond.split_by_column().items(): + if col in float_cols: + parts = [] + for part in conj.conditions: + if isinstance(part, IsInCondition): + part = IsInCondition( + column=part.column, value=tuple((float(v) for v in part.value)) + ) + elif isinstance(part, InIntervalCondition): + part = InIntervalCondition( + column=part.column, + start=float(part.start), + stop=float(part.stop), + ) + else: + part = part.__class__(column=part.column, value=float(part.value)) + parts.append(part) + conj = Conjunction(parts) + cond2 &= conj + + # apply conditions + df = cond2.filter_df(df).reset_index(drop=True) + + # convert float columns to int columns + for col in df.columns: + if df[col].notnull().all(): + dtype = df[col].dtype + if dtype == np.float64: + dtype = np.int64 + elif dtype == np.float32: + dtype = np.int32 + elif dtype == np.float16: + dtype = np.int16 + + df[col] = df[col].astype(dtype) + + return df + + +@pytest.mark.parametrize( + "cond", + [ + C("v1") >= 7, + C("v1") >= 10000, + C("v2") >= 7, + C("v3") >= 3, + C("i1") >= 7, + C("i1") >= 10000, + C("i2") >= 7, + C("i2") != 0, + C("i3") >= 3, + C("p") >= 1, + C("q") >= 1, + C("x") >= 1, + C("y") >= 1, + (C("x") == 3) & (C("y") == 3), + (C("i1") > 0) & (C("i2") > 0), + Conjunction([]), + ], +) +def test_condition(driver, module_store, test_cube, test_df, cond): + result = driver(cube=test_cube, store=module_store, conditions=cond) + + df_expected = apply_condition_unsafe(test_df, cond) + + if df_expected.empty: + assert len(result) == 0 + else: + assert len(result) == 1 + df_actual = result[0] + pdt.assert_frame_equal(df_actual, df_expected) + + +@pytest.mark.parametrize("payload_columns", [["v1", "v2"], ["v2", "v3"], ["v3"]]) +def test_select(driver, module_store, test_cube, test_df, payload_columns): + result = driver(cube=test_cube, store=module_store, payload_columns=payload_columns) + assert len(result) == 1 + df_actual = result[0] + df_expected = test_df.loc[ + :, sorted(set(payload_columns) | {"x", "y", "z", "p", "q"}) + ] + pdt.assert_frame_equal(df_actual, df_expected) + + +def test_filter_select(driver, module_store, test_cube, test_df): + result = driver( + cube=test_cube, + store=module_store, + payload_columns=["v1", "v2"], + conditions=(C("i3") >= 3), # completely unrelated to the payload + ) + assert len(result) == 1 + df_actual = result[0] + df_expected = test_df.loc[ + test_df["i3"] >= 3, ["p", "q", "v1", "v2", "x", "y", "z"] + ].reset_index(drop=True) + pdt.assert_frame_equal(df_actual, df_expected) + + +@pytest.mark.parametrize( + "partition_by", + [["i1"], ["i2"], ["i3"], ["x"], ["y"], ["p"], ["q"], ["i1", "i2"], ["x", "y"]], +) +def test_partition_by(driver, module_store, test_cube, test_df, partition_by): + dfs_actual = driver(cube=test_cube, store=module_store, partition_by=partition_by) + + dfs_expected = [ + df_g.reset_index(drop=True) + for g, df_g in test_df.groupby(partition_by, sort=True) + ] + for df_expected in dfs_expected: + for col in df_expected.columns: + if df_expected[col].dtype == float: + try: + df_expected[col] = df_expected[col].astype(int) + except Exception: + pass + + assert len(dfs_actual) == len(dfs_expected) + for df_actual, df_expected in zip(dfs_actual, dfs_expected): + pdt.assert_frame_equal(df_actual, df_expected) + + +@pytest.mark.parametrize("dimension_columns", list(permutations(["x", "y", "z"]))) +def test_sort(driver, module_store, test_cube, test_df, dimension_columns): + result = driver( + cube=test_cube, store=module_store, dimension_columns=dimension_columns + ) + assert len(result) == 1 + df_actual = result[0] + df_expected = test_df.sort_values( + list(dimension_columns) + list(test_cube.partition_columns) + ).reset_index(drop=True) + pdt.assert_frame_equal(df_actual, df_expected) + + +@pytest.mark.parametrize("payload_columns", [["y", "z"], ["y", "z", "v3"]]) +def test_projection(driver, module_store, test_cube, test_df, payload_columns): + result = driver( + cube=test_cube, + store=module_store, + dimension_columns=["y", "z"], + payload_columns=payload_columns, + ) + assert len(result) == 1 + df_actual = result[0] + df_expected = ( + test_df.loc[:, sorted(set(payload_columns) | {"y", "z", "p", "q"})] + .drop_duplicates() + .sort_values(["y", "z", "p", "q"]) + .reset_index(drop=True) + ) + pdt.assert_frame_equal(df_actual, df_expected) + + +def test_stresstest_index_select_row(driver, function_store): + n_indices = 100 + n_rows = 1000 + + data = {"x": np.arange(n_rows), "p": 0} + for i in range(n_indices): + data["i{}".format(i)] = np.arange(n_rows) + df = pd.DataFrame(data) + + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + index_columns=["i{}".format(i) for i in range(n_indices)], + ) + + build_cube(data=df, cube=cube, store=function_store) + + conditions = Conjunction([(C("i{}".format(i)) == 0) for i in range(n_indices)]) + + result = driver( + cube=cube, + store=function_store, + conditions=conditions, + payload_columns=["p", "x"], + ) + assert len(result) == 1 + df_actual = result[0] + df_expected = df.loc[df["x"] == 0].reindex(columns=["p", "x"]) + pdt.assert_frame_equal(df_actual, df_expected) + + +def test_fail_missing_dimension_columns(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver(cube=test_cube, store=module_store, dimension_columns=["x", "a", "b"]) + assert ( + "Following dimension columns were requested but are missing from the cube: a, b" + in str(exc.value) + ) + + +def test_fail_empty_dimension_columns(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver(cube=test_cube, store=module_store, dimension_columns=[]) + assert "Dimension columns cannot be empty." in str(exc.value) + + +def test_fail_missing_partition_by(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver(cube=test_cube, store=module_store, partition_by=["foo"]) + assert ( + "Following partition-by columns were requested but are missing from the cube: foo" + in str(exc.value) + ) + + +def test_fail_unindexed_partition_by(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver(cube=test_cube, store=module_store, partition_by=["v1", "v2"]) + assert ( + "Following partition-by columns are not indexed and cannot be used: v1, v2" + in str(exc.value) + ) + + +def test_fail_missing_condition_columns(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver( + cube=test_cube, + store=module_store, + conditions=(C("foo") == 1) & (C("bar") == 2), + ) + assert ( + "Following condition columns are required but are missing from the cube: bar, foo" + in str(exc.value) + ) + + +def test_fail_missing_payload_columns(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver(cube=test_cube, store=module_store, payload_columns=["foo", "bar"]) + assert "Cannot find the following requested payload columns: bar, foo" in str( + exc.value + ) + + +def test_fail_projection(driver, module_store, test_cube, test_df): + with pytest.raises(ValueError) as exc: + driver( + cube=test_cube, + store=module_store, + dimension_columns=["y", "z"], + payload_columns=["v1"], + ) + assert ( + 'Cannot project dataset "seed" with dimensionality [x, y, z] to [y, z] ' + "while keeping the following payload intact: v1" in str(exc.value) + ) + + +def test_fail_unstable_dimension_columns(driver, module_store, test_cube, test_df): + with pytest.raises(TypeError) as exc: + driver(cube=test_cube, store=module_store, dimension_columns={"x", "y"}) + assert "which has type set has an unstable iteration order" in str(exc.value) + + +def test_fail_unstable_partition_by(driver, module_store, test_cube, test_df): + with pytest.raises(TypeError) as exc: + driver(cube=test_cube, store=module_store, partition_by={"x", "y"}) + assert "which has type set has an unstable iteration order" in str(exc.value) + + +def test_wrong_condition_type(driver, function_store, driver_name): + types = { + "int": pd.Series([-1], dtype=np.int64), + "uint": pd.Series([1], dtype=np.uint64), + "float": pd.Series([1.3], dtype=np.float64), + "bool": pd.Series([True], dtype=np.bool_), + "str": pd.Series(["foo"], dtype=object), + } + cube = Cube( + dimension_columns=["d_{}".format(t) for t in sorted(types.keys())], + partition_columns=["p_{}".format(t) for t in sorted(types.keys())], + uuid_prefix="typed_cube", + index_columns=["i_{}".format(t) for t in sorted(types.keys())], + ) + data = { + "seed": pd.DataFrame( + { + "{}_{}".format(prefix, t): types[t] + for t in sorted(types.keys()) + for prefix in ["d", "p", "v1"] + } + ), + "enrich": pd.DataFrame( + { + "{}_{}".format(prefix, t): types[t] + for t in sorted(types.keys()) + for prefix in ["d", "p", "i", "v2"] + } + ), + } + build_cube(data=data, store=function_store, cube=cube) + + df = pd.DataFrame( + { + "{}_{}".format(prefix, t): types[t] + for t in sorted(types.keys()) + for prefix in ["d", "p", "i", "v1", "v2"] + } + ) + + for col in df.columns: + t1 = col.split("_")[1] + + for t2 in sorted(types.keys()): + cond = C(col) == types[t2].values[0] + + if t1 == t2: + result = driver(cube=cube, store=function_store, conditions=cond) + assert len(result) == 1 + df_actual = result[0] + df_expected = cond.filter_df(df).reset_index(drop=True) + pdt.assert_frame_equal(df_actual, df_expected, check_like=True) + else: + with pytest.raises(TypeError) as exc: + driver(cube=cube, store=function_store, conditions=cond) + assert "has wrong type" in str(exc.value) + + +def test_condition_on_null(driver, function_store): + df = pd.DataFrame( + { + "x": pd.Series([0, 1, 2], dtype=np.int64), + "p": pd.Series([0, 0, 1], dtype=np.int64), + "v_f1": pd.Series([0, np.nan, 2], dtype=np.float64), + "v_f2": pd.Series([0, 1, np.nan], dtype=np.float64), + "v_f3": pd.Series([np.nan, np.nan, np.nan], dtype=np.float64), + "v_s1": pd.Series(["a", None, "c"], dtype=object), + "v_s2": pd.Series(["a", "b", None], dtype=object), + "v_s3": pd.Series([None, None, None], dtype=object), + } + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="nulled_cube", + index_columns=[], + ) + build_cube(data=df, store=function_store, cube=cube) + + for col in df.columns: + # only iterate over the value columns (not the dimension / partition column): + if not col.startswith("v"): + continue + + # col_type will be either 'f' for float or 's' for string; see column + # names above + col_type = col.split("_")[1][0] + if col_type == "f": + value = 1.2 + elif col_type == "s": + value = "foo" + else: + raise RuntimeError("unknown type") + + cond = C(col) == value + + df_expected = cond.filter_df(df).reset_index(drop=True) + + result = driver(cube=cube, store=function_store, conditions=cond) + + if df_expected.empty: + assert len(result) == 0 + else: + assert len(result) == 1 + df_actual = result[0] + pdt.assert_frame_equal(df_actual, df_expected, check_like=True) + + +def test_fail_no_store_factory(driver, module_store, test_cube, skip_eager): + store = module_store() + with pytest.raises(TypeError) as exc: + driver(cube=test_cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_delayed_index_build_partition_by(driver, function_store): + df_seed = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1]}) + df_extend = pd.DataFrame({"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v": [0, 0, 0, 1]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="delayed_index_cube", + index_columns=[], + ) + build_cube( + data={"seed": df_seed, "extend": df_extend}, store=function_store, cube=cube + ) + + build_dataset_indices( + store=function_store, + dataset_uuid=cube.ktk_dataset_uuid("extend"), + columns=["v"], + ) + + results = driver(cube=cube, store=function_store, partition_by=["v"]) + assert len(results) == 2 + + df_result1 = pd.DataFrame( + data={"x": [0, 1, 2], "p": [0, 0, 1], "v": [0, 0, 0]}, columns=["p", "v", "x"] + ) + df_result2 = pd.DataFrame( + data={"x": [3], "p": [1], "v": [1]}, columns=["p", "v", "x"] + ) + pdt.assert_frame_equal(results[0], df_result1) + pdt.assert_frame_equal(results[1], df_result2) + + +def test_fail_blocksize_wrong_type( + driver, module_store, test_cube, skip_eager, driver_name +): + if driver_name == "dask_dataframe": + pytest.skip("not relevant for dask.dataframe") + + with pytest.raises(TypeError, match="blocksize must be an integer but is str"): + driver(cube=test_cube, store=module_store, blocksize="foo") + + +def test_fail_blocksize_negative( + driver, module_store, test_cube, skip_eager, driver_name +): + if driver_name == "dask_dataframe": + pytest.skip("not relevant for dask.dataframe") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): + driver(cube=test_cube, store=module_store, blocksize=-1) + + +def test_fail_blocksize_zero(driver, module_store, test_cube, skip_eager, driver_name): + if driver_name == "dask_dataframe": + pytest.skip("not relevant for dask.dataframe") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): + driver(cube=test_cube, store=module_store, blocksize=0) + + +def test_delayed_index_build_correction_restriction(driver, function_store): + """ + Ensure that adding extra indices for dimension columns does not mark other datasets as restrictive. + """ + df_seed = pd.DataFrame({"x": [0, 1, 2, 3, 4, 5], "p": [0, 0, 1, 1, 2, 2]}) + df_extend = pd.DataFrame({"x": [0, 1, 2], "p": [0, 0, 1], "v": [0, 1, 2]}) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="delayed_index_cube", + index_columns=[], + ) + build_cube( + data={"seed": df_seed, "extend": df_extend}, store=function_store, cube=cube + ) + + build_dataset_indices( + store=function_store, + dataset_uuid=cube.ktk_dataset_uuid("extend"), + columns=["x"], + ) + + results = driver(cube=cube, store=function_store, conditions=C("x") >= 0) + assert len(results) == 1 + + df_actual = results[0] + df_expected = pd.DataFrame( + { + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v": [0, 1, 2, np.nan, np.nan, np.nan], + }, + columns=["p", "v", "x"], + ) + pdt.assert_frame_equal(df_actual, df_expected) + + +time_travel_stages_ops_df = [ + ( + partial( + build_cube, + data={ + "source": pd.DataFrame( + { + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v1": [0, 1, 2, 3, 4, 5], + "i1": [0, 1, 2, 3, 4, 5], + } + ), + "enrich": pd.DataFrame( + { + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v2": [0, 1, 2, 3, 4, 5], + "i2": [0, 1, 2, 3, 4, 5], + } + ), + }, + ), + pd.DataFrame( + data={ + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v1": [0, 1, 2, 3, 4, 5], + "i1": [0, 1, 2, 3, 4, 5], + "v2": [0, 1, 2, 3, 4, 5], + "i2": [0, 1, 2, 3, 4, 5], + }, + columns=["i1", "i2", "p", "v1", "v2", "x"], + ), + ), + ( + partial( + remove_partitions, ktk_cube_dataset_ids=["enrich"], conditions=C("p") > 0 + ), + pd.DataFrame( + data={ + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v1": [0, 1, 2, 3, 4, 5], + "i1": [0, 1, 2, 3, 4, 5], + "v2": [0, 1, np.nan, np.nan, np.nan, np.nan], + "i2": [0, 1, np.nan, np.nan, np.nan, np.nan], + }, + columns=["i1", "i2", "p", "v1", "v2", "x"], + ), + ), + ( + partial( + append_to_cube, + data={"enrich": pd.DataFrame({"x": [2], "p": [1], "v2": [20], "i2": [20]})}, + ), + pd.DataFrame( + data={ + "x": [0, 1, 2, 3, 4, 5], + "p": [0, 0, 1, 1, 2, 2], + "v1": [0, 1, 2, 3, 4, 5], + "i1": [0, 1, 2, 3, 4, 5], + "v2": [0, 1, 20, np.nan, np.nan, np.nan], + "i2": [0, 1, 20, np.nan, np.nan, np.nan], + }, + columns=["i1", "i2", "p", "v1", "v2", "x"], + ), + ), + ( + partial( + append_to_cube, + data={ + "source": pd.DataFrame( + { + "x": [4, 5, 6, 7], + "p": [2, 2, 3, 3], + "v1": [40, 50, 60, 70], + "i1": [40, 50, 60, 70], + } + ) + }, + ), + pd.DataFrame( + data={ + "x": [0, 1, 2, 3, 4, 5, 6, 7], + "p": [0, 0, 1, 1, 2, 2, 3, 3], + "v1": [0, 1, 2, 3, 40, 50, 60, 70], + "i1": [0, 1, 2, 3, 40, 50, 60, 70], + "v2": [0, 1, 20, np.nan, np.nan, np.nan, np.nan, np.nan], + "i2": [0, 1, 20, np.nan, np.nan, np.nan, np.nan, np.nan], + }, + columns=["i1", "i2", "p", "v1", "v2", "x"], + ), + ), +] + + +def test_overlay_tricky(driver, function_store): + cube = Cube( + dimension_columns=["x", "y"], + partition_columns=["p", "q"], + uuid_prefix="time_travel_cube_tricky", + seed_dataset="source", + ) + + build_cube( + data={ + cube.seed_dataset: pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": 1, + } + ), + "no_part": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "v2": 1, + } + ), + "q": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v3": 1, + } + ), + "a": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "a": [0, 1, 0, 0, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1, 0, 1], + "v4": 1, + } + ), + }, + cube=cube, + store=function_store, + partition_on={"no_part": [], "q": ["q"], "a": ["a"]}, + ) + append_to_cube( + data={ + cube.seed_dataset: pd.DataFrame( + { + "x": [0, 1, 0, 1, 2, 3, 2, 3], + "y": [2, 2, 3, 3, 0, 0, 1, 1], + "p": [1, 1, 1, 1, 0, 0, 0, 0], + "q": [0, 0, 0, 0, 1, 1, 1, 1], + "v1": 2, + } + ), + "no_part": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "v2": 2, + } + ), + "q": pd.DataFrame( + { + "x": [0, 1, 0, 1, 0, 1, 0, 1], + "y": [0, 0, 1, 1, 2, 2, 3, 3], + "q": [0, 0, 0, 0, 0, 0, 0, 0], + "v3": 2, + } + ), + "a": pd.DataFrame( + { + "x": [1, 0, 1, 2, 3, 2, 3, 3], + "y": [0, 2, 2, 1, 1, 2, 2, 3], + "a": [1, 1, 1, 1, 1, 1, 1, 1], + "v4": 2, + } + ), + }, + cube=cube, + store=function_store, + ) + + df_expected = ( + pd.DataFrame( + data={ + "x": [0, 1, 0, 1, 0, 1, 0, 1, 2, 3, 2, 3, 2, 3, 2, 3], + "y": [0, 0, 1, 1, 2, 2, 3, 3, 0, 0, 1, 1, 2, 2, 3, 3], + "p": [0, 0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1], + "q": [0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1], + "v1": [1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1], + "v2": [2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2], + "v3": [2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1], + "v4": [1, 2, 1, 1, 2, 2, 1, 1, 1, 1, 2, 2, 2, 2, 1, 2], + "a": [0, 1, 0, 0, 1, 1, 0, 0, 0, 0, 1, 1, 1, 1, 0, 1], + }, + columns=["a", "p", "q", "v1", "v2", "v3", "v4", "x", "y"], + ) + .sort_values(["x", "y", "p", "q"]) + .reset_index(drop=True) + ) + + result = driver(cube=cube, store=function_store) + assert len(result) == 1 + df_actual = result[0] + pdt.assert_frame_equal(df_actual, df_expected) + + +cond_types_simple = [ + EqualityCondition, + LessEqualCondition, + LessThanCondition, + GreaterEqualCondition, + GreaterThanCondition, + InequalityCondition, +] + +cond_types_all = cond_types_simple + [IsInCondition, InIntervalCondition] # type:ignore + + +def _tuple_to_condition(t): + col, cond_type, v1, v2, vset = t + if issubclass(cond_type, tuple(cond_types_simple)): + return cond_type(col, v1) + elif cond_type == IsInCondition: + return cond_type(col, vset) + elif cond_type == InIntervalCondition: + return cond_type(col, v1, v2) + raise ValueError("Unknown condition type {}".format(cond_type)) + + +st_columns = st.sampled_from( + ["x", "y", "z", "p", "q", "i1", "i2", "i3", "v1", "v2", "v3"] +) +st_values = st.integers(min_value=-1, max_value=17) +st_cond_types = st.sampled_from(cond_types_all) +st_conditions = st.tuples( + st_columns, st_cond_types, st_values, st_values, st.sets(st_values) +).map(_tuple_to_condition) + + +@given( + conditions=st.lists(st_conditions).map(Conjunction), + dimension_columns=st.permutations(["x", "y", "z"]), + payload_columns=st.sets(st_columns), +) +@settings(deadline=timedelta(seconds=5)) +def test_hypothesis( + driver, + driver_name, + module_store, + test_cube, + test_df, + dimension_columns, + payload_columns, + conditions, +): + if driver_name != "eager": + pytest.skip("only eager is fast enough") + + result = driver( + cube=test_cube, + store=module_store, + dimension_columns=dimension_columns, + payload_columns=payload_columns, + conditions=conditions, + ) + + df_expected = ( + apply_condition_unsafe(test_df, conditions) + .sort_values(dimension_columns + list(test_cube.partition_columns)) + .loc[:, sorted({"x", "y", "z", "p", "q"} | payload_columns)] + .reset_index(drop=True) + ) + + if df_expected.empty: + assert len(result) == 0 + else: + assert len(result) == 1 + df_actual = result[0] + pdt.assert_frame_equal(df_actual, df_expected) diff --git a/tests/io/cube/test_stats.py b/tests/io/cube/test_stats.py index 218f36a9..6b799cbf 100644 --- a/tests/io/cube/test_stats.py +++ b/tests/io/cube/test_stats.py @@ -1,10 +1,12 @@ # -*- coding: utf-8 -*- +import dask.bag as db +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_stats -from kartothek.io.dask.bag_cube import collect_stats_bag -from kartothek.io.eager_cube import collect_stats -from kartothek.io.testing.stats_cube import * # noqa +from kartothek.core.cube.cube import Cube +from kartothek.io.dask.bag_cube import build_cube_from_bag, collect_stats_bag +from kartothek.io.eager_cube import build_cube, collect_stats @pytest.fixture @@ -19,3 +21,83 @@ def driver(driver_name): return collect_stats else: raise ValueError("Unknown driver: {}".format(driver_name)) + + +def test_simple(driver, function_store, function_store_rwro): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + df_enrich = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "foo": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube( + data={cube.seed_dataset: df_seed, "enrich": df_enrich}, + cube=cube, + store=function_store, + ) + result = driver(cube=cube, store=function_store_rwro) + + assert set(result.keys()) == {cube.seed_dataset, "enrich"} + stats_seed = result[cube.seed_dataset] + + assert stats_seed["partitions"] == 2 + assert stats_seed["files"] == 2 + assert stats_seed["rows"] == 4 + assert stats_seed["blobsize"] > 0 + + stats_enrich = result["enrich"] + assert stats_enrich["partitions"] == stats_seed["partitions"] + assert stats_enrich["files"] == stats_seed["files"] + assert stats_enrich["rows"] == stats_seed["rows"] + assert stats_enrich["blobsize"] != stats_seed["blobsize"] + + +def test_multifile(driver, function_store): + dfs = [pd.DataFrame({"x": [i], "p": [0], "v1": [10]}) for i in range(2)] + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube_from_bag( + data=db.from_sequence(dfs, partition_size=1), cube=cube, store=function_store + ).compute() + + result = driver(cube=cube, store=function_store) + + assert set(result.keys()) == {cube.seed_dataset} + stats_seed = result[cube.seed_dataset] + assert stats_seed["partitions"] == 1 + assert stats_seed["files"] == 2 + assert stats_seed["rows"] == 2 + assert stats_seed["blobsize"] > 0 + + +def test_fail_no_store_factory(driver, function_store, skip_eager): + df_seed = pd.DataFrame( + {"x": [0, 1, 2, 3], "p": [0, 0, 1, 1], "v1": [10, 11, 12, 13]} + ) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + build_cube(data=df_seed, cube=cube, store=function_store) + store = function_store() + with pytest.raises(TypeError) as exc: + driver(cube=cube, store=store, no_run=True) + assert str(exc.value) == "store must be a factory but is HFilesystemStore" + + +def test_fail_blocksize_wrong_type(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(TypeError, match="blocksize must be an integer but is str"): + driver(cube=cube, store=function_store, blocksize="foo") + + +def test_fail_blocksize_negative(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is -1"): + driver(cube=cube, store=function_store, blocksize=-1) + + +def test_fail_blocksize_zero(driver, function_store, skip_eager): + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="cube") + + with pytest.raises(ValueError, match="blocksize must be > 0 but is 0"): + driver(cube=cube, store=function_store, blocksize=0) diff --git a/tests/io/cube/test_update.py b/tests/io/cube/test_update.py index 10bcb790..940e95a8 100644 --- a/tests/io/cube/test_update.py +++ b/tests/io/cube/test_update.py @@ -1,8 +1,56 @@ +from typing import Tuple + +import numpy as np +import pandas as pd import pytest from tests.io.cube.utils import wrap_bag_write +from kartothek.core.cube.conditions import C +from kartothek.core.cube.cube import Cube +from kartothek.core.dataset import DatasetMetadata from kartothek.io.dask.bag_cube import update_cube_from_bag -from kartothek.io.testing.update_cube import * # noqa +from kartothek.io.eager import read_table +from kartothek.io.eager_cube import build_cube, extend_cube, query_cube +from kartothek.serialization._parquet import ParquetSerializer + +from .utils import assert_num_row_groups + + +def _write_cube(function_store) -> Tuple[pd.DataFrame, Cube]: + """ + Write a cube with dimension column "x" and partition column "p" + + returns the 'source' and 'enrich' dataframes and the cube specification. + """ + df_source = pd.DataFrame( + { + "i1": [10, 11, 12, 13], + "p": [0, 0, 1, 1], + "v1": [10, 11, 12, 13], + "x": [0, 1, 2, 3], + } + ) + cube = Cube( + dimension_columns=["x"], + partition_columns=["p"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["i1", "i2", "i3"], + ) + build_cube( + data={"source": df_source}, + cube=cube, + store=function_store, + metadata={"source": {"meta_at_create": "data"}}, + ) + return df_source, cube + + +def _extend_cube(cube, function_store) -> pd.DataFrame: + # extend the existing cube by a dataset 'ex' with columns a = x + 1000 + df = pd.DataFrame({"a": [1000, 1001], "p": [0, 1], "x": [0, 2]}) + extend_cube({"ex": df}, cube, function_store) + return df @pytest.fixture @@ -13,3 +61,320 @@ def driver(driver_name): return wrap_bag_write(update_cube_from_bag, blocksize=3) else: pytest.skip() + + +@pytest.mark.parametrize( + "remove_partitions,new_partitions", + [ + # only append: + ([], [4, 5]), + # do nothing: + ([], []), + # partial overwrite with new data for p=0 + ([0], [0, 1, 4]), + # explicitly remove p=0 without overwriting it + ([0], [1, 4]), + # overwrite all: + ([0, 1], [0, 1]), + ], +) +def test_update_partitions(driver, function_store, remove_partitions, new_partitions): + df_source, cube = _write_cube(function_store) + + df_source_new = pd.DataFrame( + { + "i1": range(200, 200 + len(new_partitions)), + "p": np.array(new_partitions, np.int64), + "v1": range(300, 300 + len(new_partitions)), + "x": range(100, 100 + len(new_partitions)), + } + ) + + # what should remain of the old data: + df_source_of_old = df_source.loc[~df_source["p"].isin(set(remove_partitions))] + df_source_expected_after = pd.concat( + [df_source_of_old, df_source_new], sort=False, ignore_index=True + ) + + remove_conditions = C("p").isin(remove_partitions) + + result = driver( + data={"source": df_source_new}, + remove_conditions=remove_conditions, + cube=cube, + store=function_store, + ktk_cube_dataset_ids={"source"}, + metadata={"source": {"some_new_meta": 42}}, + ) + + assert set(result.keys()) == {"source"} + + dm_source_after = DatasetMetadata.load_from_store( + cube.ktk_dataset_uuid("source"), function_store(), load_all_indices=True + ) + + assert "some_new_meta" in dm_source_after.metadata + assert "meta_at_create" in dm_source_after.metadata + + # check values for "p" are as expected: + expected_p_source = (set(df_source["p"].unique()) - set(remove_partitions)) | set( + new_partitions + ) + assert set(dm_source_after.indices["p"].index_dct) == expected_p_source + + df_read = query_cube(cube, function_store)[0] + + assert set(df_read.columns) == set(df_source_expected_after.columns) + + for df in (df_read, df_source_expected_after): + df.sort_values("x", inplace=True) + df.reset_index(drop=True, inplace=True) + + pd.testing.assert_frame_equal(df_read, df_source_expected_after) + + +@pytest.mark.parametrize("chunk_size_build", [None, 2]) +@pytest.mark.parametrize("chunk_size_update", [None, 2]) +def test_rowgroups_are_applied_when_df_serializer_is_passed_to_update_cube( + driver, function_store, chunk_size_build, chunk_size_update +): + """ + Test that the dataset is split into row groups depending on the chunk size + + Partitions build with ``chunk_size=None`` should keep a single row group if they + are not touched by the update. Partitions that are newly created or replaced with + ``chunk_size>0`` should be split into row groups accordingly. + """ + # Build cube + df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size_build), + ) + + # Update cube - replace p=1 and append p=2 partitions + df_update = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], + ) + result = driver( + data={"seed": df_update}, + remove_conditions=(C("p") == 1), # Remove p=1 partition + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(chunk_size=chunk_size_update), + ) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 2, 2: 2} + part_chunk_size = { + 0: chunk_size_build, + 1: chunk_size_update, + 2: chunk_size_update, + } + + assert len(dataset.partitions) == 3 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_single_rowgroup_when_df_serializer_is_not_passed_to_update_cube( + driver, function_store +): + """ + Test that the dataset has a single row group as default path + """ + # Build cube + df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, cube=cube, store=function_store, + ) + + # Update cube - replace p=1 and append p=2 partitions + df_update = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], + ) + result = driver( + data={"seed": df_update}, + remove_conditions=(C("p") == 1), # Remove p=1 partition + cube=cube, + store=function_store, + ) + dataset = result["seed"].load_all_indices(function_store()) + + part_num_rows = {0: 1, 1: 2, 2: 2} + part_chunk_size = {0: None, 1: None, 2: None} + + assert len(dataset.partitions) == 3 + assert_num_row_groups(function_store(), dataset, part_num_rows, part_chunk_size) + + +def test_compression_is_compatible_on_update_cube(driver, function_store): + """ + Test that partitons written with different compression algorithms are compatible + + The compression algorithms are not parametrized because their availability depends + on the arrow build. 'SNAPPY' and 'GZIP' are already assumed to be available in parts + of the code. A fully parametrized test would also increase runtime and test complexity + unnecessarily. + """ + # Build cube + df = pd.DataFrame(data={"x": [0, 1], "p": [0, 1]}, columns=["x", "p"],) + cube = Cube(dimension_columns=["x"], partition_columns=["p"], uuid_prefix="rg-cube") + build_cube( + data=df, + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="SNAPPY"), + ) + + # Update cube - replace p=1 and append p=2 partitions + df_update = pd.DataFrame( + data={"x": [0, 1, 2, 3], "p": [1, 1, 2, 2]}, columns=["x", "p"], + ) + result = driver( + data={"seed": df_update}, + remove_conditions=(C("p") == 1), # Remove p=1 partition + cube=cube, + store=function_store, + df_serializer=ParquetSerializer(compression="GZIP"), + ) + dataset = result["seed"].load_all_indices(function_store()) + + assert len(dataset.partitions) == 3 + + +@pytest.mark.parametrize( + "ktk_cube_dataset_ids", [{"source", "ex"}, {"source"}, {"ex"}, set()] +) +def test_update_respects_ktk_cube_dataset_ids( + driver, function_store, ktk_cube_dataset_ids +): + df_source, cube = _write_cube(function_store) + df_ex = _extend_cube(cube, function_store) + + remove_conditions = C("p") == 0 + + # This implicitly also tests that `data={}` behaves as expected and still deletes partitions + # as requested via ktk_cube_dataset_ids and remove_conditions + result = driver( + data={}, + remove_conditions=remove_conditions, + cube=cube, + store=function_store, + ktk_cube_dataset_ids=ktk_cube_dataset_ids, + ) + assert set(result) == ktk_cube_dataset_ids + df_read = query_cube(cube, function_store)[0] + + # expected result: df_source left joined with df_ex; choosing the subset of p!=0 from each + # that is in `ktk_cube_dataset_ids`: + if "source" in ktk_cube_dataset_ids: + df_source = df_source.loc[df_source["p"] != 0] + if "ex" in ktk_cube_dataset_ids: + df_ex = df_ex.loc[df_ex["p"] != 0] + df_expected = df_source.merge(df_ex[["x", "a"]], how="left", on="x") + df_expected = df_expected[sorted(df_expected.columns)] + pd.testing.assert_frame_equal(df_read, df_expected) + + # test "ex" separately, because the test above based on the *left* merge does not tell us much about + # "ex" in case the partitions were removed from "source" + df_ex_read = read_table(cube.ktk_dataset_uuid("ex"), function_store) + if "ex" in ktk_cube_dataset_ids: + assert set(df_ex_read["p"]) == {1} + else: + assert set(df_ex_read["p"]) == {0, 1} + + +def test_cube_update_secondary_indices_subset(function_store, driver): + + cube1 = Cube( + dimension_columns=["A"], + partition_columns=["P"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["indexed"], + ) + df_1 = pd.DataFrame({"A": range(10), "P": 1, "indexed": 1, "not-indexed": 1}) + build_cube( + data={"source": df_1}, + cube=cube1, + store=function_store, + metadata={"source": {"meta_at_create": "data"}}, + ) + + cube2 = Cube( + dimension_columns=["A"], + partition_columns=["P"], + uuid_prefix="cube", + seed_dataset="source", + ) + df_2 = pd.DataFrame({"A": range(10, 20), "P": 1, "indexed": 2, "not-indexed": 1}) + driver( + data={"source": df_2}, cube=cube2, store=function_store, remove_conditions=None + ) + + dataset_uuid = cube2.ktk_dataset_uuid(cube2.seed_dataset) + dm = DatasetMetadata.load_from_store( + dataset_uuid, function_store(), load_all_indices=True + ) + obs_values = dm.indices["indexed"].observed_values() + + assert sorted(obs_values) == [1, 2] + + cube2 = Cube( + dimension_columns=["A"], + partition_columns=["P"], + uuid_prefix="cube", + seed_dataset="source", + index_columns=["not-indexed"], + ) + with pytest.raises( + ValueError, + match='ExplicitSecondaryIndex or PartitionIndex "not-indexed" is missing in dataset', + ): + driver( + data={"source": df_2}, + cube=cube2, + store=function_store, + remove_conditions=None, + ) + + +def test_cube_blacklist_dimension_index(function_store, driver): + + cube1 = Cube( + dimension_columns=["A", "B"], + partition_columns=["P"], + uuid_prefix="cube", + seed_dataset="source", + ) + df_1 = pd.DataFrame({"A": range(10), "P": 1, "B": 1, "payload": ""}) + build_cube( + data={"source": df_1}, + cube=cube1, + store=function_store, + metadata={"source": {"meta_at_create": "data"}}, + ) + + cube2 = Cube( + dimension_columns=["A", "B"], + partition_columns=["P"], + uuid_prefix="cube", + seed_dataset="source", + suppress_index_on=["B"], + ) + df_2 = pd.DataFrame({"A": range(10), "P": 1, "B": 2, "payload": ""}) + driver( + data={"source": df_2}, cube=cube2, store=function_store, remove_conditions=None + ) + + dataset_uuid = cube2.ktk_dataset_uuid(cube2.seed_dataset) + dm = DatasetMetadata.load_from_store( + dataset_uuid, function_store(), load_all_indices=True + ) + obs_values = dm.indices["B"].observed_values() + + assert sorted(obs_values) == [1, 2] diff --git a/tests/io/cube/utils.py b/tests/io/cube/utils.py index 9f875f07..40d3a4c4 100644 --- a/tests/io/cube/utils.py +++ b/tests/io/cube/utils.py @@ -1,9 +1,11 @@ +import math import pickle from collections import defaultdict import dask import dask.bag as db import dask.dataframe as ddf +from pyarrow.parquet import ParquetFile def pickle_roundtrip(obj): @@ -161,3 +163,21 @@ def _driver(no_run=False, blocksize=None, **kwargs): b.compute() return _driver + + +def assert_num_row_groups(store, dataset, part_num_rows, part_chunk_size): + """ + Assert that the row groups of each partition match the expectation based on the + number of rows and the chunk size + """ + # Iterate over the partitions of each index value + for index, partitions in dataset.indices["p"].index_dct.items(): + for part_key in partitions: + key = dataset.partitions[part_key].files["table"] + parquet_file = ParquetFile(store.open(key)) + if part_chunk_size[index] is None: + assert parquet_file.num_row_groups == 1 + else: + assert parquet_file.num_row_groups == math.ceil( + part_num_rows[index] / part_chunk_size[index] + ) diff --git a/tests/io/dask/bag/test_index.py b/tests/io/dask/bag/test_index.py deleted file mode 100644 index 747235a7..00000000 --- a/tests/io/dask/bag/test_index.py +++ /dev/null @@ -1,22 +0,0 @@ -# -*- coding: utf-8 -*- -import pickle - -import pytest - -from kartothek.io.dask.bag import build_dataset_indices__bag -from kartothek.io.testing.index import * # noqa: F4 - - -def _build_indices(*args, **kwargs): - bag = build_dataset_indices__bag(*args, **kwargs) - - # pickle roundtrip to ensure we don't need the inefficient cloudpickle fallback - s = pickle.dumps(bag, pickle.HIGHEST_PROTOCOL) - bag = pickle.loads(s) - - bag.compute() - - -@pytest.fixture() -def bound_build_dataset_indices(): - return _build_indices diff --git a/tests/io/dask/bag/test_read.py b/tests/io/dask/bag/test_read.py index 9083e2a6..3fac30bd 100644 --- a/tests/io/dask/bag/test_read.py +++ b/tests/io/dask/bag/test_read.py @@ -9,7 +9,6 @@ read_dataset_as_metapartitions_bag, ) from kartothek.io.iter import store_dataframes_as_dataset__iter -from kartothek.io.testing.read import * # noqa @pytest.fixture(params=["dataframe"]) diff --git a/tests/io/dask/bag/test_write.py b/tests/io/dask/bag/test_write.py deleted file mode 100644 index 8a99bd94..00000000 --- a/tests/io/dask/bag/test_write.py +++ /dev/null @@ -1,19 +0,0 @@ -import pickle - -import dask.bag as db -import pytest - -from kartothek.io.dask.bag import store_bag_as_dataset -from kartothek.io.testing.write import * # noqa - - -def _store_dataframes(df_list, *args, **kwargs): - bag = store_bag_as_dataset(db.from_sequence(df_list), *args, **kwargs) - s = pickle.dumps(bag, pickle.HIGHEST_PROTOCOL) - bag = pickle.loads(s) - return bag.compute() - - -@pytest.fixture() -def bound_store_dataframes(request): - return _store_dataframes diff --git a/tests/io/dask/dataframe/test_read.py b/tests/io/dask/dataframe/test_read.py index 176e4b7c..38bbc2b3 100644 --- a/tests/io/dask/dataframe/test_read.py +++ b/tests/io/dask/dataframe/test_read.py @@ -9,11 +9,11 @@ from dask.dataframe.utils import assert_eq as assert_dask_eq from pandas import testing as pdt from pandas.testing import assert_frame_equal +from tests.io.common.test_read import test_read_dataset_as_dataframes from kartothek.core.testing import get_dataframe_not_nested from kartothek.io.dask.dataframe import read_dataset_as_ddf from kartothek.io.eager import store_dataframes_as_dataset -from kartothek.io.testing.read import * # noqa from kartothek.io_components.metapartition import SINGLE_TABLE @@ -76,7 +76,7 @@ def bound_load_dataframes(): def test_load_dataframe_categoricals_with_index(dataset_with_index_factory): func = partial(_read_as_ddf, dataset_has_index=True) - test_read_dataset_as_dataframes( # noqa: F405 + test_read_dataset_as_dataframes( dataset_factory=dataset_with_index_factory, dataset=dataset_with_index_factory, store_session_factory=dataset_with_index_factory.store_factory, diff --git a/tests/io/dask/dataframe/test_update.py b/tests/io/dask/dataframe/test_update.py index 712f8d0d..48cb418f 100644 --- a/tests/io/dask/dataframe/test_update.py +++ b/tests/io/dask/dataframe/test_update.py @@ -1,49 +1,13 @@ -import pickle - import dask -import dask.dataframe as dd -import pandas as pd import pytest +from tests.io.common.conftest import update_dataset_dataframe from kartothek.io.dask.dataframe import update_dataset_from_ddf -from kartothek.io.testing.update import * # noqa @pytest.fixture def bound_update_dataset(): - return _update_dataset - - -def _id(part): - if isinstance(part, pd.DataFrame): - return part - else: - return part[0] - - -def _update_dataset(partitions, *args, **kwargs): - # TODO: Simplify once parse_input_to_metapartition is removed / obsolete - - if isinstance(partitions, pd.DataFrame): - partitions = dd.from_pandas(partitions, npartitions=1) - elif partitions is not None: - delayed_partitions = [dask.delayed(_id)(part) for part in partitions] - partitions = dd.from_delayed(delayed_partitions) - else: - partitions = None - - # Replace `table_name` with `table` keyword argument to enable shared test code - # via `bound_update_dataset` fixture - if "table_name" in kwargs: - kwargs["table"] = kwargs["table_name"] - del kwargs["table_name"] - - ddf = update_dataset_from_ddf(partitions, *args, **kwargs) - - s = pickle.dumps(ddf, pickle.HIGHEST_PROTOCOL) - ddf = pickle.loads(s) - - return ddf.compute() + return update_dataset_dataframe def _return_none(): diff --git a/tests/io/dask/delayed/test_delete.py b/tests/io/dask/delayed/test_delete.py deleted file mode 100644 index e31c2212..00000000 --- a/tests/io/dask/delayed/test_delete.py +++ /dev/null @@ -1,19 +0,0 @@ -import pickle - -import dask -import pytest - -from kartothek.io.dask.delayed import delete_dataset__delayed -from kartothek.io.testing.delete import * # noqa - - -def _delete(*args, **kwargs): - tasks = delete_dataset__delayed(*args, **kwargs) - s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) - tasks = pickle.loads(s) - dask.compute(tasks) - - -@pytest.fixture -def bound_delete_dataset(): - return _delete diff --git a/tests/io/dask/delayed/test_gc.py b/tests/io/dask/delayed/test_gc.py deleted file mode 100644 index 02c23fc5..00000000 --- a/tests/io/dask/delayed/test_gc.py +++ /dev/null @@ -1,19 +0,0 @@ -import pickle - -import dask -import pytest - -from kartothek.io.dask.delayed import garbage_collect_dataset__delayed -from kartothek.io.testing.gc import * # noqa - - -def _run_garbage_collect(*args, **kwargs): - tasks = garbage_collect_dataset__delayed(*args, **kwargs) - s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) - tasks = pickle.loads(s) - dask.compute(tasks) - - -@pytest.fixture() -def garbage_collect_callable(): - return _run_garbage_collect diff --git a/tests/io/dask/delayed/test_read.py b/tests/io/dask/delayed/test_read.py deleted file mode 100644 index 3ead1b15..00000000 --- a/tests/io/dask/delayed/test_read.py +++ /dev/null @@ -1,31 +0,0 @@ -import pickle -from functools import partial - -import pytest - -from kartothek.io.dask.delayed import read_dataset_as_delayed -from kartothek.io.testing.read import * # noqa - - -@pytest.fixture(params=["table"]) -def output_type(request): - return request.param - - -def _load_dataframes(output_type, *args, **kwargs): - if "tables" in kwargs: - param_tables = kwargs.pop("tables") - kwargs["table"] = param_tables - func = partial(read_dataset_as_delayed) - tasks = func(*args, **kwargs) - - s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) - tasks = pickle.loads(s) - - result = [task.compute() for task in tasks] - return result - - -@pytest.fixture() -def bound_load_dataframes(output_type): - return partial(_load_dataframes, output_type) diff --git a/tests/io/dask/delayed/test_update.py b/tests/io/dask/delayed/test_update.py deleted file mode 100644 index ba4ba7c9..00000000 --- a/tests/io/dask/delayed/test_update.py +++ /dev/null @@ -1,28 +0,0 @@ -import pickle - -import dask -import pytest - -from kartothek.io.dask.delayed import update_dataset_from_delayed -from kartothek.io.testing.update import * # noqa - - -@pytest.fixture -def bound_update_dataset(): - return _update_dataset - - -@dask.delayed -def _unwrap_partition(part): - return next(iter(dict(part["data"]).values())) - - -def _update_dataset(partitions, *args, **kwargs): - if not isinstance(partitions, list): - partitions = [partitions] - tasks = update_dataset_from_delayed(partitions, *args, **kwargs) - - s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) - tasks = pickle.loads(s) - - return tasks.compute() diff --git a/tests/io/dask/delayed/test_write.py b/tests/io/dask/delayed/test_write.py deleted file mode 100644 index 8c41c909..00000000 --- a/tests/io/dask/delayed/test_write.py +++ /dev/null @@ -1,33 +0,0 @@ -import pickle -from functools import partial - -import dask.bag as db -import pytest - -from kartothek.io.dask.bag import store_bag_as_dataset -from kartothek.io.dask.delayed import store_delayed_as_dataset -from kartothek.io.testing.write import * # noqa - - -def _store_dataframes(execution_mode, df_list, *args, **kwargs): - if execution_mode == "dask.bag": - bag = store_bag_as_dataset(db.from_sequence(df_list), *args, **kwargs) - - s = pickle.dumps(bag, pickle.HIGHEST_PROTOCOL) - bag = pickle.loads(s) - - return bag.compute() - elif execution_mode == "dask.delayed": - tasks = store_delayed_as_dataset(df_list, *args, **kwargs) - - s = pickle.dumps(tasks, pickle.HIGHEST_PROTOCOL) - tasks = pickle.loads(s) - - return tasks.compute() - else: - raise ValueError("Unknown execution mode: {}".format(execution_mode)) - - -@pytest.fixture(params=["dask.delayed", "dask.bag"]) -def bound_store_dataframes(request): - return partial(_store_dataframes, request.param) diff --git a/tests/io/eager/test_copy.py b/tests/io/eager/test_copy.py new file mode 100644 index 00000000..53be64fd --- /dev/null +++ b/tests/io/eager/test_copy.py @@ -0,0 +1,144 @@ +import pytest + +from kartothek.core.factory import DatasetFactory +from kartothek.core.utils import lazy_store +from kartothek.io.eager import copy_dataset, read_table, store_dataframes_as_dataset +from kartothek.serialization.testing import get_dataframe_not_nested +from kartothek.utils.ktk_adapters import get_dataset_keys + +SRC_DS_UUID = "test_copy_ds_with_index" +TGT_DS_UUID = "copy_target" + + +@pytest.fixture +def dataset_to_copy(store): + df = get_dataframe_not_nested(10) + store_dataframes_as_dataset( + dfs=[df], + dataset_uuid=SRC_DS_UUID, + store=store, + partition_on=[df.columns[0]], + secondary_indices=[df.columns[1]], + ) + + +def assert_target_ktk_readable(tgt_store, tgt_ds): + """ + Try to read the target dataset using high level KTK functionality + """ + df_result = read_table(store=tgt_store, dataset_uuid=tgt_ds,) + assert df_result is not None + assert len(df_result) == 10 + df_result = read_table( + store=tgt_store, dataset_uuid=tgt_ds, predicates=[[("bool", "==", True)]] + ) + assert len(df_result) == 5 + df_result = read_table( + store=tgt_store, dataset_uuid=tgt_ds, predicates=[[("bytes", "==", b"2")]] + ) + assert len(df_result) == 1 + + +def assert_target_keys(src_store, src_uuid, tgt_store, tgt_uuid): + """ + Check that the expected keys exist in the target data set, and the corresponding + values are equal to the source data set (or modified as expected) + """ + df_source = DatasetFactory( + dataset_uuid=src_uuid, store_factory=lazy_store(src_store), + ) + src_keys = get_dataset_keys(df_source.dataset_metadata) + df_target = DatasetFactory( + dataset_uuid=tgt_uuid, store_factory=lazy_store(tgt_store), + ) + tgt_keys = get_dataset_keys(df_target.dataset_metadata) + + for src_key in src_keys: + # check for each source key if the corresponding target key exists + tgt_key = src_key.replace(src_uuid, tgt_uuid) + assert tgt_key in tgt_keys + + # check if the files for source and target key are equal (exception: + # metadata => here the target must contain the modified metadata) + b1 = src_store.get(src_key) + b2 = tgt_store.get(tgt_key) + + if tgt_key.endswith("by-dataset-metadata.json"): + b1_mod = b1.decode("utf-8").replace(src_uuid, tgt_uuid).encode("utf-8") + assert b1_mod == b2 + else: + assert b1 == b2 + + +def test_copy_rename_eager_same_store(dataset_to_copy, store): + """ + Copies and renames DS within one store + """ + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=TGT_DS_UUID, + store=store, + target_store=store, + ) + assert_target_keys(store, SRC_DS_UUID, store, TGT_DS_UUID) + assert_target_ktk_readable(store, TGT_DS_UUID) + + +def test_copy_eager_with_rename_different_store(dataset_to_copy, store, store2): + """ + Copies and renames DS between stores + """ + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=TGT_DS_UUID, + store=store, + target_store=store2, + ) + assert_target_keys(store, SRC_DS_UUID, store2, TGT_DS_UUID) + assert_target_ktk_readable(store2, TGT_DS_UUID) + + +def test_copy_eager_no_target_uuid(dataset_to_copy, store, store2): + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=None, + store=store, + target_store=store2, + ) + assert_target_keys(store, SRC_DS_UUID, store2, SRC_DS_UUID) + assert_target_ktk_readable(store2, SRC_DS_UUID) + + +def test_copy_eager_no_target_store(dataset_to_copy, store, store2): + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=TGT_DS_UUID, + store=store, + target_store=None, + ) + assert_target_keys(store, SRC_DS_UUID, store, TGT_DS_UUID) + assert_target_ktk_readable(store, TGT_DS_UUID) + + +def test_copy_eager_without_rename_different_store(dataset_to_copy, store, store2): + """ + Copies DS between stores while keeping the name + """ + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=SRC_DS_UUID, + store=store, + target_store=store2, + ) + assert_target_keys(store, SRC_DS_UUID, store2, SRC_DS_UUID) + assert_target_ktk_readable(store2, SRC_DS_UUID) + + +def test_copy_same_source_and_target(dataset_to_copy, store): + with pytest.raises(ValueError): + copy_dataset( + source_dataset_uuid=SRC_DS_UUID, + target_dataset_uuid=SRC_DS_UUID, + store=store, + target_store=store, + ) diff --git a/tests/io/eager/test_delete.py b/tests/io/eager/test_delete.py deleted file mode 100644 index a2199ac2..00000000 --- a/tests/io/eager/test_delete.py +++ /dev/null @@ -1,20 +0,0 @@ -import pytest - -from kartothek.io.eager import delete_dataset -from kartothek.io.testing.delete import * # noqa - - -def _delete_store_factory(dataset_uuid, store_factory): - delete_dataset(dataset_uuid, store_factory) - - -def _delete_store(dataset_uuid, store_factory): - delete_dataset(dataset_uuid, store_factory()) - - -@pytest.fixture(params=["factory", "store-factory"]) -def bound_delete_dataset(request): - if request.param == "factory": - return _delete_store_factory - else: - return _delete_store diff --git a/tests/io/eager/test_gc.py b/tests/io/eager/test_gc.py deleted file mode 100644 index 83ef4d1c..00000000 --- a/tests/io/eager/test_gc.py +++ /dev/null @@ -1,11 +0,0 @@ -# -*- coding: utf-8 -*- - -import pytest - -from kartothek.io.eager import garbage_collect_dataset -from kartothek.io.testing.gc import * # noqa: F4 - - -@pytest.fixture() -def garbage_collect_callable(): - return garbage_collect_dataset diff --git a/tests/io/eager/test_index.py b/tests/io/eager/test_index.py deleted file mode 100644 index dbc532fe..00000000 --- a/tests/io/eager/test_index.py +++ /dev/null @@ -1,11 +0,0 @@ -# -*- coding: utf-8 -*- - -import pytest - -from kartothek.io.eager import build_dataset_indices -from kartothek.io.testing.index import * # noqa: F4 - - -@pytest.fixture() -def bound_build_dataset_indices(): - return build_dataset_indices diff --git a/tests/io/eager/test_read.py b/tests/io/eager/test_read.py index 01278221..bffb1bfd 100644 --- a/tests/io/eager/test_read.py +++ b/tests/io/eager/test_read.py @@ -10,7 +10,6 @@ read_table, store_dataframes_as_dataset, ) -from kartothek.io.testing.read import * # noqa @pytest.fixture( diff --git a/tests/io/eager/test_update.py b/tests/io/eager/test_update.py index 77a68c6c..01b954b0 100644 --- a/tests/io/eager/test_update.py +++ b/tests/io/eager/test_update.py @@ -9,7 +9,6 @@ update_dataset_from_dataframes, write_single_partition, ) -from kartothek.io.testing.update import * # noqa: F40 @pytest.fixture() diff --git a/tests/io/eager/test_write.py b/tests/io/eager/test_write.py index faacb56d..1670b390 100644 --- a/tests/io/eager/test_write.py +++ b/tests/io/eager/test_write.py @@ -13,22 +13,14 @@ store_dataframes_as_dataset, write_single_partition, ) -from kartothek.io.testing.write import * # noqa: F40 from kartothek.io_components.metapartition import MetaPartition - -def _store_dataframes(dfs, **kwargs): - # Positional arguments in function but `None` is acceptable input - for kw in ("dataset_uuid", "store"): - if kw not in kwargs: - kwargs[kw] = None - - return store_dataframes_as_dataset(dfs=dfs, **kwargs) +from ..common.conftest import store_dataframes_eager @pytest.fixture() def bound_store_dataframes(): - return _store_dataframes + return store_dataframes_eager def test_write_single_partition(store_factory, mock_uuid, metadata_version): diff --git a/tests/io/iter/test_read.py b/tests/io/iter/test_read.py deleted file mode 100644 index ccf6f804..00000000 --- a/tests/io/iter/test_read.py +++ /dev/null @@ -1,36 +0,0 @@ -from functools import partial - -import pytest - -from kartothek.io.iter import ( - read_dataset_as_dataframes__iterator, - read_dataset_as_metapartitions__iterator, -) -from kartothek.io.testing.read import * # noqa - - -@pytest.fixture(params=["dataframe"]) -def output_type(request): - return request.param - - -def _load_dataframes(output_type, *args, **kwargs): - if output_type == "dataframe": - func = read_dataset_as_dataframes__iterator - else: - raise ValueError("Unknown output type {}".format(output_type)) - return list(func(*args, **kwargs)) - - -@pytest.fixture() -def bound_load_dataframes(output_type): - return partial(_load_dataframes, output_type) - - -def _load_metapartitions(*args, **kwargs): - return list(read_dataset_as_metapartitions__iterator(*args, **kwargs)) - - -@pytest.fixture() -def bound_load_metapartitions(): - return _load_metapartitions diff --git a/tests/io/iter/test_update.py b/tests/io/iter/test_update.py deleted file mode 100644 index 0f0e56ad..00000000 --- a/tests/io/iter/test_update.py +++ /dev/null @@ -1,19 +0,0 @@ -# -*- coding: utf-8 -*- - -import pandas as pd -import pytest - -from kartothek.io.iter import update_dataset_from_dataframes__iter -from kartothek.io.testing.update import * # noqa - - -@pytest.fixture() -def bound_update_dataset(): - return _update_dataset - - -def _update_dataset(df_list, *args, **kwargs): - if isinstance(df_list, pd.DataFrame): - df_list = [df_list] - df_generator = (x for x in df_list) - return update_dataset_from_dataframes__iter(df_generator, *args, **kwargs) diff --git a/tests/io/iter/test_write.py b/tests/io/iter/test_write.py deleted file mode 100644 index 71061737..00000000 --- a/tests/io/iter/test_write.py +++ /dev/null @@ -1,16 +0,0 @@ -# -*- coding: utf-8 -*- - -import pytest - -from kartothek.io.iter import store_dataframes_as_dataset__iter -from kartothek.io.testing.write import * # noqa - - -def _store_dataframes(df_list, *args, **kwargs): - df_generator = (x for x in df_list) - return store_dataframes_as_dataset__iter(df_generator, *args, **kwargs) - - -@pytest.fixture() -def bound_store_dataframes(): - return _store_dataframes