def cleanup_cube_bag(cube, store, blocksize=100): """ Remove unused keys from cube datasets. .. important:: All untracked keys which start with the cube's `uuid_prefix` followed by the `KTK_CUBE_UUID_SEPERATOR` (e.g. `my_cube_uuid++seed...`) will be deleted by this routine. These keys may be leftovers from past overwrites or index updates. Parameters ---------- cube: Cube Cube specification. store: Union[simplekv.KeyValueStore, Callable[[], simplekv.KeyValueStore]] KV store. blocksize: int Number of keys to delete at once. Returns ------- bag: dask.bag.Bag A dask bag that performs the given operation. May contain multiple partitions. """ check_store_factory(store) check_blocksize(blocksize) store_obj = store() datasets = discover_datasets_unchecked(uuid_prefix=cube.uuid_prefix, store=store) keys = get_keys_to_clean(cube.uuid_prefix, datasets, store_obj) return db.from_sequence( seq=sorted(keys), partition_size=blocksize).map_partitions(_delete, store=store)
def copy_cube_bag( cube, src_store: StoreFactory, tgt_store: StoreFactory, blocksize: int = 100, overwrite: bool = False, datasets: Optional[Union[Iterable[str], Dict[str, DatasetMetadata]]] = None, ): """ Copy cube 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. blocksize Number of keys to copy at once. datasets 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). Returns ------- bag: dask.bag.Bag A dask bag that performs the given operation. May contain multiple partitions. """ check_store_factory(src_store) check_store_factory(tgt_store) check_blocksize(blocksize) assert_stores_different( 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, ) return db.from_sequence(seq=sorted(keys), partition_size=blocksize).map_partitions( copy_keys, src_store=src_store, tgt_store=tgt_store )
def collect_stats_bag( cube: Cube, store: StoreFactory, datasets: Optional[Union[Iterable[str], Dict[str, DatasetMetadata]]] = None, blocksize: int = 100, ): """ Collect statistics for given cube. Parameters ---------- cube Cube specification. store KV store that preserves the cube. datasets Datasets to query, 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 auto-discovery will be used). blocksize Number of partitions to scan at once. Returns ------- bag: dask.bag.Bag A dask bag that returns a single result of the form ``Dict[str, Dict[str, int]]`` and contains statistics per ktk_cube dataset ID. """ check_store_factory(store) check_blocksize(blocksize) if not isinstance(datasets, dict): datasets = discover_datasets_unchecked( uuid_prefix=cube.uuid_prefix, store=store, filter_ktk_cube_dataset_ids=datasets, ) all_metapartitions = get_metapartitions_for_stats(datasets) return ( db.from_sequence(seq=all_metapartitions, partition_size=blocksize) .map_partitions(collect_stats_block, store=store) .reduction( perpartition=_obj_to_list, aggregate=_reduce_stats, split_every=False, out_type=db.Bag, ) )
def delete_cube_bag( cube: Cube, store: StoreFactory, blocksize: int = 100, datasets: Optional[Union[Iterable[str], Dict[str, DatasetMetadata]]] = None, ): """ Delete cube from store. .. important:: This routine only deletes tracked files. Garbage and leftovers from old cubes and failed operations are NOT removed. Parameters ---------- cube Cube specification. store KV store. blocksize Number of keys to delete at once. datasets Datasets to delete, 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 deleted). Returns ------- bag: dask.bag.Bag A dask bag that performs the given operation. May contain multiple partitions. """ check_store_factory(store) check_blocksize(blocksize) if not isinstance(datasets, dict): datasets = discover_datasets_unchecked( uuid_prefix=cube.uuid_prefix, store=store, filter_ktk_cube_dataset_ids=datasets, ) keys = set() for ktk_cube_dataset_id in sorted(datasets.keys()): ds = datasets[ktk_cube_dataset_id] keys |= get_dataset_keys(ds) return db.from_sequence(seq=sorted(keys), partition_size=blocksize).map_partitions( _delete, store=store )
def build_cube_from_bag_internal( data: db.Bag, cube: Cube, store: StoreFactory, ktk_cube_dataset_ids: Optional[Iterable[str]], metadata: Optional[Dict[str, Dict[str, Any]]], overwrite: bool, partition_on: Optional[Dict[str, Iterable[str]]], df_serializer: Optional[ParquetSerializer] = None, ) -> db.Bag: """ Create dask computation graph that builds a cube with the data supplied from a dask bag. Parameters ---------- data: dask.bag.Bag Bag containing dataframes cube: Cube specification. store: Store to which the data should be written to. ktk_cube_dataset_ids: Datasets that will be written, must be specified in advance. If left unprovided, it is assumed that only the seed dataset will be written. metadata: Metadata for every dataset. overwrite: If possibly existing datasets should be overwritten. partition_on: Optional parition-on attributes for datasets (dictionary mapping :term:`Dataset ID` -> columns). df_serializer: Optional Dataframe to Parquet serializer Returns ------- metadata_dict: dask.bag.Bag A dask bag object containing the compute graph to build a cube returning the dict of dataset metadata objects. The bag has a single partition with a single element. """ check_store_factory(store) if ktk_cube_dataset_ids is None: ktk_cube_dataset_ids = [cube.seed_dataset] else: ktk_cube_dataset_ids = sorted(ktk_cube_dataset_ids) metadata = check_provided_metadata_dict(metadata, ktk_cube_dataset_ids) existing_datasets = discover_datasets_unchecked(cube.uuid_prefix, store) check_datasets_prebuild(ktk_cube_dataset_ids, cube, existing_datasets) prep_partition_on = prepare_ktk_partition_on(cube, ktk_cube_dataset_ids, partition_on) cube = ensure_valid_cube_indices(existing_datasets, cube) data = (data.map(multiplex_user_input, cube=cube).map( _check_dataset_ids, ktk_cube_dataset_ids=ktk_cube_dataset_ids).map( _multiplex_prepare_data_for_ktk, cube=cube, existing_payload=set(), partition_on=prep_partition_on, )) data = _store_bag_as_dataset_parallel( bag=data, store=store, cube=cube, ktk_cube_dataset_ids=ktk_cube_dataset_ids, metadata={ ktk_cube_dataset_id: prepare_ktk_metadata(cube, ktk_cube_dataset_id, metadata) for ktk_cube_dataset_id in ktk_cube_dataset_ids }, overwrite=overwrite, update=False, existing_datasets=existing_datasets, df_serializer=df_serializer, ) data = data.map( apply_postwrite_checks, cube=cube, store=store, existing_datasets=existing_datasets, ) return data
def append_to_cube_from_bag_internal( data: db.Bag, cube: Cube, store: StoreFactory, ktk_cube_dataset_ids: Optional[Iterable[str]], metadata: Optional[Dict[str, Dict[str, Any]]], remove_conditions=None, df_serializer: Optional[ParquetSerializer] = None, ) -> db.Bag: """ Append data to existing cube. For details on ``data`` and ``metadata``, see :func:`~kartothek.io.eager_cube.build_cube`. .. important:: Physical partitions must be updated as a whole. If only single rows within a physical partition are updated, the old data is treated as "removed". Parameters ---------- data: dask.bag.Bag Bag containing dataframes cube: Cube specification. store: Store to which the data should be written to. ktk_cube_dataset_ids: Datasets that will be written, must be specified in advance. metadata: Metadata for every dataset, optional. For every dataset, only given keys are updated/replaced. Deletion of metadata keys is not possible. remove_conditions: Conditions that select which partitions to remove. df_serializer: Optional Dataframe to Parquet serializer Returns ------- metadata_dict: dask.bag.Bag A dask bag object containing the compute graph to append to the cube returning the dict of dataset metadata objects. The bag has a single partition with a single element. """ check_store_factory(store) if ktk_cube_dataset_ids: ktk_cube_dataset_ids = sorted(ktk_cube_dataset_ids) else: ktk_cube_dataset_ids = [] metadata = check_provided_metadata_dict(metadata, ktk_cube_dataset_ids) existing_datasets = discover_datasets(cube, store) cube = ensure_valid_cube_indices(existing_datasets, cube) # existing_payload is set to empty because we're not checking against any existing payload. ktk will account for the # compat check within 1 dataset existing_payload: Set[str] = set() partition_on = {k: v.partition_keys for k, v in existing_datasets.items()} check_existing_datasets(existing_datasets=existing_datasets, ktk_cube_dataset_ids=ktk_cube_dataset_ids) if remove_conditions is not None: remove_metapartitions = prepare_metapartitions_for_removal_action( cube, store, remove_conditions, ktk_cube_dataset_ids, existing_datasets) delete_scopes = { k: delete_scope for k, (_, _, delete_scope) in remove_metapartitions.items() } else: delete_scopes = {} data = (data.map(multiplex_user_input, cube=cube).map( _check_dataset_ids, ktk_cube_dataset_ids=ktk_cube_dataset_ids).map( _fill_dataset_ids, ktk_cube_dataset_ids=ktk_cube_dataset_ids).map( _multiplex_prepare_data_for_ktk, cube=cube, existing_payload=existing_payload, partition_on=partition_on, )) data = _store_bag_as_dataset_parallel( bag=data, store=store, cube=cube, ktk_cube_dataset_ids=ktk_cube_dataset_ids, metadata={ ktk_cube_dataset_id: prepare_ktk_metadata(cube, ktk_cube_dataset_id, metadata) for ktk_cube_dataset_id in ktk_cube_dataset_ids }, update=True, existing_datasets=existing_datasets, delete_scopes=delete_scopes, df_serializer=df_serializer, ) data = data.map( apply_postwrite_checks, cube=cube, store=store, existing_datasets=existing_datasets, ) return data
def query_cube_bag_internal( cube, store, conditions, datasets, dimension_columns, partition_by, payload_columns, blocksize, ): """ Query cube. For detailed documentation, see :func:`~kartothek.io.eager_cube.query_cube`. Parameters ---------- cube: Cube Cube specification. store: simplekv.KeyValueStore KV store that preserves the cube. conditions: Union[None, Condition, Iterable[Condition], Conjunction] Conditions that should be applied, optional. datasets: Union[None, Iterable[str], Dict[str, kartothek.core.dataset.DatasetMetadata]] Datasets to query, 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 auto-discovery will be used). dimension_columns: Union[None, str, Iterable[str]] Dimension columns of the query, may result in projection. If not provided, dimension columns from cube specification will be used. partition_by: Union[None, str, Iterable[str]] By which column logical partitions should be formed. If not provided, a single partition will be generated. payload_columns: Union[None, str, Iterable[str]] Which columns apart from ``dimension_columns`` and ``partition_by`` should be returned. blocksize: int Partition size of the bag. Returns ------- empty: pandas.DataFrame Empty DataFrame with correct dtypes and column order. bag: dask.bag.Bag Bag of 1-sized partitions of non-empty DataFrames, order by ``partition_by``. Column of DataFrames is alphabetically ordered. Data types are provided on best effort (they are restored based on the preserved data, but may be different due to Pandas NULL-handling, e.g. integer columns may be floats). """ check_store_factory(store) check_blocksize(blocksize) intention, empty, groups = plan_query( cube=cube, store=store, conditions=conditions, datasets=datasets, dimension_columns=dimension_columns, partition_by=partition_by, payload_columns=payload_columns, ) b = (db.from_sequence(seq=groups, partition_size=blocksize).map( load_group, store=store, cube=cube).filter(_not_empty)) if not intention.partition_by: b = (b.reduction( perpartition=list, aggregate=_collect_dfs, split_every=False, out_type=db.Bag, ).map( _quick_concat_or_none, dimension_columns=intention.dimension_columns, partition_columns=cube.partition_columns, ).filter(_not_none)) return empty, b
def extend_cube_from_bag_internal( data: db.Bag, cube: Cube, store: KeyValueStore, ktk_cube_dataset_ids: Optional[Iterable[str]], metadata: Optional[Dict[str, Dict[str, Any]]], overwrite: bool, partition_on: Optional[Dict[str, Iterable[str]]], df_serializer: Optional[ParquetSerializer] = None, ) -> db.Bag: """ Create dask computation graph that extends a cube by the data supplied from a dask bag. For details on ``data`` and ``metadata``, see :func:`~kartothek.io.eager_cube.build_cube`. Parameters ---------- data: dask.bag.Bag Bag containing dataframes (see :func:`~kartothek.io.eager_cube.build_cube` for possible format and types). cube: kartothek.core.cube.cube.Cube Cube specification. store: Store to which the data should be written to. ktk_cube_dataset_ids: Datasets that will be written, must be specified in advance. metadata: Metadata for every dataset. overwrite: If possibly existing datasets should be overwritten. partition_on: Optional parition-on attributes for datasets (dictionary mapping :term:`Dataset ID` -> columns). df_serializer: Optional Dataframe to Parquet serializer Returns ------- metadata_dict: dask.bag.Bag A dask bag object containing the compute graph to extend a cube returning the dict of dataset metadata objects. The bag has a single partition with a single element. """ check_store_factory(store) check_datasets_preextend(ktk_cube_dataset_ids, cube) if ktk_cube_dataset_ids: ktk_cube_dataset_ids = sorted(ktk_cube_dataset_ids) else: ktk_cube_dataset_ids = [] metadata = check_provided_metadata_dict(metadata, ktk_cube_dataset_ids) prep_partition_on = prepare_ktk_partition_on(cube, ktk_cube_dataset_ids, partition_on) existing_datasets = discover_datasets(cube, store) cube = ensure_valid_cube_indices(existing_datasets, cube) if overwrite: existing_datasets_cut = { ktk_cube_dataset_id: ds for ktk_cube_dataset_id, ds in existing_datasets.items() if ktk_cube_dataset_id not in ktk_cube_dataset_ids } else: existing_datasets_cut = existing_datasets existing_payload = get_cube_payload(existing_datasets_cut, cube) data = (data.map(multiplex_user_input, cube=cube).map( _check_dataset_ids, ktk_cube_dataset_ids=ktk_cube_dataset_ids).map( _multiplex_prepare_data_for_ktk, cube=cube, existing_payload=existing_payload, partition_on=prep_partition_on, )) data = _store_bag_as_dataset_parallel( bag=data, store=store, cube=cube, ktk_cube_dataset_ids=ktk_cube_dataset_ids, metadata={ ktk_cube_dataset_id: prepare_ktk_metadata(cube, ktk_cube_dataset_id, metadata) for ktk_cube_dataset_id in ktk_cube_dataset_ids }, overwrite=overwrite, update=False, existing_datasets=existing_datasets, df_serializer=df_serializer, ) data = data.map( apply_postwrite_checks, cube=cube, store=store, existing_datasets=existing_datasets, ) return data
def build_cube_from_dataframe( data: Union[dd.DataFrame, Dict[str, dd.DataFrame]], cube: Cube, store: StoreFactory, metadata: Optional[Dict[str, Dict[str, Any]]] = None, overwrite: bool = False, partition_on: Optional[Dict[str, Iterable[str]]] = None, shuffle: bool = False, num_buckets: int = 1, bucket_by: Optional[Iterable[str]] = None, df_serializer: Optional[ParquetSerializer] = None, ) -> Delayed: """ Create dask computation graph that builds a cube with the data supplied from a dask dataframe. Parameters ---------- data Data that should be written to the cube. If only a single dataframe is given, it is assumed to be the seed dataset. cube Cube specification. store Store to which the data should be written to. metadata Metadata for every dataset. overwrite If possibly existing datasets should be overwritten. partition_on Optional parition-on attributes for datasets (dictionary mapping :term:`Dataset ID` -> columns). df_serializer: Optional Dataframe to Parquet serializer Returns ------- metadata_dict: dask.delayed.Delayed A dask delayed object containing the compute graph to build a cube returning the dict of dataset metadata objects. """ check_store_factory(store) if not isinstance(data, dict): data = {cube.seed_dataset: data} ktk_cube_dataset_ids = sorted(data.keys()) metadata = check_provided_metadata_dict(metadata, ktk_cube_dataset_ids) existing_datasets = discover_datasets_unchecked(cube.uuid_prefix, store) check_datasets_prebuild(ktk_cube_dataset_ids, cube, existing_datasets) partition_on_checked = prepare_ktk_partition_on( cube, ktk_cube_dataset_ids, partition_on ) del partition_on dct = {} for table_name, ddf in data.items(): check_user_df(table_name, ddf, cube, set(), partition_on_checked[table_name]) indices_to_build = set(cube.index_columns) & set(ddf.columns) if table_name == cube.seed_dataset: indices_to_build |= set(cube.dimension_columns) - cube.suppress_index_on indices_to_build -= set(partition_on_checked[table_name]) ddf = ddf.map_partitions( assert_dimesion_index_cols_notnull, ktk_cube_dataset_id=table_name, cube=cube, partition_on=partition_on_checked[table_name], meta=ddf._meta, ) graph = store_dataset_from_ddf( ddf, dataset_uuid=cube.ktk_dataset_uuid(table_name), store=store, metadata=prepare_ktk_metadata(cube, table_name, metadata), partition_on=partition_on_checked[table_name], secondary_indices=sorted(indices_to_build), sort_partitions_by=sorted( (set(cube.dimension_columns) - set(cube.partition_columns)) & set(ddf.columns) ), overwrite=overwrite, shuffle=shuffle, num_buckets=num_buckets, bucket_by=bucket_by, df_serializer=df_serializer, ) dct[table_name] = graph return dask.delayed(apply_postwrite_checks)( dct, cube=cube, store=store, existing_datasets=existing_datasets )
def append_to_cube_from_bag_internal(data, cube, store, ktk_cube_dataset_ids, metadata): """ Append data to existing cube. For details on ``data`` and ``metadata``, see :meth:`build_cube`. .. important:: Physical partitions must be updated as a whole. If only single rows within a physical partition are updated, the old data is treated as "removed". .. hint:: To have better control over the overwrite "mask" (i.e. which partitions are overwritten), you should use :meth:`remove_partitions` beforehand. Parameters ---------- data: dask.Bag Bag containing dataframes cube: kartothek.core.cube.cube.Cube Cube specification. store: Callable[[], simplekv.KeyValueStore] Store to which the data should be written to. ktk_cube_dataset_ids: Optional[Iterable[str]] Datasets that will be written, must be specified in advance. metadata: Dict[str, Dict[str, Any]] Metadata for every dataset, optional. For every dataset, only given keys are updated/replaced. Deletion of metadata keys is not possible. Returns ------- metadata_dict: dask.bag.Bag A dask bag object containing the compute graph to append to the cube returning the dict of dataset metadata objects. The bag has a single partition with a single element. """ check_store_factory(store) ktk_cube_dataset_ids = sorted(ktk_cube_dataset_ids) metadata = check_provided_metadata_dict(metadata, ktk_cube_dataset_ids) existing_datasets = discover_datasets(cube, store) # existing_payload is set to empty because we're not checking against any existing payload. ktk will account for the # compat check within 1 dataset existing_payload = set() partition_on = {k: v.partition_keys for k, v in existing_datasets.items()} check_existing_datasets(existing_datasets=existing_datasets, ktk_cube_dataset_ids=ktk_cube_dataset_ids) data = (data.map(multiplex_user_input, cube=cube).map( _check_dataset_ids, ktk_cube_dataset_ids=ktk_cube_dataset_ids).map( _multiplex_prepare_data_for_ktk, cube=cube, existing_payload=existing_payload, partition_on=partition_on, )) data = _store_bag_as_dataset_parallel( bag=data, store=store, cube=cube, ktk_cube_dataset_ids=ktk_cube_dataset_ids, metadata={ ktk_cube_dataset_id: prepare_ktk_metadata(cube, ktk_cube_dataset_id, metadata) for ktk_cube_dataset_id in ktk_cube_dataset_ids }, update=True, existing_datasets=existing_datasets, ) data = data.map( apply_postwrite_checks, cube=cube, store=store, existing_datasets=existing_datasets, ) return data