diff --git a/.gitignore b/.gitignore index 208813d293..c21c191bed 100644 --- a/.gitignore +++ b/.gitignore @@ -16,7 +16,7 @@ featuretools/tests/integration_data/products.gzip featuretools/tests/integration_data/regions.gzip featuretools/tests/integration_data/sessions.gzip featuretools/tests/integration_data/stores.gzip -dask-worker-space/* +**/dask-worker-space/* *.dirlock *.~lock* diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 8c798ef858..3a269dd5d3 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -4,6 +4,7 @@ Changelog --------- **Future Release** * Enhancements + * Support use of Dask DataFrames in entitysets (:pr:`783`) * Add ``make_index`` when initializing an EntitySet by passing in an ``entities`` dictionary (:pr:`1010`) * Fixes * Changes @@ -12,7 +13,7 @@ Changelog * Update tests for numpy v1.19.0 compatability (:pr:`1016`) Thanks to the following people for contributing to this release: - :user:`gsheni`, :user:`frances-h` + :user:`gsheni`, :user:`frances-h`, :user:`rwedge`, :user:`thehomebrewnerd` **v0.15.0 May 29, 2020** * Enhancements diff --git a/docs/source/frequently_asked_questions.ipynb b/docs/source/frequently_asked_questions.ipynb index 7bcb5480b7..5ad8b0d318 100644 --- a/docs/source/frequently_asked_questions.ipynb +++ b/docs/source/frequently_asked_questions.ipynb @@ -419,6 +419,21 @@ "feature_matrix[[\"COUNT(sessions WHERE product_id_device = 5 and tablet)\"]]" ] }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Can I create an `EntitySet` using Dask dataframes? (BETA)\n", + "\n", + "Support for Dask EntitySets is still in Beta - if you encounter any errors using this approach, please let us know by creating a [new issue on Github](https://github.com/FeatureLabs/featuretools/issues).\n", + "\n", + "Yes! Featuretools supports creating an `EntitySet` from Dask dataframes. You can simply follow the same process you would when creating an `EntitySet` from pandas dataframes.\n", + "\n", + "There are some limitations to be aware of when using Dask dataframes. When creating an `Entity` from a Dask dataframe, variable type inference is not performed as it is for pandas entities, so the user must supply a list of variable types during creation. Also, other quality checks are not performed, such as checking for unique index values. An `EntitySet` must be created entirely of Dask entities or pandas entities - you cannot mix pandas entities with Dask entitites in the same `EntitySet`.\n", + "\n", + "For more information on creating an `EntitySet` from Dask dataframes, see the [Using Dask EntitySets](https://docs.featuretools.com/en/stable/guides/using_dask_entitysets.html) guide." + ] + }, { "cell_type": "markdown", "metadata": {}, @@ -1509,7 +1524,7 @@ "source": [ "### How do I get a list of all Aggregation and Transform primitives?\n", "\n", - "You can do `featuretools.list_primitives()` to get all the primitive in Featuretools. It will return a Dataframe with the names, type, and description of the primitives. You can also visit [primitives.featurelabs.com](https://primitives.featurelabs.com/) to obtain a list of all available primitives." + "You can do `featuretools.list_primitives()` to get all the primitive in Featuretools. It will return a Dataframe with the names, type, and description of the primitives, and if the primitive can be used with entitysets created from Dask dataframes. You can also visit [primitives.featurelabs.com](https://primitives.featurelabs.com/) to obtain a list of all available primitives." ] }, { @@ -1531,6 +1546,19 @@ "df_primitives.tail()" ] }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### What primitives can I use when creating a feature matrix from a Dask `EntitySet`? (BETA)\n", + "\n", + "Support for Dask EntitySets is still in Beta - if you encounter any errors using this approach, please let us know by creating a [new issue on Github](https://github.com/FeatureLabs/featuretools/issues).\n", + "\n", + "When creating a feature matrix from a Dask `EntitySet`, only certain primitives can be used. Computation of certain features is quite expensive in a distributed environment, and as a result only a subset of Featuretools primitives are currently supported when using a Dask `EntitySet`.\n", + "\n", + "The table returned by `featuretools.list_primitives()` will contain a column labeled `dask_compatible`. Any primitive that has a value of `True` in this column can be used safely when computing a feature matrix from a Dask `EntitySet`." + ] + }, { "cell_type": "markdown", "metadata": {}, diff --git a/docs/source/guides/parallel.rst b/docs/source/guides/parallel.rst deleted file mode 100644 index fde4450db0..0000000000 --- a/docs/source/guides/parallel.rst +++ /dev/null @@ -1,64 +0,0 @@ -.. _parallel: - -Parallel Feature Computation -============================ -Featuretools can optionally compute features on multiple cores. The simplest way to control the amount of parallelism is to specify the ``n_jobs`` parameter:: - - fm = ft.calculate_feature_matrix(features=features, - entityset=entityset, - cutoff_time=cutoff_time, - n_jobs=2, - verbose=True) - -The above command will start 2 processes to compute chunks of the feature matrix in parallel. Each process receives its own copy of the entity set, so memory use will be proportional to the number of parallel processes. Because the entity set has to be copied to each process, there is overhead to perform this operation before calculation can begin. To avoid this overhead on successive calls to ``calculate_feature_matrix``, read the section below on using a persistent cluster. - -Running Featuretools with Spark and Dask ----------------------------------------- -The Featuretools development team is continually working to improve integration with Dask and Spark for performing feature engineering at scale. If you have a big data problem and are interested in testing our latest Dask or Spark integrations for free, please let us know by completing `this simple request form `__. - -Continue reading below to learn how to perform parallel feature computation with the current integrations. - -Using persistent cluster ------------------------- -Behind the scenes, Featuretools uses `dask's `_ distributed scheduler to implement multiprocessing. When you only specify the ``n_jobs`` parameter, a cluster will be created for that specific feature matrix calculation and destroyed once calculations have finished. A drawback of this is that each time a feature matrix is calculated, the entity set has to be transmitted to the workers again. To avoid this, we would like to reuse the same cluster between calls. The way to do this is by creating a cluster first and telling featuretools to use it with the ``dask_kwargs`` parameter:: - - import featuretools as ft - from dask.distributed import LocalCluster - - cluster = LocalCluster() - fm_1 = ft.calculate_feature_matrix(features=features_1, - entityset=entityset, - cutoff_time=cutoff_time, - dask_kwargs={'cluster': cluster}, - verbose=True) - -The 'cluster' value can either be the actual cluster object or a string of the address the cluster's scheduler can be reached at. The call below would also work. This second feature matrix calculation will not need to resend the entityset data to the workers because it has already been saved on the cluster.:: - - fm_2 = ft.calculate_feature_matrix(features=features_2, - entityset=entityset, - cutoff_time=cutoff_time, - dask_kwargs={'cluster': cluster.scheduler.address}, - verbose=True) - -.. note:: - - When using a persistent cluster, Featuretools publishes a copy of the ``EntitySet`` to the cluster the first time it calculates a feature matrix. Based on the ``EntitySet``'s metadata the cluster will reuse it for successive computations. This means if two ``EntitySets`` have the same metadata but different row values (e.g. new data is added to the ``EntitySet``), Featuretools won’t recopy the second ``EntitySet`` in later calls. A simple way to avoid this scenario is to use a unique ``EntitySet`` id. - -Using the distributed dashboard -------------------------------- -Dask.distributed has a web-based diagnostics dashboard that can be used to analyze the state of the workers and tasks. It can also be useful for tracking memory use or visualizing task run-times. An in-depth description of the web interface can be found `here `_. - -.. image:: /images/dashboard.png - -The dashboard requires an additional python package, bokeh, to work. Once bokeh is installed, the web interface will be launched by default when a LocalCluster is created. The cluster created by featuretools when using ``n_jobs`` does not enable the web interface automatically. To do so, the port to launch the main web interface on must be specified in ``dask_kwargs``:: - - fm = ft.calculate_feature_matrix(features=features, - entityset=entityset, - cutoff_time=cutoff_time, - n_jobs=2, - dask_kwargs={'diagnostics_port': 8787} - verbose=True) - -Parallel Computation by Partitioning Data ------------------------------------------ -As an alternative to Featuretool's parallelization, the data can be partitioned and the feature calculations run on multiple cores or a cluster using Dask or Apache Spark with PySpark. This approach may be necessary with a large ``EntitySet`` because the current parallel implementation sends the entire ``EntitySet`` to each worker which may exhaust the worker memory. For more information on partitioning the data and using Dask or Spark, see :doc:`/guides/performance`. Dask and Spark allow Featuretools to scale to multiple cores on a single machine or multiple machines on a cluster. diff --git a/docs/source/guides/performance.rst b/docs/source/guides/performance.rst index 859592b0e0..a8ff0f6607 100644 --- a/docs/source/guides/performance.rst +++ b/docs/source/guides/performance.rst @@ -15,9 +15,35 @@ Each row in a feature matrix created by Featuretools is calculated at a specific If there are many unique cutoff times, it is often worthwhile to figure out how to have fewer. This can be done manually by figuring out which unique times are necessary for the prediction problem or automatically using :ref:`approximate `. +Parallel Feature Computation +---------------------------- +Computational performance can often be improved by parallelizing the feature calculation process. There are several different approaches that can be used to perform parallel feature computation with Featuretools. An overview of the most commonly used approaches is provided below. + +Computation with Dask EntitySets (BETA) +*************************************** +.. note:: + Support for Dask EntitySets is still in Beta. While the key functionality has been implemented, development is ongoing to add the remaining functionality. + + All planned improvements to the Featuretools/Dask integration are `documented on Github `_. If you see an open issue that is important for your application, please let us know by upvoting or commenting on the issue. If you encounter any errors using Dask entities, or find missing functionality that does not yet have an open issue, please create a `new issue on Github `_. + +Dask can be used with Featuretools to perform parallel feature computation with virtually no changes to the workflow required. Featuretools supports creating an ``EntitySet`` directly from Dask dataframes instead of using pandas dataframes, enabling the parallel and distributed computation capabilities of Dask to be used. By creating an ``EntitySet`` directly from Dask dataframes, Featuretools can be used to generate a larger-than-memory feature matrix, something that may be difficult with other approaches. When computing a feature matrix from an ``EntitySet`` created from Dask dataframes, the resulting feature matrix will be returned as a Dask dataframe. + +This method does have some limitations in terms of the primitives that are available and the optional parameters that can be used when calculating the feature matrix. For more information on generating a feature matrix with this approach, refer to the guide :doc:`/guides/using_dask_entitysets`. + +Simple Parallel Feature Computation +*********************************** +If using a pandas ``EntitySet``, Featuretools can optionally compute features on multiple cores. The simplest way to control the amount of parallelism is to specify the ``n_jobs`` parameter:: + + fm = ft.calculate_feature_matrix(features=features, + entityset=entityset, + cutoff_time=cutoff_time, + n_jobs=2, + verbose=True) + +The above command will start 2 processes to compute chunks of the feature matrix in parallel. Each process receives its own copy of the entity set, so memory use will be proportional to the number of parallel processes. Because the entity set has to be copied to each process, there is overhead to perform this operation before calculation can begin. To avoid this overhead on successive calls to ``calculate_feature_matrix``, read the section below on using a persistent cluster. Adjust chunk size ------------------ ++++++++++++++++++ By default, Featuretools calculates rows with the same cutoff time simultaneously. The `chunk_size` parameter limits the maximum number of rows that will be grouped and then calculated together. If calculation is done using parallel processing, the default chunk size is set to be ``1 / n_jobs`` to ensure the computation can be spread across available workers. Normally, this behavior works well, but if there are only a few unique cutoff times it can lead to higher peak memory usage (due to more intermediate calculations stored in memory) or limited parallelism (if the number of chunks is less than `n_jobs`). By setting ``chunk_size``, we can limit the maximum number of rows in each group to specific number or a percentage of the overall data when calling ``ft.dfs`` or ``ft.calculate_feature_matrix``:: @@ -35,9 +61,54 @@ We can also set chunk size to be a percentage of total rows:: target_entity="customers", chunk_size=.05) +Using persistent cluster +++++++++++++++++++++++++ +Behind the scenes, Featuretools uses `Dask's `_ distributed scheduler to implement multiprocessing. When you only specify the ``n_jobs`` parameter, a cluster will be created for that specific feature matrix calculation and destroyed once calculations have finished. A drawback of this is that each time a feature matrix is calculated, the entity set has to be transmitted to the workers again. To avoid this, we would like to reuse the same cluster between calls. The way to do this is by creating a cluster first and telling featuretools to use it with the ``dask_kwargs`` parameter:: + + import featuretools as ft + from dask.distributed import LocalCluster + + cluster = LocalCluster() + fm_1 = ft.calculate_feature_matrix(features=features_1, + entityset=entityset, + cutoff_time=cutoff_time, + dask_kwargs={'cluster': cluster}, + verbose=True) + +The 'cluster' value can either be the actual cluster object or a string of the address the cluster's scheduler can be reached at. The call below would also work. This second feature matrix calculation will not need to resend the entityset data to the workers because it has already been saved on the cluster.:: + + fm_2 = ft.calculate_feature_matrix(features=features_2, + entityset=entityset, + cutoff_time=cutoff_time, + dask_kwargs={'cluster': cluster.scheduler.address}, + verbose=True) + +.. note:: + + When using a persistent cluster, Featuretools publishes a copy of the ``EntitySet`` to the cluster the first time it calculates a feature matrix. Based on the ``EntitySet``'s metadata the cluster will reuse it for successive computations. This means if two ``EntitySets`` have the same metadata but different row values (e.g. new data is added to the ``EntitySet``), Featuretools won’t recopy the second ``EntitySet`` in later calls. A simple way to avoid this scenario is to use a unique ``EntitySet`` id. + +Using the distributed dashboard ++++++++++++++++++++++++++++++++ +Dask.distributed has a web-based diagnostics dashboard that can be used to analyze the state of the workers and tasks. It can also be useful for tracking memory use or visualizing task run-times. An in-depth description of the web interface can be found `here `_. + +.. image:: /images/dashboard.png + +The dashboard requires an additional python package, bokeh, to work. Once bokeh is installed, the web interface will be launched by default when a LocalCluster is created. The cluster created by featuretools when using ``n_jobs`` does not enable the web interface automatically. To do so, the port to launch the main web interface on must be specified in ``dask_kwargs``:: + + fm = ft.calculate_feature_matrix(features=features, + entityset=entityset, + cutoff_time=cutoff_time, + n_jobs=2, + dask_kwargs={'diagnostics_port': 8787} + verbose=True) + +Parallel Computation by Partitioning Data +***************************************** +As an alternative to Featuretools' parallelization, the data can be partitioned and the feature calculations run on multiple cores or a cluster using Dask or Apache Spark with PySpark. This approach may be necessary with a large pandas ``EntitySet`` because the current parallel implementation sends the entire ``EntitySet`` to each worker which may exhaust the worker memory. Dask and Spark allow Featuretools to scale to multiple cores on a single machine or multiple machines on a cluster. + +.. note:: + Partitioning data is not necessary when using a Dask ``EntitySet``, as the Dask dataframes that make up the ``EntitySet`` are already partitioned. Partitioning is only needed when working with pandas entities. -Partition and Distribute Data ------------------------------ When an entire dataset is not required to calculate the features for a given set of instances, we can split the data into independent partitions and calculate on each partition. For example, imagine we are calculating features for customers and the features are "number of other customers in this zip code" or "average age of other customers in this zip code". In this case, we can load in data partitioned by zip code. As long as we have all of the data for a zip code when calculating, we can calculate all features for a subset of customers. An example of this approach can be seen in the `Predict Next Purchase demo notebook `_. In this example, we partition data by customer and only load a fixed number of customers into memory at any given time. We implement this easily using `Dask `_, which could also be used to scale the computation to a cluster of computers. A framework like `Spark `_ could be used similarly. @@ -46,10 +117,6 @@ An additional example of partitioning data to distribute on multiple cores or a For a similar partition and distribute implementation using Apache Spark with PySpark, refer to the `Feature Engineering on Spark notebook `_. This implementation shows how to carry out feature engineering on a cluster of EC2 instances using Spark as the distributed framework. A write-up of this approach is described in the `Featuretools on Spark article `_ on the Feature Labs engineering blog. -Running Featuretools with Spark and Dask ----------------------------------------- -The Featuretools development team is continually working to improve integration with Dask and Spark for performing feature engineering at scale. If you have a big data problem and are interested in testing our latest Dask or Spark integrations for free, please let us know by completing `this simple request form `__. - -Featuretools Enterprise ------------------------ -If you don't want to build it yourself, Featuretools Enterprise has native integrations with Apache Spark and Dask. More information is available `here `__. +Running Featuretoools with Spark +******************************** +The Featuretools development team is continually working to improve integration with Spark for performing feature engineering at scale. If you have a big data problem and are interested in testing our latest integrations for free, please let us know by completing `this simple request form `__. diff --git a/docs/source/guides/using_dask_entitysets.rst b/docs/source/guides/using_dask_entitysets.rst new file mode 100644 index 0000000000..5197aa2795 --- /dev/null +++ b/docs/source/guides/using_dask_entitysets.rst @@ -0,0 +1,127 @@ +Using Dask EntitySets (BETA) +============================ +.. note:: + Support for Dask EntitySets is still in Beta. While the key functionality has been implemented, development is ongoing to add the remaining functionality. + + All planned improvements to the Featuretools/Dask integration are `documented on Github `_. If you see an open issue that is important for your application, please let us know by upvoting or commenting on the issue. If you encounter any errors using Dask entities, or find missing functionality that does not yet have an open issue, please create a `new issue on Github `_. + +Creating a feature matrix from a very large dataset can be problematic if the underlying pandas dataframes that make up the entities cannot easily fit in memory. To help get around this issue, Featuretools supports creating ``Entity`` and ``EntitySet`` objects from Dask dataframes. A Dask ``EntitySet`` can then be passed to ``featuretools.dfs`` or ``featuretools.calculate_feature_matrix`` to create a feature matrix, which will be returned as a Dask dataframe. In addition to working on larger than memory datasets, this approach also allows users to take advantage of the parallel and distributed processing capabilities offered by Dask. + +This guide will provide an overview of how to create a Dask ``EntitySet`` and then generate a feature matrix from it. If you are already familiar with creating a feature matrix starting from pandas dataframes, this process will seem quite familiar, as there are no differences in the process. There are, however, some limitations when using Dask dataframes, and those limitations are reviewed in more detail below. + +Creating Entities and EntitySets +-------------------------------- +For this example, we will create a very small pandas dataframe and then convert this into a Dask dataframe to use in the remainder of the process. Normally when using Dask, you would just read your data directly into a Dask dataframe without the intermediate step of using pandas. + +.. ipython:: python + + import featuretools as ft + import pandas as pd + import dask.dataframe as dd + id = [0, 1, 2, 3, 4] + values = [12, -35, 14, 103, -51] + df = pd.DataFrame({"id": id, "values": values}) + dask_df = dd.from_pandas(df, npartitions=2) + dask_df + + +Now that we have our Dask dataframe, we can start to create the ``EntitySet``. The current implementation does not support variable type inference for Dask entities, so we must pass a dictionary of variable types using the ``variable_types`` parameter when calling ``es.entity_from_dataframe()``. Aside from needing to supply the variable types, the rest of the process of creating an ``EntitySet`` is the same as if we were using pandas dataframes. + +.. ipython:: python + + es = ft.EntitySet(id="dask_es") + es = es.entity_from_dataframe(entity_id="dask_entity", + dataframe=dask_df, + index="id", + variable_types={"id": ft.variable_types.Id, + "values": ft.variable_types.Numeric}) + es + + +Notice that when we print our ``EntitySet``, the number of rows for the ``dask_entity`` entity is returned as a Dask ``Delayed`` object. This is because obtaining the length of a Dask dataframe may require an expensive compute operation to sum up the lengths of all the individual partitions that make up the dataframe and that operation is not performed by default. + + +Running DFS +----------- +We can pass the ``EntitySet`` we created above to ``featuretools.dfs`` in order to create a feature matrix. If the ``EntitySet`` we pass to ``dfs`` is made of Dask entities, the feature matrix we get back will be a Dask dataframe. + +.. ipython:: python + + feature_matrix, features = ft.dfs(entityset=es, + target_entity="dask_entity", + trans_primitives=["negate"]) + feature_matrix + + +This feature matrix can be saved to disk or computed and brought into memory, using the appropriate Dask dataframe methods. + +.. ipython:: python + + fm_computed = feature_matrix.compute() + fm_computed + + +While this is a simple example to illustrate the process of using Dask dataframes with Featuretools, this process will also work with an ``EntitySet`` containing multiple entities, as well as with aggregation primitives. + +Limitations +----------- +The key functionality of Featuretools is available for use with a Dask ``EntitySet``, and work is ongoing to add the remaining functionality that is available when using a pandas ``EntitySet``. There are, however, some limitations to be aware of when creating a Dask ``Entityset`` and then using it to generate a feature matrix. The most significant limitations are reviewed in more detail in this section. + +.. note:: + If the limitations of using a Dask ``EntitySet`` are problematic for your problem, you may still be able to compute a larger-than-memory feature matrix by partitioning your data as described in :doc:`performance`. + +Supported Primitives +******************** +When creating a feature matrix from a Dask ``EntitySet``, only certain primitives can be used. Primitives that rely on the order of the entire dataframe or require an entire column for computation are currently not supported when using a Dask ``EntitySet``. Multivariable and time-dependent aggregation primitives also are not currently supported. + +To obtain a list of the primitives that can be used with a Dask ``EntitySet``, you can call ``featuretools.list_primitives()``. This will return a table of all primitives. Any primitive that can be used with a Dask ``EntitySet`` will have a value of ``True`` in the ``dask_compatible`` column. + + +.. ipython:: python + + primitives_df = ft.list_primitives() + dask_compatible_df = primitives_df[primitives_df["dask_compatible"] == True] + dask_compatible_df.head() + dask_compatible_df.tail() + +Primitive Limitations +********************* +At this time, custom primitives created with ``featuretools.primitives.make_trans_primitive()`` or ``featuretools.primitives.make_agg_primitive()`` cannot be used for running deep feature synthesis on a Dask ``EntitySet``. While it is possible to create custom primitives for use with a Dask ``EntitySet`` by extending the proper primitive class, there are several potential problems in doing so, and those issues are beyond the scope of this guide. + +Entity Limitations +****************** +When creating a Featuretools ``Entity`` from Dask dataframes, variable type inference is not performed as it is when creating entities from pandas dataframes. This is done to improve speed as sampling the data to infer the variable types would require an expensive compute operation on the underlying Dask dataframe. As a consequence, users must define the variable types for each column in the supplied Dataframe. This step is needed so that the deep feature synthesis process can build the proper features based on the column types. A list of available variable types can be obtained by running ``featuretools.variable_types.find_variable_types()``. + +By default, Featuretools checks that entities created from pandas dataframes have unique index values. Because performing this same check with Dask would require an expensive compute operation, this check is not performed when creating an entity from a Dask dataframe. When using Dask dataframes, users must ensure that the supplied index values are unique. + +When an ``Entity`` is created from a pandas dataframe, the ordering of the underlying dataframe rows is maintained. For a Dask ``Entity``, the ordering of the dataframe rows is not guaranteed, and Featuretools does not attempt to maintain row order in a Dask ``Entity``. If ordering is important, close attention must be paid to any output to avoid issues. + +The ``Entity.add_interesting_values()`` method is not supported when using a Dask ``Entity``. If needed, users can manually set ``interesing_values`` on entities by assigning them directly with syntax similar to this: ``es["entity_name"]["variable_name"].interesting_values = ["Value 1", "Value 2"]``. + +EntitySet Limitations +********************* +When creating a Featuretools ``EntitySet`` that will be made of Dask entities, all of the entities used to create the ``EntitySet`` must be of the same type, either all Dask entities or all pandas entities. Featuretools does not support creating an ``EntitySet`` containing a mix of Dask and pandas entities. + +Additionally, the ``EntitySet.add_interesting_values()`` method is not supported when using a Dask ``EntitySet``. Users can manually set ``interesing_values`` on entities, as described above. + +DFS Limitations +*************** +There are a few key limitations when generating a feature matrix from a Dask ``EntitySet``. + +If a ``cutoff_time`` parammeter is passed to ``featuretools.dfs()`` it must either be a single cutoff time value, or a pandas dataframe. The current implementation does not support the use of a Dask dataframe for cutoff time values. + +Additionally, Featuretools does not currently support the use of the ``approximate`` or ``training_window`` parameters when working with Dask entitiysets, but should in future releases. + +Finally, if the output feature matrix contains a boolean column with ``NaN`` values included, the column type may have a different datatype than the same feature matrix generated from a pandas ``EntitySet``. If feature matrix column data types are critical, the feature matrix should be inspected to make sure the types are of the proper types, and recast as necessary. + +Other Limitations +***************** +In some instances, generating a feature matrix with a large number of features has resulted in memory issues on Dask workers. The underlying reason for this is that the partition size of the feature matrix grows too large for Dask to handle as the number of feature columns grows large. This issue is most prevalent when the feature matrix contains a large number of columns compared to the dataframes that make up the entities. Possible solutions to this problem include reducing the partition size used when creating the entity dataframes or increasing the memory available on Dask workers. + +Currently ``featuretools.encode_features()`` does not work with a Dask dataframe as input. This will hopefully be resolved in a future release of Featuretools. + +The utility function ``featuretools.make_temporal_cutoffs()`` will not work properly with Dask inputs for ``instance_ids`` or ``cutoffs``. However, as noted above, if a ``cutoff_time`` dataframe is supplied to ``dfs``, the supplied dataframe must be a pandas dataframe, and this can be generated by supplying pandas inputs to ``make_temporal_cutoffs()``. + +The use of ``featuretools.remove_low_information_features()`` cannot currently be used with a Dask feature matrix. + +When manually defining a ``Feature``, the ``use_previous`` parameter cannot be used if this feature will be applied to calculate a feature matrix from a Dask ``EntitySet``. diff --git a/docs/source/index.rst b/docs/source/index.rst index 8dcfb584f4..c532b2aa9c 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -164,7 +164,7 @@ Table of contents guides/tuning_dfs guides/specifying_primitive_options guides/performance - guides/parallel + guides/using_dask_entitysets guides/deployment guides/advanced_custom_primitives diff --git a/docs/source/loading_data/using_entitysets.rst b/docs/source/loading_data/using_entitysets.rst index e518785f05..53cb645943 100644 --- a/docs/source/loading_data/using_entitysets.rst +++ b/docs/source/loading_data/using_entitysets.rst @@ -159,3 +159,8 @@ Finally, we are ready to use this EntitySet with any functionality within Featur feature_matrix As we can see, the features from DFS use the relational structure of our entity set. Therefore it is important to think carefully about the entities that we create. + +Dask EntitySets +~~~~~~~~~~~~~~~ + +EntitySets can also be created using Dask dataframes. For more information refer to :doc:`../guides/using_dask_entitysets`. \ No newline at end of file diff --git a/featuretools/computational_backends/calculate_feature_matrix.py b/featuretools/computational_backends/calculate_feature_matrix.py index 520ac0c591..4b6d0e21c5 100644 --- a/featuretools/computational_backends/calculate_feature_matrix.py +++ b/featuretools/computational_backends/calculate_feature_matrix.py @@ -7,6 +7,7 @@ from datetime import datetime import cloudpickle +import dask.dataframe as dd import numpy as np import pandas as pd @@ -138,6 +139,14 @@ def calculate_feature_matrix(features, entityset=None, cutoff_time=None, instanc if entities is not None and relationships is not None: entityset = EntitySet("entityset", entities, relationships) + if any(isinstance(es.df, dd.DataFrame) for es in entityset.entities): + if approximate: + msg = "Using approximate is not supported with Dask Entities" + raise ValueError(msg) + if training_window: + msg = "Using training_window is not supported with Dask Entities" + raise ValueError(msg) + target_entity = entityset[features[0].entity.id] pass_columns = [] @@ -145,6 +154,11 @@ def calculate_feature_matrix(features, entityset=None, cutoff_time=None, instanc if isinstance(cutoff_time, list): raise TypeError("cutoff_time must be a single value or DataFrame") + if isinstance(cutoff_time, dd.DataFrame): + msg = "cannot use Dask DataFrame for cutoff_time: "\ + "cutoff_time must a single value or a Pandas DataFrame" + raise TypeError(msg) + if cutoff_time is None: if entityset.time_type == NumericTimeIndex: cutoff_time = np.inf @@ -157,7 +171,7 @@ def calculate_feature_matrix(features, entityset=None, cutoff_time=None, instanc time_last=cutoff_time, training_window=training_window, include_cutoff_time=include_cutoff_time) - instance_ids = df[index_var].tolist() + instance_ids = list(df[index_var]) cutoff_time = [cutoff_time] * len(instance_ids) map_args = [(id, time) for id, time in zip(instance_ids, cutoff_time)] @@ -197,7 +211,8 @@ def calculate_feature_matrix(features, entityset=None, cutoff_time=None, instanc "via pd.to_numeric(cutoff_time['time'])") elif entityset.time_type == DatetimeTimeIndex: if cutoff_time['time'].dtype.name not in PandasTypes._pandas_datetimes: - raise TypeError("cutoff_time times must be datetime type: try casting via pd.to_datetime(cutoff_time['time'])") + raise TypeError( + "cutoff_time times must be datetime type: try casting via pd.to_datetime(cutoff_time['time'])") assert (cutoff_time[['instance_id', 'time']].duplicated().sum() == 0), \ "Duplicated rows in cutoff time dataframe." @@ -303,11 +318,12 @@ def calculate_feature_matrix(features, entityset=None, cutoff_time=None, instanc include_cutoff_time=include_cutoff_time) # ensure rows are sorted by input order - feature_matrix = feature_matrix.reindex(pd.MultiIndex.from_frame(cutoff_time[["instance_id", "time"]], - names=feature_matrix.index.names)) - - if not cutoff_time_in_index: - feature_matrix.reset_index(level='time', drop=True, inplace=True) + if isinstance(feature_matrix, pd.DataFrame): + feature_matrix = feature_matrix.reindex( + pd.MultiIndex.from_frame(cutoff_time[["instance_id", "time"]], + names=feature_matrix.index.names)) + if not cutoff_time_in_index: + feature_matrix.reset_index(level='time', drop=True, inplace=True) if save_progress and os.path.exists(os.path.join(save_progress, 'temp')): shutil.rmtree(os.path.join(save_progress, 'temp')) @@ -362,7 +378,8 @@ def update_progress_callback(done): previous_progress = progress_bar.n progress_bar.update(done * group.shape[0]) if progress_callback is not None: - update, progress_percent, time_elapsed = update_progress_callback_parameters(progress_bar, previous_progress) + update, progress_percent, time_elapsed = update_progress_callback_parameters(progress_bar, + previous_progress) progress_callback(update, progress_percent, time_elapsed) calculator = FeatureSetCalculator(entityset, @@ -402,7 +419,10 @@ def update_progress_callback(done): training_window=window, include_cutoff_time=include_cutoff_time) - id_name = _feature_matrix.index.name + if isinstance(_feature_matrix, dd.DataFrame): + id_name = _feature_matrix.columns[-1] + else: + id_name = _feature_matrix.index.name # if approximate, merge feature matrix with group frame to get original # cutoff times and passed columns @@ -417,20 +437,32 @@ def update_progress_callback(done): _feature_matrix.sort_index(level=1, kind='mergesort', inplace=True) else: # all rows have same cutoff time. set time and add passed columns - num_rows = _feature_matrix.shape[0] - time_index = pd.Index([time_last] * num_rows, name='time') - _feature_matrix.set_index(time_index, append=True, inplace=True) + num_rows = len(ids) if len(pass_columns) > 0: pass_through = group[['instance_id', cutoff_df_time_var] + pass_columns] pass_through.rename(columns={'instance_id': id_name, cutoff_df_time_var: 'time'}, inplace=True) - pass_through.set_index([id_name, 'time'], inplace=True) + if isinstance(_feature_matrix, pd.DataFrame): + time_index = pd.Index([time_last] * num_rows, name='time') + _feature_matrix = _feature_matrix.set_index(time_index, append=True) + if len(pass_columns) > 0: + pass_through.set_index([id_name, 'time'], inplace=True) + for col in pass_columns: + _feature_matrix[col] = pass_through[col] + elif isinstance(_feature_matrix, dd.DataFrame) and (len(pass_columns) > 0): + _feature_matrix['time'] = time_last for col in pass_columns: - _feature_matrix[col] = pass_through[col] + pass_df = dd.from_pandas(pass_through[[id_name, 'time', col]], npartitions=_feature_matrix.npartitions) + _feature_matrix = _feature_matrix.merge(pass_df, how="outer") + _feature_matrix = _feature_matrix.drop(columns=['time']) + feature_matrix.append(_feature_matrix) - feature_matrix = pd.concat(feature_matrix) + if any(isinstance(fm, dd.DataFrame) for fm in feature_matrix): + feature_matrix = dd.concat(feature_matrix) + else: + feature_matrix = pd.concat(feature_matrix) return feature_matrix @@ -614,7 +646,8 @@ def parallel_calculate_chunks(cutoff_time, chunk_size, feature_set, approximate, previous_progress = progress_bar.n progress_bar.update(result.shape[0]) if progress_callback is not None: - update, progress_percent, time_elapsed = update_progress_callback_parameters(progress_bar, previous_progress) + update, progress_percent, time_elapsed = update_progress_callback_parameters(progress_bar, + previous_progress) progress_callback(update, progress_percent, time_elapsed) except Exception: @@ -651,7 +684,6 @@ def _add_approx_entity_index_var(es, target_entity_id, cutoffs, path): new_var_name = '%s.%s' % (last_child_var, relationship.child_variable.id) to_rename = {relationship.child_variable.id: new_var_name} child_df = child_df.rename(columns=to_rename) - cutoffs = cutoffs.merge(child_df, left_on=last_child_var, right_on=last_parent_var) diff --git a/featuretools/computational_backends/feature_set_calculator.py b/featuretools/computational_backends/feature_set_calculator.py index dc7a83fdeb..b1c808fd77 100644 --- a/featuretools/computational_backends/feature_set_calculator.py +++ b/featuretools/computational_backends/feature_set_calculator.py @@ -2,6 +2,7 @@ from datetime import datetime from functools import partial +import dask.dataframe as dd import numpy as np import pandas as pd import pandas.api.types as pdtypes @@ -118,28 +119,39 @@ def progress_callback(*args): # df_trie. df = df_trie.value - if df.empty: - return self.generate_default_df(instance_ids=instance_ids) + # Fill in empty rows with default values. This only works for pandas dataframes + # and is not currently supported for Dask dataframes. + if isinstance(df, pd.DataFrame): + if df.empty: + return self.generate_default_df(instance_ids=instance_ids) - # fill in empty rows with default values - missing_ids = [i for i in instance_ids if i not in - df[target_entity.index]] - if missing_ids: - default_df = self.generate_default_df(instance_ids=missing_ids, - extra_columns=df.columns) - df = df.append(default_df, sort=True) + missing_ids = [i for i in instance_ids if i not in + df[target_entity.index]] + if missing_ids: + default_df = self.generate_default_df(instance_ids=missing_ids, + extra_columns=df.columns) + df = df.append(default_df, sort=True) + + df.index.name = self.entityset[self.feature_set.target_eid].index - df.index.name = self.entityset[self.feature_set.target_eid].index column_list = [] # Order by instance_ids unique_instance_ids = pd.unique(instance_ids) - # pd.unique changes the dtype for Categorical, so reset it. - unique_instance_ids = unique_instance_ids.astype(instance_ids.dtype) - df = df.reindex(unique_instance_ids) + + if isinstance(df, dd.DataFrame): + unique_instance_ids = unique_instance_ids.astype(object) + else: + # pd.unique changes the dtype for Categorical, so reset it. + unique_instance_ids = unique_instance_ids.astype(instance_ids.dtype) + df = df.reindex(unique_instance_ids) for feat in self.feature_set.target_features: column_list.extend(feat.get_feature_names()) + + if isinstance(df, dd.DataFrame): + column_list.extend([target_entity.index]) + df.index.name = target_entity.index return df[column_list] def _calculate_features_for_entity(self, entity_id, feature_trie, df_trie, @@ -235,6 +247,9 @@ def _calculate_features_for_entity(self, entity_id, feature_trie, df_trie, # Pass filtered values, even if we are using a full df. if need_full_entity: + if isinstance(filter_values, dd.Series): + msg = "Cannot use primitives that require full entity with Dask EntitySets" + raise ValueError(msg) filtered_df = df[df[filter_variable].isin(filter_values)] else: filtered_df = df @@ -363,7 +378,9 @@ def _add_ancestor_relationship_variables(self, child_df, parent_df, right_on=relationship.child_variable.id) # ensure index is maintained - df.set_index(relationship.child_entity.index, drop=False, inplace=True) + # TODO: Review for dask dataframes + if isinstance(df, pd.DataFrame): + df.set_index(relationship.child_entity.index, drop=False, inplace=True) return df, new_relationship_variables @@ -403,7 +420,7 @@ def _feature_type_handler(self, f): def _calculate_identity_features(self, features, df, _df_trie, progress_callback): for f in features: - assert f.get_name() in df, ( + assert f.get_name() in df.columns, ( 'Column "%s" missing frome dataframe' % f.get_name()) progress_callback(len(features) / float(self.num_features)) @@ -411,9 +428,11 @@ def _calculate_identity_features(self, features, df, _df_trie, progress_callback return df def _calculate_transform_features(self, features, frame, _df_trie, progress_callback): + frame_empty = frame.empty if isinstance(frame, pd.DataFrame) else False + feature_values = [] for f in features: # handle when no data - if frame.shape[0] == 0: + if frame_empty: set_default_column(frame, f) progress_callback(1 / float(self.num_features)) @@ -421,6 +440,7 @@ def _calculate_transform_features(self, features, frame, _df_trie, progress_call continue # collect only the variables we need for this transformation + variable_data = [frame[bf.get_name()] for bf in f.base_features] @@ -437,10 +457,12 @@ def _calculate_transform_features(self, features, frame, _df_trie, progress_call values = [strip_values_if_series(value) for value in values] else: values = [strip_values_if_series(values)] - update_feature_columns(f, frame, values) + + feature_values.append((f, values)) progress_callback(1 / float(self.num_features)) + frame = update_feature_columns(feature_values, frame) return frame def _calculate_groupby_features(self, features, frame, _df_trie, progress_callback): @@ -523,14 +545,19 @@ def _calculate_direct_features(self, features, child_df, df_trie, progress_callb # merge the identity feature from the parent entity into the child merge_df = parent_df[list(col_map.keys())].rename(columns=col_map) - if index_as_feature is not None: - merge_df.set_index(index_as_feature.get_name(), inplace=True, - drop=False) + if isinstance(merge_df, dd.DataFrame): + new_df = child_df.merge(merge_df, left_on=merge_var, right_on=merge_var, + how='left') else: - merge_df.set_index(merge_var, inplace=True) + if index_as_feature is not None: + merge_df.set_index(index_as_feature.get_name(), + inplace=True, + drop=False) + else: + merge_df.set_index(merge_var, inplace=True) - new_df = child_df.merge(merge_df, left_on=merge_var, right_index=True, - how='left') + new_df = child_df.merge(merge_df, left_on=merge_var, right_index=True, + how='left') progress_callback(len(features) / float(self.num_features)) @@ -540,6 +567,7 @@ def _calculate_agg_features(self, features, frame, df_trie, progress_callback): test_feature = features[0] child_entity = test_feature.base_features[0].entity base_frame = df_trie.get_node(test_feature.relationship_path).value + parent_merge_var = test_feature.relationship_path[0][1].parent_variable.id # Sometimes approximate features get computed in a previous filter frame # and put in the current one dynamically, # so there may be existing features here @@ -555,15 +583,19 @@ def _calculate_agg_features(self, features, frame, df_trie, progress_callback): return frame # handle where + base_frame_empty = base_frame.empty if isinstance(base_frame, pd.DataFrame) else False where = test_feature.where - if where is not None and not base_frame.empty: + if where is not None and not base_frame_empty: base_frame = base_frame.loc[base_frame[where.get_name()]] # when no child data, just add all the features to frame with nan - if base_frame.empty: + base_frame_empty = base_frame.empty if isinstance(base_frame, pd.DataFrame) else False + if base_frame_empty: + feature_values = [] for f in features: - update_feature_columns(f, frame, np.full(f.number_output_features, np.nan)) + feature_values.append((f, np.full(f.number_output_features, np.nan))) progress_callback(1 / float(self.num_features)) + frame = update_feature_columns(feature_values, frame) else: relationship_path = test_feature.relationship_path @@ -572,7 +604,7 @@ def _calculate_agg_features(self, features, frame, df_trie, progress_callback): # if the use_previous property exists on this feature, include only the # instances from the child entity included in that Timedelta use_previous = test_feature.use_previous - if use_previous and not base_frame.empty: + if use_previous: # Filter by use_previous values time_last = self.time_last if use_previous.has_no_observations(): @@ -599,7 +631,10 @@ def last_n(df): variable_id = f.base_features[0].get_name() if variable_id not in to_agg: to_agg[variable_id] = [] - func = f.get_function() + if isinstance(base_frame, dd.DataFrame): + func = f.get_dask_aggregation() + else: + func = f.get_function() # for some reason, using the string count is significantly # faster than any method a primitive can return @@ -619,6 +654,11 @@ def last_n(df): func.__name__ = funcname + if isinstance(func, dd.Aggregation): + # TODO: handle aggregation being applied to same variable twice + # (see above partial wrapping of functions) + funcname = func.__name__ + to_agg[variable_id].append(func) # this is used below to rename columns that pandas names for us agg_rename[u"{}-{}".format(variable_id, funcname)] = f.get_name() @@ -634,7 +674,9 @@ def last_n(df): # to silence pandas warning about ambiguity we explicitly pass # the column (in actuality grouping by both index and group would # work) - to_merge = base_frame.groupby(base_frame[groupby_var], observed=True, sort=False).apply(wrap) + to_merge = base_frame.groupby(base_frame[groupby_var], + observed=True, + sort=False).apply(wrap) frame = pd.merge(left=frame, right=to_merge, left_index=True, right_index=True, how='left') @@ -648,9 +690,12 @@ def last_n(df): # to silence pandas warning about ambiguity we explicitly pass # the column (in actuality grouping by both index and group would # work) - to_merge = base_frame.groupby(base_frame[groupby_var], - observed=True, sort=False).agg(to_agg) + if isinstance(base_frame, dd.DataFrame): + to_merge = base_frame.groupby(groupby_var).agg(to_agg) + else: + to_merge = base_frame.groupby(base_frame[groupby_var], + observed=True, sort=False).agg(to_agg) # rename columns to the correct feature names to_merge.columns = [agg_rename["-".join(x)] for x in to_merge.columns.ravel()] to_merge = to_merge[list(agg_rename.values())] @@ -661,8 +706,11 @@ def last_n(df): categories = pdtypes.CategoricalDtype(categories=frame.index.categories) to_merge.index = to_merge.index.astype(object).astype(categories) - frame = pd.merge(left=frame, right=to_merge, - left_index=True, right_index=True, how='left') + if isinstance(frame, dd.DataFrame): + frame = frame.merge(to_merge, left_on=parent_merge_var, right_index=True, how='left') + else: + frame = pd.merge(left=frame, right=to_merge, + left_index=True, right_index=True, how='left') # determine number of features that were just merged progress_callback(len(to_merge.columns) / float(self.num_features)) @@ -674,7 +722,7 @@ def last_n(df): for name in f.get_feature_names()} fillna_dict.update(feature_defaults) - frame.fillna(fillna_dict, inplace=True) + frame = frame.fillna(fillna_dict) # convert boolean dtypes to floats as appropriate # pandas behavior: https://github.com/pydata/pandas/issues/3752 @@ -716,6 +764,7 @@ def _can_agg(feature): def agg_wrapper(feats, time_last): def wrap(df): d = {} + feature_values = [] for f in feats: func = f.get_function() variable_ids = [bf.get_name() for bf in f.base_features] @@ -728,7 +777,9 @@ def wrap(df): if f.number_output_features == 1: values = [values] - update_feature_columns(f, d, values) + feature_values.append((f, values)) + + d = update_feature_columns(feature_values, d) return pd.Series(d) return wrap @@ -739,11 +790,21 @@ def set_default_column(frame, f): frame[name] = f.default_value -def update_feature_columns(feature, data, values): - names = feature.get_feature_names() - assert len(names) == len(values) - for name, value in zip(names, values): - data[name] = value +def update_feature_columns(feature_data, data): + new_cols = {} + for item in feature_data: + names = item[0].get_feature_names() + values = item[1] + assert len(names) == len(values) + for name, value in zip(names, values): + new_cols[name] = value + + # Handle the case where a dict is being updated + if isinstance(data, dict): + data.update(new_cols) + return data + + return data.assign(**new_cols) def strip_values_if_series(values): diff --git a/featuretools/entityset/deserialize.py b/featuretools/entityset/deserialize.py index e5b6288a7d..dc0fc91436 100644 --- a/featuretools/entityset/deserialize.py +++ b/featuretools/entityset/deserialize.py @@ -5,6 +5,7 @@ from pathlib import Path import pandas as pd +from dask import dataframe as dd from featuretools.entityset.relationship import Relationship from featuretools.entityset.serialize import FORMATS @@ -125,15 +126,20 @@ def read_entity_data(description, path): file = os.path.join(path, description['loading_info']['location']) kwargs = description['loading_info'].get('params', {}) load_format = description['loading_info']['type'] + entity_type = description['loading_info'].get('entity_type', 'pandas') + if entity_type == 'dask': + lib = dd + else: + lib = pd if load_format == 'csv': - dataframe = pd.read_csv( + dataframe = lib.read_csv( file, engine=kwargs['engine'], compression=kwargs['compression'], encoding=kwargs['encoding'], ) elif load_format == 'parquet': - dataframe = pd.read_parquet(file, engine=kwargs['engine']) + dataframe = lib.read_parquet(file, engine=kwargs['engine']) elif load_format == 'pickle': dataframe = pd.read_pickle(file, **kwargs) else: @@ -152,7 +158,12 @@ def parse_latlong(x): return tuple(float(y) for y in x[1:-1].split(",")) for column in latlongs: - dataframe[column] = dataframe[column].apply(parse_latlong) + if entity_type == 'dask': + meta = (column, tuple([float, float])) + dataframe[column] = dataframe[column].apply(parse_latlong, + meta=meta) + else: + dataframe[column] = dataframe[column].apply(parse_latlong) return dataframe diff --git a/featuretools/entityset/entity.py b/featuretools/entityset/entity.py index 76bce24bfc..1ec0caf0e0 100644 --- a/featuretools/entityset/entity.py +++ b/featuretools/entityset/entity.py @@ -1,6 +1,7 @@ import logging import warnings +import dask.dataframe as dd import numpy as np import pandas as pd import pandas.api.types as pdtypes @@ -244,13 +245,17 @@ def query_by_values(self, instance_vals, variable_id=None, columns=None, if instance_vals is None: df = self.df.copy() - elif instance_vals.shape[0] == 0: + elif isinstance(instance_vals, pd.Series) and instance_vals.empty: df = self.df.head(0) else: - df = self.df[self.df[variable_id].isin(instance_vals)] + if isinstance(instance_vals, dd.Series): + df = self.df.merge(instance_vals.to_frame(), how="inner", on=variable_id) + else: + df = self.df[self.df[variable_id].isin(instance_vals)] - df = df.set_index(self.index, drop=False) + if isinstance(self.df, pd.DataFrame): + df = df.set_index(self.index, drop=False) # ensure filtered df has same categories as original # workaround for issue below @@ -409,7 +414,7 @@ def delete_variables(self, variable_ids): Remove variables from entity's dataframe and from self.variables """ - self.df.drop(variable_ids, axis=1, inplace=True) + self.df = self.df.drop(variable_ids, axis=1) for v_id in variable_ids: v = self._get_variable(v_id) @@ -417,12 +422,12 @@ def delete_variables(self, variable_ids): def set_time_index(self, variable_id, already_sorted=False): # check time type - if self.df.empty: + if isinstance(self.df, dd.DataFrame) or self.df.empty: time_to_check = vtypes.DEFAULT_DTYPE_VALUES[self[variable_id]._default_pandas_dtype] else: time_to_check = self.df[variable_id].iloc[0] - time_type = _check_time_type(time_to_check) + if time_type is None: raise TypeError("%s time index not recognized as numeric or" " datetime" % (self.id)) @@ -434,14 +439,19 @@ def set_time_index(self, variable_id, already_sorted=False): " other entityset time indexes" % (self.id, time_type)) + if isinstance(self.df, dd.DataFrame): + t = time_type # skip checking values + already_sorted = True # skip sorting + else: + t = vtypes.NumericTimeIndex + if col_is_datetime(self.df[variable_id]): + t = vtypes.DatetimeTimeIndex + # use stable sort if not already_sorted: # sort by time variable, then by index - self.df.sort_values([variable_id, self.index], inplace=True) + self.df = self.df.sort_values([variable_id, self.index]) - t = vtypes.NumericTimeIndex - if col_is_datetime(self.df[variable_id]): - t = vtypes.DatetimeTimeIndex self.convert_variable_type(variable_id, t, convert_data=False) self.time_index = variable_id @@ -452,20 +462,22 @@ def set_index(self, variable_id, unique=True): variable_id (string) : Name of an existing variable to set as index. unique (bool) : Whether to assert that the index is unique. """ - self.df = self.df.set_index(self.df[variable_id], drop=False) - self.df.index.name = None - if unique: - assert self.df.index.is_unique, "Index is not unique on dataframe (Entity {})".format(self.id) + if isinstance(self.df, pd.DataFrame): + self.df = self.df.set_index(self.df[variable_id], drop=False) + self.df.index.name = None + if unique: + assert self.df.index.is_unique, "Index is not unique on dataframe " \ + "(Entity {})".format(self.id) self.convert_variable_type(variable_id, vtypes.Index, convert_data=False) self.index = variable_id def set_secondary_time_index(self, secondary_time_index): for time_index, columns in secondary_time_index.items(): - if self.df.empty: + if len(self.df) == 0: time_to_check = vtypes.DEFAULT_DTYPE_VALUES[self[time_index]._default_pandas_dtype] else: - time_to_check = self.df[time_index].iloc[0] + time_to_check = self.df[time_index].head(1).iloc[0] time_type = _check_time_type(time_to_check) if time_type is None: raise TypeError("%s time index not recognized as numeric or" @@ -494,7 +506,7 @@ def _vals_to_series(self, instance_vals, variable_id): # convert iterable to pd.Series if type(instance_vals) == pd.DataFrame: out_vals = instance_vals[variable_id] - elif type(instance_vals) == pd.Series: + elif type(instance_vals) == pd.Series or type(instance_vals) == dd.Series: out_vals = instance_vals.rename(variable_id) else: out_vals = pd.Series(instance_vals) @@ -514,7 +526,8 @@ def _handle_time(self, df, time_last=None, training_window=None, include_cutoff_ dataframe. """ if self.time_index: - if time_last is not None and not df.empty: + df_empty = df.empty if isinstance(df, pd.DataFrame) else False + if time_last is not None and not df_empty: if include_cutoff_time: df = df[df[self.time_index] <= time_last] else: @@ -537,13 +550,19 @@ def _handle_time(self, df, time_last=None, training_window=None, include_cutoff_ "Using training_window but last_time_index is " "not set on entity %s" % (self.id) ) + df = df[mask] for secondary_time_index, columns in self.secondary_time_index.items(): # should we use ignore time last here? - if time_last is not None and not df.empty: + df_empty = df.empty if isinstance(df, pd.DataFrame) else False + if time_last is not None and not df_empty: mask = df[secondary_time_index] >= time_last - df.loc[mask, columns] = np.nan + if isinstance(df, dd.DataFrame): + for col in columns: + df[col] = df[col].mask(mask, np.nan) + else: + df.loc[mask, columns] = np.nan return df @@ -570,7 +589,11 @@ def _create_index(index, make_index, df): "integer column", index) # Case 5: make_index with no errors or warnings # (Case 4 also uses this code path) - df.insert(0, index, range(0, len(df))) + if isinstance(df, dd.DataFrame): + df[index] = 1 + df[index] = df[index].cumsum() - 1 + else: + df.insert(0, index, range(len(df))) created_index = index # Case 6: user specified index, which is already in df. No action needed. return created_index, index, df diff --git a/featuretools/entityset/entityset.py b/featuretools/entityset/entityset.py index 346c3cffe5..7904f89e35 100644 --- a/featuretools/entityset/entityset.py +++ b/featuretools/entityset/entityset.py @@ -2,6 +2,7 @@ import logging from collections import defaultdict +import dask.dataframe as dd import numpy as np import pandas as pd from pandas.api.types import is_dtype_equal, is_numeric_dtype @@ -258,8 +259,9 @@ def add_relationship(self, relationship): # default to object dtypes for discrete variables, but # indexes/ids default to ints. In this case, we convert # the empty column's type to int - if (child_e.df.empty and child_e.df[child_v].dtype == object and - is_numeric_dtype(parent_e.df[parent_v])): + if isinstance(child_e.df, pd.DataFrame) and \ + (child_e.df.empty and child_e.df[child_v].dtype == object and + is_numeric_dtype(parent_e.df[parent_v])): child_e.df[child_v] = pd.Series(name=child_v, dtype=np.int64) parent_dtype = parent_e.df[parent_v].dtype @@ -488,6 +490,12 @@ def entity_from_dataframe(self, if variable_type == vtypes.DatetimeTimeIndex: raise ValueError("DatetimeTimeIndex variable %s must be set using time_index parameter" % (variable)) + if len(self.entities) > 0: + if not isinstance(dataframe, type(self.entities[0].df)): + raise ValueError("All entity dataframes must be of the same type. " + "Cannot add entity of type {} to an entityset with existing entities " + "of type {}".format(type(dataframe), type(self.entities[0].df))) + entity = Entity( entity_id, dataframe, @@ -634,7 +642,7 @@ def normalize_entity(self, base_entity_id, new_entity_id, index, drop_duplicates(index, keep='first')[selected_variables] if make_time_index: - new_entity_df2.rename(columns={base_time_index: new_entity_time_index}, inplace=True) + new_entity_df2 = new_entity_df2.rename(columns={base_time_index: new_entity_time_index}) if make_secondary_time_index: assert len(make_secondary_time_index) == 1, "Can only provide 1 secondary time index" secondary_time_index = list(make_secondary_time_index.keys())[0] @@ -643,12 +651,11 @@ def normalize_entity(self, base_entity_id, new_entity_id, index, secondary_df = new_entity_df. \ drop_duplicates(index, keep='last')[secondary_variables] if new_entity_secondary_time_index: - secondary_df.rename(columns={secondary_time_index: new_entity_secondary_time_index}, - inplace=True) + secondary_df = secondary_df.rename(columns={secondary_time_index: new_entity_secondary_time_index}) secondary_time_index = new_entity_secondary_time_index else: new_entity_secondary_time_index = secondary_time_index - secondary_df.set_index(index, inplace=True) + secondary_df = secondary_df.set_index(index) new_entity_df = new_entity_df2.join(secondary_df, on=index) else: new_entity_df = new_entity_df2 @@ -786,9 +793,17 @@ def add_last_time_indexes(self, updated_entities=None): if entity.last_time_index is None: if entity.time_index is not None: lti = entity.df[entity.time_index].copy() + if isinstance(entity.df, dd.DataFrame): + # The current Dask implementation doesn't set the index of the dataframe + # to the entity's index, so we have to do it manually here + lti.index = entity.df[entity.index].copy() else: lti = entity.df[entity.index].copy() - lti[:] = None + if isinstance(entity.df, dd.DataFrame): + lti.index = entity.df[entity.index].copy() + lti = lti.apply(lambda x: None) + else: + lti[:] = None entity.last_time_index = lti if entity.id in children: @@ -813,21 +828,45 @@ def add_last_time_indexes(self, updated_entities=None): continue link_var = child_vars[entity.id][child_e.id].id - lti_df = pd.DataFrame({'last_time': child_e.last_time_index, - entity.index: child_e.df[link_var]}) - - # sort by time and keep only the most recent - lti_df.sort_values(['last_time', entity.index], - kind="mergesort", inplace=True) - - lti_df.drop_duplicates(entity.index, - keep='last', - inplace=True) + if isinstance(child_e.last_time_index, dd.Series): + to_join = child_e.df[link_var] + to_join.index = child_e.df[child_e.index] + + lti_df = child_e.last_time_index.to_frame(name='last_time').join( + to_join.to_frame(name=entity.index) + ) + new_index = lti_df.index.copy() + new_index.name = None + lti_df.index = new_index + lti_df = lti_df.groupby(lti_df[entity.index]).agg('max') + + lti_df = entity.last_time_index.to_frame(name='last_time_old').join(lti_df) + + else: + lti_df = pd.DataFrame({'last_time': child_e.last_time_index, + entity.index: child_e.df[link_var]}) + + # sort by time and keep only the most recent + lti_df.sort_values(['last_time', entity.index], + kind="mergesort", inplace=True) + + lti_df.drop_duplicates(entity.index, + keep='last', + inplace=True) + + lti_df.set_index(entity.index, inplace=True) + lti_df = lti_df.reindex(entity.last_time_index.index) + lti_df['last_time_old'] = entity.last_time_index + if not isinstance(lti_df, dd.DataFrame) and lti_df.empty: + # Pandas errors out if it tries to do fillna and then max on an empty dataframe + lti_df = pd.Series() + else: + lti_df['last_time'] = lti_df['last_time'].astype('datetime64[ns]') + lti_df['last_time_old'] = lti_df['last_time_old'].astype('datetime64[ns]') + lti_df = lti_df.fillna(pd.to_datetime('1800-01-01 00:00')).max(axis=1) + lti_df = lti_df.replace(pd.to_datetime('1800-01-01 00:00'), pd.NaT) + # lti_df = lti_df.apply(lambda x: x.dropna().max(), axis=1) - lti_df.set_index(entity.index, inplace=True) - lti_df = lti_df.reindex(entity.last_time_index.index) - lti_df['last_time_old'] = entity.last_time_index - lti_df = lti_df.apply(lambda x: x.dropna().max(), axis=1) entity.last_time_index = lti_df entity.last_time_index.name = 'last_time' diff --git a/featuretools/entityset/serialize.py b/featuretools/entityset/serialize.py index 3c9758aba2..9b9432d580 100644 --- a/featuretools/entityset/serialize.py +++ b/featuretools/entityset/serialize.py @@ -4,6 +4,8 @@ import tarfile import tempfile +import dask.dataframe as dd + from featuretools.utils.s3_utils import get_transport_params, use_smartopen_es from featuretools.utils.wrangle import _is_s3, _is_url @@ -22,6 +24,10 @@ def entity_to_description(entity): ''' index = entity.df.columns.isin([variable.id for variable in entity.variables]) dtypes = entity.df[entity.df.columns[index]].dtypes.astype(str).to_dict() + if isinstance(entity.df, dd.DataFrame): + entity_type = 'dask' + else: + entity_type = 'pandas' description = { "id": entity.id, "index": entity.index, @@ -32,6 +38,7 @@ def entity_to_description(entity): }, "variables": [variable.to_data_description() for variable in entity.variables], "loading_info": { + 'entity_type': entity_type, 'params': {}, 'properties': { 'dtypes': dtypes @@ -76,11 +83,16 @@ def write_entity_data(entity, path, format='csv', **kwargs): loading_info (dict) : Information on storage location and format of entity data. ''' format = format.lower() - basename = '.'.join([entity.id, format]) + if isinstance(entity.df, dd.DataFrame) and format == 'csv': + basename = "{}-*.{}".format(entity.id, format) + else: + basename = '.'.join([entity.id, format]) location = os.path.join('data', basename) file = os.path.join(path, location) + df = entity.df + if format == 'csv': - entity.df.to_csv( + df.to_csv( file, index=kwargs['index'], sep=kwargs['sep'], @@ -91,12 +103,17 @@ def write_entity_data(entity, path, format='csv', **kwargs): # Serializing to parquet format raises an error when columns contain tuples. # Columns containing tuples are mapped as dtype object. # Issue is resolved by casting columns of dtype object to string. - df = entity.df.copy() - columns = df.select_dtypes('object').columns + df = df.copy() + columns = list(df.select_dtypes('object').columns) df[columns] = df[columns].astype('unicode') df.to_parquet(file, **kwargs) elif format == 'pickle': - entity.df.to_pickle(file, **kwargs) + # Dask currently does not support to_pickle + if isinstance(df, dd.DataFrame): + msg = 'Cannot serialize Dask EntitySet to pickle' + raise ValueError(msg) + else: + df.to_pickle(file, **kwargs) else: error = 'must be one of the following formats: {}' raise ValueError(error.format(', '.join(FORMATS))) diff --git a/featuretools/feature_base/feature_base.py b/featuretools/feature_base/feature_base.py index bb5b922b05..3d54a78510 100644 --- a/featuretools/feature_base/feature_base.py +++ b/featuretools/feature_base/feature_base.py @@ -634,6 +634,9 @@ def get_arguments(self): 'use_previous': self.use_previous and self.use_previous.get_arguments(), } + def get_dask_aggregation(self): + return self.primitive.get_dask_aggregation() + def relationship_path_name(self): if self._path_is_unique: return self.child_entity.id diff --git a/featuretools/primitives/base/aggregation_primitive_base.py b/featuretools/primitives/base/aggregation_primitive_base.py index 07e76f7c0f..a23321b26f 100755 --- a/featuretools/primitives/base/aggregation_primitive_base.py +++ b/featuretools/primitives/base/aggregation_primitive_base.py @@ -35,6 +35,9 @@ def generate_names(self, base_feature_names, relationship_path_name, use_prev_str) return [base_name + "[%s]" % i for i in range(n)] + def get_dask_aggregation(self): + raise NotImplementedError("Subclass must implement") + def make_agg_primitive(function, input_types, return_type, name=None, stack_on_self=True, stack_on=None, diff --git a/featuretools/primitives/base/primitive_base.py b/featuretools/primitives/base/primitive_base.py index 3a5f680e72..33af37fc8e 100644 --- a/featuretools/primitives/base/primitive_base.py +++ b/featuretools/primitives/base/primitive_base.py @@ -31,6 +31,8 @@ class PrimitiveBase(object): base_of_exclude = None # (bool) If True will only make one feature per unique set of base features commutative = False + # (bool) If True, is compatible with Dask EntitySets + dask_compatible = False def __init__(self): pass diff --git a/featuretools/primitives/standard/aggregation_primitives.py b/featuretools/primitives/standard/aggregation_primitives.py index c6ab0b6702..c8889f358f 100644 --- a/featuretools/primitives/standard/aggregation_primitives.py +++ b/featuretools/primitives/standard/aggregation_primitives.py @@ -2,6 +2,7 @@ import numpy as np import pandas as pd +from dask import dataframe as dd from scipy import stats from featuretools.primitives.base.aggregation_primitive_base import ( @@ -32,10 +33,14 @@ class Count(AggregationPrimitive): return_type = Numeric stack_on_self = False default_value = 0 + dask_compatible = True def get_function(self): return pd.Series.count + def get_dask_aggregation(self): + return 'count' + def generate_name(self, base_feature_names, relationship_path_name, parent_entity_id, where_str, use_prev_str): return u"COUNT(%s%s%s)" % (relationship_path_name, @@ -56,10 +61,14 @@ class Sum(AggregationPrimitive): stack_on_self = False stack_on_exclude = [Count] default_value = 0 + dask_compatible = True def get_function(self): return np.sum + def get_dask_aggregation(self): + return 'sum' + class Mean(AggregationPrimitive): """Computes the average for a list of values. @@ -82,6 +91,7 @@ class Mean(AggregationPrimitive): name = "mean" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, skipna=True): self.skipna = skipna @@ -96,6 +106,9 @@ def mean(series): return mean + def get_dask_aggregation(self): + return 'mean' + class Mode(AggregationPrimitive): """Determines the most commonly repeated value. @@ -133,10 +146,14 @@ class Min(AggregationPrimitive): input_types = [Numeric] return_type = Numeric stack_on_self = False + dask_compatible = True def get_function(self): return np.min + def get_dask_aggregation(self): + return 'min' + class Max(AggregationPrimitive): """Calculates the highest value, ignoring `NaN` values. @@ -150,10 +167,14 @@ class Max(AggregationPrimitive): input_types = [Numeric] return_type = Numeric stack_on_self = False + dask_compatible = True def get_function(self): return np.max + def get_dask_aggregation(self): + return 'max' + class NumUnique(AggregationPrimitive): """Determines the number of distinct values, ignoring `NaN` values. @@ -172,10 +193,31 @@ class NumUnique(AggregationPrimitive): input_types = [Discrete] return_type = Numeric stack_on_self = False + dask_compatible = True def get_function(self): return pd.Series.nunique + def get_dask_aggregation(self): + def chunk(s): + def inner_chunk(x): + x = x[:].dropna() + return set(x.unique()) + + return s.agg(inner_chunk) + + def agg(s): + def inner_agg(x): + x = x[:].dropna() + return(set().union(*x.values)) + + return s.agg(inner_agg) + + def finalize(s): + return s.apply(lambda x: len(x)) + + return dd.Aggregation(self.name, chunk=chunk, agg=agg, finalize=finalize) + class NumTrue(AggregationPrimitive): """Counts the number of `True` values. @@ -195,10 +237,23 @@ class NumTrue(AggregationPrimitive): default_value = 0 stack_on = [] stack_on_exclude = [] + dask_compatible = True def get_function(self): return np.sum + def get_dask_aggregation(self): + def chunk(s): + chunk_sum = s.agg(np.sum) + if chunk_sum.dtype == 'bool': + chunk_sum = chunk_sum.astype('int64') + return chunk_sum + + def agg(s): + return s.agg(np.sum) + + return dd.Aggregation(self.name, chunk=chunk, agg=agg) + class PercentTrue(AggregationPrimitive): """Determines the percent of `True` values. @@ -220,6 +275,7 @@ class PercentTrue(AggregationPrimitive): stack_on = [] stack_on_exclude = [] default_value = 0 + dask_compatible = True def get_function(self): def percent_true(s): @@ -227,6 +283,27 @@ def percent_true(s): return percent_true + def get_dask_aggregation(self): + def chunk(s): + def format_chunk(x): + return x[:].fillna(0) + + chunk_sum = s.agg(lambda x: format_chunk(x).sum()) + chunk_len = s.agg(lambda x: len(format_chunk(x))) + if chunk_sum.dtype == 'bool': + chunk_sum = chunk_sum.astype('int64') + if chunk_len.dtype == 'bool': + chunk_len = chunk_len.astype('int64') + return (chunk_sum, chunk_len) + + def agg(val, length): + return (val.sum(), length.sum()) + + def finalize(total, length): + return total / length + + return dd.Aggregation(self.name, chunk=chunk, agg=agg, finalize=finalize) + class NMostCommon(AggregationPrimitive): """Determines the `n` most common elements. @@ -385,10 +462,14 @@ class Std(AggregationPrimitive): input_types = [Numeric] return_type = Numeric stack_on_self = False + dask_compatible = True def get_function(self): return np.std + def get_dask_aggregation(self): + return 'std' + class First(AggregationPrimitive): """Determines the first value in a list. @@ -446,10 +527,20 @@ class Any(AggregationPrimitive): input_types = [Boolean] return_type = Boolean stack_on_self = False + dask_compatible = True def get_function(self): return np.any + def get_dask_aggregation(self): + def chunk(s): + return s.agg(np.any) + + def agg(s): + return s.agg(np.any) + + return dd.Aggregation(self.name, chunk=chunk, agg=agg) + class All(AggregationPrimitive): """Calculates if all values are 'True' in a list. @@ -467,10 +558,20 @@ class All(AggregationPrimitive): input_types = [Boolean] return_type = Boolean stack_on_self = False + dask_compatible = True def get_function(self): return np.all + def get_dask_aggregation(self): + def chunk(s): + return s.agg(np.all) + + def agg(s): + return s.agg(np.all) + + return dd.Aggregation(self.name, chunk=chunk, agg=agg) + class TimeSinceLast(AggregationPrimitive): """Calculates the time elapsed since the last datetime (default in seconds). diff --git a/featuretools/primitives/standard/binary_transform.py b/featuretools/primitives/standard/binary_transform.py index 98dae7d8aa..9035f8c116 100644 --- a/featuretools/primitives/standard/binary_transform.py +++ b/featuretools/primitives/standard/binary_transform.py @@ -29,6 +29,7 @@ class GreaterThan(TransformPrimitive): name = "greater_than" input_types = [[Numeric, Numeric], [Datetime, Datetime], [Ordinal, Ordinal]] return_type = Boolean + dask_compatible = True def get_function(self): return np.greater @@ -53,14 +54,14 @@ class GreaterThanScalar(TransformPrimitive): name = "greater_than_scalar" input_types = [[Numeric], [Datetime], [Ordinal]] return_type = Boolean + dask_compatible = True def __init__(self, value=0): self.value = value def get_function(self): def greater_than_scalar(vals): - # convert series to handle both numeric and datetime case - return pd.Series(vals) > self.value + return vals > self.value return greater_than_scalar def generate_name(self, base_feature_names): @@ -83,6 +84,7 @@ class GreaterThanEqualTo(TransformPrimitive): name = "greater_than_equal_to" input_types = [[Numeric, Numeric], [Datetime, Datetime], [Ordinal, Ordinal]] return_type = Boolean + dask_compatible = True def get_function(self): return np.greater_equal @@ -107,14 +109,14 @@ class GreaterThanEqualToScalar(TransformPrimitive): name = "greater_than_equal_to_scalar" input_types = [[Numeric], [Datetime], [Ordinal]] return_type = Boolean + dask_compatible = True def __init__(self, value=0): self.value = value def get_function(self): def greater_than_equal_to_scalar(vals): - # convert series to handle both numeric and datetime case - return pd.Series(vals) >= self.value + return vals >= self.value return greater_than_equal_to_scalar def generate_name(self, base_feature_names): @@ -137,6 +139,7 @@ class LessThan(TransformPrimitive): name = "less_than" input_types = [[Numeric, Numeric], [Datetime, Datetime], [Ordinal, Ordinal]] return_type = Boolean + dask_compatible = True def get_function(self): return np.less @@ -161,14 +164,14 @@ class LessThanScalar(TransformPrimitive): name = "less_than_scalar" input_types = [[Numeric], [Datetime], [Ordinal]] return_type = Boolean + dask_compatible = True def __init__(self, value=0): self.value = value def get_function(self): def less_than_scalar(vals): - # convert series to handle both numeric and datetime case - return pd.Series(vals) < self.value + return vals < self.value return less_than_scalar def generate_name(self, base_feature_names): @@ -191,6 +194,7 @@ class LessThanEqualTo(TransformPrimitive): name = "less_than_equal_to" input_types = [[Numeric, Numeric], [Datetime, Datetime], [Ordinal, Ordinal]] return_type = Boolean + dask_compatible = True def get_function(self): return np.less_equal @@ -215,14 +219,14 @@ class LessThanEqualToScalar(TransformPrimitive): name = "less_than_equal_to_scalar" input_types = [[Numeric], [Datetime], [Ordinal]] return_type = Boolean + dask_compatible = True def __init__(self, value=0): self.value = value def get_function(self): def less_than_equal_to_scalar(vals): - # convert series to handle both numeric and datetime case - return pd.Series(vals) <= self.value + return vals <= self.value return less_than_equal_to_scalar def generate_name(self, base_feature_names): @@ -277,13 +281,14 @@ class EqualScalar(TransformPrimitive): name = "equal_scalar" input_types = [Variable] return_type = Boolean + dask_compatible = True def __init__(self, value=None): self.value = value def get_function(self): def equal_scalar(vals): - return pd.Series(vals) == self.value + return vals == self.value return equal_scalar def generate_name(self, base_feature_names): @@ -338,13 +343,14 @@ class NotEqualScalar(TransformPrimitive): name = "not_equal_scalar" input_types = [Variable] return_type = Boolean + dask_compatible = True def __init__(self, value=None): self.value = value def get_function(self): def not_equal_scalar(vals): - return pd.Series(vals) != self.value + return vals != self.value return not_equal_scalar def generate_name(self, base_feature_names): @@ -368,6 +374,7 @@ class AddNumeric(TransformPrimitive): input_types = [Numeric, Numeric] return_type = Numeric commutative = True + dask_compatible = True def get_function(self): return np.add @@ -391,6 +398,7 @@ class AddNumericScalar(TransformPrimitive): name = "add_numeric_scalar" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=0): self.value = value @@ -425,6 +433,7 @@ class SubtractNumeric(TransformPrimitive): name = "subtract_numeric" input_types = [Numeric, Numeric] return_type = Numeric + dask_compatible = True def __init__(self, commutative=True): self.commutative = commutative @@ -451,6 +460,7 @@ class SubtractNumericScalar(TransformPrimitive): name = "subtract_numeric_scalar" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=0): self.value = value @@ -480,6 +490,7 @@ class ScalarSubtractNumericFeature(TransformPrimitive): name = "scalar_subtract_numeric_feature" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=0): self.value = value @@ -514,6 +525,7 @@ class MultiplyNumeric(TransformPrimitive): ] return_type = Numeric commutative = True + dask_compatible = True def get_function(self): return np.multiply @@ -537,6 +549,7 @@ class MultiplyNumericScalar(TransformPrimitive): name = "multiply_numeric_scalar" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=1): self.value = value @@ -568,6 +581,7 @@ class MultiplyBoolean(TransformPrimitive): return_type = Boolean commutative = True + dask_compatible = True def get_function(self): return np.bitwise_and @@ -597,6 +611,7 @@ class DivideNumeric(TransformPrimitive): name = "divide_numeric" input_types = [Numeric, Numeric] return_type = Numeric + dask_compatible = True def __init__(self, commutative=False): self.commutative = commutative @@ -623,6 +638,7 @@ class DivideNumericScalar(TransformPrimitive): name = "divide_numeric_scalar" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=1): self.value = value @@ -652,6 +668,7 @@ class DivideByFeature(TransformPrimitive): name = "divide_by_feature" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=1): self.value = value @@ -681,6 +698,7 @@ class ModuloNumeric(TransformPrimitive): name = "modulo_numeric" input_types = [Numeric, Numeric] return_type = Numeric + dask_compatible = True def get_function(self): return np.mod @@ -705,6 +723,7 @@ class ModuloNumericScalar(TransformPrimitive): name = "modulo_numeric_scalar" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=1): self.value = value @@ -734,6 +753,7 @@ class ModuloByFeature(TransformPrimitive): name = "modulo_by_feature" input_types = [Numeric] return_type = Numeric + dask_compatible = True def __init__(self, value=1): self.value = value @@ -764,6 +784,7 @@ class And(TransformPrimitive): input_types = [Boolean, Boolean] return_type = Boolean commutative = True + dask_compatible = True def get_function(self): return np.logical_and @@ -789,6 +810,7 @@ class Or(TransformPrimitive): input_types = [Boolean, Boolean] return_type = Boolean commutative = True + dask_compatible = True def get_function(self): return np.logical_or diff --git a/featuretools/primitives/standard/transform_primitive.py b/featuretools/primitives/standard/transform_primitive.py index ad36a08c75..0cbf91cff4 100644 --- a/featuretools/primitives/standard/transform_primitive.py +++ b/featuretools/primitives/standard/transform_primitive.py @@ -1,3 +1,4 @@ +import dask.dataframe as dd import numpy as np import pandas as pd @@ -29,9 +30,15 @@ class IsNull(TransformPrimitive): name = "is_null" input_types = [Variable] return_type = Boolean + dask_compatible = True def get_function(self): - return lambda array: pd.isnull(pd.Series(array)) + def isnull(array): + if isinstance(array, dd.Series): + return dd.Series.isnull(array) + else: + return pd.isnull(pd.Series(array)) + return isnull class Absolute(TransformPrimitive): @@ -45,6 +52,7 @@ class Absolute(TransformPrimitive): name = "absolute" input_types = [Numeric] return_type = Numeric + dask_compatible = True def get_function(self): return np.absolute @@ -102,9 +110,12 @@ class Day(TransformPrimitive): name = "day" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def day(vals): + if isinstance(vals, dd.Series): + return vals.dt.day return pd.DatetimeIndex(vals).day.values return day @@ -124,9 +135,12 @@ class Hour(TransformPrimitive): name = "hour" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def hour(vals): + if isinstance(vals, dd.Series): + return vals.dt.hour return pd.DatetimeIndex(vals).hour.values return hour @@ -146,9 +160,12 @@ class Second(TransformPrimitive): name = "second" input_types = [Datetime] return_type = Numeric + dask_compatible = True def get_function(self): def second(vals): + if isinstance(vals, dd.Series): + return vals.dt.second return pd.DatetimeIndex(vals).second.values return second @@ -168,9 +185,12 @@ class Minute(TransformPrimitive): name = "minute" input_types = [Datetime] return_type = Numeric + dask_compatible = True def get_function(self): def minute(vals): + if isinstance(vals, dd.Series): + return vals.dt.minute return pd.DatetimeIndex(vals).minute.values return minute @@ -195,9 +215,12 @@ class Week(TransformPrimitive): name = "week" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def week(vals): + if isinstance(vals, dd.Series): + return vals.dt.week return pd.DatetimeIndex(vals).week.values return week @@ -217,9 +240,12 @@ class Month(TransformPrimitive): name = "month" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def month(vals): + if isinstance(vals, dd.Series): + return vals.dt.month return pd.DatetimeIndex(vals).month.values return month @@ -239,9 +265,12 @@ class Year(TransformPrimitive): name = "year" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def year(vals): + if isinstance(vals, dd.Series): + return vals.dt.year return pd.DatetimeIndex(vals).year.values return year @@ -261,9 +290,12 @@ class IsWeekend(TransformPrimitive): name = "is_weekend" input_types = [Datetime] return_type = Boolean + dask_compatible = True def get_function(self): def is_weekend(vals): + if isinstance(vals, dd.Series): + return vals.dt.weekday > 4 return pd.DatetimeIndex(vals).weekday.values > 4 return is_weekend @@ -287,9 +319,12 @@ class Weekday(TransformPrimitive): name = "weekday" input_types = [Datetime] return_type = Ordinal + dask_compatible = True def get_function(self): def weekday(vals): + if isinstance(vals, dd.Series): + return vals.dt.weekday return pd.DatetimeIndex(vals).weekday.values return weekday @@ -307,9 +342,12 @@ class NumCharacters(TransformPrimitive): name = 'num_characters' input_types = [Text] return_type = Numeric + dask_compatible = True def get_function(self): - return lambda array: pd.Series(array).fillna('').str.len() + def character_counter(array): + return array.fillna('').str.len() + return character_counter class NumWords(TransformPrimitive): @@ -326,10 +364,11 @@ class NumWords(TransformPrimitive): name = 'num_words' input_types = [Text] return_type = Numeric + dask_compatible = True def get_function(self): def word_counter(array): - return pd.Series(array).fillna('').str.count(' ') + 1 + return array.fillna('').str.count(' ') + 1 return word_counter @@ -368,13 +407,16 @@ class TimeSince(TransformPrimitive): input_types = [[DatetimeTimeIndex], [Datetime]] return_type = Numeric uses_calc_time = True + dask_compatible = True def __init__(self, unit="seconds"): self.unit = unit.lower() def get_function(self): def pd_time_since(array, time): - return convert_time_units((time - pd.DatetimeIndex(array)).total_seconds(), self.unit) + if isinstance(array, list): + array = pd.Series(array) + return convert_time_units((time - array).dt.total_seconds(), self.unit) return pd_time_since @@ -390,13 +432,17 @@ class IsIn(TransformPrimitive): name = "isin" input_types = [Variable] return_type = Boolean + dask_compatible = True def __init__(self, list_of_outputs=None): self.list_of_outputs = list_of_outputs def get_function(self): def pd_is_in(array): - return pd.Series(array).isin(self.list_of_outputs or []) + if isinstance(array, dd.Series): + return array.isin(self.list_of_outputs or []) + else: + return pd.Series(array).isin(self.list_of_outputs or []) return pd_is_in def generate_name(self, base_feature_names): @@ -442,6 +488,7 @@ class Negate(TransformPrimitive): name = "negate" input_types = [Numeric] return_type = Numeric + dask_compatible = True def get_function(self): def negate(vals): @@ -463,6 +510,7 @@ class Not(TransformPrimitive): name = "not" input_types = [Boolean] return_type = Boolean + dask_compatible = True def generate_name(self, base_feature_names): return u"NOT({})".format(base_feature_names[0]) diff --git a/featuretools/primitives/utils.py b/featuretools/primitives/utils.py index 2c26ed6d3d..3db0fdb7f7 100644 --- a/featuretools/primitives/utils.py +++ b/featuretools/primitives/utils.py @@ -39,16 +39,20 @@ def get_transform_primitives(): def list_primitives(): trans_names, trans_primitives = _get_names_primitives(get_transform_primitives) + trans_dask = [primitive.dask_compatible for primitive in trans_primitives] transform_df = pd.DataFrame({'name': trans_names, - 'description': _get_descriptions(trans_primitives)}) + 'description': _get_descriptions(trans_primitives), + 'dask_compatible': trans_dask}) transform_df['type'] = 'transform' agg_names, agg_primitives = _get_names_primitives(get_aggregation_primitives) + agg_dask = [primitive.dask_compatible for primitive in agg_primitives] agg_df = pd.DataFrame({'name': agg_names, - 'description': _get_descriptions(agg_primitives)}) + 'description': _get_descriptions(agg_primitives), + 'dask_compatible': agg_dask}) agg_df['type'] = 'aggregation' - return pd.concat([agg_df, transform_df], ignore_index=True)[['name', 'type', 'description']] + return pd.concat([agg_df, transform_df], ignore_index=True)[['name', 'type', 'dask_compatible', 'description']] def get_default_aggregation_primitives(): diff --git a/featuretools/synthesis/deep_feature_synthesis.py b/featuretools/synthesis/deep_feature_synthesis.py index 36b2416afe..711256e72c 100644 --- a/featuretools/synthesis/deep_feature_synthesis.py +++ b/featuretools/synthesis/deep_feature_synthesis.py @@ -1,6 +1,8 @@ import logging from collections import defaultdict +from dask import dataframe as dd + from featuretools import primitives, variable_types from featuretools.entityset.relationship import RelationshipPath from featuretools.feature_base import ( @@ -178,6 +180,8 @@ def __init__(self, if agg_primitives is None: agg_primitives = primitives.get_default_aggregation_primitives() + if any(isinstance(e.df, dd.DataFrame) for e in self.es.entities): + agg_primitives = [p for p in agg_primitives if p.dask_compatible] self.agg_primitives = [] agg_prim_dict = primitives.get_aggregation_primitives() for a in agg_primitives: @@ -195,6 +199,8 @@ def __init__(self, if trans_primitives is None: trans_primitives = primitives.get_default_transform_primitives() + if any(isinstance(e.df, dd.DataFrame) for e in self.es.entities): + trans_primitives = [p for p in trans_primitives if p.dask_compatible] self.trans_primitives = [] for t in trans_primitives: t = check_trans_primitive(t) @@ -225,6 +231,11 @@ def __init__(self, primitive_options = {} all_primitives = self.trans_primitives + self.agg_primitives + \ self.where_primitives + self.groupby_trans_primitives + if any(isinstance(entity.df, dd.DataFrame) for entity in self.es.entities): + if not all([primitive.dask_compatible for primitive in all_primitives]): + bad_primitives = ", ".join([prim.name for prim in all_primitives if not prim.dask_compatible]) + raise ValueError('Selected primitives are incompatible with Dask EntitySets: {}'.format(bad_primitives)) + self.primitive_options, self.ignore_entities, self.ignore_variables =\ generate_all_primitive_options(all_primitives, primitive_options, diff --git a/featuretools/synthesis/dfs.py b/featuretools/synthesis/dfs.py index 77ad5c644e..54515f98da 100644 --- a/featuretools/synthesis/dfs.py +++ b/featuretools/synthesis/dfs.py @@ -1,4 +1,7 @@ +import warnings + import pandas as pd +from dask import dataframe as dd from featuretools.computational_backends import calculate_feature_matrix from featuretools.entityset import EntitySet @@ -252,6 +255,12 @@ def dfs(entities=None, if features_only: return features + if isinstance(cutoff_time, dd.DataFrame): + msg = "cutoff_time should be a Pandas DataFrame: "\ + "computing cutoff_time, this may take a while" + warnings.warn(msg) + cutoff_time = cutoff_time.compute() + if isinstance(cutoff_time, pd.DataFrame): feature_matrix = calculate_feature_matrix(features, entityset=entityset, diff --git a/featuretools/synthesis/encode_features.py b/featuretools/synthesis/encode_features.py index bee6f10d30..f6caa1de0c 100644 --- a/featuretools/synthesis/encode_features.py +++ b/featuretools/synthesis/encode_features.py @@ -74,6 +74,10 @@ def encode_features(feature_matrix, features, top_n=DEFAULT_TOP_N, include_unkno drop_first=True) f_encoded """ + if not isinstance(feature_matrix, pd.DataFrame): + msg = "feature_matrix must be a Pandas DataFrame" + raise TypeError(msg) + if inplace: X = feature_matrix else: diff --git a/featuretools/tests/computational_backend/test_calculate_feature_matrix.py b/featuretools/tests/computational_backend/test_calculate_feature_matrix.py index f43b8d4fff..38d6a35e44 100644 --- a/featuretools/tests/computational_backend/test_calculate_feature_matrix.py +++ b/featuretools/tests/computational_backend/test_calculate_feature_matrix.py @@ -1,3 +1,4 @@ +import copy import os import re import shutil @@ -5,10 +6,12 @@ from itertools import combinations from random import randint +import composeml as cp import numpy as np import pandas as pd import psutil import pytest +from dask import dataframe as dd from distributed.utils_test import cluster import featuretools as ft @@ -44,7 +47,10 @@ def test_scatter_warning(): assert len(record) == 1 +# TODO: final assert fails w/ Dask def test_calc_feature_matrix(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered') times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + [datetime(2011, 4, 9, 10, 40, 0)] + @@ -61,6 +67,8 @@ def test_calc_feature_matrix(es): es, cutoff_time=cutoff_time, verbose=True) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() assert (feature_matrix[property_feature.get_name()] == labels).values.all() @@ -104,10 +112,98 @@ def test_calc_feature_matrix(es): es, cutoff_time=cutoff_reordered, verbose=True) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() assert all(feature_matrix.index == cutoff_reordered["id"].values) + # fails with Dask entitysets, cutoff time not reordered; cannot verify out of order + # - can't tell if wrong/different all are false so can't check positional +def test_cfm_fails_dask_cutoff_time(es): + times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + + [datetime(2011, 4, 9, 10, 40, 0)] + + [datetime(2011, 4, 10, 10, 40, i) for i in range(2)] + + [datetime(2011, 4, 10, 10, 41, i * 3) for i in range(3)] + + [datetime(2011, 4, 10, 11, 10, i * 3) for i in range(2)]) + instances = range(17) + cutoff_time = pd.DataFrame({'time': times, + es['log'].index: instances}) + cutoff_time = dd.from_pandas(cutoff_time, npartitions=4) + + property_feature = ft.Feature(es['log']['value']) > 10 + + error_text = "cannot use Dask DataFrame for cutoff_time: "\ + "cutoff_time must a single value or a Pandas DataFrame" + with pytest.raises(TypeError, match=error_text): + calculate_feature_matrix([property_feature], + es, + cutoff_time=cutoff_time) + + +def test_cfm_compose(es): + def label_func(df): + return df['value'].sum() > 10 + + lm = cp.LabelMaker( + target_entity='id', + time_index='datetime', + labeling_function=label_func, + window_size='1m' + ) + + df = es['log'].df + if isinstance(df, dd.DataFrame): + df = df.compute() + labels = lm.search( + df, + num_examples_per_instance=-1 + ) + labels = labels.rename(columns={'cutoff_time': 'time'}) + + property_feature = ft.Feature(es['log']['value']) > 10 + + feature_matrix = calculate_feature_matrix([property_feature], + es, + cutoff_time=labels, + verbose=True) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() + + assert (feature_matrix[property_feature.get_name()] == + feature_matrix['label_func']).values.all() + + +def test_cfm_dask_compose(dask_es): + def label_func(df): + return df['value'].sum() > 10 + + lm = cp.LabelMaker( + target_entity='id', + time_index='datetime', + labeling_function=label_func, + window_size='3m' + ) + + labels = lm.search( + dask_es['log'].df.compute(), + num_examples_per_instance=-1 + ) + labels = labels.rename(columns={'cutoff_time': 'time'}) + + property_feature = ft.Feature(dask_es['log']['value']) > 10 + + feature_matrix = calculate_feature_matrix([property_feature], + dask_es, + cutoff_time=labels, + verbose=True) + feature_matrix = feature_matrix.compute() + + assert (feature_matrix[property_feature.get_name()] == feature_matrix['label_func']).values.all() + + +# tests approximate, skip for dask def test_cfm_approximate_correct_ordering(): trips = { 'trip_id': [i for i in range(1000)], @@ -155,16 +251,17 @@ def test_cfm_approximate_correct_ordering(): assert ((pd.isnull(x) and pd.isnull(y)) or (x == y)) -def test_cfm_no_cutoff_time_index(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat4 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat4, es['sessions']) +# uses approximate, skip for dask entitysets +def test_cfm_no_cutoff_time_index(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat4 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat4, pd_es['sessions']) cutoff_time = pd.DataFrame({ 'time': [datetime(2013, 4, 9, 10, 31, 19), datetime(2013, 4, 9, 11, 0, 0)], 'instance_id': [0, 2] }) feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, cutoff_time_in_index=False, approximate=Timedelta(12, 's'), cutoff_time=cutoff_time) @@ -178,7 +275,7 @@ def test_cfm_no_cutoff_time_index(es): 'instance_id': [0, 2] }) feature_matrix_2 = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, cutoff_time_in_index=False, approximate=Timedelta(10, 's'), cutoff_time=cutoff_time) @@ -188,7 +285,10 @@ def test_cfm_no_cutoff_time_index(es): assert feature_matrix_2[agg_feat.get_name()].tolist() == [5, 1] +# TODO: fails with dask entitysets def test_cfm_duplicated_index_in_cutoff_time(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered, missing duplicates') times = [datetime(2011, 4, 1), datetime(2011, 5, 1), datetime(2011, 4, 1), datetime(2011, 5, 1)] @@ -201,11 +301,16 @@ def test_cfm_duplicated_index_in_cutoff_time(es): es, cutoff_time=cutoff_time, chunk_size=1) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() assert (feature_matrix.shape[0] == cutoff_time.shape[0]) +# TODO: fails with Dask def test_saveprogress(es, tmpdir): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('saveprogress fails with Dask') times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + [datetime(2011, 4, 9, 10, 40, 0)] + @@ -239,13 +344,16 @@ def test_saveprogress(es, tmpdir): def test_cutoff_time_correctly(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered') property_feature = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) times = [datetime(2011, 4, 10), datetime(2011, 4, 11), datetime(2011, 4, 7)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 1, 2]}) feature_matrix = calculate_feature_matrix([property_feature], es, cutoff_time=cutoff_time) - + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() labels = [10, 5, 0] assert (feature_matrix[property_feature.get_name()] == labels).values.all() @@ -278,6 +386,18 @@ def test_cutoff_time_binning(): binned_cutoff_times = bin_cutoff_times(cutoff_time, Timedelta(1, 'mo')) +def test_training_window_fails_dask(dask_es): + property_feature = ft.Feature(dask_es['log']['id'], + parent_entity=dask_es['customers'], + primitive=Count) + + error_text = "Using training_window is not supported with Dask Entities" + with pytest.raises(ValueError, match=error_text): + calculate_feature_matrix([property_feature], + dask_es, + training_window='2 hours') + + def test_cutoff_time_columns_order(es): property_feature = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) times = [datetime(2011, 4, 10), datetime(2011, 4, 11), datetime(2011, 4, 7)] @@ -294,7 +414,8 @@ def test_cutoff_time_columns_order(es): cutoff_time=cutoff_time) labels = [10, 5, 0] - + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() assert (feature_matrix[property_feature.get_name()] == labels).values.all() @@ -325,12 +446,12 @@ def test_cutoff_time_df_redundant_column_names(es): cutoff_time=cutoff_time) -def test_training_window(es): - property_feature = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) - top_level_agg = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) +def test_training_window(pd_es): + property_feature = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['customers'], primitive=Count) + top_level_agg = ft.Feature(pd_es['customers']['id'], parent_entity=pd_es[u'régions'], primitive=Count) # make sure features that have a direct to a higher level agg - dagg = DirectFeature(top_level_agg, es['customers']) + dagg = DirectFeature(top_level_agg, pd_es['customers']) # for now, warns if last_time_index not present times = [datetime(2011, 4, 9, 12, 31), @@ -338,22 +459,22 @@ def test_training_window(es): datetime(2011, 4, 10, 13, 10)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 1, 2]}) feature_matrix = calculate_feature_matrix([property_feature, dagg], - es, + pd_es, cutoff_time=cutoff_time, training_window='2 hours') - es.add_last_time_indexes() + pd_es.add_last_time_indexes() error_text = 'Training window cannot be in observations' with pytest.raises(AssertionError, match=error_text): feature_matrix = calculate_feature_matrix([property_feature], - es, + pd_es, cutoff_time=cutoff_time, training_window=Timedelta(2, 'observations')) # Case1. include_cutoff_time = True feature_matrix = calculate_feature_matrix([property_feature, dagg], - es, + pd_es, cutoff_time=cutoff_time, training_window='2 hours', include_cutoff_time=True) @@ -364,7 +485,7 @@ def test_training_window(es): # Case2. include_cutoff_time = False feature_matrix = calculate_feature_matrix([property_feature, dagg], - es, + pd_es, cutoff_time=cutoff_time, training_window='2 hours', include_cutoff_time=False) @@ -374,12 +495,12 @@ def test_training_window(es): assert (feature_matrix[dagg.get_name()] == dagg_values).values.all() -def test_include_cutoff_time_with_training_window(es): - es.add_last_time_indexes() +def test_training_window_overlap(pd_es): + pd_es.add_last_time_indexes() count_log = ft.Feature( - base=es['log']['id'], - parent_entity=es['customers'], + base=pd_es['log']['id'], + parent_entity=pd_es['customers'], primitive=Count, ) @@ -391,7 +512,7 @@ def test_include_cutoff_time_with_training_window(es): # Case1. include_cutoff_time = True actual = ft.calculate_feature_matrix( features=[count_log], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, cutoff_time_in_index=True, training_window='10 minutes', @@ -402,7 +523,7 @@ def test_include_cutoff_time_with_training_window(es): # Case2. include_cutoff_time = False actual = ft.calculate_feature_matrix( features=[count_log], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, cutoff_time_in_index=True, training_window='10 minutes', @@ -446,14 +567,14 @@ def test_include_cutoff_time_without_training_window(es): np.testing.assert_array_equal(actual.values, [0, 5]) -def test_approximate_dfeat_of_agg_on_target_include_cutoff_time(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_approximate_dfeat_of_agg_on_target_include_cutoff_time(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) cutoff_time = pd.DataFrame({'time': [datetime(2011, 4, 9, 10, 31, 19)], 'instance_id': [0]}) feature_matrix = calculate_feature_matrix([dfeat, agg_feat2, agg_feat], - es, + pd_es, approximate=Timedelta(20, 's'), cutoff_time=cutoff_time, include_cutoff_time=False) @@ -465,7 +586,7 @@ def test_approximate_dfeat_of_agg_on_target_include_cutoff_time(es): assert feature_matrix[agg_feat.get_name()].tolist() == [5] feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(20, 's'), cutoff_time=cutoff_time, include_cutoff_time=True) @@ -477,7 +598,7 @@ def test_approximate_dfeat_of_agg_on_target_include_cutoff_time(es): assert feature_matrix[agg_feat.get_name()].tolist() == [5] -def test_training_window_recent_time_index(es): +def test_training_window_recent_time_index(pd_es): # customer with no sessions row = { 'id': [3], @@ -497,7 +618,7 @@ def test_training_window_recent_time_index(es): to_add_df.index = range(3, 4) # have to convert category to int in order to concat - old_df = es['customers'].df + old_df = pd_es['customers'].df old_df.index = old_df.index.astype("int") old_df["id"] = old_df["id"].astype(int) @@ -507,12 +628,12 @@ def test_training_window_recent_time_index(es): df.index = df.index.astype("category") df["id"] = df["id"].astype("category") - es['customers'].update_data(df=df, recalculate_last_time_indexes=False) - es.add_last_time_indexes() + pd_es['customers'].update_data(df=df, recalculate_last_time_indexes=False) + pd_es.add_last_time_indexes() - property_feature = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) - top_level_agg = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) - dagg = DirectFeature(top_level_agg, es['customers']) + property_feature = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['customers'], primitive=Count) + top_level_agg = ft.Feature(pd_es['customers']['id'], parent_entity=pd_es[u'régions'], primitive=Count) + dagg = DirectFeature(top_level_agg, pd_es['customers']) instance_ids = [0, 1, 2, 3] times = [datetime(2011, 4, 9, 12, 31), datetime(2011, 4, 10, 11), datetime(2011, 4, 10, 13, 10, 1), datetime(2011, 4, 10, 1, 59, 59)] @@ -521,7 +642,7 @@ def test_training_window_recent_time_index(es): # Case1. include_cutoff_time = True feature_matrix = calculate_feature_matrix( [property_feature, dagg], - es, + pd_es, cutoff_time=cutoff_time, training_window='2 hours', include_cutoff_time=True, @@ -536,7 +657,7 @@ def test_training_window_recent_time_index(es): # Case2. include_cutoff_time = False feature_matrix = calculate_feature_matrix( [property_feature, dagg], - es, + pd_es, cutoff_time=cutoff_time, training_window='2 hours', include_cutoff_time=False, @@ -549,45 +670,56 @@ def test_training_window_recent_time_index(es): assert (feature_matrix[dagg.get_name()] == dagg_values).values.all() -def test_approximate_multiple_instances_per_cutoff_time(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_approximate_fails_dask(dask_es): + agg_feat = ft.Feature(dask_es['log']['id'], + parent_entity=dask_es['sessions'], + primitive=Count) + error_text = "Using approximate is not supported with Dask Entities" + with pytest.raises(ValueError, match=error_text): + calculate_feature_matrix([agg_feat], + dask_es, + approximate=Timedelta(1, 'week')) + + +def test_approximate_multiple_instances_per_cutoff_time(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(1, 'week'), cutoff_time=cutoff_time) assert feature_matrix.shape[0] == 2 assert feature_matrix[agg_feat.get_name()].tolist() == [5, 1] -def test_approximate_with_multiple_paths(diamond_es): - es = diamond_es - path = backward_path(es, ['regions', 'customers', 'transactions']) - agg_feat = ft.AggregationFeature(es['transactions']['id'], - parent_entity=es['regions'], +def test_approximate_with_multiple_paths(pd_diamond_es): + pd_es = pd_diamond_es + path = backward_path(pd_es, ['regions', 'customers', 'transactions']) + agg_feat = ft.AggregationFeature(pd_es['transactions']['id'], + parent_entity=pd_es['regions'], relationship_path=path, primitive=Count) - dfeat = DirectFeature(agg_feat, es['customers']) + dfeat = DirectFeature(agg_feat, pd_es['customers']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat], - es, + pd_es, approximate=Timedelta(1, 'week'), cutoff_time=cutoff_time) assert feature_matrix[dfeat.get_name()].tolist() == [6, 2] -def test_approximate_dfeat_of_agg_on_target(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_approximate_dfeat_of_agg_on_target(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, instance_ids=[0, 2], approximate=Timedelta(10, 's'), cutoff_time=cutoff_time) @@ -595,19 +727,19 @@ def test_approximate_dfeat_of_agg_on_target(es): assert feature_matrix[agg_feat.get_name()].tolist() == [5, 1] -def test_approximate_dfeat_of_need_all_values(es): - p = ft.Feature(es['log']['value'], primitive=Percentile) - agg_feat = ft.Feature(p, parent_entity=es['sessions'], primitive=Sum) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_approximate_dfeat_of_need_all_values(pd_es): + p = ft.Feature(pd_es['log']['value'], primitive=Percentile) + agg_feat = ft.Feature(p, parent_entity=pd_es['sessions'], primitive=Sum) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time_in_index=True, cutoff_time=cutoff_time) - log_df = es['log'].df + log_df = pd_es['log'].df instances = [0, 2] cutoffs = [pd.Timestamp('2011-04-09 10:31:19'), pd.Timestamp('2011-04-09 11:00:00')] approxes = [pd.Timestamp('2011-04-09 10:31:10'), pd.Timestamp('2011-04-09 11:00:00')] @@ -629,12 +761,12 @@ def test_approximate_dfeat_of_need_all_values(es): assert test_list == true_vals -def test_uses_full_entity_feat_of_approximate(es): - agg_feat = ft.Feature(es['log']['value'], parent_entity=es['sessions'], primitive=Sum) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - agg_feat3 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Max) - dfeat = DirectFeature(agg_feat2, es['sessions']) - dfeat2 = DirectFeature(agg_feat3, es['sessions']) +def test_uses_full_entity_feat_of_approximate(pd_es): + agg_feat = ft.Feature(pd_es['log']['value'], parent_entity=pd_es['sessions'], primitive=Sum) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + agg_feat3 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Max) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) + dfeat2 = DirectFeature(agg_feat3, pd_es['sessions']) p = ft.Feature(dfeat, primitive=Percentile) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) @@ -643,7 +775,7 @@ def test_uses_full_entity_feat_of_approximate(es): feature_matrix_only_dfeat2 = calculate_feature_matrix( [dfeat2], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time_in_index=True, cutoff_time=cutoff_time) @@ -651,7 +783,7 @@ def test_uses_full_entity_feat_of_approximate(es): feature_matrix_approx = calculate_feature_matrix( [p, dfeat, dfeat2, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time_in_index=True, cutoff_time=cutoff_time) @@ -659,14 +791,14 @@ def test_uses_full_entity_feat_of_approximate(es): feature_matrix_small_approx = calculate_feature_matrix( [p, dfeat, dfeat2, agg_feat], - es, + pd_es, approximate=Timedelta(10, 'ms'), cutoff_time_in_index=True, cutoff_time=cutoff_time) feature_matrix_no_approx = calculate_feature_matrix( [p, dfeat, dfeat2, agg_feat], - es, + pd_es, cutoff_time_in_index=True, cutoff_time=cutoff_time) for f in [p, dfeat, agg_feat]: @@ -676,28 +808,28 @@ def test_uses_full_entity_feat_of_approximate(es): assert fm1[f.get_name()].tolist() == fm2[f.get_name()].tolist() -def test_approximate_dfeat_of_dfeat_of_agg_on_target(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(ft.Feature(agg_feat2, es["sessions"]), es['log']) +def test_approximate_dfeat_of_dfeat_of_agg_on_target(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(ft.Feature(agg_feat2, pd_es["sessions"]), pd_es['log']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_time) assert feature_matrix[dfeat.get_name()].tolist() == [7, 10] -def test_empty_path_approximate_full(es): - es['sessions'].df['customer_id'] = pd.Series([np.nan, np.nan, np.nan, 1, 1, 2], dtype="category") - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_empty_path_approximate_full(pd_es): + pd_es['sessions'].df['customer_id'] = pd.Series([np.nan, np.nan, np.nan, 1, 1, 2], dtype="category") + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_time) vals1 = feature_matrix[dfeat.get_name()].tolist() @@ -706,16 +838,16 @@ def test_empty_path_approximate_full(es): assert feature_matrix[agg_feat.get_name()].tolist() == [5, 1] # todo: do we need to test this situation? -# def test_empty_path_approximate_partial(es): -# es = copy.deepcopy(es) -# es['sessions'].df['customer_id'] = pd.Categorical([0, 0, np.nan, 1, 1, 2]) -# agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) -# agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) -# dfeat = DirectFeature(agg_feat2, es['sessions']) +# def test_empty_path_approximate_partial(pd_es): +# pd_es = copy.deepcopy(pd_es) +# pd_es['sessions'].df['customer_id'] = pd.Categorical([0, 0, np.nan, 1, 1, 2]) +# agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) +# agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) +# dfeat = DirectFeature(agg_feat2, pd_es['sessions']) # times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] # cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) # feature_matrix = calculate_feature_matrix([dfeat, agg_feat], -# es, +# pd_es, # approximate=Timedelta(10, 's'), # cutoff_time=cutoff_time) # vals1 = feature_matrix[dfeat.get_name()].tolist() @@ -724,17 +856,17 @@ def test_empty_path_approximate_full(es): # assert feature_matrix[agg_feat.get_name()].tolist() == [5, 1] -def test_approx_base_feature_is_also_first_class_feature(es): - log_to_products = DirectFeature(es['products']['rating'], es['log']) +def test_approx_base_feature_is_also_first_class_feature(pd_es): + log_to_products = DirectFeature(pd_es['products']['rating'], pd_es['log']) # This should still be computed properly - agg_feat = ft.Feature(log_to_products, parent_entity=es['sessions'], primitive=Min) - customer_agg_feat = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) + agg_feat = ft.Feature(log_to_products, parent_entity=pd_es['sessions'], primitive=Min) + customer_agg_feat = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) # This is to be approximated - sess_to_cust = DirectFeature(customer_agg_feat, es['sessions']) + sess_to_cust = DirectFeature(customer_agg_feat, pd_es['sessions']) times = [datetime(2011, 4, 9, 10, 31, 19), datetime(2011, 4, 9, 11, 0, 0)] cutoff_time = pd.DataFrame({'time': times, 'instance_id': [0, 2]}) feature_matrix = calculate_feature_matrix([sess_to_cust, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_time) vals1 = feature_matrix[sess_to_cust.get_name()].tolist() @@ -743,17 +875,17 @@ def test_approx_base_feature_is_also_first_class_feature(es): assert vals2 == [4, 1.5] -def test_approximate_time_split_returns_the_same_result(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) - agg_feat2 = ft.Feature(agg_feat, parent_entity=es['customers'], primitive=Sum) - dfeat = DirectFeature(agg_feat2, es['sessions']) +def test_approximate_time_split_returns_the_same_result(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['sessions'], primitive=Count) + agg_feat2 = ft.Feature(agg_feat, parent_entity=pd_es['customers'], primitive=Sum) + dfeat = DirectFeature(agg_feat2, pd_es['sessions']) cutoff_df = pd.DataFrame({'time': [pd.Timestamp('2011-04-09 10:07:30'), pd.Timestamp('2011-04-09 10:07:40')], 'instance_id': [0, 0]}) feature_matrix_at_once = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_df) divided_matrices = [] @@ -764,7 +896,7 @@ def test_approximate_time_split_returns_the_same_result(es): separate_cutoff[1].index = [1] for ct in separate_cutoff: fm = calculate_feature_matrix([dfeat, agg_feat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=ct) divided_matrices.append(fm) @@ -777,34 +909,34 @@ def test_approximate_time_split_returns_the_same_result(es): assert (pd.isnull(i1) and pd.isnull(i2)) or (i1 == i2) -def test_approximate_returns_correct_empty_default_values(es): - agg_feat = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) - dfeat = DirectFeature(agg_feat, es['sessions']) +def test_approximate_returns_correct_empty_default_values(pd_es): + agg_feat = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['customers'], primitive=Count) + dfeat = DirectFeature(agg_feat, pd_es['sessions']) cutoff_df = pd.DataFrame({'time': [pd.Timestamp('2011-04-08 11:00:00'), pd.Timestamp('2011-04-09 11:00:00')], 'instance_id': [0, 0]}) fm = calculate_feature_matrix([dfeat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_df) assert fm[dfeat.get_name()].tolist() == [0, 10] -# def test_approximate_deep_recurse(es): - # es = es - # agg_feat = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) - # dfeat1 = DirectFeature(agg_feat, es['sessions']) - # agg_feat2 = Sum(dfeat1, es['customers']) - # dfeat2 = DirectFeature(agg_feat2, es['sessions']) +# def test_approximate_deep_recurse(pd_es): + # pd_es = pd_es + # agg_feat = ft.Feature(pd_es['customers']['id'], parent_entity=pd_es[u'régions'], primitive=Count) + # dfeat1 = DirectFeature(agg_feat, pd_es['sessions']) + # agg_feat2 = Sum(dfeat1, pd_es['customers']) + # dfeat2 = DirectFeature(agg_feat2, pd_es['sessions']) - # agg_feat3 = ft.Feature(es['log']['id'], parent_entity=es['products'], primitive=Count) - # dfeat3 = DirectFeature(agg_feat3, es['log']) - # agg_feat4 = Sum(dfeat3, es['sessions']) + # agg_feat3 = ft.Feature(pd_es['log']['id'], parent_entity=pd_es['products'], primitive=Count) + # dfeat3 = DirectFeature(agg_feat3, pd_es['log']) + # agg_feat4 = Sum(dfeat3, pd_es['sessions']) # feature_matrix = calculate_feature_matrix([dfeat2, agg_feat4], - # es, + # pd_es, # instance_ids=[0, 2], # approximate=Timedelta(10, 's'), # cutoff_time=[datetime(2011, 4, 9, 10, 31, 19), @@ -812,20 +944,20 @@ def test_approximate_returns_correct_empty_default_values(es): # # dfeat2 and agg_feat4 should both be approximated -def test_approximate_child_aggs_handled_correctly(es): - agg_feat = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) - dfeat = DirectFeature(agg_feat, es['customers']) - agg_feat_2 = ft.Feature(es['log']['value'], parent_entity=es['customers'], primitive=Sum) +def test_approximate_child_aggs_handled_correctly(pd_es): + agg_feat = ft.Feature(pd_es['customers']['id'], parent_entity=pd_es[u'régions'], primitive=Count) + dfeat = DirectFeature(agg_feat, pd_es['customers']) + agg_feat_2 = ft.Feature(pd_es['log']['value'], parent_entity=pd_es['customers'], primitive=Sum) cutoff_df = pd.DataFrame({'time': [pd.Timestamp('2011-04-08 10:30:00'), pd.Timestamp('2011-04-09 10:30:06')], 'instance_id': [0, 0]}) fm = calculate_feature_matrix([dfeat], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_df) fm_2 = calculate_feature_matrix([dfeat, agg_feat_2], - es, + pd_es, approximate=Timedelta(10, 's'), cutoff_time=cutoff_df) assert fm[dfeat.get_name()].tolist() == [2, 3] @@ -843,7 +975,11 @@ def test_cutoff_time_naming(es): cutoff_df_wrong_time_name = cutoff_df.rename(columns={"time": "cutoff_time"}) fm1 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df) + if isinstance(fm1, dd.DataFrame): + fm1 = fm1.compute().set_index('id').sort_index() fm2 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df_index_name) + if isinstance(fm2, dd.DataFrame): + fm2 = fm2.compute().set_index('id').sort_index() assert all((fm1 == fm2.values).values) error_text = 'Cutoff time DataFrame must contain a column with either the same name' \ @@ -857,7 +993,11 @@ def test_cutoff_time_naming(es): calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df_wrong_time_name) +# TODO: order doesn't match, but output matches +# TODO: split out approximate test into seperate test for only pandas def test_cutoff_time_extra_columns(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered') agg_feat = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) dfeat = DirectFeature(agg_feat, es['customers']) @@ -868,19 +1008,22 @@ def test_cutoff_time_extra_columns(es): 'label': [True, True, False]}, columns=['time', 'instance_id', 'label']) fm = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df) + if isinstance(fm, dd.DataFrame): + fm = fm.compute() # check column was added to end of matrix assert 'label' == fm.columns[-1] assert (fm['label'].values == cutoff_df['label'].values).all() - fm_2 = calculate_feature_matrix([dfeat], - es, - cutoff_time=cutoff_df, - approximate="2 days") - # check column was added to end of matrix - assert 'label' in fm_2.columns + if any(isinstance(entity.df, pd.DataFrame) for entity in es.entities): + fm_2 = calculate_feature_matrix([dfeat], + es, + cutoff_time=cutoff_df, + approximate="2 days") + # check column was added to end of matrix + assert 'label' in fm_2.columns - assert (fm_2['label'].values == cutoff_df['label'].values).all() + assert (fm_2['label'].values == cutoff_df['label'].values).all() def test_instances_after_cutoff_time_removed(es): @@ -890,13 +1033,20 @@ def test_instances_after_cutoff_time_removed(es): es, cutoff_time=cutoff_time, cutoff_time_in_index=True) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') + actual_ids = fm.index + else: + actual_ids = [id for (id, _) in fm.index] # Customer with id 1 should be removed - actual_ids = [id for (id, _) in fm.index] assert set(actual_ids) == set([2, 0]) +# TODO: Dask doesn't keep instance_id after cutoff def test_instances_with_id_kept_after_cutoff(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered, missing extra instances') property_feature = ft.Feature(es['log']['id'], parent_entity=es['customers'], primitive=Count) cutoff_time = datetime(2011, 4, 8) fm = calculate_feature_matrix([property_feature], @@ -907,11 +1057,19 @@ def test_instances_with_id_kept_after_cutoff(es): # Customer #1 is after cutoff, but since it is included in instance_ids it # should be kept. - actual_ids = [id for (id, _) in fm.index] + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') + actual_ids = fm.index + else: + actual_ids = [id for (id, _) in fm.index] assert set(actual_ids) == set([0, 1, 2]) +# TODO: Fails with Dask +# TODO: split out approximate portion into seperate test for pandas def test_cfm_returns_original_time_indexes(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask result not ordered, indexes are lost due to not multiindexing') agg_feat = ft.Feature(es['customers']['id'], parent_entity=es[u'régions'], primitive=Count) dfeat = DirectFeature(agg_feat, es['customers']) agg_feat_2 = ft.Feature(es['sessions']['id'], parent_entity=es['customers'], primitive=Count) @@ -924,45 +1082,53 @@ def test_cfm_returns_original_time_indexes(es): fm = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df, cutoff_time_in_index=True) - instance_level_vals = fm.index.get_level_values(0).values - time_level_vals = fm.index.get_level_values(1).values - assert (instance_level_vals == cutoff_df['instance_id'].values).all() - assert (time_level_vals == cutoff_df['time'].values).all() - - # approximate, in different windows, no unapproximated aggs - fm2 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df, - cutoff_time_in_index=True, approximate="1 m") - instance_level_vals = fm2.index.get_level_values(0).values - time_level_vals = fm2.index.get_level_values(1).values - assert (instance_level_vals == cutoff_df['instance_id'].values).all() - assert (time_level_vals == cutoff_df['time'].values).all() - - # approximate, in different windows, unapproximated aggs - fm2 = calculate_feature_matrix([dfeat, agg_feat_2], es, cutoff_time=cutoff_df, - cutoff_time_in_index=True, approximate="1 m") - instance_level_vals = fm2.index.get_level_values(0).values - time_level_vals = fm2.index.get_level_values(1).values - assert (instance_level_vals == cutoff_df['instance_id'].values).all() - assert (time_level_vals == cutoff_df['time'].values).all() - - # approximate, in same window, no unapproximated aggs - fm3 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df, - cutoff_time_in_index=True, approximate="2 d") - instance_level_vals = fm3.index.get_level_values(0).values - time_level_vals = fm3.index.get_level_values(1).values - assert (instance_level_vals == cutoff_df['instance_id'].values).all() - assert (time_level_vals == cutoff_df['time'].values).all() - - # approximate, in same window, unapproximated aggs - fm3 = calculate_feature_matrix([dfeat, agg_feat_2], es, cutoff_time=cutoff_df, - cutoff_time_in_index=True, approximate="2 d") - instance_level_vals = fm3.index.get_level_values(0).values - time_level_vals = fm3.index.get_level_values(1).values + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') + instance_level_vals = fm.index + # Dask doesn't return time (doesn't support multi-index) + time_level_vals = [] + else: + instance_level_vals = fm.index.get_level_values(0).values + time_level_vals = fm.index.get_level_values(1).values assert (instance_level_vals == cutoff_df['instance_id'].values).all() assert (time_level_vals == cutoff_df['time'].values).all() - -def test_dask_kwargs(es): + # skip approximate for Dask + if any(isinstance(entity.df, pd.DataFrame) for entity in es.entities): + # approximate, in different windows, no unapproximated aggs + fm2 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df, + cutoff_time_in_index=True, approximate="1 m") + instance_level_vals = fm2.index.get_level_values(0).values + time_level_vals = fm2.index.get_level_values(1).values + assert (instance_level_vals == cutoff_df['instance_id'].values).all() + assert (time_level_vals == cutoff_df['time'].values).all() + + # approximate, in different windows, unapproximated aggs + fm2 = calculate_feature_matrix([dfeat, agg_feat_2], es, cutoff_time=cutoff_df, + cutoff_time_in_index=True, approximate="1 m") + instance_level_vals = fm2.index.get_level_values(0).values + time_level_vals = fm2.index.get_level_values(1).values + assert (instance_level_vals == cutoff_df['instance_id'].values).all() + assert (time_level_vals == cutoff_df['time'].values).all() + + # approximate, in same window, no unapproximated aggs + fm3 = calculate_feature_matrix([dfeat], es, cutoff_time=cutoff_df, + cutoff_time_in_index=True, approximate="2 d") + instance_level_vals = fm3.index.get_level_values(0).values + time_level_vals = fm3.index.get_level_values(1).values + assert (instance_level_vals == cutoff_df['instance_id'].values).all() + assert (time_level_vals == cutoff_df['time'].values).all() + + # approximate, in same window, unapproximated aggs + fm3 = calculate_feature_matrix([dfeat, agg_feat_2], es, cutoff_time=cutoff_df, + cutoff_time_in_index=True, approximate="2 d") + instance_level_vals = fm3.index.get_level_values(0).values + time_level_vals = fm3.index.get_level_values(1).values + assert (instance_level_vals == cutoff_df['instance_id'].values).all() + assert (time_level_vals == cutoff_df['time'].values).all() + + +def test_dask_kwargs(pd_es): times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + [datetime(2011, 4, 9, 10, 40, 0)] + @@ -971,12 +1137,12 @@ def test_dask_kwargs(es): [datetime(2011, 4, 10, 11, 10, i * 3) for i in range(2)]) labels = [False] * 3 + [True] * 2 + [False] * 9 + [True] + [False] * 2 cutoff_time = pd.DataFrame({'time': times, 'instance_id': range(17)}) - property_feature = IdentityFeature(es['log']['value']) > 10 + property_feature = IdentityFeature(pd_es['log']['value']) > 10 with cluster() as (scheduler, [a, b]): dkwargs = {'cluster': scheduler['address']} feature_matrix = calculate_feature_matrix([property_feature], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, verbose=True, chunk_size=.13, @@ -986,7 +1152,7 @@ def test_dask_kwargs(es): assert (feature_matrix[property_feature.get_name()] == labels).values.all() -def test_dask_persisted_es(es, capsys): +def test_dask_persisted_es(pd_es, capsys): times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + [datetime(2011, 4, 9, 10, 40, 0)] + @@ -995,12 +1161,12 @@ def test_dask_persisted_es(es, capsys): [datetime(2011, 4, 10, 11, 10, i * 3) for i in range(2)]) labels = [False] * 3 + [True] * 2 + [False] * 9 + [True] + [False] * 2 cutoff_time = pd.DataFrame({'time': times, 'instance_id': range(17)}) - property_feature = IdentityFeature(es['log']['value']) > 10 + property_feature = IdentityFeature(pd_es['log']['value']) > 10 with cluster() as (scheduler, [a, b]): dkwargs = {'cluster': scheduler['address']} feature_matrix = calculate_feature_matrix([property_feature], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, verbose=True, chunk_size=.13, @@ -1008,7 +1174,7 @@ def test_dask_persisted_es(es, capsys): approximate='1 hour') assert (feature_matrix[property_feature.get_name()] == labels).values.all() feature_matrix = calculate_feature_matrix([property_feature], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, verbose=True, chunk_size=.13, @@ -1081,7 +1247,7 @@ def test_not_enough_memory(self, monkeypatch): entityset_size=total_memory * .75) -def test_parallel_failure_raises_correct_error(es): +def test_parallel_failure_raises_correct_error(pd_es): times = list([datetime(2011, 4, 9, 10, 30, i * 6) for i in range(5)] + [datetime(2011, 4, 9, 10, 31, i * 9) for i in range(4)] + [datetime(2011, 4, 9, 10, 40, 0)] + @@ -1089,12 +1255,12 @@ def test_parallel_failure_raises_correct_error(es): [datetime(2011, 4, 10, 10, 41, i * 3) for i in range(3)] + [datetime(2011, 4, 10, 11, 10, i * 3) for i in range(2)]) cutoff_time = pd.DataFrame({'time': times, 'instance_id': range(17)}) - property_feature = IdentityFeature(es['log']['value']) > 10 + property_feature = IdentityFeature(pd_es['log']['value']) > 10 error_text = 'Need at least one worker' with pytest.raises(AssertionError, match=error_text): calculate_feature_matrix([property_feature], - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, verbose=True, chunk_size=.13, @@ -1102,13 +1268,13 @@ def test_parallel_failure_raises_correct_error(es): approximate='1 hour') -def test_warning_not_enough_chunks(es, capsys): - property_feature = IdentityFeature(es['log']['value']) > 10 +def test_warning_not_enough_chunks(pd_es, capsys): + property_feature = IdentityFeature(pd_es['log']['value']) > 10 with cluster(nworkers=3) as (scheduler, [a, b, c]): dkwargs = {'cluster': scheduler['address']} calculate_feature_matrix([property_feature], - entityset=es, + entityset=pd_es, chunk_size=.5, verbose=True, dask_kwargs=dkwargs) @@ -1136,6 +1302,7 @@ def test_n_jobs(): n_jobs_to_workers(0) +# TODO: add dask version of int_es def test_integer_time_index(int_es): times = list(range(8, 18)) + list(range(19, 26)) labels = [False] * 3 + [True] * 2 + [False] * 9 + [True] + [False] * 2 @@ -1153,6 +1320,7 @@ def test_integer_time_index(int_es): assert (feature_matrix[property_feature.get_name()] == labels).values.all() +# TODO: add dask version of int_es def test_integer_time_index_datetime_cutoffs(int_es): times = [datetime.now()] * 17 cutoff_df = pd.DataFrame({'time': times, 'instance_id': range(17)}) @@ -1167,6 +1335,7 @@ def test_integer_time_index_datetime_cutoffs(int_es): cutoff_time_in_index=True) +# TODO: add Dask version of int_es def test_integer_time_index_passes_extra_columns(int_es): times = list(range(8, 18)) + list(range(19, 23)) + [25, 24, 23] labels = [False] * 3 + [True] * 2 + [False] * 9 + [False] * 2 + [True] @@ -1185,6 +1354,7 @@ def test_integer_time_index_passes_extra_columns(int_es): assert (fm[property_feature.get_name()] == fm['labels']).all() +# TODO: add Dask version of int_es def test_integer_time_index_mixed_cutoff(int_es): times_dt = list(range(8, 17)) + [datetime(2011, 1, 1), 19, 20, 21, 22, 25, 24, 23] labels = [False] * 3 + [True] * 2 + [False] * 9 + [False] * 2 + [True] @@ -1277,8 +1447,29 @@ def test_string_time_values_in_cutoff_time(es): calculate_feature_matrix([agg_feature], es, cutoff_time=cutoff_time) -def test_no_data_for_cutoff_time(): - es = ft.demo.load_mock_customer(return_entityset=True, random_seed=0) +@pytest.fixture +def pd_mock_customer(): + return ft.demo.load_mock_customer(return_entityset=True, random_seed=0) + + +@pytest.fixture +def dd_mock_customer(pd_mock_customer): + dd_mock_customer = copy.deepcopy(pd_mock_customer) + for entity in dd_mock_customer.entities: + entity.df = dd.from_pandas(entity.df.reset_index(drop=True), npartitions=4) + return dd_mock_customer + + +@pytest.fixture(params=['pd_mock_customer', 'dd_mock_customer']) +def mock_customer(request): + return request.getfixturevalue(request.param) + + +# TODO: Dask version fails (feature matrix is empty) +def test_no_data_for_cutoff_time(mock_customer): + if any(isinstance(entity.df, dd.DataFrame) for entity in mock_customer.entities): + pytest.xfail('Dask version fails, returned feature matrix is empty') + es = mock_customer cutoff_times = pd.DataFrame({"customer_id": [4], "time": pd.Timestamp('2011-04-08 20:08:13')}) @@ -1287,29 +1478,32 @@ def test_no_data_for_cutoff_time(): features = [trans_per_customer, ft.Feature(trans_per_session, parent_entity=es["customers"], primitive=Max)] fm = ft.calculate_feature_matrix(features, entityset=es, cutoff_time=cutoff_times) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('customer_id') # due to default values for each primitive # count will be 0, but max will nan np.testing.assert_array_equal(fm.values, [[0, np.nan]]) -def test_instances_not_in_data(es): - last_instance = max(es['log'].df.index.values) +# adding missing instances not supported in Dask +def test_instances_not_in_data(pd_es): + last_instance = max(pd_es['log'].df.index.values) instances = list(range(last_instance + 1, last_instance + 11)) - identity_feature = IdentityFeature(es['log']['value']) + identity_feature = IdentityFeature(pd_es['log']['value']) property_feature = identity_feature > 10 - agg_feat = AggregationFeature(es['log']['value'], - parent_entity=es["sessions"], + agg_feat = AggregationFeature(pd_es['log']['value'], + parent_entity=pd_es["sessions"], primitive=Max) - direct_feature = DirectFeature(agg_feat, es["log"]) + direct_feature = DirectFeature(agg_feat, pd_es["log"]) features = [identity_feature, property_feature, direct_feature] - fm = calculate_feature_matrix(features, entityset=es, instance_ids=instances) + fm = calculate_feature_matrix(features, entityset=pd_es, instance_ids=instances) assert all(fm.index.values == instances) for column in fm.columns: assert fm[column].isnull().all() fm = calculate_feature_matrix(features, - entityset=es, + entityset=pd_es, instance_ids=instances, approximate="730 days") assert all(fm.index.values == instances) @@ -1317,7 +1511,7 @@ def test_instances_not_in_data(es): assert fm[column].isnull().all() -def test_some_instances_not_in_data(es): +def test_some_instances_not_in_data(pd_es): a_time = datetime(2011, 4, 10, 10, 41, 9) # only valid data b_time = datetime(2011, 4, 10, 11, 10, 5) # some missing data c_time = datetime(2011, 4, 10, 12, 0, 0) # all missing data @@ -1325,17 +1519,16 @@ def test_some_instances_not_in_data(es): times = [a_time, b_time, a_time, a_time, b_time, b_time] + [c_time] * 4 cutoff_time = pd.DataFrame({"instance_id": list(range(12, 22)), "time": times}) - identity_feature = IdentityFeature(es['log']['value']) + identity_feature = IdentityFeature(pd_es['log']['value']) property_feature = identity_feature > 10 - agg_feat = AggregationFeature(es['log']['value'], - parent_entity=es["sessions"], + agg_feat = AggregationFeature(pd_es['log']['value'], + parent_entity=pd_es["sessions"], primitive=Max) - direct_feature = DirectFeature(agg_feat, es["log"]) + direct_feature = DirectFeature(agg_feat, pd_es["log"]) features = [identity_feature, property_feature, direct_feature] fm = calculate_feature_matrix(features, - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time) - ifeat_answer = [0, 7, 14, np.nan] + [np.nan] * 6 prop_answer = [0, 0, 1, np.nan, 0] + [np.nan] * 5 dfeat_answer = [14, 14, 14, np.nan] + [np.nan] * 6 @@ -1345,7 +1538,7 @@ def test_some_instances_not_in_data(es): np.testing.assert_array_equal(fm[x], y) fm = calculate_feature_matrix(features, - entityset=es, + entityset=pd_es, cutoff_time=cutoff_time, approximate="5 seconds") @@ -1403,7 +1596,8 @@ def test_chunk_dataframe_groups(): assert fourth[0] == 3 and fourth[1].shape[0] == 1 -def test_calls_progress_callback(es): +# TODO: split out cluster tests into seperate test for pandas +def test_calls_progress_callback(mock_customer): class MockProgressCallback: def __init__(self): self.progress_history = [] @@ -1417,7 +1611,7 @@ def __call__(self, update, progress_percent, time_elapsed): mock_progress_callback = MockProgressCallback() - es = ft.demo.load_mock_customer(return_entityset=True, random_seed=0) + es = mock_customer # make sure to calculate features that have different paths to same base feature trans_per_session = ft.Feature(es["transactions"]["transaction_id"], parent_entity=es["sessions"], primitive=Count) @@ -1430,15 +1624,16 @@ def __call__(self, update, progress_percent, time_elapsed): assert np.isclose(mock_progress_callback.total_update, 100.0) assert np.isclose(mock_progress_callback.total_progress_percent, 100.0) - # test with multiple jobs - mock_progress_callback = MockProgressCallback() + # test with multiple jobs, pandas only + if any(isinstance(entity.df, pd.DataFrame) for entity in es.entities): + mock_progress_callback = MockProgressCallback() - with cluster() as (scheduler, [a, b]): - dkwargs = {'cluster': scheduler['address']} - ft.calculate_feature_matrix(features, - entityset=es, - progress_callback=mock_progress_callback, - dask_kwargs=dkwargs) + with cluster() as (scheduler, [a, b]): + dkwargs = {'cluster': scheduler['address']} + ft.calculate_feature_matrix(features, + entityset=es, + progress_callback=mock_progress_callback, + dask_kwargs=dkwargs) - assert np.isclose(mock_progress_callback.total_update, 100.0) - assert np.isclose(mock_progress_callback.total_progress_percent, 100.0) + assert np.isclose(mock_progress_callback.total_update, 100.0) + assert np.isclose(mock_progress_callback.total_progress_percent, 100.0) diff --git a/featuretools/tests/computational_backend/test_dask_features.py b/featuretools/tests/computational_backend/test_dask_features.py index ac90b512c9..bb66148339 100644 --- a/featuretools/tests/computational_backend/test_dask_features.py +++ b/featuretools/tests/computational_backend/test_dask_features.py @@ -4,19 +4,19 @@ from featuretools.tests.testing_utils import make_ecommerce_entityset -def test_tokenize_entityset(es, int_es): +def test_tokenize_entityset(pd_es, int_es): dupe = make_ecommerce_entityset() # check identitcal entitysets hash to same token - assert tokenize(es) == tokenize(dupe) + assert tokenize(pd_es) == tokenize(dupe) # not same if product relationship is missing productless = make_ecommerce_entityset() productless.relationships.pop() - assert tokenize(es) != tokenize(productless) + assert tokenize(pd_es) != tokenize(productless) # not same if integer entityset - assert tokenize(es) != tokenize(int_es) + assert tokenize(pd_es) != tokenize(int_es) # add row to cohorts cohorts_df = dupe['cohorts'].df @@ -27,4 +27,4 @@ def test_tokenize_entityset(es, int_es): index=[2]) more_cohorts = cohorts_df.append(new_row, ignore_index=True, sort=True) dupe['cohorts'].update_data(more_cohorts) - assert tokenize(es) == tokenize(dupe) + assert tokenize(pd_es) == tokenize(dupe) diff --git a/featuretools/tests/computational_backend/test_feature_set.py b/featuretools/tests/computational_backend/test_feature_set.py index c795f960df..49b960da67 100644 --- a/featuretools/tests/computational_backend/test_feature_set.py +++ b/featuretools/tests/computational_backend/test_feature_set.py @@ -50,18 +50,18 @@ def test_feature_trie_without_needs_full_entity(diamond_es): def test_feature_trie_with_needs_full_entity(diamond_es): - es = diamond_es - amount = ft.IdentityFeature(es['transactions']['amount']) + pd_es = diamond_es + amount = ft.IdentityFeature(pd_es['transactions']['amount']) - path_through_customers = backward_path(es, ['regions', 'customers', 'transactions']) - agg = ft.AggregationFeature(amount, es['regions'], + path_through_customers = backward_path(pd_es, ['regions', 'customers', 'transactions']) + agg = ft.AggregationFeature(amount, pd_es['regions'], primitive=ft.primitives.Mean, relationship_path=path_through_customers) trans_of_agg = ft.TransformFeature(agg, ft.primitives.CumSum) - path_through_stores = backward_path(es, ['regions', 'stores', 'transactions']) + path_through_stores = backward_path(pd_es, ['regions', 'stores', 'transactions']) trans = ft.TransformFeature(amount, ft.primitives.CumSum) - agg_of_trans = ft.AggregationFeature(trans, es['regions'], + agg_of_trans = ft.AggregationFeature(trans, pd_es['regions'], primitive=ft.primitives.Mean, relationship_path=path_through_stores) diff --git a/featuretools/tests/computational_backend/test_feature_set_calculator.py b/featuretools/tests/computational_backend/test_feature_set_calculator.py index a17b67610d..d372b35d46 100644 --- a/featuretools/tests/computational_backend/test_feature_set_calculator.py +++ b/featuretools/tests/computational_backend/test_feature_set_calculator.py @@ -3,10 +3,11 @@ import numpy as np import pandas as pd import pytest +from dask import dataframe as dd from numpy.testing import assert_array_equal import featuretools as ft -from featuretools import Timedelta +from featuretools import Timedelta, variable_types from featuretools.computational_backends.feature_set import FeatureSet from featuretools.computational_backends.feature_set_calculator import ( FeatureSetCalculator @@ -25,6 +26,7 @@ Mean, Min, Mode, + Negate, NMostCommon, NotEqualScalar, NumTrue, @@ -46,6 +48,9 @@ def test_make_identity(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() + v = df[f.get_name()][0] assert (v == datetime(2011, 4, 9, 10, 30, 0)) @@ -59,6 +64,9 @@ def test_make_dfeat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() + v = df[f.get_name()][0] assert (v == 33) @@ -71,17 +79,21 @@ def test_make_agg_feat_of_identity_variable(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() + v = df[agg_feat.get_name()][0] assert (v == 50) -def test_full_entity_trans_of_agg(es): - agg_feat = ft.Feature(es['log']['value'], parent_entity=es['customers'], +# full_entity not supported with Dask +def test_full_entity_trans_of_agg(pd_es): + agg_feat = ft.Feature(pd_es['log']['value'], parent_entity=pd_es['customers'], primitive=Sum) trans_feat = ft.Feature(agg_feat, primitive=CumSum) feature_set = FeatureSet([trans_feat]) - calculator = FeatureSetCalculator(es, + calculator = FeatureSetCalculator(pd_es, time_last=None, feature_set=feature_set) df = calculator.run(np.array([1])) @@ -90,6 +102,21 @@ def test_full_entity_trans_of_agg(es): assert v == 82 +def test_full_entity_error_dask(dask_es): + agg_feat = ft.Feature(dask_es['log']['value'], parent_entity=dask_es['customers'], + primitive=Sum) + trans_feat = ft.Feature(agg_feat, primitive=CumSum) + + feature_set = FeatureSet([trans_feat]) + calculator = FeatureSetCalculator(dask_es, + time_last=None, + feature_set=feature_set) + error_text = "Cannot use primitives that require full entity with Dask" + + with pytest.raises(ValueError, match=error_text): + calculator.run(np.array([1])) + + def test_make_agg_feat_of_identity_index_variable(es): agg_feat = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) @@ -98,6 +125,9 @@ def test_make_agg_feat_of_identity_index_variable(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() + v = df[agg_feat.get_name()][0] assert (v == 5) @@ -113,6 +143,8 @@ def test_make_agg_feat_where_count(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[agg_feat.get_name()][0] assert (v == 3) @@ -129,6 +161,8 @@ def test_make_agg_feat_using_prev_time(es): time_last=datetime(2011, 4, 9, 10, 30, 10), feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[agg_feat.get_name()][0] assert (v == 2) @@ -137,12 +171,16 @@ def test_make_agg_feat_using_prev_time(es): time_last=datetime(2011, 4, 9, 10, 30, 30), feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[agg_feat.get_name()][0] assert (v == 1) def test_make_agg_feat_using_prev_n_events(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Dask currently does not support use_previous') agg_feat_1 = ft.Feature(es['log']['value'], parent_entity=es['sessions'], use_previous=Timedelta(1, 'observations'), @@ -180,6 +218,8 @@ def test_make_agg_feat_using_prev_n_events(es): def test_make_agg_feat_multiple_dtypes(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('Currently no dask compatible agg prims that use multiple dtypes') compare_prod = IdentityFeature(es['log']['product_id']) == 'coke zero' agg_feat = ft.Feature(es['log']['id'], @@ -197,6 +237,8 @@ def test_make_agg_feat_multiple_dtypes(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[agg_feat.get_name()][0] v2 = df[agg_feat2.get_name()][0] @@ -215,6 +257,8 @@ def test_make_agg_feat_where_different_identity_feat(es): primitive=Count)) df = ft.calculate_feature_matrix(entityset=es, features=feats, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, where_cmp in enumerate(where_cmps): name = feats[i].get_name() @@ -260,6 +304,9 @@ def test_make_agg_feat_of_grandchild_entity(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[agg_feat.get_name()][0] assert (v == 10) @@ -282,6 +329,8 @@ def test_make_agg_feat_where_count_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0, 1])) + if isinstance(df, dd.DataFrame): + df = df.compute() name = feat.get_name() instances = df[name] v0, v1 = instances[0:2] @@ -308,6 +357,8 @@ def test_make_compare_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0, 1, 2])) + if isinstance(df, dd.DataFrame): + df = df.compute() name = feat.get_name() instances = df[name] v0, v1, v2 = instances[0:3] @@ -337,6 +388,9 @@ def test_make_agg_feat_where_count_and_device_type_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) name = feat.get_name() instances = df[name] assert (instances[0] == 1) @@ -363,6 +417,9 @@ def test_make_agg_feat_where_count_or_device_type_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) name = feat.get_name() instances = df[name] assert (instances[0] == 3) @@ -378,19 +435,16 @@ def test_make_agg_feat_of_agg_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[customer_sum_feat.get_name()][0] assert (v == 10) -def test_make_3_stacked_agg_feats(): - """ - Tests stacking 3 agg features. - - The test specifically uses non numeric indices to test how ancestor variables are handled - as dataframes are merged together - - """ - df = pd.DataFrame({ +@pytest.fixture +def pd_df(): + return pd.DataFrame({ "id": ["a", "b", "c", "d", "e"], "e1": ["h", "h", "i", "i", "j"], "e2": ["x", "x", "y", "y", "x"], @@ -398,10 +452,39 @@ def test_make_3_stacked_agg_feats(): "val": [1, 1, 1, 1, 1] }) + +@pytest.fixture +def dd_df(pd_df): + return dd.from_pandas(pd_df, npartitions=2) + + +@pytest.fixture(params=['pd_df', 'dd_df']) +def df(request): + return request.getfixturevalue(request.param) + + +def test_make_3_stacked_agg_feats(df): + """ + Tests stacking 3 agg features. + + The test specifically uses non numeric indices to test how ancestor variables are handled + as dataframes are merged together + + """ + if isinstance(df, dd.DataFrame): + pytest.xfail('normalize_entity fails with dask DataFrame') es = ft.EntitySet() + vtypes = { + 'id': variable_types.Index, + 'e1': variable_types.Categorical, + 'e2': variable_types.Categorical, + 'e3': variable_types.Categorical, + 'val': variable_types.Numeric + } es.entity_from_dataframe(dataframe=df, index="id", - entity_id="e0") + entity_id="e0", + variable_types=vtypes) es.normalize_entity(base_entity_id="e0", new_entity_id="e1", @@ -451,6 +534,9 @@ def test_make_dfeat_of_agg_feat_on_self(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[num_customers_feat.get_name()][0] assert (v == 3) @@ -478,6 +564,9 @@ def test_make_dfeat_of_agg_feat_through_parent(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[num_stores_feat.get_name()][0] assert (v == 3) @@ -507,6 +596,9 @@ def test_make_deep_agg_feat_of_dfeat_of_agg_feat(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[purchase_popularity.get_name()][0] assert (v == 38.0 / 10.0) @@ -525,21 +617,23 @@ def test_deep_agg_feat_chain(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array(['United States'])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') v = df[region_avg_feat.get_name()][0] assert (v == 17 / 3.) -def test_topn(es): - topn = ft.Feature(es['log']['product_id'], - parent_entity=es['customers'], +# NMostCommon not supported with Dask +def test_topn(pd_es): + topn = ft.Feature(pd_es['log']['product_id'], + parent_entity=pd_es['customers'], primitive=NMostCommon(n=2)) feature_set = FeatureSet([topn]) - calculator = FeatureSetCalculator(es, + calculator = FeatureSetCalculator(pd_es, time_last=None, feature_set=feature_set) df = calculator.run(np.array([0, 1, 2])) - true_results = pd.DataFrame([ ['toothpaste', 'coke zero'], ['coke zero', 'Haribo sugar-free gummy bears'], @@ -558,13 +652,14 @@ def test_topn(es): assert (pd.isnull(i1) and pd.isnull(i2)) or (i1 == i2) -def test_trend(es): - trend = ft.Feature([es['log']['value'], es['log']['datetime']], - parent_entity=es['customers'], +# Trend not supported with Dask +def test_trend(pd_es): + trend = ft.Feature([pd_es['log']['value'], pd_es['log']['datetime']], + parent_entity=pd_es['customers'], primitive=Trend) feature_set = FeatureSet([trend]) - calculator = FeatureSetCalculator(es, + calculator = FeatureSetCalculator(pd_es, time_last=None, feature_set=feature_set) df = calculator.run(np.array([0, 1, 2])) @@ -582,6 +677,8 @@ def test_direct_squared(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0, 1, 2])) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, row in df.iterrows(): assert (row[0] * row[0]) == row[1] @@ -595,6 +692,8 @@ def test_agg_empty_child(es): time_last=datetime(2011, 4, 8), feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() assert df["COUNT(log)"].iloc[0] == 0 @@ -617,6 +716,8 @@ def test_diamond_entityset(diamond_es): time_last=datetime(2011, 4, 8), feature_set=feature_set) df = calculator.run(np.array([0, 1, 2])) + if isinstance(df, dd.DataFrame): + df = df.compute() assert (df['SUM(stores.transactions.amount)'] == [94, 261, 128]).all() assert (df['SUM(customers.transactions.amount)'] == [72, 411, 0]).all() @@ -644,21 +745,62 @@ def test_two_relationships_to_single_entity(games_es): time_last=datetime(2011, 8, 28), feature_set=feature_set) df = calculator.run(np.array(range(3))) + if isinstance(df, dd.DataFrame): + df = df.compute() assert (df[home_team_mean.get_name()] == [1.5, 1.5, 2.5]).all() assert (df[away_team_mean.get_name()] == [1, 0.5, 2]).all() -def test_empty_child_dataframe(): +@pytest.fixture +def pd_parent_child(): parent_df = pd.DataFrame({"id": [1]}) child_df = pd.DataFrame({"id": [1, 2, 3], "parent_id": [1, 1, 1], "time_index": pd.date_range(start='1/1/2018', periods=3), "value": [10, 5, 2], "cat": ['a', 'a', 'b']}) - + return (parent_df, child_df) + + +@pytest.fixture +def dd_parent_child(pd_parent_child): + parent_df, child_df = pd_parent_child + parent_df = dd.from_pandas(parent_df, npartitions=2) + child_df = dd.from_pandas(child_df, npartitions=2) + return (parent_df, child_df) + + +@pytest.fixture(params=['pd_parent_child', 'dd_parent_child']) +def parent_child(request): + return request.getfixturevalue(request.param) + + +def test_empty_child_dataframe(parent_child): + parent_df, child_df = parent_child + if isinstance(parent_df, dd.DataFrame): + parent_vtypes = { + 'id': variable_types.Index + } + child_vtypes = { + 'id': variable_types.Index, + 'parent_id': variable_types.Numeric, + 'time_index': variable_types.Datetime, + 'value': variable_types.Numeric, + 'cat': variable_types.Categorical + } + else: + parent_vtypes = None + child_vtypes = None es = ft.EntitySet(id="blah") - es.entity_from_dataframe(entity_id="parent", dataframe=parent_df, index="id") - es.entity_from_dataframe(entity_id="child", dataframe=child_df, index="id", time_index="time_index") + es.entity_from_dataframe(entity_id="parent", + dataframe=parent_df, + index="id", + variable_types=parent_vtypes) + es.entity_from_dataframe(entity_id="child", + dataframe=child_df, + index="id", + time_index="time_index", + variable_types=child_vtypes) es.add_relationship(ft.Relationship(es["parent"]["id"], es["child"]["parent_id"])) # create regular agg @@ -676,24 +818,43 @@ def test_empty_child_dataframe(): trend_where = ft.Feature([es["child"]['value'], es["child"]['time_index']], parent_entity=es["parent"], where=where, primitive=Trend) n_most_common_where = ft.Feature(es["child"]['cat'], parent_entity=es["parent"], where=where, primitive=NMostCommon) + if isinstance(parent_df, dd.DataFrame): + features = [count, count_where] + names = [count.get_name(), count_where.get_name()] + values = [0, 0] + else: + features = [count, count_where, trend, trend_where, n_most_common, n_most_common_where] + names = [count.get_name(), count_where.get_name(), + trend.get_name(), trend_where.get_name(), + *n_most_common.get_names(), *n_most_common_where.get_names()] + values = [0, 0, + np.nan, np.nan, + *np.full(n_most_common.number_output_features, np.nan), *np.full(n_most_common_where.number_output_features, np.nan)] + # cutoff time before all rows fm = ft.calculate_feature_matrix(entityset=es, - features=[count, count_where, trend, trend_where, n_most_common, n_most_common_where], + features=features, cutoff_time=pd.Timestamp("12/31/2017")) - names = [count.get_name(), count_where.get_name(), - trend.get_name(), trend_where.get_name(), - *n_most_common.get_names(), *n_most_common_where.get_names()] - values = [0, 0, - np.nan, np.nan, - *np.full(n_most_common.number_output_features, np.nan), *np.full(n_most_common_where.number_output_features, np.nan)] + if isinstance(fm, dd.DataFrame): + fm = fm.compute() assert_array_equal(fm[names], [values]) # cutoff time after all rows, but where clause filters all rows + if isinstance(parent_df, dd.DataFrame): + features = [count_where] + names = [count_where.get_name()] + values = [0] + else: + features = [count_where, trend_where, n_most_common_where] + names = [count_where.get_name(), trend_where.get_name(), *n_most_common_where.get_names()] + values = [0, np.nan, *np.full(n_most_common_where.number_output_features, np.nan)] + fm2 = ft.calculate_feature_matrix(entityset=es, - features=[count_where, trend_where, n_most_common_where], + features=features, cutoff_time=pd.Timestamp("1/4/2018")) - names = [count_where.get_name(), trend_where.get_name(), *n_most_common_where.get_names()] - assert_array_equal(fm2[names], [[0, np.nan, *np.full(n_most_common_where.number_output_features, np.nan)]]) + if isinstance(fm2, dd.DataFrame): + fm2 = fm2.compute() + assert_array_equal(fm2[names], [values]) def test_with_features_built_from_es_metadata(es): @@ -705,11 +866,18 @@ def test_with_features_built_from_es_metadata(es): time_last=None, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + df.index = pd.Int64Index(df.index) v = df[agg_feat.get_name()][0] assert (v == 10) +# TODO: Fails with Dask (conflicting aggregation primitives) def test_handles_primitive_function_name_uniqueness(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Fails with Dask due conflicting aggregation primitive names") + class SumTimesN(AggregationPrimitive): name = "sum_times_n" input_types = [Numeric] @@ -724,11 +892,25 @@ def my_function(values): return my_function + def get_dask_aggregation(self): + def chunk(s): + return s.sum() + + def agg(s): + return s.sum() + + def finalize(s): + return s * self.n + + return dd.Aggregation(self.name, chunk=chunk, agg=agg, finalize=finalize) + # works as expected f1 = ft.Feature(es["log"]["value"], parent_entity=es["customers"], primitive=SumTimesN(n=1)) fm = ft.calculate_feature_matrix(features=[f1], entityset=es) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') value_sum = pd.Series([56, 26, 0]) assert all(fm[f1.get_name()].sort_index() == value_sum) @@ -737,11 +919,15 @@ def my_function(values): parent_entity=es["customers"], primitive=SumTimesN(n=2)) fm = ft.calculate_feature_matrix(features=[f2], entityset=es) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') double_value_sum = pd.Series([112, 52, 0]) assert all(fm[f2.get_name()].sort_index() == double_value_sum) # same primitive, same variable, different args fm = ft.calculate_feature_matrix(features=[f1, f2], entityset=es) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') assert all(fm[f1.get_name()].sort_index() == value_sum) assert all(fm[f2.get_name()].sort_index() == double_value_sum) @@ -754,6 +940,8 @@ def my_function(values): parent_entity=es["customers"], primitive=NumTrue) fm = ft.calculate_feature_matrix(features=[f3, f4], entityset=es) + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') purchased_sum = pd.Series([10, 1, 1]) assert all(fm[f3.get_name()].sort_index() == value_sum) assert all(fm[f4.get_name()].sort_index() == purchased_sum)\ @@ -812,14 +1000,15 @@ def get_function(self): assert all(fm[f7.get_name()].sort_index() == value_sum) -def test_returns_order_of_instance_ids(es): - feature_set = FeatureSet([ft.Feature(es['customers']['age'])]) - calculator = FeatureSetCalculator(es, +# No order guarantees w/ Dask +def test_returns_order_of_instance_ids(pd_es): + feature_set = FeatureSet([ft.Feature(pd_es['customers']['age'])]) + calculator = FeatureSetCalculator(pd_es, time_last=None, feature_set=feature_set) instance_ids = [0, 1, 2] - assert list(es['customers'].df['id']) != instance_ids + assert list(pd_es['customers'].df['id']) != instance_ids df = calculator.run(np.array(instance_ids)) @@ -832,9 +1021,14 @@ def test_calls_progress_callback(es): direct = ft.Feature(es['cohorts']['cohort_name'], es['customers']) agg = ft.Feature(es["sessions"]["id"], parent_entity=es['customers'], primitive=Count) agg_apply = ft.Feature(es["log"]["datetime"], parent_entity=es['customers'], primitive=TimeSinceLast) # this feature is handle differently than simple features - trans = ft.Feature(agg, primitive=CumSum) + trans = ft.Feature(agg, primitive=Negate) + trans_full = ft.Feature(agg, primitive=CumSum) groupby_trans = ft.Feature(agg, primitive=CumSum, groupby=es["customers"]["cohort"]) - all_features = [identity, direct, agg, agg_apply, trans, groupby_trans] + + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + all_features = [identity, direct, agg, trans] + else: + all_features = [identity, direct, agg, agg_apply, trans, trans_full, groupby_trans] feature_set = FeatureSet(all_features) calculator = FeatureSetCalculator(es, @@ -867,7 +1061,8 @@ def __call__(self, update): assert np.isclose(mock_progress_callback.total, 1) -def test_precalculated_features(es): +# precalcualted_features is only used with approximate +def test_precalculated_features(pd_es): error_msg = 'This primitive should never be used because the features are precalculated' class ErrorPrim(AggregationPrimitive): @@ -881,14 +1076,14 @@ def error(s): raise RuntimeError(error_msg) return error - value = ft.Feature(es['log']['value']) + value = ft.Feature(pd_es['log']['value']) agg = ft.Feature(value, - parent_entity=es['sessions'], + parent_entity=pd_es['sessions'], primitive=ErrorPrim) agg2 = ft.Feature(agg, - parent_entity=es['customers'], + parent_entity=pd_es['customers'], primitive=ErrorPrim) - direct = ft.Feature(agg2, entity=es['sessions']) + direct = ft.Feature(agg2, entity=pd_es['sessions']) # Set up a FeatureSet which knows which features are precalculated. precalculated_feature_trie = Trie(default=set, path_constructor=RelationshipPath) @@ -901,7 +1096,7 @@ def error(s): precalculated_fm_trie = Trie(path_constructor=RelationshipPath) precalculated_fm_trie.get_node(direct.relationship_path).value = parent_fm - calculator = FeatureSetCalculator(es, + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set, precalculated_features=precalculated_fm_trie) @@ -912,4 +1107,4 @@ def error(s): # Calculating without precalculated features should error. with pytest.raises(RuntimeError, match=error_msg): - FeatureSetCalculator(es, feature_set=FeatureSet([direct])).run(instance_ids) + FeatureSetCalculator(pd_es, feature_set=FeatureSet([direct])).run(instance_ids) diff --git a/featuretools/tests/conftest.py b/featuretools/tests/conftest.py index 863403c320..aa761b3a3b 100644 --- a/featuretools/tests/conftest.py +++ b/featuretools/tests/conftest.py @@ -1,5 +1,6 @@ import copy +import dask.dataframe as dd import pandas as pd import pytest @@ -18,7 +19,7 @@ def make_int_es(): @pytest.fixture -def es(make_es): +def pd_es(make_es): return copy.deepcopy(make_es) @@ -28,7 +29,25 @@ def int_es(make_int_es): @pytest.fixture -def diamond_es(): +def dask_es(make_es): + dask_es = copy.deepcopy(make_es) + for entity in dask_es.entities: + entity.df = dd.from_pandas(entity.df.reset_index(drop=True), npartitions=2) + return dask_es + + +@pytest.fixture(params=['pd_es', 'dask_es']) +def es(request): + return request.getfixturevalue(request.param) + + +@pytest.fixture(params=['pd_diamond_es', 'dask_diamond_es']) +def diamond_es(request): + return request.getfixturevalue(request.param) + + +@pytest.fixture +def pd_diamond_es(): countries_df = pd.DataFrame({ 'id': range(2), 'name': ['US', 'Canada'] @@ -75,7 +94,26 @@ def diamond_es(): @pytest.fixture -def home_games_es(): +def dask_diamond_es(pd_diamond_es): + entities = {} + for entity in pd_diamond_es.entities: + entities[entity.id] = (dd.from_pandas(entity.df, npartitions=2), entity.index, None, entity.variable_types) + + relationships = [(rel.parent_entity.id, + rel.parent_variable.name, + rel.child_entity.id, + rel.child_variable.name) for rel in pd_diamond_es.relationships] + + return ft.EntitySet(id=pd_diamond_es.id, entities=entities, relationships=relationships) + + +@pytest.fixture(params=['pd_home_games_es', 'dask_home_games_es']) +def home_games_es(request): + return request.getfixturevalue(request.param) + + +@pytest.fixture +def pd_home_games_es(): teams = pd.DataFrame({ 'id': range(3), 'name': ['Breakers', 'Spirit', 'Thorns'] @@ -93,6 +131,20 @@ def home_games_es(): relationships=relationships) +@pytest.fixture +def dask_home_games_es(pd_home_games_es): + entities = {} + for entity in pd_home_games_es.entities: + entities[entity.id] = (dd.from_pandas(entity.df, npartitions=2), entity.index, None, entity.variable_types) + + relationships = [(rel.parent_entity.id, + rel.parent_variable.name, + rel.child_entity.id, + rel.child_variable.name) for rel in pd_home_games_es.relationships] + + return ft.EntitySet(id=pd_home_games_es.id, entities=entities, relationships=relationships) + + @pytest.fixture def games_es(home_games_es): away_team = ft.Relationship(home_games_es['teams']['id'], diff --git a/featuretools/tests/entityset_tests/test_dask_es.py b/featuretools/tests/entityset_tests/test_dask_es.py new file mode 100644 index 0000000000..2e6335bb9a --- /dev/null +++ b/featuretools/tests/entityset_tests/test_dask_es.py @@ -0,0 +1,504 @@ +import dask.dataframe as dd +import pandas as pd +import pytest + +import featuretools as ft +from featuretools.entityset import EntitySet, Relationship + + +def test_create_entity_from_dask_df(pd_es): + dask_es = EntitySet(id="dask_es") + log_dask = dd.from_pandas(pd_es["log"].df, npartitions=2) + dask_es = dask_es.entity_from_dataframe( + entity_id="log_dask", + dataframe=log_dask, + index="id", + time_index="datetime", + variable_types=pd_es["log"].variable_types + ) + pd.testing.assert_frame_equal(pd_es["log"].df, dask_es["log_dask"].df.compute(), check_like=True) + + +def test_create_entity_with_non_numeric_index(pd_es, dask_es): + df = pd.DataFrame({"id": ["A_1", "A_2", "C", "D"], + "values": [1, 12, -34, 27]}) + dask_df = dd.from_pandas(df, npartitions=2) + + pd_es.entity_from_dataframe( + entity_id="new_entity", + dataframe=df, + index="id") + + dask_es.entity_from_dataframe( + entity_id="new_entity", + dataframe=dask_df, + index="id", + variable_types={"id": ft.variable_types.Id, "values": ft.variable_types.Numeric}) + + pd.testing.assert_frame_equal(pd_es['new_entity'].df.reset_index(drop=True), dask_es['new_entity'].df.compute()) + + +def test_create_entityset_with_mixed_dataframe_types(pd_es, dask_es): + df = pd.DataFrame({"id": [0, 1, 2, 3], + "values": [1, 12, -34, 27]}) + dask_df = dd.from_pandas(df, npartitions=2) + + # Test error is raised when trying to add Dask entity to entitset with existing pandas entities + err_msg = "All entity dataframes must be of the same type. " \ + "Cannot add entity of type {} to an entityset with existing entities " \ + "of type {}".format(type(dask_df), type(pd_es.entities[0].df)) + + with pytest.raises(ValueError, match=err_msg): + pd_es.entity_from_dataframe( + entity_id="new_entity", + dataframe=dask_df, + index="id") + + # Test error is raised when trying to add pandas entity to entitset with existing dask entities + err_msg = "All entity dataframes must be of the same type. " \ + "Cannot add entity of type {} to an entityset with existing entities " \ + "of type {}".format(type(df), type(dask_es.entities[0].df)) + + with pytest.raises(ValueError, match=err_msg): + dask_es.entity_from_dataframe( + entity_id="new_entity", + dataframe=df, + index="id") + + +def test_add_last_time_indexes(): + pd_es = EntitySet(id="pd_es") + dask_es = EntitySet(id="dask_es") + + sessions = pd.DataFrame({"id": [0, 1, 2, 3], + "user": [1, 2, 1, 3], + "time": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')], + "strings": ["I am a string", + "23", + "abcdef ghijk", + ""]}) + sessions_dask = dd.from_pandas(sessions, npartitions=2) + sessions_vtypes = { + "id": ft.variable_types.Id, + "user": ft.variable_types.Id, + "time": ft.variable_types.DatetimeTimeIndex, + "strings": ft.variable_types.Text + } + + transactions = pd.DataFrame({"id": [0, 1, 2, 3, 4, 5], + "session_id": [0, 0, 1, 2, 2, 3], + "amount": [1.23, 5.24, 123.52, 67.93, 40.34, 50.13], + "time": [pd.to_datetime('2019-01-10 03:53'), + pd.to_datetime('2019-01-10 04:12'), + pd.to_datetime('2019-02-03 10:34'), + pd.to_datetime('2019-01-01 12:35'), + pd.to_datetime('2019-01-01 12:49'), + pd.to_datetime('2017-08-25 04:53')]}) + transactions_dask = dd.from_pandas(transactions, npartitions=2) + transactions_vtypes = { + "id": ft.variable_types.Id, + "session_id": ft.variable_types.Id, + "amount": ft.variable_types.Numeric, + "time": ft.variable_types.DatetimeTimeIndex, + } + + pd_es.entity_from_dataframe(entity_id="sessions", dataframe=sessions, index="id", time_index="time") + dask_es.entity_from_dataframe(entity_id="sessions", dataframe=sessions_dask, index="id", time_index="time", variable_types=sessions_vtypes) + + pd_es.entity_from_dataframe(entity_id="transactions", dataframe=transactions, index="id", time_index="time") + dask_es.entity_from_dataframe(entity_id="transactions", dataframe=transactions_dask, index="id", time_index="time", variable_types=transactions_vtypes) + + new_rel = Relationship(pd_es["sessions"]["id"], + pd_es["transactions"]["session_id"]) + dask_rel = Relationship(dask_es["sessions"]["id"], + dask_es["transactions"]["session_id"]) + + pd_es = pd_es.add_relationship(new_rel) + dask_es = dask_es.add_relationship(dask_rel) + + assert pd_es['sessions'].last_time_index is None + assert dask_es['sessions'].last_time_index is None + + pd_es.add_last_time_indexes() + dask_es.add_last_time_indexes() + + pd.testing.assert_series_equal(pd_es['sessions'].last_time_index.sort_index(), dask_es['sessions'].last_time_index.compute(), check_names=False) + + +def test_create_entity_with_make_index(): + values = [1, 12, -23, 27] + df = pd.DataFrame({"values": values}) + dask_df = dd.from_pandas(df, npartitions=2) + dask_es = EntitySet(id="dask_es") + vtypes = {"values": ft.variable_types.Numeric} + dask_es.entity_from_dataframe(entity_id="new_entity", dataframe=dask_df, make_index=True, index="new_index", variable_types=vtypes) + + expected_df = pd.DataFrame({"new_index": range(len(values)), "values": values}) + pd.testing.assert_frame_equal(expected_df, dask_es['new_entity'].df.compute()) + + +def test_single_table_dask_entityset(): + primitives_list = ['absolute', 'is_weekend', 'year', 'day', 'num_characters', 'num_words'] + + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [0, 1, 2, 3], + "values": [1, 12, -34, 27], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')], + "strings": ["I am a string", + "23", + "abcdef ghijk", + ""]}) + values_dd = dd.from_pandas(df, npartitions=2) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.Datetime, + "strings": ft.variable_types.Text + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + variable_types=vtypes) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + variable_types={"strings": ft.variable_types.Text}) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list) + + # Use the same columns and make sure both indexes are sorted the same + dask_computed_fm = dask_fm.compute().set_index('id').loc[fm.index][fm.columns] + pd.testing.assert_frame_equal(fm, dask_computed_fm) + + +def test_single_table_dask_entityset_ids_not_sorted(): + primitives_list = ['absolute', 'is_weekend', 'year', 'day', 'num_characters', 'num_words'] + + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [2, 0, 1, 3], + "values": [1, 12, -34, 27], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')], + "strings": ["I am a string", + "23", + "abcdef ghijk", + ""]}) + values_dd = dd.from_pandas(df, npartitions=2) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.Datetime, + "strings": ft.variable_types.Text + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + variable_types=vtypes) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + variable_types={"strings": ft.variable_types.Text}) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list) + + # Make sure both indexes are sorted the same + pd.testing.assert_frame_equal(fm, dask_fm.compute().set_index('id').loc[fm.index]) + + +def test_single_table_dask_entityset_with_instance_ids(): + primitives_list = ['absolute', 'is_weekend', 'year', 'day', 'num_characters', 'num_words'] + instance_ids = [0, 1, 3] + + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [0, 1, 2, 3], + "values": [1, 12, -34, 27], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')], + "strings": ["I am a string", + "23", + "abcdef ghijk", + ""]}) + + values_dd = dd.from_pandas(df, npartitions=2) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.Datetime, + "strings": ft.variable_types.Text + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + variable_types=vtypes) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list, + instance_ids=instance_ids) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + variable_types={"strings": ft.variable_types.Text}) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list, + instance_ids=instance_ids) + + # Make sure both indexes are sorted the same + pd.testing.assert_frame_equal(fm, dask_fm.compute().set_index('id').loc[fm.index]) + + +def test_single_table_dask_entityset_single_cutoff_time(): + primitives_list = ['absolute', 'is_weekend', 'year', 'day', 'num_characters', 'num_words'] + + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [0, 1, 2, 3], + "values": [1, 12, -34, 27], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')], + "strings": ["I am a string", + "23", + "abcdef ghijk", + ""]}) + values_dd = dd.from_pandas(df, npartitions=2) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.Datetime, + "strings": ft.variable_types.Text + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + variable_types=vtypes) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list, + cutoff_time=pd.Timestamp("2019-01-05 04:00")) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + variable_types={"strings": ft.variable_types.Text}) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list, + cutoff_time=pd.Timestamp("2019-01-05 04:00")) + + # Make sure both indexes are sorted the same + pd.testing.assert_frame_equal(fm, dask_fm.compute().set_index('id').loc[fm.index]) + + +def test_single_table_dask_entityset_cutoff_time_df(): + primitives_list = ['absolute', 'is_weekend', 'year', 'day', 'num_characters', 'num_words'] + + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [0, 1, 2], + "values": [1, 12, -34], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01')], + "strings": ["I am a string", + "23", + "abcdef ghijk"]}) + values_dd = dd.from_pandas(df, npartitions=2) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.DatetimeTimeIndex, + "strings": ft.variable_types.Text + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + time_index="dates", + variable_types=vtypes) + ids = [0, 1, 2, 0] + times = [pd.Timestamp("2019-01-05 04:00"), + pd.Timestamp("2019-01-05 04:00"), + pd.Timestamp("2019-01-05 04:00"), + pd.Timestamp("2019-01-15 04:00")] + labels = [True, False, True, False] + cutoff_times = pd.DataFrame({"id": ids, "time": times, "labels": labels}, columns=["id", "time", "labels"]) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list, + cutoff_time=cutoff_times) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + time_index="dates", + variable_types={"strings": ft.variable_types.Text}) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list, + cutoff_time=cutoff_times) + # Because row ordering with Dask is not guaranteed, we need to sort on two columns to make sure that values + # for instance id 0 are compared correctly. Also, make sure the boolean column has the same dtype. + fm = fm.sort_values(['id', 'labels']) + dask_fm = dask_fm.compute().set_index('id').sort_values(['id', 'labels']) + dask_fm['IS_WEEKEND(dates)'] = dask_fm['IS_WEEKEND(dates)'].astype(fm['IS_WEEKEND(dates)'].dtype) + pd.testing.assert_frame_equal(fm, dask_fm) + + +def test_single_table_dask_entityset_dates_not_sorted(): + dask_es = EntitySet(id="dask_es") + df = pd.DataFrame({"id": [0, 1, 2, 3], + "values": [1, 12, -34, 27], + "dates": [pd.to_datetime('2019-01-10'), + pd.to_datetime('2019-02-03'), + pd.to_datetime('2019-01-01'), + pd.to_datetime('2017-08-25')]}) + + primitives_list = ['absolute', 'is_weekend', 'year', 'day'] + values_dd = dd.from_pandas(df, npartitions=1) + vtypes = { + "id": ft.variable_types.Id, + "values": ft.variable_types.Numeric, + "dates": ft.variable_types.Datetime, + } + dask_es.entity_from_dataframe(entity_id="data", + dataframe=values_dd, + index="id", + time_index="dates", + variable_types=vtypes) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="data", + trans_primitives=primitives_list, + max_depth=1) + + pd_es = ft.EntitySet(id="pd_es") + pd_es.entity_from_dataframe(entity_id="data", + dataframe=df, + index="id", + time_index="dates") + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="data", + trans_primitives=primitives_list, + max_depth=1) + + pd.testing.assert_frame_equal(fm, dask_fm.compute().set_index('id').loc[fm.index]) + + +def test_secondary_time_index(): + log_df = pd.DataFrame() + log_df['id'] = [0, 1, 2, 3] + log_df['scheduled_time'] = pd.to_datetime([ + "2019-01-01", + "2019-01-01", + "2019-01-01", + "2019-01-01" + ]) + log_df['departure_time'] = pd.to_datetime([ + "2019-02-01 09:00", + "2019-02-06 10:00", + "2019-02-12 10:00", + "2019-03-01 11:30"]) + log_df['arrival_time'] = pd.to_datetime([ + "2019-02-01 11:23", + "2019-02-06 12:45", + "2019-02-12 13:53", + "2019-03-01 14:07" + ]) + log_df['delay'] = [-2, 10, 60, 0] + log_df['flight_id'] = [0, 1, 0, 1] + log_dask = dd.from_pandas(log_df, npartitions=2) + + flights_df = pd.DataFrame() + flights_df['id'] = [0, 1, 2, 3] + flights_df['origin'] = ["BOS", "LAX", "BOS", "LAX"] + flights_dask = dd.from_pandas(flights_df, npartitions=2) + + pd_es = ft.EntitySet("flights") + dask_es = ft.EntitySet("flights_dask") + + pd_es.entity_from_dataframe(entity_id='logs', + dataframe=log_df, + index="id", + time_index="scheduled_time", + secondary_time_index={ + 'arrival_time': ['departure_time', 'delay']}) + + log_vtypes = { + "id": ft.variable_types.Id, + "scheduled_time": ft.variable_types.DatetimeTimeIndex, + "departure_time": ft.variable_types.DatetimeTimeIndex, + "arrival_time": ft.variable_types.DatetimeTimeIndex, + "delay": ft.variable_types.Numeric, + "flight_id": ft.variable_types.Id + } + dask_es.entity_from_dataframe(entity_id='logs', + dataframe=log_dask, + index="id", + variable_types=log_vtypes, + time_index="scheduled_time", + secondary_time_index={ + 'arrival_time': ['departure_time', 'delay']}) + + pd_es.entity_from_dataframe('flights', flights_df, index="id") + flights_vtypes = pd_es['flights'].variable_types + dask_es.entity_from_dataframe('flights', flights_dask, index="id", variable_types=flights_vtypes) + + new_rel = ft.Relationship(pd_es['flights']['id'], pd_es['logs']['flight_id']) + dask_rel = ft.Relationship(dask_es['flights']['id'], dask_es['logs']['flight_id']) + pd_es.add_relationship(new_rel) + dask_es.add_relationship(dask_rel) + + cutoff_df = pd.DataFrame() + cutoff_df['id'] = [0, 1, 1] + cutoff_df['time'] = pd.to_datetime(['2019-02-02', '2019-02-02', '2019-02-20']) + + fm, _ = ft.dfs(entityset=pd_es, + target_entity="logs", + cutoff_time=cutoff_df, + agg_primitives=["max"], + trans_primitives=["month"]) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity="logs", + cutoff_time=cutoff_df, + agg_primitives=["max"], + trans_primitives=["month"]) + + # Make sure both matrixes are sorted the same + pd.testing.assert_frame_equal(fm.sort_values('delay'), dask_fm.compute().set_index('id').sort_values('delay')) diff --git a/featuretools/tests/entityset_tests/test_entity.py b/featuretools/tests/entityset_tests/test_entity.py index 7e47707806..ffe7b4c7bf 100644 --- a/featuretools/tests/entityset_tests/test_entity.py +++ b/featuretools/tests/entityset_tests/test_entity.py @@ -3,6 +3,7 @@ import numpy as np import pandas as pd import pytest +from dask import dataframe as dd import featuretools as ft from featuretools import variable_types @@ -59,7 +60,7 @@ def test_eq(es): assert es['log'].__eq__(es['log'], deep=True) assert es['log'].__eq__(other_es['log'], deep=True) - assert (es['log'].df['latlong'] == latlong).all() + assert all(es['log'].df['latlong'].eq(latlong)) other_es['log'].add_interesting_values() assert not es['log'].__eq__(other_es['log'], deep=True) @@ -81,7 +82,7 @@ def test_eq(es): def test_update_data(es): df = es['customers'].df.copy() - df['new'] = [1, 2, 3] + df['new'] = pd.Series([1, 2, 3]) error_text = 'Updated dataframe is missing new cohort column' with pytest.raises(ValueError, match=error_text): @@ -92,20 +93,42 @@ def test_update_data(es): es['customers'].update_data(df) # test already_sorted on entity without time index - df = es["sessions"].df.copy(deep=True) - df["id"].iloc[1:3] = [2, 1] - es["sessions"].update_data(df.copy(deep=True)) - assert es["sessions"].df["id"].iloc[1] == 2 # no sorting since time index not defined - es["sessions"].update_data(df.copy(deep=True), already_sorted=True) - assert es["sessions"].df["id"].iloc[1] == 2 + df = es["sessions"].df.copy() + if isinstance(df, dd.DataFrame): + updated_id = df['id'].compute() + else: + updated_id = df['id'] + updated_id.iloc[1:3] = [2, 1] + df["id"] = updated_id + es["sessions"].update_data(df.copy()) + sessions_df = es['sessions'].df + if isinstance(sessions_df, dd.DataFrame): + sessions_df = sessions_df.compute() + assert sessions_df["id"].iloc[1] == 2 # no sorting since time index not defined + es["sessions"].update_data(df.copy(), already_sorted=True) + sessions_df = es['sessions'].df + if isinstance(sessions_df, dd.DataFrame): + sessions_df = sessions_df.compute() + assert sessions_df["id"].iloc[1] == 2 # test already_sorted on entity with time index - df = es["customers"].df.copy(deep=True) - df["signup_date"].iloc[0] = datetime(2011, 4, 11) - es["customers"].update_data(df.copy(deep=True)) - assert es["customers"].df["id"].iloc[0] == 0 - es["customers"].update_data(df.copy(deep=True), already_sorted=True) - assert es["customers"].df["id"].iloc[0] == 2 + df = es["customers"].df.copy() + if isinstance(df, dd.DataFrame): + updated_signup = df['signup_date'].compute() + else: + updated_signup = df['signup_date'] + updated_signup.iloc[0] = datetime(2011, 4, 11) + df['signup_date'] = updated_signup + es["customers"].update_data(df.copy(), already_sorted=True) + customers_df = es['customers'].df + if isinstance(customers_df, dd.DataFrame): + customers_df = customers_df.compute() + assert customers_df["id"].iloc[0] == 2 + + # only pandas allows for sorting: + if isinstance(df, pd.DataFrame): + es["customers"].update_data(df.copy()) + assert es['customers'].df["id"].iloc[0] == 0 def test_query_by_values_returns_rows_in_given_order(): @@ -118,7 +141,7 @@ def test_query_by_values_returns_rows_in_given_order(): es = ft.EntitySet() es = es.entity_from_dataframe(entity_id="test", dataframe=data, index="id", time_index="time", variable_types={ - "value": ft.variable_types.Categorical + "value": ft.variable_types.Categorical }) query = es['test'].query_by_values(['b', 'a'], variable_id='value') assert np.array_equal(query['id'], [1, 3, 4, 5]) @@ -129,6 +152,8 @@ def test_query_by_values_secondary_time_index(es): all_instances = [0, 1, 2] result = es['customers'].query_by_values(all_instances, time_last=end) + if isinstance(result, dd.DataFrame): + result = result.compute().set_index('id') for col in ["cancel_date", "cancel_reason"]: nulls = result.loc[all_instances][col].isnull() == [False, True, True] assert nulls.all(), "Some instance has data it shouldn't for column %s" % col @@ -143,7 +168,7 @@ def test_delete_variables(es): for var in to_delete: assert var not in variable_names - assert var not in entity.df + assert var not in entity.df.columns def test_variable_types_unmodified(): diff --git a/featuretools/tests/entityset_tests/test_es.py b/featuretools/tests/entityset_tests/test_es.py index 8d9565384f..6ca991269b 100644 --- a/featuretools/tests/entityset_tests/test_es.py +++ b/featuretools/tests/entityset_tests/test_es.py @@ -1,6 +1,7 @@ import copy from datetime import datetime +import dask.dataframe as dd import pandas as pd import pytest @@ -33,13 +34,23 @@ def test_operations_invalidate_metadata(es): assert new_es._data_description is None assert new_es.metadata is not None # generated after access assert new_es._data_description is not None - + if isinstance(es['customers'].df, dd.DataFrame): + customers_vtypes = es["customers"].variable_types + customers_vtypes['signup_date'] = variable_types.Datetime + else: + customers_vtypes = None new_es.entity_from_dataframe("customers", es["customers"].df, - index=es["customers"].index) + index=es["customers"].index, + variable_types=customers_vtypes) + if isinstance(es['sessions'].df, dd.DataFrame): + sessions_vtypes = es["sessions"].variable_types + else: + sessions_vtypes = None new_es.entity_from_dataframe("sessions", es["sessions"].df, - index=es["sessions"].index) + index=es["sessions"].index, + variable_types=sessions_vtypes) assert new_es._data_description is None assert new_es.metadata is not None assert new_es._data_description is not None @@ -61,10 +72,12 @@ def test_operations_invalidate_metadata(es): assert new_es.metadata is not None assert new_es._data_description is not None - new_es.add_interesting_values() - assert new_es._data_description is None - assert new_es.metadata is not None - assert new_es._data_description is not None + # automatically adding interesting values not supported in Dask + if any(isinstance(entity.df, pd.DataFrame) for entity in new_es.entities): + new_es.add_interesting_values() + assert new_es._data_description is None + assert new_es.metadata is not None + assert new_es._data_description is not None def test_reset_metadata(es): @@ -101,11 +114,15 @@ def test_add_relationship_errors_on_dtype_mismatch(es): 'value_2': variable_types.Numeric, 'latlong': variable_types.LatLong, 'latlong2': variable_types.LatLong, + 'zipcode': variable_types.ZIPCode, + 'countrycode': variable_types.CountryCode, + 'subregioncode': variable_types.SubRegionCode, 'value_many_nans': variable_types.Numeric, 'priority_level': variable_types.Ordinal, 'purchased': variable_types.Boolean, 'comments': variable_types.Text } + assert set(log_variable_types) == set(log_2_df.columns) es.entity_from_dataframe(entity_id='log2', dataframe=log_2_df, index='id', @@ -118,8 +135,23 @@ def test_add_relationship_errors_on_dtype_mismatch(es): es.add_relationship(mismatch) +def test_add_relationship_empty_child_convert_dtype(es): + relationship = ft.Relationship(es["sessions"]["id"], es["log"]["session_id"]) + es['log'].df = pd.DataFrame(columns=es['log'].df.columns) + assert len(es['log'].df) == 0 + assert es['log'].df['session_id'].dtype == 'object' + + es.relationships.remove(relationship) + assert(relationship not in es.relationships) + + es.add_relationship(relationship) + assert es['log'].df['session_id'].dtype == 'int64' + + def test_query_by_id(es): df = es['log'].query_by_values(instance_vals=[0]) + if isinstance(df, dd.DataFrame): + df = df.compute() assert df['id'].values[0] == 0 @@ -127,19 +159,23 @@ def test_query_by_id_with_time(es): df = es['log'].query_by_values( instance_vals=[0, 1, 2, 3, 4], time_last=datetime(2011, 4, 9, 10, 30, 2 * 6)) - - assert df['id'].tolist() == [0, 1, 2] + if isinstance(df, dd.DataFrame): + df = df.compute() + assert list(df['id'].values) == [0, 1, 2] def test_query_by_variable_with_time(es): df = es['log'].query_by_values( instance_vals=[0, 1, 2], variable_id='session_id', time_last=datetime(2011, 4, 9, 10, 50, 0)) + if isinstance(df, dd.DataFrame): + df = df.compute() true_values = [ i * 5 for i in range(5)] + [i * 1 for i in range(4)] + [0] - assert df['id'].tolist() == list(range(10)) - assert df['value'].tolist() == true_values + + assert list(df['id'].values) == list(range(10)) + assert list(df['value'].values) == true_values def test_query_by_variable_with_training_window(es): @@ -147,22 +183,40 @@ def test_query_by_variable_with_training_window(es): instance_vals=[0, 1, 2], variable_id='session_id', time_last=datetime(2011, 4, 9, 10, 50, 0), training_window='15m') + if isinstance(df, dd.DataFrame): + df = df.compute() - assert df['id'].tolist() == [9] - assert df['value'].tolist() == [0] + assert list(df['id'].values) == [9] + assert list(df['value'].values) == [0] def test_query_by_indexed_variable(es): df = es['log'].query_by_values( instance_vals=['taco clock'], variable_id='product_id') + if isinstance(df, dd.DataFrame): + df = df.compute() + + assert list(df['id'].values) == [15, 16] + + +@pytest.fixture +def pd_df(): + return pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'c']}) + + +@pytest.fixture +def dd_df(pd_df): + return dd.from_pandas(pd_df, npartitions=2) + - assert df['id'].tolist() == [15, 16] +@pytest.fixture(params=['pd_df', 'dd_df']) +def df(request): + return request.getfixturevalue(request.param) -def test_check_variables_and_dataframe(): +def test_check_variables_and_dataframe(df): # matches - df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a']}) vtypes = {'id': variable_types.Categorical, 'category': variable_types.Categorical} es = EntitySet(id='test') @@ -171,8 +225,7 @@ def test_check_variables_and_dataframe(): assert es.entity_dict['test_entity'].variable_types['category'] == variable_types.Categorical -def test_make_index_variable_ordering(): - df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a']}) +def test_make_index_variable_ordering(df): vtypes = {'id': variable_types.Categorical, 'category': variable_types.Categorical} @@ -185,9 +238,8 @@ def test_make_index_variable_ordering(): assert es.entity_dict['test_entity'].df.columns[0] == 'id1' -def test_extra_variable_type(): +def test_extra_variable_type(df): # more variables - df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a']}) vtypes = {'id': variable_types.Categorical, 'category': variable_types.Categorical, 'category2': variable_types.Categorical} @@ -207,35 +259,59 @@ def test_add_parent_not_index_varible(es): es['customers'][u'région_id'])) -def test_none_index(): - df = pd.DataFrame({'category': [1, 2, 3], 'category2': ['1', '2', '3']}) +@pytest.fixture +def pd_df2(): + return pd.DataFrame({'category': [1, 2, 3], 'category2': ['1', '2', '3']}) + + +@pytest.fixture +def dd_df2(pd_df2): + return dd.from_pandas(pd_df2, npartitions=2) + + +@pytest.fixture(params=['pd_df2', 'dd_df2']) +def df2(request): + return request.getfixturevalue(request.param) + + +def test_none_index(df2): vtypes = {'category': variable_types.Categorical, 'category2': variable_types.Categorical} es = EntitySet(id='test') es.entity_from_dataframe(entity_id='test_entity', - dataframe=df, + dataframe=df2, variable_types=vtypes) assert es['test_entity'].index == 'category' assert isinstance(es['test_entity']['category'], variable_types.Index) -def test_unknown_index(): - # more variables - df = pd.DataFrame({'category': ['a', 'b', 'a']}) +@pytest.fixture +def pd_df3(): + return pd.DataFrame({'category': [1, 2, 3]}) + + +@pytest.fixture +def dd_df3(pd_df3): + return dd.from_pandas(pd_df3, npartitions=2) + + +@pytest.fixture(params=['pd_df3', 'dd_df3']) +def df3(request): + return request.getfixturevalue(request.param) + + +def test_unknown_index(df3): vtypes = {'category': variable_types.Categorical} es = EntitySet(id='test') es.entity_from_dataframe(entity_id='test_entity', index='id', - variable_types=vtypes, dataframe=df) + variable_types=vtypes, dataframe=df3) assert es['test_entity'].index == 'id' - assert es['test_entity'].df['id'].tolist() == list(range(3)) + assert list(es['test_entity'].df['id']) == list(range(3)) -def test_doesnt_remake_index(): - # more variables - df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a']}) - +def test_doesnt_remake_index(df): error_text = "Cannot make index: index variable already present" with pytest.raises(RuntimeError, match=error_text): es = EntitySet(id='test') @@ -245,32 +321,47 @@ def test_doesnt_remake_index(): dataframe=df) -def test_bad_time_index_variable(): - df = pd.DataFrame({'category': ['a', 'b', 'a']}) - +def test_bad_time_index_variable(df3): error_text = "Time index not found in dataframe" with pytest.raises(LookupError, match=error_text): es = EntitySet(id='test') es.entity_from_dataframe(entity_id='test_entity', index="id", - dataframe=df, + dataframe=df3, time_index='time') -def test_converts_variable_types_on_init(): +@pytest.fixture +def pd_df4(): df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a'], 'category_int': [1, 2, 3], 'ints': ['1', '2', '3'], 'floats': ['1', '2', '3.0']}) df["category_int"] = df["category_int"].astype("category") + return df + + +@pytest.fixture +def dd_df4(pd_df4): + return dd.from_pandas(pd_df4, npartitions=2) + +@pytest.fixture(params=['pd_df4', 'dd_df4']) +def df4(request): + return request.getfixturevalue(request.param) + + +def test_converts_variable_types_on_init(df4): vtypes = {'id': variable_types.Categorical, 'ints': variable_types.Numeric, 'floats': variable_types.Numeric} + if isinstance(df4, dd.DataFrame): + vtypes['category'] = variable_types.Categorical + vtypes['category_int'] = variable_types.Categorical es = EntitySet(id='test') es.entity_from_dataframe(entity_id='test_entity', index='id', - variable_types=vtypes, dataframe=df) + variable_types=vtypes, dataframe=df4) entity_df = es['test_entity'].df assert entity_df['ints'].dtype.name in variable_types.PandasTypes._pandas_numerics @@ -281,16 +372,19 @@ def test_converts_variable_types_on_init(): assert isinstance(e['category_int'], variable_types.Categorical) -def test_converts_variable_type_after_init(): - df = pd.DataFrame({'id': [0, 1, 2], - 'category': ['a', 'b', 'a'], - 'ints': ['1', '2', '1']}) - - df["category"] = df["category"].astype("category") - +def test_converts_variable_type_after_init(df4): + df4["category"] = df4["category"].astype("category") + if isinstance(df4, dd.DataFrame): + vtypes = {'id': variable_types.Categorical, + 'category': variable_types.Categorical, + 'category_int': variable_types.Categorical, + 'ints': variable_types.Numeric, + 'floats': variable_types.Numeric} + else: + vtypes = None es = EntitySet(id='test') es.entity_from_dataframe(entity_id='test_entity', index='id', - dataframe=df) + dataframe=df4, variable_types=vtypes) e = es['test_entity'] df = es['test_entity'].df @@ -310,13 +404,36 @@ def test_converts_variable_type_after_init(): assert df['ints'].dtype.name == 'bool' -def test_converts_datetime(): +def test_errors_no_vtypes_dask(dd_df4): + es = EntitySet(id='test') + msg = 'Variable types cannot be inferred from Dask DataFrames, ' \ + 'use variable_types to provide type metadata for entity' + with pytest.raises(ValueError, match=msg): + es.entity_from_dataframe(entity_id='test_entity', index='id', + dataframe=dd_df4) + + +@pytest.fixture +def pd_datetime1(): + times = pd.date_range('1/1/2011', periods=3, freq='H') + time_strs = times.strftime('%Y-%m-%d') + return pd.DataFrame({'id': [0, 1, 2], 'time': time_strs}) + + +@pytest.fixture +def dd_datetime1(pd_datetime1): + return dd.from_pandas(pd_datetime1, npartitions=2) + + +@pytest.fixture(params=['pd_datetime1', 'dd_datetime1']) +def datetime1(request): + return request.getfixturevalue(request.param) + + +def test_converts_datetime(datetime1): # string converts to datetime correctly # This test fails without defining vtypes. Entityset # infers time column should be numeric type - times = pd.date_range('1/1/2011', periods=3, freq='H') - time_strs = times.strftime('%Y-%m-%d') - df = pd.DataFrame({'id': [0, 1, 2], 'time': time_strs}) vtypes = {'id': variable_types.Categorical, 'time': variable_types.Datetime} @@ -326,20 +443,39 @@ def test_converts_datetime(): index='id', time_index="time", variable_types=vtypes, - dataframe=df) + dataframe=datetime1) pd_col = es['test_entity'].df['time'] + if isinstance(pd_col, dd.Series): + pd_col = pd_col.compute() # assert type(es['test_entity']['time']) == variable_types.Datetime assert type(pd_col[0]) == pd.Timestamp -def test_handles_datetime_format(): - # check if we load according to the format string - # pass in an ambigious date +@pytest.fixture +def pd_datetime2(): datetime_format = "%d-%m-%Y" actual = pd.Timestamp('Jan 2, 2011') time_strs = [actual.strftime(datetime_format)] * 3 - df = pd.DataFrame( + return pd.DataFrame( {'id': [0, 1, 2], 'time_format': time_strs, 'time_no_format': time_strs}) + + +@pytest.fixture +def dd_datetime2(pd_datetime2): + return dd.from_pandas(pd_datetime2, npartitions=2) + + +@pytest.fixture(params=['pd_datetime2', 'dd_datetime2']) +def datetime2(request): + return request.getfixturevalue(request.param) + + +def test_handles_datetime_format(datetime2): + # check if we load according to the format string + # pass in an ambigious date + datetime_format = "%d-%m-%Y" + actual = pd.Timestamp('Jan 2, 2011') + vtypes = {'id': variable_types.Categorical, 'time_format': (variable_types.Datetime, {"format": datetime_format}), 'time_no_format': variable_types.Datetime} @@ -349,10 +485,13 @@ def test_handles_datetime_format(): entity_id='test_entity', index='id', variable_types=vtypes, - dataframe=df) + dataframe=datetime2) col_format = es['test_entity'].df['time_format'] col_no_format = es['test_entity'].df['time_no_format'] + if isinstance(col_format, dd.Series): + col_format = col_format.compute() + col_no_format = col_no_format.compute() # without formatting pandas gets it wrong assert (col_no_format != actual).all() @@ -360,6 +499,7 @@ def test_handles_datetime_format(): assert (col_format == actual).all() +# Inferring variable types and verifying typing not supported in dask def test_handles_datetime_mismatch(): # can't convert arbitrary strings df = pd.DataFrame({'id': [0, 1, 2], 'time': ['a', 'b', 'tomorrow']}) @@ -381,11 +521,28 @@ def test_entity_init(es): for i in range(3)], 'category': ['a', 'b', 'a'], 'number': [4, 5, 6]}) + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + df = dd.from_pandas(df, npartitions=2) vtypes = {'time': variable_types.Datetime} + if isinstance(df, dd.DataFrame): + extra_vtypes = { + 'id': variable_types.Categorical, + 'category': variable_types.Categorical, + 'number': variable_types.Numeric + } + vtypes.update(extra_vtypes) es.entity_from_dataframe('test_entity', df, index='id', time_index='time', variable_types=vtypes) - assert es['test_entity'].df.shape == df.shape + if isinstance(df, dd.DataFrame): + df_shape = (df.shape[0].compute(), df.shape[1]) + else: + df_shape = df.shape + if isinstance(es['test_entity'].df, dd.DataFrame): + es_df_shape = (es['test_entity'].df.shape[0].compute(), es['test_entity'].df.shape[1]) + else: + es_df_shape = es['test_entity'].df.shape + assert es_df_shape == df_shape assert es['test_entity'].index == 'id' assert es['test_entity'].time_index == 'time' assert set([v.id for v in es['test_entity'].variables]) == set(df.columns) @@ -394,14 +551,27 @@ def test_entity_init(es): assert set(es['test_entity'].df['id']) == set(df['id']) -def test_nonstr_column_names(): - df = pd.DataFrame({'a': [1, 2, 3], 'b': [4, 5, 6], 3: ['a', 'b', 'c']}) - es = ft.EntitySet(id='Failure') +@pytest.fixture +def pd_bad_df(): + return pd.DataFrame({'a': [1, 2, 3], 'b': [4, 5, 6], 3: ['a', 'b', 'c']}) + +@pytest.fixture +def dd_bad_df(pd_bad_df): + return dd.from_pandas(pd_bad_df, npartitions=2) + + +@pytest.fixture(params=['pd_bad_df', 'dd_bad_df']) +def bad_df(request): + return request.getfixturevalue(request.param) + + +def test_nonstr_column_names(bad_df): + es = ft.EntitySet(id='Failure') error_text = r"All column names must be strings \(Column 3 is not a string\)" with pytest.raises(ValueError, match=error_text): es.entity_from_dataframe(entity_id='str_cols', - dataframe=df, + dataframe=bad_df, index='index') @@ -410,8 +580,8 @@ def test_sort_time_id(): "transaction_time": pd.date_range(start="10:00", periods=6, freq="10s")[::-1]}) es = EntitySet("test", entities={"t": (transactions_df, "id", "transaction_time")}) - times = es["t"].df.transaction_time.tolist() - assert times == sorted(transactions_df.transaction_time.tolist()) + times = list(es["t"].df.transaction_time) + assert times == sorted(list(transactions_df.transaction_time)) def test_already_sorted_parameter(): @@ -433,12 +603,18 @@ def test_already_sorted_parameter(): index='id', time_index="transaction_time", already_sorted=True) - times = es["t"].df.transaction_time.tolist() - assert times == transactions_df.transaction_time.tolist() + times = list(es["t"].df.transaction_time) + assert times == list(transactions_df.transaction_time) +# TODO: equality check fails, dask series have no .equals method; error computing lti if categorical index def test_concat_entitysets(es): df = pd.DataFrame({'id': [0, 1, 2], 'category': ['a', 'b', 'a']}) + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask has no .equals method and issue with categoricals " + "and add_last_time_indexes") + df = dd.from_pandas(df, npartitions=2) + vtypes = {'id': variable_types.Categorical, 'category': variable_types.Categorical} es.entity_from_dataframe(entity_id='test_entity', @@ -522,16 +698,43 @@ def test_concat_entitysets(es): assert es_4[entity.id].last_time_index is None -def test_set_time_type_on_init(): +@pytest.fixture +def pd_transactions_df(): + return pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], + "card_id": [1, 2, 1, 3, 4, 5], + "transaction_time": [10, 12, 13, 20, 21, 20], + "fraud": [True, False, False, False, True, True]}) + + +@pytest.fixture +def dd_transactions_df(pd_transactions_df): + return dd.from_pandas(pd_transactions_df, npartitions=3) + + +@pytest.fixture(params=['pd_transactions_df', 'dd_transactions_df']) +def transactions_df(request): + return request.getfixturevalue(request.param) + + +def test_set_time_type_on_init(transactions_df): # create cards entity cards_df = pd.DataFrame({"id": [1, 2, 3, 4, 5]}) - transactions_df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], - "card_id": [1, 2, 1, 3, 4, 5], - "transaction_time": [10, 12, 13, 20, 21, 20], - "fraud": [True, False, False, False, True, True]}) + if isinstance(transactions_df, dd.DataFrame): + cards_df = dd.from_pandas(cards_df, npartitions=3) + cards_vtypes = {'id': variable_types.Categorical} + transactions_vtypes = { + 'id': variable_types.Categorical, + 'card_id': variable_types.Categorical, + 'transaction_time': variable_types.Numeric, + 'fraud': variable_types.Boolean + } + else: + cards_vtypes = None + transactions_vtypes = None + entities = { - "cards": (cards_df, "id"), - "transactions": (transactions_df, "id", "transaction_time") + "cards": (cards_df, "id", None, cards_vtypes), + "transactions": (transactions_df, "id", "transaction_time", transactions_vtypes) } relationships = [("cards", "id", "transactions", "card_id")] es = EntitySet("fraud", entities, relationships) @@ -539,11 +742,7 @@ def test_set_time_type_on_init(): assert es.time_type == variable_types.NumericTimeIndex -def test_sets_time_when_adding_entity(): - transactions_df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], - "card_id": [1, 2, 1, 3, 4, 5], - "transaction_time": [10, 12, 13, 20, 21, 20], - "fraud": [True, False, False, False, True, True]}) +def test_sets_time_when_adding_entity(transactions_df): accounts_df = pd.DataFrame({"id": [3, 4, 5], "signup_date": [datetime(2002, 5, 1), datetime(2006, 3, 20), @@ -552,6 +751,19 @@ def test_sets_time_when_adding_entity(): "signup_date": ["element", "exporting", "editable"]}) + if isinstance(transactions_df, dd.DataFrame): + accounts_df = dd.from_pandas(accounts_df, npartitions=2) + accounts_vtypes = {'id': variable_types.Categorical, 'signup_date': variable_types.Datetime} + transactions_vtypes = { + 'id': variable_types.Categorical, + 'card_id': variable_types.Categorical, + 'transaction_time': variable_types.Numeric, + 'fraud': variable_types.Boolean + } + else: + accounts_vtypes = None + transactions_vtypes = None + # create empty entityset es = EntitySet("fraud") # assert it's not set @@ -560,7 +772,8 @@ def test_sets_time_when_adding_entity(): es.entity_from_dataframe("transactions", transactions_df, index="id", - time_index="transaction_time") + time_index="transaction_time", + variable_types=transactions_vtypes) # assert time_type is set assert es.time_type == variable_types.NumericTimeIndex # add another entity @@ -576,19 +789,25 @@ def test_sets_time_when_adding_entity(): es.entity_from_dataframe("accounts", accounts_df, index="id", - time_index="signup_date") - # add non time type as time index - error_text = "Attempted to convert all string column signup_date to numeric" - with pytest.raises(TypeError, match=error_text): - es.entity_from_dataframe("accounts", - accounts_df_string, - index="id", - time_index="signup_date") + time_index="signup_date", + variable_types=accounts_vtypes) + # add non time type as time index, only valid for pandas + if isinstance(transactions_df, pd.DataFrame): + error_text = "Attempted to convert all string column signup_date to numeric" + with pytest.raises(TypeError, match=error_text): + es.entity_from_dataframe("accounts", + accounts_df_string, + index="id", + time_index="signup_date") def test_checks_time_type_setting_time_index(es): - # set non time type as time index - error_text = 'log time index not recognized as numeric or datetime' + # set non time type as time index, Dask and Pandas error differently + if isinstance(es['log'].df, pd.DataFrame): + error_text = 'log time index not recognized as numeric or datetime' + else: + error_text = "log time index is %s type which differs from" \ + " other entityset time indexes" % (variable_types.NumericTimeIndex) with pytest.raises(TypeError, match=error_text): es['log'].set_time_index('purchased') @@ -732,7 +951,11 @@ def test_normalize_time_index_from_none(es): make_time_index='date_of_birth', copy_variables=['date_of_birth']) assert es['birthdays'].time_index == 'date_of_birth' - assert es['birthdays'].df['date_of_birth'].is_monotonic_increasing + df = es['birthdays'].df + + # only pandas sorts by time index + if isinstance(df, pd.DataFrame): + assert df['date_of_birth'].is_monotonic_increasing def test_raise_error_if_dupicate_additional_variables_passed(es): @@ -771,6 +994,7 @@ def test_normalize_entity_copies_variable_types(es): assert es['values_2'].variable_types['value'] == variable_types.Ordinal +# sorting not supported in dask def test_make_time_index_keeps_original_sorting(): trips = { 'trip_id': [999 - i for i in range(1000)], @@ -795,13 +1019,16 @@ def test_make_time_index_keeps_original_sorting(): def test_normalize_entity_new_time_index(es): new_time_index = 'value_time' es.normalize_entity('log', 'values', 'value', - make_time_index=True, - new_entity_time_index=new_time_index) + make_time_index=True, + new_entity_time_index=new_time_index) assert es['values'].time_index == new_time_index assert new_time_index in es['values'].df.columns assert len(es['values'].df.columns) == 2 - assert es['values'].df[new_time_index].is_monotonic_increasing + df = es['values'].df + if isinstance(df, dd.DataFrame): + df = df.compute() + assert df[new_time_index].is_monotonic_increasing def test_normalize_entity_same_index(es): @@ -822,11 +1049,14 @@ def test_normalize_entity_same_index(es): make_time_index=True) +# TODO: normalize entity fails with Dask, doesn't specify all vtypes when creating new entity def test_secondary_time_index(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('vtype error when attempting to normalize entity') es.normalize_entity('log', 'values', 'value', make_time_index=True, make_secondary_time_index={ - 'datetime': ['comments']}, + 'datetime': ['comments']}, new_entity_time_index="value_time", new_entity_secondary_time_index='second_ti') @@ -865,14 +1095,40 @@ def test_metadata_without_id(): assert es.metadata.id is None -def test_datetime64_conversion(): - df = pd.DataFrame({'id': [0, 1, 2], - 'ints': ['1', '2', '1']}) +@pytest.fixture +def pd_datetime3(): + return pd.DataFrame({'id': [0, 1, 2], + 'ints': ['1', '2', '1']}) + + +@pytest.fixture +def dd_datetime3(pd_datetime3): + return dd.from_pandas(pd_datetime3, npartitions=2) + + +@pytest.fixture(params=['pd_datetime3', 'dd_datetime3']) +def datetime3(request): + return request.getfixturevalue(request.param) + + +def test_datetime64_conversion(datetime3): + df = datetime3 df["time"] = pd.Timestamp.now() df["time"] = df["time"].astype("datetime64[ns, UTC]") + if isinstance(df, dd.DataFrame): + vtypes = { + 'id': variable_types.Categorical, + 'ints': variable_types.Numeric, + 'time': variable_types.Datetime + } + else: + vtypes = None es = EntitySet(id='test') - es.entity_from_dataframe(entity_id='test_entity', index='id', dataframe=df) + es.entity_from_dataframe(entity_id='test_entity', + index='id', + dataframe=df, + variable_types=vtypes) vtype_time_index = variable_types.variable.DatetimeTimeIndex es['test_entity'].convert_variable_type('time', vtype_time_index) @@ -936,10 +1192,33 @@ def _check_schema_version(version, es, warning_text): deserialize.description_to_entityset(dictionary) -def test_same_index_values(): - transactions_df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], - "transaction_time": pd.date_range(start="10:00", periods=6, freq="10s"), - "first_entity_time": [1, 2, 3, 5, 6, 6]}) +@pytest.fixture +def pd_index_df(): + return pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], + "transaction_time": pd.date_range(start="10:00", periods=6, freq="10s"), + "first_entity_time": [1, 2, 3, 5, 6, 6]}) + + +@pytest.fixture +def dd_index_df(pd_index_df): + return dd.from_pandas(pd_index_df, npartitions=3) + + +@pytest.fixture(params=['pd_index_df', 'dd_index_df']) +def index_df(request): + return request.getfixturevalue(request.param) + + +def test_same_index_values(index_df): + if isinstance(index_df, dd.DataFrame): + vtypes = { + 'id': variable_types.Categorical, + 'transaction_time': variable_types.Datetime, + 'first_entity_time': variable_types.Numeric + } + else: + vtypes = None + es = ft.EntitySet("example") error_text = "time_index and index cannot be the same value" @@ -947,12 +1226,14 @@ def test_same_index_values(): es.entity_from_dataframe(entity_id="entity", index="id", time_index="id", - dataframe=transactions_df) + dataframe=index_df, + variable_types=vtypes) es.entity_from_dataframe(entity_id="entity", index="id", time_index="transaction_time", - dataframe=transactions_df) + dataframe=index_df, + variable_types=vtypes) with pytest.raises(ValueError, match=error_text): es.normalize_entity(base_entity_id="entity", @@ -961,21 +1242,36 @@ def test_same_index_values(): make_time_index=True) -def test_use_time_index(): - df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], - "transaction_time": pd.date_range(start="10:00", periods=6, freq="10s")}) +def test_use_time_index(index_df): + if isinstance(index_df, dd.DataFrame): + bad_vtypes = { + 'id': variable_types.Categorical, + 'transaction_time': variable_types.DatetimeTimeIndex, + 'first_entity_time': variable_types.Numeric + } + vtypes = { + 'id': variable_types.Categorical, + 'transaction_time': variable_types.Datetime, + 'first_entity_time': variable_types.Numeric + } + else: + bad_vtypes = {"transaction_time": variable_types.DatetimeTimeIndex} + vtypes = None + es = ft.EntitySet() + error_text = "DatetimeTimeIndex variable transaction_time must be set using time_index parameter" with pytest.raises(ValueError, match=error_text): es.entity_from_dataframe(entity_id="entity", index="id", - variable_types={"transaction_time": variable_types.DatetimeTimeIndex}, - dataframe=df) + variable_types=bad_vtypes, + dataframe=index_df) es.entity_from_dataframe(entity_id="entity", index="id", time_index="transaction_time", - dataframe=df) + variable_types=vtypes, + dataframe=index_df) def test_normalize_with_datetime_time_index(es): diff --git a/featuretools/tests/entityset_tests/test_es_metadata.py b/featuretools/tests/entityset_tests/test_es_metadata.py index 35a4bdcd51..2de87722b0 100644 --- a/featuretools/tests/entityset_tests/test_es_metadata.py +++ b/featuretools/tests/entityset_tests/test_es_metadata.py @@ -1,5 +1,6 @@ import pandas as pd import pytest +from dask import dataframe as dd import featuretools as ft from featuretools import EntitySet, Relationship, variable_types @@ -134,9 +135,24 @@ def test_find_forward_paths_multiple_relationships(games_es): assert r2.parent_variable.id == 'id' -def test_find_forward_paths_ignores_loops(): - employee_df = pd.DataFrame({'id': [0], 'manager_id': [0]}) - entities = {'employees': (employee_df, 'id')} +@pytest.fixture +def pd_employee_df(): + return pd.DataFrame({'id': [0], 'manager_id': [0]}) + + +@pytest.fixture +def dd_employee_df(pd_employee_df): + return dd.from_pandas(pd_employee_df, npartitions=2) + + +@pytest.fixture(params=['pd_employee_df', 'dd_employee_df']) +def employee_df(request): + return request.getfixturevalue(request.param) + + +def test_find_forward_paths_ignores_loops(employee_df): + entities = {'employees': (employee_df, 'id', None, {'id': variable_types.Id, + 'manager_id': variable_types.Id})} relationships = [('employees', 'id', 'employees', 'manager_id')] es = ft.EntitySet(entities=entities, relationships=relationships) diff --git a/featuretools/tests/entityset_tests/test_last_time_index.py b/featuretools/tests/entityset_tests/test_last_time_index.py index c5223c3134..501b2679d4 100644 --- a/featuretools/tests/entityset_tests/test_last_time_index.py +++ b/featuretools/tests/entityset_tests/test_last_time_index.py @@ -2,7 +2,9 @@ import pandas as pd import pytest +from dask import dataframe as dd +import featuretools as ft from featuretools import Relationship @@ -65,7 +67,12 @@ def extra_session_df(es): 'device_type': 0, 'id': 6} row = pd.DataFrame(row_values, index=pd.Index([6], name='id')) - df = es['sessions'].df.append(row, sort=True).sort_index() + df = es['sessions'].df + if isinstance(df, dd.DataFrame): + df = df.compute() + df = df.append(row, sort=True).sort_index() + if isinstance(es['sessions'].df, dd.DataFrame): + df = dd.from_pandas(df, npartitions=3) return df @@ -85,17 +92,26 @@ def test_leaf_no_time_index(self, es): for v1, v2 in zip(stores.last_time_index, true_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 + # TODO: possible issue with either normalize_entity or add_last_time_indexes def test_parent(self, values_es, true_values_lti): # test entity with time index and all instances in child entity + if any(isinstance(entity.df, dd.DataFrame) for entity in values_es.entities): + pytest.xfail('possible issue with either normalize_entity or add_last_time_indexes') values_es.add_last_time_indexes() values = values_es['values'] assert len(values.last_time_index) == 11 - sorted_lti = values.last_time_index.sort_index() + sorted_lti = values.last_time_index + if isinstance(sorted_lti, dd.Series): + sorted_lti = sorted_lti.compute() + sorted_lti = sorted_lti.sort_index() for v1, v2 in zip(sorted_lti, true_values_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 + # TODO: fails with Dask, tests needs to be reworked def test_parent_some_missing(self, values_es, true_values_lti): # test entity with time index and not all instances have children + if any(isinstance(entity.df, dd.DataFrame) for entity in values_es.entities): + pytest.xfail('fails with Dask, tests needs to be reworked') values = values_es['values'] # add extra value instance with no children @@ -123,7 +139,10 @@ def test_parent_no_time_index(self, es, true_sessions_lti): es.add_last_time_indexes() sessions = es['sessions'] assert len(sessions.last_time_index) == 6 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -139,18 +158,28 @@ def test_parent_no_time_index_missing(self, es, extra_session_df, true_sessions_lti[6] = pd.NaT assert len(sessions.last_time_index) == 7 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 def test_multiple_children(self, es, wishlist_df, true_sessions_lti): # test all instances in both children + if isinstance(es.entities[0].df, dd.DataFrame): + wishlist_df = dd.from_pandas(wishlist_df, npartitions=2) + variable_types = {'id': ft.variable_types.variable.Index, + 'session_id': ft.variable_types.variable.Numeric, + 'datetime': ft.variable_types.variable.DatetimeTimeIndex, + 'product_id': ft.variable_types.variable.Categorical} es.entity_from_dataframe(entity_id="wishlist_log", dataframe=wishlist_df, index='id', make_index=True, - time_index='datetime') + time_index='datetime', + variable_types=variable_types) relationship = Relationship(es['sessions']['id'], es['wishlist_log']['session_id']) es.add_relationship(relationship) @@ -161,7 +190,10 @@ def test_multiple_children(self, es, wishlist_df, true_sessions_lti[3] = pd.Timestamp("2011-4-10 10:41:00") assert len(sessions.last_time_index) == 6 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -172,11 +204,18 @@ def test_multiple_children_right_missing(self, es, wishlist_df, # drop wishlist instance related to id 3 so it's only in log wishlist_df.drop(4, inplace=True) + if isinstance(es.entities[0].df, dd.DataFrame): + wishlist_df = dd.from_pandas(wishlist_df, npartitions=2) + variable_types = {'id': ft.variable_types.variable.Index, + 'session_id': ft.variable_types.variable.Numeric, + 'datetime': ft.variable_types.variable.DatetimeTimeIndex, + 'product_id': ft.variable_types.variable.Categorical} es.entity_from_dataframe(entity_id="wishlist_log", dataframe=wishlist_df, index='id', make_index=True, - time_index='datetime') + time_index='datetime', + variable_types=variable_types) relationship = Relationship(es['sessions']['id'], es['wishlist_log']['session_id']) es.add_relationship(relationship) @@ -186,7 +225,10 @@ def test_multiple_children_right_missing(self, es, wishlist_df, true_sessions_lti[1] = pd.Timestamp("2011-4-9 10:31:30") assert len(sessions.last_time_index) == 6 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -204,11 +246,18 @@ def test_multiple_children_left_missing(self, es, extra_session_df, 'product_id': 'toothpaste'} row = pd.DataFrame(row_values, index=pd.RangeIndex(start=7, stop=8)) df = wishlist_df.append(row) + if isinstance(es.entities[0].df, dd.DataFrame): + df = dd.from_pandas(df, npartitions=2) + variable_types = {'id': ft.variable_types.variable.Index, + 'session_id': ft.variable_types.variable.Numeric, + 'datetime': ft.variable_types.variable.DatetimeTimeIndex, + 'product_id': ft.variable_types.variable.Categorical} es.entity_from_dataframe(entity_id="wishlist_log", dataframe=df, index='id', make_index=True, - time_index='datetime') + time_index='datetime', + variable_types=variable_types) relationship = Relationship(es['sessions']['id'], es['wishlist_log']['session_id']) es.add_relationship(relationship) @@ -220,7 +269,10 @@ def test_multiple_children_left_missing(self, es, extra_session_df, true_sessions_lti[6] = pd.Timestamp("2011-04-11 11:11:11") assert len(sessions.last_time_index) == 7 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -241,11 +293,18 @@ def test_multiple_children_all_combined(self, es, extra_session_df, # drop instance 4 so wishlist_log does not have session id 3 instance df.drop(4, inplace=True) + if isinstance(es.entities[0].df, dd.DataFrame): + df = dd.from_pandas(df, npartitions=2) + variable_types = {'id': ft.variable_types.variable.Index, + 'session_id': ft.variable_types.variable.Numeric, + 'datetime': ft.variable_types.variable.DatetimeTimeIndex, + 'product_id': ft.variable_types.variable.Categorical} es.entity_from_dataframe(entity_id="wishlist_log", dataframe=df, index='id', make_index=True, - time_index='datetime') + time_index='datetime', + variable_types=variable_types) relationship = Relationship(es['sessions']['id'], es['wishlist_log']['session_id']) es.add_relationship(relationship) @@ -256,7 +315,10 @@ def test_multiple_children_all_combined(self, es, extra_session_df, true_sessions_lti[6] = pd.Timestamp("2011-04-11 11:11:11") assert len(sessions.last_time_index) == 7 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -265,6 +327,13 @@ def test_multiple_children_both_missing(self, es, extra_session_df, # test all instances in neither child sessions = es['sessions'] + if isinstance(es.entities[0].df, dd.DataFrame): + wishlist_df = dd.from_pandas(wishlist_df, npartitions=2) + + variable_types = {'id': ft.variable_types.variable.Index, + 'session_id': ft.variable_types.variable.Numeric, + 'datetime': ft.variable_types.variable.DatetimeTimeIndex, + 'product_id': ft.variable_types.variable.Categorical} # add row to sessions to create session with no events sessions.update_data(extra_session_df) @@ -272,7 +341,8 @@ def test_multiple_children_both_missing(self, es, extra_session_df, dataframe=wishlist_df, index='id', make_index=True, - time_index='datetime') + time_index='datetime', + variable_types=variable_types) relationship = Relationship(es['sessions']['id'], es['wishlist_log']['session_id']) es.add_relationship(relationship) @@ -285,7 +355,10 @@ def test_multiple_children_both_missing(self, es, extra_session_df, true_sessions_lti[6] = pd.NaT assert len(sessions.last_time_index) == 7 - sorted_lti = sessions.last_time_index.sort_index() + lti = sessions.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_sessions_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 @@ -297,11 +370,16 @@ def test_grandparent(self, es): # For one user, change a log event to be newer than the user's normal # last time index. This event should be from a different session than # the current last time index. - log.df['datetime'][5] = pd.Timestamp("2011-4-09 10:40:01") - log.df = (log.df.set_index('datetime', append=True) - .sort_index(level=[1, 0], kind="mergesort") - .reset_index('datetime', drop=False)) - log.update_data(log.df) + df = log.df + if isinstance(df, dd.DataFrame): + df = log.df.compute() + df['datetime'][5] = pd.Timestamp("2011-4-09 10:40:01") + df = (df.set_index('datetime', append=True) + .sort_index(level=[1, 0], kind="mergesort") + .reset_index('datetime', drop=False)) + if isinstance(log.df, dd.DataFrame): + df = dd.from_pandas(df, npartitions=2) + log.update_data(df) es.add_last_time_indexes() true_customers_lti = pd.Series([datetime(2011, 4, 9, 10, 40, 1), @@ -309,6 +387,9 @@ def test_grandparent(self, es): datetime(2011, 4, 10, 11, 10, 3)]) assert len(customers.last_time_index) == 3 - sorted_lti = customers.last_time_index.sort_index() + lti = customers.last_time_index + if isinstance(lti, dd.Series): + lti = lti.compute() + sorted_lti = lti.sort_index() for v1, v2 in zip(sorted_lti, true_customers_lti): assert (pd.isnull(v1) and pd.isnull(v2)) or v1 == v2 diff --git a/featuretools/tests/entityset_tests/test_plotting.py b/featuretools/tests/entityset_tests/test_plotting.py index d1ec9a7740..ddb124e6c8 100644 --- a/featuretools/tests/entityset_tests/test_plotting.py +++ b/featuretools/tests/entityset_tests/test_plotting.py @@ -4,17 +4,44 @@ import graphviz import pandas as pd import pytest +from dask import dataframe as dd import featuretools as ft +@pytest.fixture +def pd_simple(): + es = ft.EntitySet("test") + df = pd.DataFrame({"foo": [1]}) + es.entity_from_dataframe("test", df) + return es + + +@pytest.fixture +def dd_simple(): + es = ft.EntitySet("test") + df = pd.DataFrame({"foo": [1]}) + df = dd.from_pandas(df, npartitions=2) + es.entity_from_dataframe("test", df) + return es + + +@pytest.fixture(params=['pd_simple', 'dd_simple']) +def simple_es(request): + return request.getfixturevalue(request.param) + + def test_returns_digraph_object(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Plotting cannot use dask delayed length") graph = es.plot() assert isinstance(graph, graphviz.Digraph) def test_saving_png_file(es, tmpdir): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Plotting cannot use dask delayed length") output_path = str(tmpdir.join("test1.png")) es.plot(to_file=output_path) @@ -42,17 +69,18 @@ def test_invalid_format(es): def test_multiple_rows(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Plotting cannot use dask delayed length") plot_ = es.plot() result = re.findall(r"\((\d+\srows?)\)", plot_.source) expected = ["{} rows".format(str(i.shape[0])) for i in es.entities] assert result == expected -def test_single_row(): - es = ft.EntitySet("test") - df = pd.DataFrame({"foo": [1]}) - es.entity_from_dataframe("test", df) - plot_ = es.plot() +def test_single_row(simple_es): + if any(isinstance(entity.df, dd.DataFrame) for entity in simple_es.entities): + pytest.xfail("Plotting cannot use dask delayed length") + plot_ = simple_es.plot() result = re.findall(r"\((\d+\srows?)\)", plot_.source) expected = ["1 row"] assert result == expected diff --git a/featuretools/tests/entityset_tests/test_serialization.py b/featuretools/tests/entityset_tests/test_serialization.py index 471f9ffeea..d2f880566c 100644 --- a/featuretools/tests/entityset_tests/test_serialization.py +++ b/featuretools/tests/entityset_tests/test_serialization.py @@ -4,6 +4,7 @@ import boto3 import pandas as pd import pytest +from dask import dataframe as dd from featuretools.demo import load_mock_customer from featuretools.entityset import EntitySet, deserialize, serialize @@ -84,12 +85,28 @@ def test_entity_descriptions(es): assert entity.__eq__(_entity, deep=True) +def test_dask_entity_descriptions(dask_es): + _es = EntitySet(dask_es.id) + for entity in dask_es.metadata.entities: + description = serialize.entity_to_description(entity) + deserialize.description_to_entity(description, _es) + _entity = _es[description['id']] + _entity.last_time_index = entity.last_time_index + assert entity.__eq__(_entity, deep=True) + + def test_entityset_description(es): description = serialize.entityset_to_description(es) _es = deserialize.description_to_entityset(description) assert es.metadata.__eq__(_es, deep=True) +def test_dask_entityset_description(dask_es): + description = serialize.entityset_to_description(dask_es) + _es = deserialize.description_to_entityset(description) + assert dask_es.metadata.__eq__(_es, deep=True) + + def test_invalid_formats(es, tmpdir): error_text = 'must be one of the following formats: {}' error_text = error_text.format(', '.join(serialize.FORMATS)) @@ -111,52 +128,90 @@ def test_to_csv(es, tmpdir): es.to_csv(str(tmpdir), encoding='utf-8', engine='python') new_es = deserialize.read_entityset(str(tmpdir)) assert es.__eq__(new_es, deep=True) - assert type(es['log'].df['latlong'][0]) == tuple + df = es['log'].df + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') + new_df = new_es['log'].df + if isinstance(new_df, dd.DataFrame): + new_df = new_df.compute().set_index('id') + assert type(df['latlong'][0]) == tuple + assert type(new_df['latlong'][0]) == tuple + + +# Dask does not support to_pickle +def test_to_pickle(pd_es, tmpdir): + pd_es.to_pickle(str(tmpdir)) + new_es = deserialize.read_entityset(str(tmpdir)) + assert pd_es.__eq__(new_es, deep=True) + assert type(pd_es['log'].df['latlong'][0]) == tuple assert type(new_es['log'].df['latlong'][0]) == tuple -def test_to_pickle(es, tmpdir): - es.to_pickle(str(tmpdir)) - new_es = deserialize.read_entityset(str(tmpdir)) - assert es.__eq__(new_es, deep=True) - assert type(es['log'].df['latlong'][0]) == tuple - assert type(new_es['log'].df['latlong'][0]) == tuple +def test_to_pickle_errors_dask(dask_es, tmpdir): + msg = 'Cannot serialize Dask EntitySet to pickle' + with pytest.raises(ValueError, match=msg): + dask_es.to_pickle(str(tmpdir)) -def test_to_pickle_interesting_values(es, tmpdir): - es.add_interesting_values() - es.to_pickle(str(tmpdir)) +# Dask does not support to_pickle +def test_to_pickle_interesting_values(pd_es, tmpdir): + pd_es.add_interesting_values() + pd_es.to_pickle(str(tmpdir)) new_es = deserialize.read_entityset(str(tmpdir)) - assert es.__eq__(new_es, deep=True) + assert pd_es.__eq__(new_es, deep=True) -def test_to_pickle_manual_interesting_values(es, tmpdir): - es['log']['product_id'].interesting_values = ["coke_zero"] - es.to_pickle(str(tmpdir)) +# Dask does not support to_pickle +def test_to_pickle_manual_interesting_values(pd_es, tmpdir): + pd_es['log']['product_id'].interesting_values = ["coke_zero"] + pd_es.to_pickle(str(tmpdir)) new_es = deserialize.read_entityset(str(tmpdir)) - assert es.__eq__(new_es, deep=True) + assert pd_es.__eq__(new_es, deep=True) def test_to_parquet(es, tmpdir): es.to_parquet(str(tmpdir)) new_es = deserialize.read_entityset(str(tmpdir)) assert es.__eq__(new_es, deep=True) - assert type(es['log'].df['latlong'][0]) == tuple - assert type(new_es['log'].df['latlong'][0]) == tuple + df = es['log'].df + new_df = new_es['log'].df + if isinstance(df, dd.DataFrame): + df = df.compute() + if isinstance(new_df, dd.DataFrame): + new_df = new_df.compute() + assert type(df['latlong'][0]) == tuple + assert type(df['latlong'][0]) == tuple + + +def test_dask_to_parquet(dask_es, tmpdir): + dask_es.to_parquet(str(tmpdir)) + new_es = deserialize.read_entityset(str(tmpdir)) + assert dask_es.__eq__(new_es, deep=True) + assert type(dask_es['log'].df.set_index('id')['latlong'].compute()[0]) == tuple + assert type(new_es['log'].df.set_index('id')['latlong'].compute()[0]) == tuple def test_to_parquet_manual_interesting_values(es, tmpdir): es['log']['product_id'].interesting_values = ["coke_zero"] - es.to_pickle(str(tmpdir)) + es.to_parquet(str(tmpdir)) new_es = deserialize.read_entityset(str(tmpdir)) assert es.__eq__(new_es, deep=True) -def test_to_parquet_interesting_values(es, tmpdir): - es.add_interesting_values() - es.to_parquet(str(tmpdir)) +# Dask does not support es.add_interesting_values +def test_dask_to_parquet_manual_interesting_values(dask_es, tmpdir): + dask_es['log']['product_id'].interesting_values = ["coke_zero"] + dask_es.to_parquet(str(tmpdir)) new_es = deserialize.read_entityset(str(tmpdir)) - assert es.__eq__(new_es, deep=True) + assert dask_es.__eq__(new_es, deep=True) + + +# Dask doesn't support es.add_interesting_values +def test_to_parquet_interesting_values(pd_es, tmpdir): + pd_es.add_interesting_values() + pd_es.to_parquet(str(tmpdir)) + new_es = deserialize.read_entityset(str(tmpdir)) + assert pd_es.__eq__(new_es, deep=True) def test_to_parquet_with_lti(tmpdir): @@ -196,42 +251,56 @@ def make_public(s3_client, s3_bucket): s3_client.ObjectAcl(BUCKET_NAME, obj).put(ACL='public-read-write') +# TODO: tmp file disappears after deserialize step, cannot check equality with Dask def test_serialize_s3_csv(es, s3_client, s3_bucket): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('tmp file disappears after deserialize step, cannot check equality with Dask') es.to_csv(TEST_S3_URL, encoding='utf-8', engine='python') make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL) assert es.__eq__(new_es, deep=True) -def test_serialize_s3_pickle(es, s3_client, s3_bucket): - es.to_pickle(TEST_S3_URL) +# Dask does not support to_pickle +def test_serialize_s3_pickle(pd_es, s3_client, s3_bucket): + pd_es.to_pickle(TEST_S3_URL) make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL) - assert es.__eq__(new_es, deep=True) + assert pd_es.__eq__(new_es, deep=True) +# TODO: tmp file disappears after deserialize step, cannot check equality with Dask def test_serialize_s3_parquet(es, s3_client, s3_bucket): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('tmp file disappears after deserialize step, cannot check equality with Dask') es.to_parquet(TEST_S3_URL) make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL) assert es.__eq__(new_es, deep=True) +# TODO: tmp file disappears after deserialize step, cannot check equality with Dask def test_serialize_s3_anon_csv(es, s3_client, s3_bucket): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('tmp file disappears after deserialize step, cannot check equality with Dask') es.to_csv(TEST_S3_URL, encoding='utf-8', engine='python', profile_name=False) make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL, profile_name=False) assert es.__eq__(new_es, deep=True) -def test_serialize_s3_anon_pickle(es, s3_client, s3_bucket): - es.to_pickle(TEST_S3_URL, profile_name=False) +# Dask does not support to_pickle +def test_serialize_s3_anon_pickle(pd_es, s3_client, s3_bucket): + pd_es.to_pickle(TEST_S3_URL, profile_name=False) make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL, profile_name=False) - assert es.__eq__(new_es, deep=True) + assert pd_es.__eq__(new_es, deep=True) +# TODO: tmp file disappears after deserialize step, cannot check equality with Dask def test_serialize_s3_anon_parquet(es, s3_client, s3_bucket): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('tmp file disappears after deserialize step, cannot check equality with Dask') es.to_parquet(TEST_S3_URL, profile_name=False) make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL, profile_name=False) @@ -277,6 +346,8 @@ def setup_test_profile(monkeypatch, tmpdir): def test_s3_test_profile(es, s3_client, s3_bucket, setup_test_profile): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail('tmp file disappears after deserialize step, cannot check equality with Dask') es.to_csv(TEST_S3_URL, encoding='utf-8', engine='python', profile_name='test') make_public(s3_client, s3_bucket) new_es = deserialize.read_entityset(TEST_S3_URL, profile_name='test') diff --git a/featuretools/tests/entityset_tests/test_timedelta.py b/featuretools/tests/entityset_tests/test_timedelta.py index 2da0752e57..933b12c21e 100644 --- a/featuretools/tests/entityset_tests/test_timedelta.py +++ b/featuretools/tests/entityset_tests/test_timedelta.py @@ -1,3 +1,4 @@ +import dask.dataframe as dd import pandas as pd import pytest from dateutil.relativedelta import relativedelta @@ -40,8 +41,12 @@ def test_delta_with_observations(es): def test_delta_with_time_unit_matches_pandas(es): customer_id = 0 sessions_df = es['sessions'].df + if isinstance(sessions_df, dd.DataFrame): + sessions_df = sessions_df.compute() sessions_df = sessions_df[sessions_df['customer_id'] == customer_id] log_df = es['log'].df + if isinstance(log_df, dd.DataFrame): + log_df = log_df.compute() log_df = log_df[log_df['session_id'].isin(sessions_df['id'])] all_times = log_df['datetime'].sort_values().tolist() @@ -117,8 +122,12 @@ def test_feature_takes_timedelta_string(es): def test_deltas_week(es): customer_id = 0 sessions_df = es['sessions'].df + if isinstance(sessions_df, dd.DataFrame): + sessions_df = sessions_df.compute() sessions_df = sessions_df[sessions_df['customer_id'] == customer_id] log_df = es['log'].df + if isinstance(log_df, dd.DataFrame): + log_df = log_df.compute() log_df = log_df[log_df['session_id'].isin(sessions_df['id'])] all_times = log_df['datetime'].sort_values().tolist() delta_week = Timedelta(1, "w") diff --git a/featuretools/tests/primitive_tests/test_agg_feats.py b/featuretools/tests/primitive_tests/test_agg_feats.py index d02eb5e6c0..ecc87063b7 100644 --- a/featuretools/tests/primitive_tests/test_agg_feats.py +++ b/featuretools/tests/primitive_tests/test_agg_feats.py @@ -1,6 +1,7 @@ from datetime import datetime from math import isnan +import dask.dataframe as dd import numpy as np import pandas as pd import pytest @@ -90,7 +91,7 @@ def test_makes_count(es): assert feature_with_name(features, 'customers.COUNT(log)') -def test_count_null_and_make_agg_primitive(es): +def test_count_null_and_make_agg_primitive(pd_es): def count_func(values, count_null=False): if len(values) == 0: return 0 @@ -109,8 +110,8 @@ def count_generate_name(self, base_feature_names, relationship_path_name, Count = make_agg_primitive(count_func, [[Index], [Variable]], Numeric, name="count", stack_on_self=False, cls_attributes={"generate_name": count_generate_name}) - count_null = ft.Feature(es['log']['value'], parent_entity=es['sessions'], primitive=Count(count_null=True)) - feature_matrix = ft.calculate_feature_matrix([count_null], entityset=es) + count_null = ft.Feature(pd_es['log']['value'], parent_entity=pd_es['sessions'], primitive=Count(count_null=True)) + feature_matrix = ft.calculate_feature_matrix([count_null], entityset=pd_es) values = [5, 4, 1, 2, 3, 2] assert (values == feature_matrix[count_null.get_name()]).all() @@ -218,8 +219,12 @@ def test_init_and_name(es): log = es['log'] features = [ft.Feature(v) for v in log.variables] - for agg_prim in get_aggregation_primitives().values(): + agg_primitives = get_aggregation_primitives().values() + # If Dask EntitySet use only Dask compatible primitives + if isinstance(es['sessions'].df, dd.DataFrame): + agg_primitives = [prim for prim in agg_primitives if prim.dask_compatible] + for agg_prim in agg_primitives: input_types = agg_prim.input_types if type(input_types[0]) != list: input_types = [input_types] @@ -235,7 +240,7 @@ def test_init_and_name(es): # try to get name and calculate instance.get_name() - ft.calculate_feature_matrix([instance], entityset=es).head(5) + ft.calculate_feature_matrix([instance], entityset=es) def test_invalid_init_args(diamond_es): @@ -381,10 +386,10 @@ def test_serialization(es): _assert_agg_feats_equal(max2, deserialized) -def test_time_since_last(es): - f = ft.Feature(es["log"]["datetime"], parent_entity=es["customers"], primitive=TimeSinceLast) +def test_time_since_last(pd_es): + f = ft.Feature(pd_es["log"]["datetime"], parent_entity=pd_es["customers"], primitive=TimeSinceLast) fm = ft.calculate_feature_matrix([f], - entityset=es, + entityset=pd_es, instance_ids=[0, 1, 2], cutoff_time=datetime(2015, 6, 8)) @@ -393,10 +398,10 @@ def test_time_since_last(es): assert all(fm[f.get_name()].round().values == correct) -def test_time_since_first(es): - f = ft.Feature(es["log"]["datetime"], parent_entity=es["customers"], primitive=TimeSinceFirst) +def test_time_since_first(pd_es): + f = ft.Feature(pd_es["log"]["datetime"], parent_entity=pd_es["customers"], primitive=TimeSinceFirst) fm = ft.calculate_feature_matrix([f], - entityset=es, + entityset=pd_es, instance_ids=[0, 1, 2], cutoff_time=datetime(2015, 6, 8)) @@ -405,10 +410,10 @@ def test_time_since_first(es): assert all(fm[f.get_name()].round().values == correct) -def test_median(es): - f = ft.Feature(es["log"]["value_many_nans"], parent_entity=es["customers"], primitive=Median) +def test_median(pd_es): + f = ft.Feature(pd_es["log"]["value_many_nans"], parent_entity=pd_es["customers"], primitive=Median) fm = ft.calculate_feature_matrix([f], - entityset=es, + entityset=pd_es, instance_ids=[0, 1, 2], cutoff_time=datetime(2015, 6, 8)) @@ -423,8 +428,11 @@ def test_agg_same_method_name(es): can't differentiate them. We have a work around to this based on the name property that we test here. """ - + # TODO: Update to work with Dask + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Cannot use primitives made with make_agg_primitives with Dask EntitySets") # test with normally defined functions + def custom_primitive(x): return x.sum() @@ -455,7 +463,7 @@ def custom_primitive(x): assert fm.columns.tolist() == [f_sum.get_name(), f_max.get_name()] -def test_time_since_last_custom(es): +def test_time_since_last_custom(pd_es): def time_since_last(values, time=None): time_since = time - values.iloc[0] return time_since.total_seconds() @@ -465,9 +473,9 @@ def time_since_last(values, time=None): Numeric, name="time_since_last", uses_calc_time=True) - f = ft.Feature(es["log"]["datetime"], parent_entity=es["customers"], primitive=TimeSinceLast) + f = ft.Feature(pd_es["log"]["datetime"], parent_entity=pd_es["customers"], primitive=TimeSinceLast) fm = ft.calculate_feature_matrix([f], - entityset=es, + entityset=pd_es, instance_ids=[0, 1, 2], cutoff_time=datetime(2015, 6, 8)) @@ -483,7 +491,7 @@ def time_since_last(values, time=None): uses_calc_time=False) -def test_custom_primitive_time_as_arg(es): +def test_custom_primitive_time_as_arg(pd_es): def time_since_last(values, time): time_since = time - values.iloc[0] return time_since.total_seconds() @@ -493,9 +501,9 @@ def time_since_last(values, time): Numeric, uses_calc_time=True) assert TimeSinceLast.name == "time_since_last" - f = ft.Feature(es["log"]["datetime"], parent_entity=es["customers"], primitive=TimeSinceLast) + f = ft.Feature(pd_es["log"]["datetime"], parent_entity=pd_es["customers"], primitive=TimeSinceLast) fm = ft.calculate_feature_matrix([f], - entityset=es, + entityset=pd_es, instance_ids=[0, 1, 2], cutoff_time=datetime(2015, 6, 8)) @@ -511,7 +519,7 @@ def time_since_last(values, time): uses_calc_time=False) -def test_custom_primitive_multiple_inputs(es): +def test_custom_primitive_multiple_inputs(pd_es): def mean_sunday(numeric, datetime): ''' Finds the mean of non-null values of a feature that occurred on Sundays @@ -524,7 +532,7 @@ def mean_sunday(numeric, datetime): input_types=[Numeric, Datetime], return_type=Numeric) - fm, features = ft.dfs(entityset=es, + fm, features = ft.dfs(entityset=pd_es, target_entity="sessions", agg_primitives=[MeanSunday], trans_primitives=[]) @@ -533,9 +541,9 @@ def mean_sunday(numeric, datetime): for x, y in iterator: assert ((pd.isnull(x) and pd.isnull(y)) or (x == y)) - es.add_interesting_values() + pd_es.add_interesting_values() mean_sunday_value_priority_0 = pd.Series([None, None, None, 2.5, 0, None]) - fm, features = ft.dfs(entityset=es, + fm, features = ft.dfs(entityset=pd_es, target_entity="sessions", agg_primitives=[MeanSunday], trans_primitives=[], @@ -575,7 +583,7 @@ def test_makes_numtrue(es): assert feature_with_name(features, 'NUM_TRUE(log.purchased)') -def test_make_three_most_common(es): +def test_make_three_most_common(pd_es): def pd_top3(x): array = np.array(x.value_counts()[:3].index) if len(array) < 3: @@ -588,7 +596,7 @@ def pd_top3(x): return_type=Discrete, number_output_features=3) - fm, features = ft.dfs(entityset=es, + fm, features = ft.dfs(entityset=pd_es, target_entity="customers", instance_ids=[0, 1, 2], agg_primitives=[NMostCommoner], @@ -603,15 +611,15 @@ def pd_top3(x): assert df.iloc[2].reset_index(drop=True).equals(pd.Series(['taco clock', np.nan, np.nan])) -def test_stacking_multi(es): +def test_stacking_multi(pd_es): threecommon = NMostCommon(3) - tc = ft.Feature(es['log']['product_id'], parent_entity=es["sessions"], primitive=threecommon) + tc = ft.Feature(pd_es['log']['product_id'], parent_entity=pd_es["sessions"], primitive=threecommon) stacked = [] for i in range(3): - stacked.append(ft.Feature(tc[i], parent_entity=es['customers'], primitive=NumUnique)) + stacked.append(ft.Feature(tc[i], parent_entity=pd_es['customers'], primitive=NumUnique)) - fm = ft.calculate_feature_matrix(stacked, entityset=es, instance_ids=[0, 1, 2]) + fm = ft.calculate_feature_matrix(stacked, entityset=pd_es, instance_ids=[0, 1, 2]) correct_vals = [[3, 2, 1], [2, 1, 0], [0, 0, 0]] correct_vals1 = [[3, 1, 1], [2, 1, 0], [0, 0, 0]] @@ -634,6 +642,8 @@ def test_use_previous_pd_dateoffset(es): feature_matrix = ft.calculate_feature_matrix([total_events_pd], es, cutoff_time=pd.Timestamp('2011-04-11 10:31:30'), instance_ids=[0, 1, 2]) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id').sort_index() col_name = list(feature_matrix.head().keys())[0] assert (feature_matrix[col_name] == [1, 5, 2]).all() @@ -646,7 +656,7 @@ def _assert_agg_feats_equal(f1, f2): assert f1.use_previous == f2.use_previous -def test_override_multi_feature_names(es): +def test_override_multi_feature_names(pd_es): def gen_custom_names(primitive, base_feature_names, relationship_path_name, parent_entity_id, where_str, use_prev_str): base_string = 'Custom_%s({}.{})'.format(parent_entity_id, base_feature_names) @@ -666,7 +676,7 @@ def pd_top3(x): number_output_features=num_features, cls_attributes={"generate_names": gen_custom_names}) - fm, features = ft.dfs(entityset=es, + fm, features = ft.dfs(entityset=pd_es, target_entity="products", instance_ids=[0, 1, 2], agg_primitives=[NMostCommoner], diff --git a/featuretools/tests/primitive_tests/test_dask_primitives.py b/featuretools/tests/primitive_tests/test_dask_primitives.py new file mode 100644 index 0000000000..000ba07155 --- /dev/null +++ b/featuretools/tests/primitive_tests/test_dask_primitives.py @@ -0,0 +1,67 @@ +import pandas as pd + +import featuretools as ft +from featuretools.primitives import ( + get_aggregation_primitives, + get_transform_primitives +) + +UNSUPPORTED = [p.name for p in get_transform_primitives().values() if not p.dask_compatible] +UNSUPPORTED += [p.name for p in get_aggregation_primitives().values() if not p.dask_compatible] + + +def test_transform(pd_es, dask_es): + primitives = ft.list_primitives() + trans_list = primitives[primitives['type'] == 'transform']['name'].tolist() + trans_primitives = [prim for prim in trans_list if prim not in UNSUPPORTED] + agg_primitives = [] + + assert pd_es == dask_es + # Run DFS using each entity as a target and confirm results match + for entity in pd_es.entities: + fm, _ = ft.dfs(entityset=pd_es, + target_entity=entity.id, + trans_primitives=trans_primitives, + agg_primitives=agg_primitives, + cutoff_time=pd.Timestamp("2019-01-05 04:00"), + max_depth=2, + max_features=100) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity=entity.id, + trans_primitives=trans_primitives, + agg_primitives=agg_primitives, + cutoff_time=pd.Timestamp("2019-01-05 04:00"), + max_depth=2, + max_features=100) + # Use the same columns and make sure both indexes are sorted the same + dask_computed_fm = dask_fm.compute().set_index(entity.index).loc[fm.index][fm.columns] + pd.testing.assert_frame_equal(fm, dask_computed_fm) + + +def test_aggregation(pd_es, dask_es): + primitives = ft.list_primitives() + trans_primitives = [] + agg_list = primitives[primitives['type'] == 'aggregation']['name'].tolist() + agg_primitives = [prim for prim in agg_list if prim not in UNSUPPORTED] + + assert pd_es == dask_es + + # Run DFS using each entity as a target and confirm results match + for entity in pd_es.entities: + fm, _ = ft.dfs(entityset=pd_es, + target_entity=entity.id, + trans_primitives=trans_primitives, + agg_primitives=agg_primitives, + cutoff_time=pd.Timestamp("2019-01-05 04:00"), + max_depth=2) + + dask_fm, _ = ft.dfs(entityset=dask_es, + target_entity=entity.id, + trans_primitives=trans_primitives, + agg_primitives=agg_primitives, + cutoff_time=pd.Timestamp("2019-01-05 04:00"), + max_depth=2) + # Use the same columns and make sure both indexes are sorted the same + dask_computed_fm = dask_fm.compute().set_index(entity.index).loc[fm.index][fm.columns] + pd.testing.assert_frame_equal(fm, dask_computed_fm, check_dtype=False) diff --git a/featuretools/tests/primitive_tests/test_direct_features.py b/featuretools/tests/primitive_tests/test_direct_features.py index 70ba18ec01..40b65c5070 100644 --- a/featuretools/tests/primitive_tests/test_direct_features.py +++ b/featuretools/tests/primitive_tests/test_direct_features.py @@ -1,3 +1,4 @@ +import dask.dataframe as dd import numpy as np import pandas as pd import pytest @@ -30,6 +31,8 @@ def test_direct_from_identity(es): feature_set = FeatureSet([d]) calculator = FeatureSetCalculator(es, feature_set=feature_set, time_last=None) df = calculator.run(np.array([0, 5])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id').sort_index() v = df[d.get_name()].tolist() assert v == [0, 1] @@ -43,6 +46,8 @@ def test_direct_from_variable(es): feature_set = FeatureSet([d]) calculator = FeatureSetCalculator(es, feature_set=feature_set, time_last=None) df = calculator.run(np.array([0, 5])) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id').sort_index() v = df[d.get_name()].tolist() assert v == [0, 1] @@ -70,6 +75,10 @@ def test_direct_copy(games_es): def test_direct_of_multi_output_transform_feat(es): + # TODO: Update to work with Dask + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Custom primitive is not compabible with Dask") + class TestTime(TransformPrimitive): name = "test_time" input_types = [Datetime] @@ -105,7 +114,7 @@ def test_f(x): assert (fm[col1] == fm[col2]).all() -def test_direct_features_of_multi_output_agg_primitives(es): +def test_direct_features_of_multi_output_agg_primitives(pd_es): class ThreeMostCommonCat(AggregationPrimitive): name = "n_most_common_categorical" input_types = [Categorical] @@ -121,7 +130,7 @@ def pd_top3(x): return array return pd_top3 - fm, fl = dfs(entityset=es, + fm, fl = dfs(entityset=pd_es, target_entity="log", agg_primitives=[ThreeMostCommonCat], trans_primitives=[], diff --git a/featuretools/tests/primitive_tests/test_feature_serialization.py b/featuretools/tests/primitive_tests/test_feature_serialization.py index a4fa30e4da..089f212d39 100644 --- a/featuretools/tests/primitive_tests/test_feature_serialization.py +++ b/featuretools/tests/primitive_tests/test_feature_serialization.py @@ -77,7 +77,7 @@ def test_pickle_features(es, tmpdir): pickle_features_test_helper(asizeof(es), features_original, str(tmpdir)) -def test_pickle_features_with_custom_primitive(es, tmpdir): +def test_pickle_features_with_custom_primitive(pd_es, tmpdir): NewMax = make_agg_primitive( lambda x: max(x), name="NewMax", @@ -85,11 +85,11 @@ def test_pickle_features_with_custom_primitive(es, tmpdir): return_type=Numeric, description="Calculate means ignoring nan values") - features_original = ft.dfs(target_entity='sessions', entityset=es, + features_original = ft.dfs(target_entity='sessions', entityset=pd_es, agg_primitives=["Last", "Mean", NewMax], features_only=True) assert any([isinstance(feat.primitive, NewMax) for feat in features_original]) - pickle_features_test_helper(asizeof(es), features_original, str(tmpdir)) + pickle_features_test_helper(asizeof(pd_es), features_original, str(tmpdir)) def test_serialized_renamed_features(es): @@ -220,7 +220,7 @@ def test_s3_test_profile(es, s3_client, s3_bucket, setup_test_profile): @pytest.mark.parametrize("url,profile_name", [(S3_URL, None), (S3_URL, False), (URL, None)]) -def test_deserialize_features_s3(es, url, profile_name): +def test_deserialize_features_s3(pd_es, url, profile_name): agg_primitives = [Sum, Std, Max, Skew, Min, Mean, Count, PercentTrue, NumUnique, Mode] @@ -228,7 +228,7 @@ def test_deserialize_features_s3(es, url, profile_name): NumCharacters] features_original = sorted(ft.dfs(target_entity='sessions', - entityset=es, + entityset=pd_es, features_only=True, agg_primitives=agg_primitives, trans_primitives=trans_primitives), diff --git a/featuretools/tests/primitive_tests/test_groupby_transform_primitives.py b/featuretools/tests/primitive_tests/test_groupby_transform_primitives.py index a7eb4362d5..5e0bee1cc8 100644 --- a/featuretools/tests/primitive_tests/test_groupby_transform_primitives.py +++ b/featuretools/tests/primitive_tests/test_groupby_transform_primitives.py @@ -150,12 +150,12 @@ def test_regular(self): np.testing.assert_array_equal(function(group), answer) -def test_cum_sum(es): - log_value_feat = es['log']['value'] - dfeat = ft.Feature(es['sessions']['device_type'], entity=es['log']) +def test_cum_sum(pd_es): + log_value_feat = pd_es['log']['value'] + dfeat = ft.Feature(pd_es['sessions']['device_type'], entity=pd_es['log']) cum_sum = ft.Feature(log_value_feat, groupby=dfeat, primitive=CumSum) features = [cum_sum] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) cvalues = df[cum_sum.get_name()].values assert len(cvalues) == 15 cum_sum_values = [0, 5, 15, 30, 50, 0, 1, 3, 6, 6, 50, 55, 55, 62, 76] @@ -163,11 +163,11 @@ def test_cum_sum(es): assert v == cvalues[i] -def test_cum_min(es): - log_value_feat = es['log']['value'] - cum_min = ft.Feature(log_value_feat, groupby=es['log']['session_id'], primitive=CumMin) +def test_cum_min(pd_es): + log_value_feat = pd_es['log']['value'] + cum_min = ft.Feature(log_value_feat, groupby=pd_es['log']['session_id'], primitive=CumMin) features = [cum_min] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) cvalues = df[cum_min.get_name()].values assert len(cvalues) == 15 cum_min_values = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0] @@ -175,11 +175,11 @@ def test_cum_min(es): assert v == cvalues[i] -def test_cum_max(es): - log_value_feat = es['log']['value'] - cum_max = ft.Feature(log_value_feat, groupby=es['log']['session_id'], primitive=CumMax) +def test_cum_max(pd_es): + log_value_feat = pd_es['log']['value'] + cum_max = ft.Feature(log_value_feat, groupby=pd_es['log']['session_id'], primitive=CumMax) features = [cum_max] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) cvalues = df[cum_max.get_name()].values assert len(cvalues) == 15 cum_max_values = [0, 5, 10, 15, 20, 0, 1, 2, 3, 0, 0, 5, 0, 7, 14] @@ -187,17 +187,17 @@ def test_cum_max(es): assert v == cvalues[i] -def test_cum_sum_group_on_nan(es): - log_value_feat = es['log']['value'] - es['log'].df['product_id'] = (['coke zero'] * 3 + ['car'] * 2 + - ['toothpaste'] * 3 + ['brown bag'] * 2 + - ['shoes'] + - [np.nan] * 4 + - ['coke_zero'] * 2) - es['log'].df['value'][16] = 10 - cum_sum = ft.Feature(log_value_feat, groupby=es['log']['product_id'], primitive=CumSum) +def test_cum_sum_group_on_nan(pd_es): + log_value_feat = pd_es['log']['value'] + pd_es['log'].df['product_id'] = (['coke zero'] * 3 + ['car'] * 2 + + ['toothpaste'] * 3 + ['brown bag'] * 2 + + ['shoes'] + + [np.nan] * 4 + + ['coke_zero'] * 2) + pd_es['log'].df['value'][16] = 10 + cum_sum = ft.Feature(log_value_feat, groupby=pd_es['log']['product_id'], primitive=CumSum) features = [cum_sum] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(17)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(17)) cvalues = df[cum_sum.get_name()].values assert len(cvalues) == 17 cum_sum_values = [0, 5, 15, @@ -215,7 +215,7 @@ def test_cum_sum_group_on_nan(es): assert v == cvalues[i] -def test_cum_sum_numpy_group_on_nan(es): +def test_cum_sum_numpy_group_on_nan(pd_es): class CumSumNumpy(TransformPrimitive): """Returns the cumulative sum after grouping""" @@ -229,17 +229,17 @@ def cum_sum(values): return values.cumsum().values return cum_sum - log_value_feat = es['log']['value'] - es['log'].df['product_id'] = (['coke zero'] * 3 + ['car'] * 2 + - ['toothpaste'] * 3 + ['brown bag'] * 2 + - ['shoes'] + - [np.nan] * 4 + - ['coke_zero'] * 2) - es['log'].df['value'][16] = 10 - cum_sum = ft.Feature(log_value_feat, groupby=es['log']['product_id'], primitive=CumSumNumpy) + log_value_feat = pd_es['log']['value'] + pd_es['log'].df['product_id'] = (['coke zero'] * 3 + ['car'] * 2 + + ['toothpaste'] * 3 + ['brown bag'] * 2 + + ['shoes'] + + [np.nan] * 4 + + ['coke_zero'] * 2) + pd_es['log'].df['value'][16] = 10 + cum_sum = ft.Feature(log_value_feat, groupby=pd_es['log']['product_id'], primitive=CumSumNumpy) assert cum_sum.get_name() == "CUM_SUM(value) by product_id" features = [cum_sum] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(17)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(17)) cvalues = df[cum_sum.get_name()].values assert len(cvalues) == 17 cum_sum_values = [0, 5, 15, @@ -257,10 +257,10 @@ def cum_sum(values): assert v == cvalues[i] -def test_cum_handles_uses_full_entity(es): +def test_cum_handles_uses_full_entity(pd_es): def check(feature): feature_set = FeatureSet([feature]) - calculator = FeatureSetCalculator(es, feature_set=feature_set, time_last=None) + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set, time_last=None) df_1 = calculator.run(np.array([0, 1, 2])) df_2 = calculator.run(np.array([2, 4])) @@ -268,16 +268,16 @@ def check(feature): assert (df_2.loc[2] == df_1.loc[2]).all() for primitive in [CumSum, CumMean, CumMax, CumMin]: - check(ft.Feature(es['log']['value'], groupby=es['log']['session_id'], primitive=primitive)) + check(ft.Feature(pd_es['log']['value'], groupby=pd_es['log']['session_id'], primitive=primitive)) - check(ft.Feature(es['log']['session_id'], groupby=es['log']['session_id'], primitive=CumCount)) + check(ft.Feature(pd_es['log']['session_id'], groupby=pd_es['log']['session_id'], primitive=CumCount)) -def test_cum_mean(es): - log_value_feat = es['log']['value'] - cum_mean = ft.Feature(log_value_feat, groupby=es['log']['session_id'], primitive=CumMean) +def test_cum_mean(pd_es): + log_value_feat = pd_es['log']['value'] + cum_mean = ft.Feature(log_value_feat, groupby=pd_es['log']['session_id'], primitive=CumMean) features = [cum_mean] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) cvalues = df[cum_mean.get_name()].values assert len(cvalues) == 15 cum_mean_values = [0, 2.5, 5, 7.5, 10, 0, .5, 1, 1.5, 0, 0, 2.5, 0, 3.5, 7] @@ -285,12 +285,12 @@ def test_cum_mean(es): assert v == cvalues[i] -def test_cum_count(es): - cum_count = ft.Feature(es['log']['session_id'], - groupby=es['log']['session_id'], +def test_cum_count(pd_es): + cum_count = ft.Feature(pd_es['log']['session_id'], + groupby=pd_es['log']['session_id'], primitive=CumCount) features = [cum_count] - df = ft.calculate_feature_matrix(entityset=es, + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) cvalues = df[cum_count.get_name()].values @@ -300,9 +300,9 @@ def test_cum_count(es): assert v == cvalues[i] -def test_rename(es): - cum_count = ft.Feature(es['log']['session_id'], - groupby=es['log']['session_id'], +def test_rename(pd_es): + cum_count = ft.Feature(pd_es['log']['session_id'], + groupby=pd_es['log']['session_id'], primitive=CumCount) copy_feat = cum_count.rename("rename_test") assert cum_count.unique_name() != copy_feat.unique_name() @@ -312,12 +312,12 @@ def test_rename(es): assert cum_count.entity == copy_feat.entity -def test_groupby_no_data(es): - cum_count = ft.Feature(es['log']['session_id'], - groupby=es['log']['session_id'], +def test_groupby_no_data(pd_es): + cum_count = ft.Feature(pd_es['log']['session_id'], + groupby=pd_es['log']['session_id'], primitive=CumCount) - last_feat = ft.Feature(cum_count, parent_entity=es['customers'], primitive=Last) - df = ft.calculate_feature_matrix(entityset=es, + last_feat = ft.Feature(cum_count, parent_entity=pd_es['customers'], primitive=Last) + df = ft.calculate_feature_matrix(entityset=pd_es, features=[last_feat], cutoff_time=pd.Timestamp("2011-04-08")) cvalues = df[last_feat.get_name()].values @@ -325,7 +325,7 @@ def test_groupby_no_data(es): assert all([pd.isnull(value) for value in cvalues]) -def test_groupby_uses_calc_time(es): +def test_groupby_uses_calc_time(pd_es): def projected_amount_left(amount, timestamp, time=None): # cumulative sum of amout, with timedelta * constant subtracted delta = time - timestamp @@ -342,10 +342,10 @@ class ProjectedAmountRemaining(TransformPrimitive): def get_function(self): return projected_amount_left - time_since_product = ft.Feature([es['log']['value'], es['log']['datetime']], - groupby=es['log']['product_id'], + time_since_product = ft.Feature([pd_es['log']['value'], pd_es['log']['datetime']], + groupby=pd_es['log']['product_id'], primitive=ProjectedAmountRemaining) - df = ft.calculate_feature_matrix(entityset=es, + df = ft.calculate_feature_matrix(entityset=pd_es, features=[time_since_product], cutoff_time=pd.Timestamp("2011-04-10 11:10:30")) answers = [-88830, -88819, -88803, -88797, -88771, -88770, -88760, -88749, @@ -355,7 +355,7 @@ def get_function(self): assert ((pd.isnull(x) and pd.isnull(y)) or x == y) -def test_groupby_multi_output_stacking(es): +def test_groupby_multi_output_stacking(pd_es): TestTime = make_trans_primitive( function=lambda x: x, name="test_time", @@ -365,7 +365,7 @@ def test_groupby_multi_output_stacking(es): ) fl = dfs( - entityset=es, + entityset=pd_es, target_entity="sessions", agg_primitives=[], trans_primitives=[TestTime], @@ -379,9 +379,9 @@ def test_groupby_multi_output_stacking(es): assert ('customers.CUM_SUM(TEST_TIME(date_of_birth)[%d]) by customer_id' % i) in fl -def test_serialization(es): - value = ft.IdentityFeature(es['log']['value']) - zipcode = ft.IdentityFeature(es['log']['zipcode']) +def test_serialization(pd_es): + value = ft.IdentityFeature(pd_es['log']['value']) + zipcode = ft.IdentityFeature(pd_es['log']['zipcode']) primitive = CumSum() groupby = ft.feature_base.GroupByTransformFeature(value, primitive, zipcode) @@ -398,12 +398,12 @@ def test_serialization(es): zipcode.unique_name(): zipcode, } assert groupby == \ - ft.feature_base.GroupByTransformFeature.from_dictionary(dictionary, es, + ft.feature_base.GroupByTransformFeature.from_dictionary(dictionary, pd_es, dependencies, PrimitivesDeserializer()) -def test_groupby_with_multioutput_primitive(es): +def test_groupby_with_multioutput_primitive(pd_es): def multi_cum_sum(x): return x.cumsum(), x.cummax(), x.cummin() @@ -413,7 +413,7 @@ def multi_cum_sum(x): return_type=Numeric, number_output_features=num_features) - fm, _ = dfs(entityset=es, + fm, _ = dfs(entityset=pd_es, target_entity='customers', trans_primitives=[], agg_primitives=[], @@ -421,7 +421,7 @@ def multi_cum_sum(x): # Calculate output in a separate DFS call to make sure the multi-output code # does not alter any values - fm2, _ = dfs(entityset=es, + fm2, _ = dfs(entityset=pd_es, target_entity='customers', trans_primitives=[], agg_primitives=[], @@ -451,7 +451,7 @@ def multi_cum_sum(x): assert x == y -def test_groupby_with_multioutput_primitive_custom_names(es): +def test_groupby_with_multioutput_primitive_custom_names(pd_es): def gen_custom_names(primitive, base_feature_names): return ["CUSTOM_SUM", "CUSTOM_MAX", "CUSTOM_MIN"] @@ -465,7 +465,7 @@ def multi_cum_sum(x): number_output_features=num_features, cls_attributes={"generate_names": gen_custom_names}) - fm, _ = dfs(entityset=es, + fm, _ = dfs(entityset=pd_es, target_entity='customers', trans_primitives=[], agg_primitives=[], diff --git a/featuretools/tests/primitive_tests/test_make_agg_primitives.py b/featuretools/tests/primitive_tests/test_make_agg_primitives.py index 5236dffed9..106641fde7 100644 --- a/featuretools/tests/primitive_tests/test_make_agg_primitives.py +++ b/featuretools/tests/primitive_tests/test_make_agg_primitives.py @@ -37,7 +37,7 @@ def maximum(column): # Check the successful default value for custom aggregation primitives def test_default_value_make_agg_primitive(): - es = ft.demo.load_mock_customer(return_entityset=True) + pd_es = ft.demo.load_mock_customer(return_entityset=True) def mean_sunday(numeric, datetime): ''' @@ -51,7 +51,7 @@ def mean_sunday(numeric, datetime): input_types=[Numeric, Datetime], return_type=Numeric) - feature_matrix, features = ft.dfs(entityset=es, + feature_matrix, features = ft.dfs(entityset=pd_es, target_entity="sessions", agg_primitives=[MeanSunday], trans_primitives=[], @@ -62,7 +62,7 @@ def mean_sunday(numeric, datetime): return_type=Numeric, default_value=0) - feature_matrix2, features = ft.dfs(entityset=es, + feature_matrix2, features = ft.dfs(entityset=pd_es, target_entity="sessions", agg_primitives=[MeanSundayDefault], trans_primitives=[], diff --git a/featuretools/tests/primitive_tests/test_overrides.py b/featuretools/tests/primitive_tests/test_overrides.py index 70b0c121c9..767c1d9a44 100644 --- a/featuretools/tests/primitive_tests/test_overrides.py +++ b/featuretools/tests/primitive_tests/test_overrides.py @@ -1,6 +1,7 @@ -import featuretools as ft +import dask.dataframe as dd -from featuretools.primitives import ( # CumCount,; CumMax,; CumMean,; CumMin,; CumSum, +import featuretools as ft +from featuretools.primitives import ( AddNumeric, AddNumericScalar, Count, @@ -90,6 +91,8 @@ def test_override_boolean(es): features.append(~(count_lo.AND(count_hi))) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() assert v == test @@ -159,6 +162,8 @@ def test_override_cmp_from_variable(es): features = [count_lo] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2]) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[count_lo.get_name()].values.tolist() for i, test in enumerate(to_test): assert v[i] == test @@ -190,6 +195,8 @@ def test_override_cmp(es): lt_other, le_hi, le_other, ne_lo, ne_other] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() assert v == test diff --git a/featuretools/tests/primitive_tests/test_transform_features.py b/featuretools/tests/primitive_tests/test_transform_features.py index 560dfbc2ca..1d965dcb80 100644 --- a/featuretools/tests/primitive_tests/test_transform_features.py +++ b/featuretools/tests/primitive_tests/test_transform_features.py @@ -1,3 +1,4 @@ +import dask.dataframe as dd import numpy as np import pandas as pd import pytest @@ -68,11 +69,13 @@ def test_init_and_name(es): # Add Timedelta feature # features.append(pd.Timestamp.now() - ft.Feature(log['datetime'])) customers_features = [ft.Feature(v) for v in es["customers"].variables] - for transform_prim in get_transform_primitives().values(): - - features_to_use = log_features - + trans_primitives = get_transform_primitives().values() + # If Dask EntitySet use only Dask compatible primitives + if isinstance(es['log'].df, dd.DataFrame): + trans_primitives = [prim for prim in trans_primitives if prim.dask_compatible] + for transform_prim in trans_primitives: # skip automated testing if a few special cases + features_to_use = log_features if transform_prim in [NotEqual, Equal]: continue if transform_prim in [Age]: @@ -92,7 +95,7 @@ def test_init_and_name(es): # try to get name and calculate instance.get_name() - ft.calculate_feature_matrix([instance], entityset=es).head(5) + ft.calculate_feature_matrix([instance], entityset=es) def test_relationship_path(es): @@ -125,6 +128,8 @@ def test_make_trans_feat(es): feature_set = FeatureSet([f]) calculator = FeatureSetCalculator(es, feature_set=feature_set) df = calculator.run(np.array([0])) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[f.get_name()][0] assert v == 10 @@ -196,14 +201,14 @@ def test_not_equal_different_dtypes(simple_es): assert df['datetime != object'].to_list() == [True, True, True, True] -def test_diff(es): - value = ft.Feature(es['log']['value']) - customer_id_feat = ft.Feature(es['sessions']['customer_id'], entity=es['log']) - diff1 = ft.Feature(value, groupby=es['log']['session_id'], primitive=Diff) +def test_diff(pd_es): + value = ft.Feature(pd_es['log']['value']) + customer_id_feat = ft.Feature(pd_es['sessions']['customer_id'], entity=pd_es['log']) + diff1 = ft.Feature(value, groupby=pd_es['log']['session_id'], primitive=Diff) diff2 = ft.Feature(value, groupby=customer_id_feat, primitive=Diff) feature_set = FeatureSet([diff1, diff2]) - calculator = FeatureSetCalculator(es, feature_set=feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set) df = calculator.run(np.array(range(15))) val1 = df[diff1.get_name()].values.tolist() @@ -225,28 +230,28 @@ def test_diff(es): assert v2 == correct_vals2[i] -def test_diff_single_value(es): - diff = ft.Feature(es['stores']['num_square_feet'], groupby=es['stores'][u'région_id'], primitive=Diff) +def test_diff_single_value(pd_es): + diff = ft.Feature(pd_es['stores']['num_square_feet'], groupby=pd_es['stores'][u'région_id'], primitive=Diff) feature_set = FeatureSet([diff]) - calculator = FeatureSetCalculator(es, feature_set=feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set) df = calculator.run(np.array([4])) assert df[diff.get_name()][4] == 6000.0 -def test_diff_reordered(es): - sum_feat = ft.Feature(es['log']['value'], parent_entity=es["sessions"], primitive=Sum) +def test_diff_reordered(pd_es): + sum_feat = ft.Feature(pd_es['log']['value'], parent_entity=pd_es["sessions"], primitive=Sum) diff = ft.Feature(sum_feat, primitive=Diff) feature_set = FeatureSet([diff]) - calculator = FeatureSetCalculator(es, feature_set=feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set) df = calculator.run(np.array([4, 2])) assert df[diff.get_name()][4] == 16 assert df[diff.get_name()][2] == -6 -def test_diff_single_value_is_nan(es): - diff = ft.Feature(es['stores']['num_square_feet'], groupby=es['stores'][u'région_id'], primitive=Diff) +def test_diff_single_value_is_nan(pd_es): + diff = ft.Feature(pd_es['stores']['num_square_feet'], groupby=pd_es['stores'][u'région_id'], primitive=Diff) feature_set = FeatureSet([diff]) - calculator = FeatureSetCalculator(es, feature_set=feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set=feature_set) df = calculator.run(np.array([5])) assert df.shape[0] == 1 assert df[diff.get_name()].dropna().shape[0] == 0 @@ -265,6 +270,8 @@ def test_compare_of_identity(es): features.append(ft.Feature(es['log']['value'], primitive=test[0](10))) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() @@ -285,6 +292,8 @@ def test_compare_of_direct(es): features.append(ft.Feature(log_rating, primitive=test[0](4.5))) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() @@ -305,6 +314,8 @@ def test_compare_of_transform(es): features.append(ft.Feature(day, primitive=test[0](10))) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 14]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() @@ -326,6 +337,8 @@ def test_compare_of_agg(es): features.append(ft.Feature(count_logs, primitive=test[0](2))) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() @@ -333,12 +346,19 @@ def test_compare_of_agg(es): def test_compare_all_nans(es): - nan_feat = ft.Feature(es['log']['product_id'], parent_entity=es['sessions'], primitive=Mode) - compare = nan_feat == 'brown bag' + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + nan_feat = ft.Feature(es['log']['value'], parent_entity=es['sessions'], primitive=ft.primitives.Min) + compare = nan_feat == 0.0 + else: + nan_feat = ft.Feature(es['log']['product_id'], parent_entity=es['sessions'], primitive=Mode) + compare = nan_feat == 'brown bag' + # before all data time_last = pd.Timestamp('1/1/1993') df = ft.calculate_feature_matrix(entityset=es, features=[nan_feat, compare], instance_ids=[0, 1, 2], cutoff_time=time_last) + if isinstance(df, dd.DataFrame): + df = df.compute() assert df[nan_feat.get_name()].dropna().shape[0] == 0 assert not df[compare.get_name()].any() @@ -358,6 +378,8 @@ def test_arithmetic_of_val(es): features.append(ft.Feature(es['log']['value']) / 0) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for f, test in zip(features, to_test): v = df[f.get_name()].values.tolist() @@ -388,6 +410,8 @@ def test_arithmetic_of_identity(es): features.append(ft.Feature([logs['value'], logs['value_2']], primitive=test[0])) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1, 2, 3]) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test[:-1]): v = df[features[i].get_name()].values.tolist() @@ -415,12 +439,21 @@ def test_arithmetic_of_direct(es): features.append(ft.Feature([log_age, log_rating], primitive=test[0])) df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 3, 5, 7]) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id').sort_index() + for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() assert v == test[1] -def test_boolean_multiply(): +@pytest.fixture(params=['pd_boolean_mult_es', 'dask_boolean_mult_es']) +def boolean_mult_es(request): + return request.getfixturevalue(request.param) + + +@pytest.fixture +def pd_boolean_mult_es(): es = ft.EntitySet() df = pd.DataFrame({"index": [0, 1, 2], "bool": [True, False, True], @@ -430,6 +463,20 @@ def test_boolean_multiply(): dataframe=df, index="index") + return es + + +@pytest.fixture +def dask_boolean_mult_es(pd_boolean_mult_es): + entities = {} + for entity in pd_boolean_mult_es.entities: + entities[entity.id] = (dd.from_pandas(entity.df, npartitions=2), entity.index, None, entity.variable_types) + + return ft.EntitySet(id=pd_boolean_mult_es.id, entities=entities) + + +def test_boolean_multiply(boolean_mult_es): + es = boolean_mult_es to_test = [ ('numeric', 'numeric'), ('numeric', 'bool'), @@ -442,6 +489,13 @@ def test_boolean_multiply(): fm = ft.calculate_feature_matrix(entityset=es, features=features) + if isinstance(fm, dd.DataFrame): + fm = fm.compute() + + df = es['test'].df + if isinstance(df, dd.DataFrame): + df = df.compute() + for row in to_test: col_name = '{} * {}'.format(row[0], row[1]) if row[0] == 'bool' and row[1] == 'bool': @@ -466,6 +520,8 @@ def test_arithmetic_of_transform(es): feature_set = FeatureSet(features) calculator = FeatureSetCalculator(es, feature_set=feature_set) df = calculator.run(np.array([0, 2, 12, 13])) + if isinstance(df, dd.DataFrame): + df = df.compute() for i, test in enumerate(to_test): v = df[features[i].get_name()].values.tolist() assert np.isnan(v.pop(0)) @@ -477,6 +533,8 @@ def test_not_feature(es): not_feat = ft.Feature(es['customers']['loves_ice_cream'], primitive=Not) features = [not_feat] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=[0, 1]) + if isinstance(df, dd.DataFrame): + df = df.compute() v = df[not_feat.get_name()].values assert not v[0] assert v[1] @@ -499,6 +557,8 @@ def test_arithmetic_of_agg(es): ids = ['United States', 'Mexico'] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=ids) + if isinstance(df, dd.DataFrame): + df = df.compute().set_index('id') df = df.loc[ids] for i, test in enumerate(to_test): @@ -513,12 +573,12 @@ def test_arithmetic_of_agg(es): # return (lat, lon) -def test_latlong(es): - log_latlong_feat = es['log']['latlong'] +def test_latlong(pd_es): + log_latlong_feat = pd_es['log']['latlong'] latitude = ft.Feature(log_latlong_feat, primitive=Latitude) longitude = ft.Feature(log_latlong_feat, primitive=Longitude) features = [latitude, longitude] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) latvalues = df[latitude.get_name()].values lonvalues = df[longitude.get_name()].values assert len(latvalues) == 15 @@ -531,14 +591,14 @@ def test_latlong(es): assert v == lonvalues[i] -def test_haversine(es): - log_latlong_feat = es['log']['latlong'] - log_latlong_feat2 = es['log']['latlong2'] +def test_haversine(pd_es): + log_latlong_feat = pd_es['log']['latlong'] + log_latlong_feat2 = pd_es['log']['latlong2'] haversine = ft.Feature([log_latlong_feat, log_latlong_feat2], primitive=Haversine) features = [haversine] - df = ft.calculate_feature_matrix(entityset=es, features=features, + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) values = df[haversine.get_name()].values real = [0, 525.318462, 1045.32190304, 1554.56176802, 2047.3294327, 0, @@ -550,7 +610,7 @@ def test_haversine(es): haversine = ft.Feature([log_latlong_feat, log_latlong_feat2], primitive=Haversine(unit='kilometers')) features = [haversine] - df = ft.calculate_feature_matrix(entityset=es, features=features, + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) values = df[haversine.get_name()].values real_km = [0, 845.41812212, 1682.2825471, 2501.82467535, 3294.85736668, @@ -570,6 +630,8 @@ def test_text_primitives(es): features = [words, chars] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + if isinstance(df, dd.DataFrame): + df = df.compute() word_counts = [514, 3, 3, 644, 1268, 1269, 177, 172, 79, 240, 1239, 3, 3, 3, 3] @@ -588,6 +650,8 @@ def test_isin_feat(es): isin = ft.Feature(es['log']['product_id'], primitive=IsIn(list_of_outputs=["toothpaste", "coke zero"])) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [True, True, True, False, False, True, True, True] v = df[isin.get_name()].values.tolist() assert true == v @@ -597,6 +661,8 @@ def test_isin_feat_other_syntax(es): isin = ft.Feature(es['log']['product_id']).isin(["toothpaste", "coke zero"]) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [True, True, True, False, False, True, True, True] v = df[isin.get_name()].values.tolist() assert true == v @@ -606,6 +672,8 @@ def test_isin_feat_other_syntax_int(es): isin = ft.Feature(es['log']['value']).isin([5, 10]) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [False, True, True, False, False, False, False, False] v = df[isin.get_name()].values.tolist() assert true == v @@ -615,6 +683,8 @@ def test_isin_feat_custom(es): def pd_is_in(array, list_of_outputs=None): if list_of_outputs is None: list_of_outputs = [] + if isinstance(array, dd.Series): + return array.isin(list_of_outputs) return pd.Series(array).isin(list_of_outputs) def isin_generate_name(self, base_feature_names): @@ -633,6 +703,8 @@ def isin_generate_name(self, base_feature_names): isin = ft.Feature(es['log']['product_id'], primitive=IsIn(list_of_outputs=["toothpaste", "coke zero"])) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [True, True, True, False, False, True, True, True] v = df[isin.get_name()].values.tolist() assert true == v @@ -640,6 +712,8 @@ def isin_generate_name(self, base_feature_names): isin = ft.Feature(es['log']['product_id']).isin(["toothpaste", "coke zero"]) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [True, True, True, False, False, True, True, True] v = df[isin.get_name()].values.tolist() assert true == v @@ -647,17 +721,19 @@ def isin_generate_name(self, base_feature_names): isin = ft.Feature(es['log']['value']).isin([5, 10]) features = [isin] df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(8)) + if isinstance(df, dd.DataFrame): + df = df.compute() true = [False, True, True, False, False, False, False, False] v = df[isin.get_name()].values.tolist() assert true == v -def test_isnull_feat(es): - value = ft.Feature(es['log']['value']) - diff = ft.Feature(value, groupby=es['log']['session_id'], primitive=Diff) +def test_isnull_feat(pd_es): + value = ft.Feature(pd_es['log']['value']) + diff = ft.Feature(value, groupby=pd_es['log']['session_id'], primitive=Diff) isnull = ft.Feature(diff, primitive=IsNull) features = [isnull] - df = ft.calculate_feature_matrix(entityset=es, features=features, instance_ids=range(15)) + df = ft.calculate_feature_matrix(entityset=pd_es, features=features, instance_ids=range(15)) # correct_vals_diff = [ # np.nan, 5, 5, 5, 5, np.nan, 1, 1, 1, np.nan, np.nan, 5, np.nan, 7, 7] correct_vals = [True, False, False, False, False, True, False, False, @@ -666,55 +742,55 @@ def test_isnull_feat(es): assert correct_vals == values -def test_percentile(es): - v = ft.Feature(es['log']['value']) +def test_percentile(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) feature_set = FeatureSet([p]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array(range(10, 17))) - true = es['log'].df[v.get_name()].rank(pct=True) + true = pd_es['log'].df[v.get_name()].rank(pct=True) true = true.loc[range(10, 17)] for t, a in zip(true.values, df[p.get_name()].values): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_dependent_percentile(es): - v = ft.Feature(es['log']['value']) +def test_dependent_percentile(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) p2 = ft.Feature(p - 1, primitive=Percentile) feature_set = FeatureSet([p, p2]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array(range(10, 17))) - true = es['log'].df[v.get_name()].rank(pct=True) + true = pd_es['log'].df[v.get_name()].rank(pct=True) true = true.loc[range(10, 17)] for t, a in zip(true.values, df[p.get_name()].values): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_agg_percentile(es): - v = ft.Feature(es['log']['value']) +def test_agg_percentile(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) - agg = ft.Feature(p, parent_entity=es['sessions'], primitive=Sum) + agg = ft.Feature(p, parent_entity=pd_es['sessions'], primitive=Sum) feature_set = FeatureSet([agg]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) - log_vals = es['log'].df[[v.get_name(), 'session_id']] + log_vals = pd_es['log'].df[[v.get_name(), 'session_id']] log_vals['percentile'] = log_vals[v.get_name()].rank(pct=True) true_p = log_vals.groupby('session_id')['percentile'].sum()[[0, 1]] for t, a in zip(true_p.values, df[agg.get_name()].values): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_percentile_agg_percentile(es): - v = ft.Feature(es['log']['value']) +def test_percentile_agg_percentile(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) - agg = ft.Feature(p, parent_entity=es['sessions'], primitive=Sum) + agg = ft.Feature(p, parent_entity=pd_es['sessions'], primitive=Sum) pagg = ft.Feature(agg, primitive=Percentile) feature_set = FeatureSet([pagg]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) - log_vals = es['log'].df[[v.get_name(), 'session_id']] + log_vals = pd_es['log'].df[[v.get_name(), 'session_id']] log_vals['percentile'] = log_vals[v.get_name()].rank(pct=True) true_p = log_vals.groupby('session_id')['percentile'].sum().fillna(0) true_p = true_p.rank(pct=True)[[0, 1]] @@ -723,15 +799,15 @@ def test_percentile_agg_percentile(es): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_percentile_agg(es): - v = ft.Feature(es['log']['value']) - agg = ft.Feature(v, parent_entity=es['sessions'], primitive=Sum) +def test_percentile_agg(pd_es): + v = ft.Feature(pd_es['log']['value']) + agg = ft.Feature(v, parent_entity=pd_es['sessions'], primitive=Sum) pagg = ft.Feature(agg, primitive=Percentile) feature_set = FeatureSet([pagg]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) - log_vals = es['log'].df[[v.get_name(), 'session_id']] + log_vals = pd_es['log'].df[[v.get_name(), 'session_id']] true_p = log_vals.groupby('session_id')[v.get_name()].sum().fillna(0) true_p = true_p.rank(pct=True)[[0, 1]] @@ -739,31 +815,31 @@ def test_percentile_agg(es): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_direct_percentile(es): - v = ft.Feature(es['customers']['age']) +def test_direct_percentile(pd_es): + v = ft.Feature(pd_es['customers']['age']) p = ft.Feature(v, primitive=Percentile) - d = ft.Feature(p, es['sessions']) + d = ft.Feature(p, pd_es['sessions']) feature_set = FeatureSet([d]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) - cust_vals = es['customers'].df[[v.get_name()]] + cust_vals = pd_es['customers'].df[[v.get_name()]] cust_vals['percentile'] = cust_vals[v.get_name()].rank(pct=True) true_p = cust_vals['percentile'].loc[[0, 0]] for t, a in zip(true_p.values, df[d.get_name()].values): assert (pd.isnull(t) and pd.isnull(a)) or t == a -def test_direct_agg_percentile(es): - v = ft.Feature(es['log']['value']) +def test_direct_agg_percentile(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) - agg = ft.Feature(p, parent_entity=es['customers'], primitive=Sum) - d = ft.Feature(agg, es['sessions']) + agg = ft.Feature(p, parent_entity=pd_es['customers'], primitive=Sum) + d = ft.Feature(agg, pd_es['sessions']) feature_set = FeatureSet([d]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) - log_vals = es['log'].df[[v.get_name(), 'session_id']] + log_vals = pd_es['log'].df[[v.get_name(), 'session_id']] log_vals['percentile'] = log_vals[v.get_name()].rank(pct=True) log_vals['customer_id'] = [0] * 10 + [1] * 5 + [2] * 2 true_p = log_vals.groupby('customer_id')['percentile'].sum().fillna(0) @@ -772,25 +848,25 @@ def test_direct_agg_percentile(es): assert (pd.isnull(t) and pd.isnull(a)) or round(t, 3) == round(a, 3) -def test_percentile_with_cutoff(es): - v = ft.Feature(es['log']['value']) +def test_percentile_with_cutoff(pd_es): + v = ft.Feature(pd_es['log']['value']) p = ft.Feature(v, primitive=Percentile) feature_set = FeatureSet([p]) - calculator = FeatureSetCalculator(es, feature_set, pd.Timestamp('2011/04/09 10:30:13')) + calculator = FeatureSetCalculator(pd_es, feature_set, pd.Timestamp('2011/04/09 10:30:13')) df = calculator.run(np.array([2])) assert df[p.get_name()].tolist()[0] == 1.0 -def test_two_kinds_of_dependents(es): - v = ft.Feature(es['log']['value']) - product = ft.Feature(es['log']['product_id']) - agg = ft.Feature(v, parent_entity=es['customers'], where=product == 'coke zero', primitive=Sum) +def test_two_kinds_of_dependents(pd_es): + v = ft.Feature(pd_es['log']['value']) + product = ft.Feature(pd_es['log']['product_id']) + agg = ft.Feature(v, parent_entity=pd_es['customers'], where=product == 'coke zero', primitive=Sum) p = ft.Feature(agg, primitive=Percentile) g = ft.Feature(agg, primitive=Absolute) - agg2 = ft.Feature(v, parent_entity=es['sessions'], where=product == 'coke zero', primitive=Sum) - agg3 = ft.Feature(agg2, parent_entity=es['customers'], primitive=Sum) + agg2 = ft.Feature(v, parent_entity=pd_es['sessions'], where=product == 'coke zero', primitive=Sum) + agg3 = ft.Feature(agg2, parent_entity=pd_es['customers'], primitive=Sum) feature_set = FeatureSet([p, g, agg3]) - calculator = FeatureSetCalculator(es, feature_set) + calculator = FeatureSetCalculator(pd_es, feature_set) df = calculator.run(np.array([0, 1])) assert df[p.get_name()].tolist() == [2. / 3, 1.0] assert df[g.get_name()].tolist() == [15, 26] @@ -865,7 +941,7 @@ def isin_generate_name(self, base_feature_names): assert isin_2_list == isin_2.primitive.kwargs['list_of_outputs'] -def test_make_transform_multiple_output_features(es): +def test_make_transform_multiple_output_features(pd_es): def test_time(x): times = pd.Series(x) units = ["year", "month", "day", "hour", "minute", "second"] @@ -884,15 +960,15 @@ def gen_feat_names(self): cls_attributes={"get_feature_names": gen_feat_names}, ) - join_time_split = ft.Feature(es["log"]["datetime"], primitive=TestTime) - alt_features = [ft.Feature(es["log"]["datetime"], primitive=Year), - ft.Feature(es["log"]["datetime"], primitive=Month), - ft.Feature(es["log"]["datetime"], primitive=Day), - ft.Feature(es["log"]["datetime"], primitive=Hour), - ft.Feature(es["log"]["datetime"], primitive=Minute), - ft.Feature(es["log"]["datetime"], primitive=Second)] + join_time_split = ft.Feature(pd_es["log"]["datetime"], primitive=TestTime) + alt_features = [ft.Feature(pd_es["log"]["datetime"], primitive=Year), + ft.Feature(pd_es["log"]["datetime"], primitive=Month), + ft.Feature(pd_es["log"]["datetime"], primitive=Day), + ft.Feature(pd_es["log"]["datetime"], primitive=Hour), + ft.Feature(pd_es["log"]["datetime"], primitive=Minute), + ft.Feature(pd_es["log"]["datetime"], primitive=Second)] fm, fl = ft.dfs( - entityset=es, + entityset=pd_es, target_entity="log", agg_primitives=[], trans_primitives=[TestTime, Year, Month, Day, Hour, Minute, Second, Diff], @@ -920,6 +996,7 @@ class Mod4(TransformPrimitive): name = "mod4" input_types = [Numeric] return_type = Numeric + dask_compatible = True def get_function(self): filepath = self.get_filepath("featuretools_unit_test_example.csv") @@ -930,14 +1007,15 @@ def _map(x): if pd.isnull(x): return x return reference[int(x) % 4] - return pd.Series(x).apply(_map) + return x.apply(_map) return map_to_word feat = ft.Feature(es['log']['value'], primitive=Mod4) df = ft.calculate_feature_matrix(features=[feat], entityset=es, instance_ids=range(17)) - + if isinstance(df, dd.DataFrame): + df = df.compute() assert pd.isnull(df["MOD4(value)"][15]) assert df["MOD4(value)"][0] == 0 assert df["MOD4(value)"][14] == 2 @@ -946,13 +1024,14 @@ def _map(x): target_entity="log", agg_primitives=[], trans_primitives=[Mod4]) - + if isinstance(fm, dd.DataFrame): + fm = fm.compute().set_index('id') assert fm["MOD4(value)"][0] == 0 assert fm["MOD4(value)"][14] == 2 assert pd.isnull(fm["MOD4(value)"][15]) -def test_override_multi_feature_names(es): +def test_override_multi_feature_names(pd_es): def gen_custom_names(primitive, base_feature_names): return ['Above18(%s)' % base_feature_names, 'Above21(%s)' % base_feature_names, @@ -968,7 +1047,7 @@ def is_greater(x): number_output_features=num_features, cls_attributes={"generate_names": gen_custom_names}) - fm, features = ft.dfs(entityset=es, + fm, features = ft.dfs(entityset=pd_es, target_entity="customers", instance_ids=[0, 1, 2], agg_primitives=[], diff --git a/featuretools/tests/selection/test_selection.py b/featuretools/tests/selection/test_selection.py index 7cd6c61d08..84b8fa7e8e 100644 --- a/featuretools/tests/selection/test_selection.py +++ b/featuretools/tests/selection/test_selection.py @@ -19,11 +19,12 @@ def feature_matrix(): @pytest.fixture -def test_es(es, feature_matrix): - es.entity_from_dataframe('test', feature_matrix, index='test') - return es +def test_es(pd_es, feature_matrix): + pd_es.entity_from_dataframe('test', feature_matrix, index='test') + return pd_es +# remove low information features not supported in Dask def test_remove_low_information_feature_names(feature_matrix): feature_matrix = remove_low_information_features(feature_matrix) assert feature_matrix.shape == (3, 5) @@ -31,6 +32,7 @@ def test_remove_low_information_feature_names(feature_matrix): assert 'all_null' not in feature_matrix.columns +# remove low information features not supported in Dask def test_remove_low_information_features(test_es, feature_matrix): features = [Feature(v) for v in test_es['test'].variables] feature_matrix, features = remove_low_information_features(feature_matrix, diff --git a/featuretools/tests/synthesis/test_deep_feature_synthesis.py b/featuretools/tests/synthesis/test_deep_feature_synthesis.py index 5fd0fd459a..bac88f11c4 100644 --- a/featuretools/tests/synthesis/test_deep_feature_synthesis.py +++ b/featuretools/tests/synthesis/test_deep_feature_synthesis.py @@ -1,5 +1,6 @@ import copy +import dask.dataframe as dd import pandas as pd import pytest @@ -10,7 +11,7 @@ IdentityFeature, TransformFeature ) -from featuretools.primitives import ( # CumMean, +from featuretools.primitives import ( Absolute, AddNumeric, Count, @@ -41,24 +42,55 @@ from featuretools.variable_types import Datetime, Numeric +@pytest.fixture(params=['pd_transform_es', 'dask_transform_es']) +def transform_es(request): + return request.getfixturevalue(request.param) + + +@pytest.fixture +def pd_transform_es(): + # Create dataframe + df = pd.DataFrame({'a': [14, 12, 10], 'b': [False, False, True], + 'b1': [True, True, False], 'b12': [4, 5, 6], + 'P': [10, 15, 12]}) + es = ft.EntitySet(id='test') + # Add dataframe to entityset + es.entity_from_dataframe(entity_id='first', dataframe=df, + index='index', + make_index=True) + + return es + + +@pytest.fixture +def dask_transform_es(pd_transform_es): + es = ft.EntitySet(id=pd_transform_es.id) + for entity in pd_transform_es.entities: + es.entity_from_dataframe(entity_id=entity.id, + dataframe=dd.from_pandas(entity.df, npartitions=2), + index=entity.index, + variable_types=entity.variable_types) + return es + + def test_makes_agg_features_from_str(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=['last'], + agg_primitives=['sum'], trans_primitives=[]) features = dfs_obj.build_features() - assert (feature_with_name(features, 'LAST(log.value)')) + assert (feature_with_name(features, 'SUM(log.value)')) def test_makes_agg_features_from_mixed_str(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Count, 'last'], + agg_primitives=[Count, 'sum'], trans_primitives=[]) features = dfs_obj.build_features() - assert (feature_with_name(features, 'LAST(log.value)')) + assert (feature_with_name(features, 'SUM(log.value)')) assert (feature_with_name(features, 'COUNT(log)')) @@ -76,11 +108,11 @@ def test_case_insensitive(es): def test_makes_agg_features(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[]) features = dfs_obj.build_features() - assert (feature_with_name(features, 'LAST(log.value)')) + assert (feature_with_name(features, 'SUM(log.value)')) def test_only_makes_supplied_agg_feat(es): @@ -89,22 +121,34 @@ def test_only_makes_supplied_agg_feat(es): entityset=es, max_depth=3, ) - dfs_obj = DeepFeatureSynthesis(agg_primitives=[Last], **kwargs) + dfs_obj = DeepFeatureSynthesis(agg_primitives=[Sum], **kwargs) features = dfs_obj.build_features() def find_other_agg_features(features): return [f for f in features if (isinstance(f, AggregationFeature) and - not isinstance(f.primitive, Last)) or + not isinstance(f.primitive, Sum)) or len([g for g in f.base_features if isinstance(g, AggregationFeature) and - not isinstance(g.primitive, Last)]) > 0] + not isinstance(g.primitive, Sum)]) > 0] other_agg_features = find_other_agg_features(features) assert len(other_agg_features) == 0 +def test_errors_unsupported_primitives_dask(dask_es): + bad_trans_prim = CumSum() + bad_agg_prim = NumUnique() + bad_trans_prim.dask_compatible, bad_agg_prim.dask_compatible = False, False + error_text = "Selected primitives are incompatible with Dask EntitySets: cum_sum, num_unique" + with pytest.raises(ValueError, match=error_text): + DeepFeatureSynthesis(target_entity_id='sessions', + entityset=dask_es, + agg_primitives=[bad_agg_prim], + trans_primitives=[bad_trans_prim]) + + def test_error_for_missing_target_entity(es): error_text = 'Provided target entity missing_entity does not exist in ecommerce' with pytest.raises(KeyError, match=error_text): @@ -129,13 +173,13 @@ def test_ignores_entities(es): with pytest.raises(TypeError, match=error_text): DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], ignore_entities='log') dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], ignore_entities=['log']) @@ -149,7 +193,7 @@ def test_ignores_entities(es): def test_ignores_variables(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], ignore_variables={'log': ['value']}) features = dfs_obj.build_features() @@ -215,9 +259,9 @@ def test_makes_trans_feat(es): assert (feature_with_name(features, 'HOUR(datetime)')) -def test_handles_diff_entity_groupby(es): +def test_handles_diff_entity_groupby(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='log', - entityset=es, + entityset=pd_es, agg_primitives=[], groupby_trans_primitives=[Diff]) @@ -226,9 +270,9 @@ def test_handles_diff_entity_groupby(es): assert (feature_with_name(features, 'DIFF(value) by product_id')) -def test_handles_time_since_previous_entity_groupby(es): +def test_handles_time_since_previous_entity_groupby(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='log', - entityset=es, + entityset=pd_es, agg_primitives=[], groupby_trans_primitives=[TimeSincePrevious]) @@ -236,9 +280,9 @@ def test_handles_time_since_previous_entity_groupby(es): assert (feature_with_name(features, 'TIME_SINCE_PREVIOUS(datetime) by session_id')) # M TODO -# def test_handles_cumsum_entity_groupby(es): +# def test_handles_cumsum_entity_groupby(pd_es): # dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', -# entityset=es, +# entityset=pd_es, # agg_primitives=[], # trans_primitives=[CumMean]) @@ -265,24 +309,29 @@ def test_only_makes_supplied_trans_feat(es): def test_makes_dfeatures_of_agg_primitives(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=['max'], trans_primitives=[]) features = dfs_obj.build_features() + assert (feature_with_name(features, - 'customers.LAST(sessions.device_type)')) + 'customers.MAX(log.value)')) def test_makes_agg_features_of_trans_primitives(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], - trans_primitives=[Hour]) + agg_primitives=[Mean], + trans_primitives=[NumCharacters]) features = dfs_obj.build_features() - assert (feature_with_name(features, 'LAST(log.HOUR(datetime))')) + assert (feature_with_name(features, 'MEAN(log.NUM_CHARACTERS(comments))')) def test_makes_agg_features_with_where(es): + # TODO: Update to work with Dask `es` fixture when issue #978 is closed + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support add_interesting_values") + es.add_interesting_values() dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', @@ -300,9 +349,9 @@ def test_makes_agg_features_with_where(es): 'COUNT(log WHERE products.department = food)')) -def test_make_groupby_features(es): +def test_make_groupby_features(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='log', - entityset=es, + entityset=pd_es, agg_primitives=[], trans_primitives=[], groupby_trans_primitives=['cum_sum']) @@ -311,9 +360,9 @@ def test_make_groupby_features(es): "CUM_SUM(value) by session_id")) -def test_make_indirect_groupby_features(es): +def test_make_indirect_groupby_features(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='log', - entityset=es, + entityset=pd_es, agg_primitives=[], trans_primitives=[], groupby_trans_primitives=['cum_sum']) @@ -322,9 +371,9 @@ def test_make_indirect_groupby_features(es): "CUM_SUM(products.rating) by session_id")) -def test_make_groupby_features_with_id(es): +def test_make_groupby_features_with_id(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', - entityset=es, + entityset=pd_es, agg_primitives=[], trans_primitives=[], groupby_trans_primitives=['cum_count']) @@ -332,9 +381,9 @@ def test_make_groupby_features_with_id(es): assert (feature_with_name(features, "CUM_COUNT(customer_id) by customer_id")) -def test_make_groupby_features_with_diff_id(es): +def test_make_groupby_features_with_diff_id(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='customers', - entityset=es, + entityset=pd_es, agg_primitives=[], trans_primitives=[], groupby_trans_primitives=['cum_count']) @@ -343,9 +392,9 @@ def test_make_groupby_features_with_diff_id(es): assert (feature_with_name(features, groupby_with_diff_id)) -def test_make_groupby_features_with_agg(es): +def test_make_groupby_features_with_agg(pd_es): dfs_obj = DeepFeatureSynthesis(target_entity_id='cohorts', - entityset=es, + entityset=pd_es, agg_primitives=['sum'], trans_primitives=[], groupby_trans_primitives=['cum_sum']) @@ -368,7 +417,7 @@ def test_abides_by_max_depth_param(es): for i in [1, 2, 3]: dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=i) @@ -381,7 +430,7 @@ def test_abides_by_max_depth_param(es): def test_drop_contains(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=1, seed_features=[], @@ -391,7 +440,7 @@ def test_drop_contains(es): partial_name = to_drop.get_name()[:5] dfs_drop = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=1, seed_features=[], @@ -403,7 +452,7 @@ def test_drop_contains(es): def test_drop_exact(es): dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=1, seed_features=[], @@ -413,7 +462,7 @@ def test_drop_exact(es): name = to_drop.get_name() dfs_drop = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=1, seed_features=[], @@ -423,12 +472,12 @@ def test_drop_exact(es): def test_seed_features(es): - seed_feature_sessions = ft.Feature(es['log']["id"], parent_entity=es['sessions'], primitive=Count) > 2 - seed_feature_log = ft.Feature(es['log']['datetime'], primitive=Hour) - session_agg = ft.Feature(seed_feature_log, parent_entity=es['sessions'], primitive=Last) + seed_feature_sessions = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) > 2 + seed_feature_log = ft.Feature(es['log']['comments'], primitive=NumCharacters) + session_agg = ft.Feature(seed_feature_log, parent_entity=es['sessions'], primitive=Mean) dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', entityset=es, - agg_primitives=[Last], + agg_primitives=[Mean], trans_primitives=[], max_depth=2, seed_features=[seed_feature_sessions, @@ -440,7 +489,11 @@ def test_seed_features(es): def test_does_not_make_agg_of_direct_of_target_entity(es): - count_sessions = ft.Feature(es['sessions']["id"], parent_entity=es['customers'], primitive=Count) + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the Last primitive") + + count_sessions = ft.Feature(es['sessions']['id'], parent_entity=es['customers'], primitive=Count) dfs_obj = DeepFeatureSynthesis(target_entity_id='customers', entityset=es, agg_primitives=[Last], @@ -455,7 +508,11 @@ def test_does_not_make_agg_of_direct_of_target_entity(es): def test_dfs_builds_on_seed_features_more_than_max_depth(es): - seed_feature_sessions = ft.Feature(es['log']["id"], parent_entity=es['sessions'], primitive=Count) + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the Last and Mode primitives") + + seed_feature_sessions = ft.Feature(es['log']['id'], parent_entity=es['sessions'], primitive=Count) seed_feature_log = ft.Feature(es['log']['datetime'], primitive=Hour) session_agg = ft.Feature(seed_feature_log, parent_entity=es['sessions'], primitive=Last) @@ -479,6 +536,10 @@ def test_dfs_builds_on_seed_features_more_than_max_depth(es): def test_allowed_paths(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the Last primitive") + kwargs = dict( target_entity_id='customers', entityset=es, @@ -509,7 +570,7 @@ def test_max_features(es): kwargs = dict( target_entity_id='customers', entityset=es, - agg_primitives=[Last], + agg_primitives=[Sum], trans_primitives=[], max_depth=2, seed_features=[] @@ -531,12 +592,12 @@ def test_where_primitives(es): kwargs = dict( target_entity_id='customers', entityset=es, - agg_primitives=[Count, Last], + agg_primitives=[Count, Sum], trans_primitives=[Absolute], max_depth=3, ) dfs_unconstrained = DeepFeatureSynthesis(**kwargs) - dfs_constrained = DeepFeatureSynthesis(where_primitives=['last'], **kwargs) + dfs_constrained = DeepFeatureSynthesis(where_primitives=['sum'], **kwargs) features_unconstrained = dfs_unconstrained.build_features() features = dfs_constrained.build_features() @@ -548,12 +609,12 @@ def test_where_primitives(es): assert len(where_feats_unconstrained) >= 1 assert len([f for f in where_feats_unconstrained - if isinstance(f.primitive, Last)]) == 0 + if isinstance(f.primitive, Sum)]) == 0 assert len([f for f in where_feats_unconstrained if isinstance(f.primitive, Count)]) > 0 assert len([f for f in where_feats - if isinstance(f.primitive, Last)]) > 0 + if isinstance(f.primitive, Sum)]) > 0 assert len([f for f in where_feats if isinstance(f.primitive, Count)]) == 0 assert len([d for f in where_feats @@ -562,6 +623,9 @@ def test_where_primitives(es): def test_stacking_where_primitives(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the Last primitive") es = copy.deepcopy(es) es['sessions']['device_type'].interesting_values = [0] es['log']['product_id'].interesting_values = ["coke_zero"] @@ -620,8 +684,8 @@ def test_where_different_base_feats(es): kwargs = dict( target_entity_id='customers', entityset=es, - agg_primitives=[Last, Count], - where_primitives=[Last, Count], + agg_primitives=[Sum, Count], + where_primitives=[Sum, Count], max_depth=3, ) dfs_unconstrained = DeepFeatureSynthesis(**kwargs) @@ -635,6 +699,9 @@ def test_where_different_base_feats(es): def test_dfeats_where(es): + # TODO: Update to work with Dask `es` fixture when issue #978 is closed + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support add_interesting_values") es.add_interesting_values() dfs_obj = DeepFeatureSynthesis(target_entity_id='sessions', @@ -673,19 +740,9 @@ def test_commutative(es): assert len(add_feats) == len(unordered_args) -def test_transform_consistency(): - # Create dataframe - df = pd.DataFrame({'a': [14, 12, 10], 'b': [False, False, True], - 'b1': [True, True, False], 'b12': [4, 5, 6], - 'P': [10, 15, 12]}) - es = ft.EntitySet(id='test') - # Add dataframe to entityset - es.entity_from_dataframe(entity_id='first', dataframe=df, - index='index', - make_index=True) - +def test_transform_consistency(transform_es): # Generate features - feature_defs = ft.dfs(entityset=es, target_entity='first', + feature_defs = ft.dfs(entityset=transform_es, target_entity='first', trans_primitives=['and', 'add_numeric', 'or'], features_only=True) @@ -706,6 +763,9 @@ def test_transform_consistency(): def test_transform_no_stack_agg(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the NMostCommon primitive") feature_defs = ft.dfs(entityset=es, target_entity="customers", agg_primitives=[NMostCommon], @@ -727,6 +787,9 @@ def test_intialized_trans_prim(es): def test_initialized_agg_prim(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the NMostCommon primitive") ThreeMost = NMostCommon(n=3) dfs_obj = DeepFeatureSynthesis(target_entity_id="sessions", entityset=es, @@ -737,6 +800,9 @@ def test_initialized_agg_prim(es): def test_return_variable_types(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the NMostCommon primitive") dfs_obj = DeepFeatureSynthesis(target_entity_id="sessions", entityset=es, agg_primitives=[Count, NMostCommon], @@ -784,13 +850,13 @@ def test_checks_primitives_correct_type(es): trans_primitives=[]) error_text = "Primitive in trans_primitives or "\ + "aggregation_primitives\\.Sum\\'> in trans_primitives or "\ "groupby_trans_primitives is not a transform primitive" with pytest.raises(ValueError, match=error_text): DeepFeatureSynthesis(target_entity_id="sessions", entityset=es, agg_primitives=[], - trans_primitives=[Last]) + trans_primitives=[Sum]) def test_makes_agg_features_along_multiple_paths(diamond_es): @@ -822,6 +888,10 @@ def test_makes_direct_features_through_multiple_relationships(games_es): def test_stacks_multioutput_features(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the NumUnique and NMostCommon primitives") + class TestTime(TransformPrimitive): name = "test_time" input_types = [Datetime] @@ -853,6 +923,9 @@ def test_f(x): def test_seed_multi_output_feature_stacking(es): + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support the NMostCommon and NumUnique primitives") threecommon = NMostCommon(3) tc = ft.Feature(es['log']['product_id'], parent_entity=es["sessions"], primitive=threecommon) @@ -895,9 +968,11 @@ def test_does_not_make_trans_of_single_direct_feature(es): def test_makes_trans_of_multiple_direct_features(diamond_es): - es = diamond_es + # TODO: Update to work with Dask supported primitive + if any(isinstance(entity.df, dd.DataFrame) for entity in diamond_es.entities): + pytest.xfail("Dask EntitySets do not support the Equal primitive") dfs_obj = DeepFeatureSynthesis(target_entity_id='transactions', - entityset=es, + entityset=diamond_es, agg_primitives=['mean'], trans_primitives=[Equal], max_depth=4) @@ -933,64 +1008,64 @@ def test_makes_direct_of_agg_of_trans_on_target(es): def test_primitive_options_errors(es): - wrong_key_options = {'mode': {'ignore_entity': ['sessions']}} - wrong_type_list = {'mode': {'ignore_entities': 'sessions'}} - wrong_type_dict = {'mode': + wrong_key_options = {'mean': {'ignore_entity': ['sessions']}} + wrong_type_list = {'mean': {'ignore_entities': 'sessions'}} + wrong_type_dict = {'mean': {'ignore_variables': {'sessions': 'product_id'}}} - conflicting_primitive_options = {('count', 'mode'): + conflicting_primitive_options = {('count', 'mean'): {'ignore_entities': ['sessions']}, - 'mode': {'include_entities': ['sessions']}} - invalid_entity = {'mode': {'include_entities': ['invalid_entity']}} - invalid_variable_entity = {'mode': {'include_variables': {'invalid_entity': ['product_id']}}} - invalid_variable = {'mode': {'include_variables': {'sessions': ['invalid_variable']}}} - key_error_text = "Unrecognized primitive option 'ignore_entity' for mode" - list_error_text = "Incorrect type formatting for 'ignore_entities' for mode" - dict_error_text = "Incorrect type formatting for 'ignore_variables' for mode" - conflicting_error_text = "Multiple options found for primitive mode" + 'mean': {'include_entities': ['sessions']}} + invalid_entity = {'mean': {'include_entities': ['invalid_entity']}} + invalid_variable_entity = {'mean': {'include_variables': {'invalid_entity': ['product_id']}}} + invalid_variable = {'mean': {'include_variables': {'sessions': ['invalid_variable']}}} + key_error_text = "Unrecognized primitive option 'ignore_entity' for mean" + list_error_text = "Incorrect type formatting for 'ignore_entities' for mean" + dict_error_text = "Incorrect type formatting for 'ignore_variables' for mean" + conflicting_error_text = "Multiple options found for primitive mean" invalid_entity_warning = "Entity 'invalid_entity' not in entityset" invalid_variable_warning = "Variable 'invalid_variable' not in entity 'sessions'" with pytest.raises(KeyError, match=key_error_text): DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=wrong_key_options) with pytest.raises(TypeError, match=list_error_text): DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=wrong_type_list) with pytest.raises(TypeError, match=dict_error_text): DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=wrong_type_dict) with pytest.raises(KeyError, match=conflicting_error_text): DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=conflicting_primitive_options) with pytest.warns(UserWarning, match=invalid_entity_warning) as record: DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=invalid_entity) assert len(record) == 1 with pytest.warns(UserWarning, match=invalid_entity_warning) as record: DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=invalid_variable_entity) assert len(record) == 1 with pytest.warns(UserWarning, match=invalid_variable_warning) as record: DeepFeatureSynthesis(target_entity_id='customers', entityset=es, - agg_primitives=['mode'], + agg_primitives=['mean'], trans_primitives=[], primitive_options=invalid_variable) assert len(record) == 1 @@ -1124,7 +1199,7 @@ def test_primitive_options_with_globals(es): identity_base.get_name() == 'age') -def test_primitive_options_groupbys(es): +def test_primitive_options_groupbys(pd_es): options = {'cum_count': {'include_groupby_entities': ['log', 'customers']}, 'cum_sum': {'ignore_groupby_entities': ['sessions']}, 'cum_mean': {'ignore_groupby_variables': {'customers': [u'région_id'], @@ -1132,7 +1207,7 @@ def test_primitive_options_groupbys(es): 'cum_min': {'include_groupby_variables': {'sessions': ['customer_id', 'device_type']}}} dfs_obj = DeepFeatureSynthesis(target_entity_id='log', - entityset=es, + entityset=pd_es, agg_primitives=[], trans_primitives=[], max_depth=3, @@ -1166,6 +1241,8 @@ def test_primitive_options_groupbys(es): def test_primitive_options_multiple_inputs(es): + if any(isinstance(entity.df, dd.DataFrame) for entity in es.entities): + pytest.xfail("Dask EntitySets do not support various primitives used in this test") too_many_options = {'mode': [{'include_entities': ['logs']}, {'ignore_entities': ['sessions']}]} error_msg = "Number of options does not match number of inputs for primitive mode" diff --git a/featuretools/tests/synthesis/test_dfs_method.py b/featuretools/tests/synthesis/test_dfs_method.py index 0b15dbf179..915c6af744 100644 --- a/featuretools/tests/synthesis/test_dfs_method.py +++ b/featuretools/tests/synthesis/test_dfs_method.py @@ -1,8 +1,11 @@ +import composeml as cp import numpy as np import pandas as pd import pytest +from dask import dataframe as dd from distributed.utils_test import cluster +from featuretools import variable_types as vtypes from featuretools.computational_backends.calculate_feature_matrix import ( FEATURE_CALCULATION_PERCENTAGE ) @@ -11,8 +14,13 @@ from featuretools.synthesis import dfs +@pytest.fixture(params=['pd_entities', 'dask_entities']) +def entities(request): + return request.getfixturevalue(request.param) + + @pytest.fixture -def entities(): +def pd_entities(): cards_df = pd.DataFrame({"id": [1, 2, 3, 4, 5]}) transactions_df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], "card_id": [1, 2, 1, 3, 4, 5], @@ -25,6 +33,33 @@ def entities(): return entities +@pytest.fixture +def dask_entities(): + cards_df = pd.DataFrame({"id": [1, 2, 3, 4, 5]}) + transactions_df = pd.DataFrame({"id": [1, 2, 3, 4, 5, 6], + "card_id": [1, 2, 1, 3, 4, 5], + "transaction_time": [10, 12, 13, 20, 21, 20], + "fraud": [True, False, False, False, True, True]}) + cards_df = dd.from_pandas(cards_df, npartitions=2) + transactions_df = dd.from_pandas(transactions_df, npartitions=2) + + cards_vtypes = { + 'id': vtypes.Index + } + transactions_vtypes = { + 'id': vtypes.Index, + 'card_id': vtypes.Id, + 'transaction_time': vtypes.NumericTimeIndex, + 'fraud': vtypes.Boolean + } + + entities = { + "cards": (cards_df, "id", None, cards_vtypes), + "transactions": (transactions_df, "id", "transaction_time", transactions_vtypes) + } + return entities + + @pytest.fixture def relationships(): return [("cards", "id", "transactions", "card_id")] @@ -63,15 +98,65 @@ def test_accepts_cutoff_time_df(entities, relationships): relationships=relationships, target_entity="transactions", cutoff_time=cutoff_times_df) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index("id") assert len(feature_matrix.index) == 3 assert len(feature_matrix.columns) == len(features) +def test_warns_cutoff_time_dask(entities, relationships): + cutoff_times_df = pd.DataFrame({"instance_id": [1, 2, 3], + "time": [10, 12, 15]}) + cutoff_times_df = dd.from_pandas(cutoff_times_df, npartitions=2) + match = "cutoff_time should be a Pandas DataFrame: " \ + "computing cutoff_time, this may take a while" + with pytest.warns(UserWarning, match=match): + feature_matrix, features = dfs(entities=entities, + relationships=relationships, + target_entity="transactions", + cutoff_time=cutoff_times_df) + + +def test_accepts_cutoff_time_compose(entities, relationships): + def fraud_occured(df): + return df['fraud'].any() + + lm = cp.LabelMaker( + target_entity='card_id', + time_index='transaction_time', + labeling_function=fraud_occured, + window_size=1 + ) + + transactions_df = entities['transactions'][0] + if isinstance(transactions_df, dd.DataFrame): + transactions_df = transactions_df.compute() + + labels = lm.search( + transactions_df, + num_examples_per_instance=-1 + ) + + labels['time'] = pd.to_numeric(labels['time']) + labels.rename({'card_id': 'id'}, axis=1, inplace=True) + + feature_matrix, features = dfs(entities=entities, + relationships=relationships, + target_entity="cards", + cutoff_time=labels) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id') + assert len(feature_matrix.index) == 6 + assert len(feature_matrix.columns) == len(features) + 1 + + def test_accepts_single_cutoff_time(entities, relationships): feature_matrix, features = dfs(entities=entities, relationships=relationships, target_entity="transactions", cutoff_time=20) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.compute().set_index('id') assert len(feature_matrix.index) == 5 assert len(feature_matrix.columns) == len(features) @@ -81,6 +166,8 @@ def test_accepts_no_cutoff_time(entities, relationships): relationships=relationships, target_entity="transactions", instance_ids=[1, 2, 3, 5, 6]) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.set_index('id').compute() assert len(feature_matrix.index) == 5 assert len(feature_matrix.columns) == len(features) @@ -94,14 +181,17 @@ def test_ignores_instance_ids_if_cutoff_df(entities, relationships): target_entity="transactions", cutoff_time=cutoff_times_df, instance_ids=instance_ids) + if isinstance(feature_matrix, dd.DataFrame): + feature_matrix = feature_matrix.set_index('id').compute() assert len(feature_matrix.index) == 3 assert len(feature_matrix.columns) == len(features) -def test_approximate_features(entities, relationships): +def test_approximate_features(pd_entities, relationships): + # TODO: Update to use Dask entities when issue #985 is closed cutoff_times_df = pd.DataFrame({"instance_id": [1, 3, 1, 5, 3, 6], "time": [11, 16, 16, 26, 17, 22]}) - feature_matrix, features = dfs(entities=entities, + feature_matrix, features = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", cutoff_time=cutoff_times_df, @@ -116,11 +206,11 @@ def test_approximate_features(entities, relationships): assert (feature_matrix[direct_agg_feat_name] == truth_values.values).all() -def test_all_variables(entities, relationships): +def test_all_variables(pd_entities, relationships): cutoff_times_df = pd.DataFrame({"instance_id": [1, 2, 3], "time": [10, 12, 15]}) instance_ids = [1, 2, 3, 4, 5] - feature_matrix, features = dfs(entities=entities, + feature_matrix, features = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", cutoff_time=cutoff_times_df, @@ -145,17 +235,17 @@ def test_features_only(entities, relationships): assert len(features) > 0 -def test_dask_kwargs(entities, relationships): +def test_dask_kwargs(pd_entities, relationships): cutoff_times_df = pd.DataFrame({"instance_id": [1, 2, 3], "time": [10, 12, 15]}) - feature_matrix, features = dfs(entities=entities, + feature_matrix, features = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", cutoff_time=cutoff_times_df) with cluster() as (scheduler, [a, b]): dask_kwargs = {'cluster': scheduler['address']} - feature_matrix_2, features_2 = dfs(entities=entities, + feature_matrix_2, features_2 = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", cutoff_time=cutoff_times_df, @@ -168,6 +258,7 @@ def test_dask_kwargs(entities, relationships): def test_accepts_relative_training_window(datetime_es): + # TODO: Update to use Dask entities when issue #882 is closed feature_matrix, features = dfs(entityset=datetime_es, target_entity="transactions") @@ -190,6 +281,7 @@ def test_accepts_relative_training_window(datetime_es): assert (feature_matrix_3.index == [2, 3, 4]).all() assert (feature_matrix_4.index == [2, 3, 4]).all() + # Test case for leap years feature_matrix_5, features_5 = dfs(entityset=datetime_es, target_entity="transactions", cutoff_time=pd.Timestamp("2012-2-29 04:00"), @@ -206,6 +298,7 @@ def test_accepts_relative_training_window(datetime_es): def test_accepts_pd_timedelta_training_window(datetime_es): + # TODO: Update to use Dask entities when issue #882 is closed feature_matrix, features = dfs(entityset=datetime_es, target_entity="transactions", cutoff_time=pd.Timestamp("2012-3-31 04:00"), @@ -215,6 +308,7 @@ def test_accepts_pd_timedelta_training_window(datetime_es): def test_accepts_pd_dateoffset_training_window(datetime_es): + # TODO: Update to use Dask entities when issue #882 is closed feature_matrix, features = dfs(entityset=datetime_es, target_entity="transactions", cutoff_time=pd.Timestamp("2012-3-31 04:00"), @@ -229,7 +323,8 @@ def test_accepts_pd_dateoffset_training_window(datetime_es): assert (feature_matrix.index == feature_matrix_2.index).all() -def test_calls_progress_callback(entities, relationships): +# TODO: split out cluster test to run on pandas seperately +def test_calls_progress_callback(pd_entities, relationships): class MockProgressCallback: def __init__(self): self.progress_history = [] @@ -243,7 +338,7 @@ def __call__(self, update, progress_percent, time_elapsed): mock_progress_callback = MockProgressCallback() - feature_matrix, features = dfs(entities=entities, + feature_matrix, features = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", progress_callback=mock_progress_callback) @@ -258,7 +353,7 @@ def __call__(self, update, progress_percent, time_elapsed): with cluster() as (scheduler, [a, b]): dkwargs = {'cluster': scheduler['address']} - feature_matrix, features = dfs(entities=entities, + feature_matrix, features = dfs(entities=pd_entities, relationships=relationships, target_entity="transactions", progress_callback=mock_progress_callback, diff --git a/featuretools/tests/synthesis/test_encode_features.py b/featuretools/tests/synthesis/test_encode_features.py index e76a581227..dfcdb14cd2 100644 --- a/featuretools/tests/synthesis/test_encode_features.py +++ b/featuretools/tests/synthesis/test_encode_features.py @@ -7,13 +7,13 @@ from featuretools.synthesis import encode_features -def test_encodes_features(es): - f1 = IdentityFeature(es["log"]["product_id"]) - f2 = IdentityFeature(es["log"]["purchased"]) - f3 = IdentityFeature(es["log"]["value"]) +def test_encodes_features(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) + f2 = IdentityFeature(pd_es["log"]["purchased"]) + f3 = IdentityFeature(pd_es["log"]["value"]) features = [f1, f2, f3] - feature_matrix = calculate_feature_matrix(features, es, instance_ids=[0, 1, 2, 3, 4, 5]) + feature_matrix = calculate_feature_matrix(features, pd_es, instance_ids=[0, 1, 2, 3, 4, 5]) feature_matrix_encoded, features_encoded = encode_features(feature_matrix, features) assert len(features_encoded) == 6 @@ -26,11 +26,26 @@ def test_encodes_features(es): assert len(features_encoded) == 5 -def test_inplace_encodes_features(es): - f1 = IdentityFeature(es["log"]["product_id"]) +def test_dask_errors_encode_features(dask_es): + f1 = IdentityFeature(dask_es["log"]["product_id"]) + f2 = IdentityFeature(dask_es["log"]["purchased"]) + f3 = IdentityFeature(dask_es["log"]["value"]) + + features = [f1, f2, f3] + feature_matrix = calculate_feature_matrix(features, + dask_es, + instance_ids=[0, 1, 2, 3, 4, 5]) + error_text = "feature_matrix must be a Pandas DataFrame" + + with pytest.raises(TypeError, match=error_text): + encode_features(feature_matrix, features) + + +def test_inplace_encodes_features(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) features = [f1] - feature_matrix = calculate_feature_matrix(features, es, instance_ids=[0, 1, 2, 3, 4, 5]) + feature_matrix = calculate_feature_matrix(features, pd_es, instance_ids=[0, 1, 2, 3, 4, 5]) feature_matrix_shape = feature_matrix.shape feature_matrix_encoded, features_encoded = encode_features(feature_matrix, features) @@ -42,12 +57,12 @@ def test_inplace_encodes_features(es): assert feature_matrix_encoded.shape == feature_matrix.shape -def test_to_encode_features(es): - f1 = IdentityFeature(es["log"]["product_id"]) - f2 = IdentityFeature(es["log"]["value"]) +def test_to_encode_features(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) + f2 = IdentityFeature(pd_es["log"]["value"]) features = [f1, f2] - feature_matrix = calculate_feature_matrix(features, es, instance_ids=[0, 1, 2, 3, 4, 5]) + feature_matrix = calculate_feature_matrix(features, pd_es, instance_ids=[0, 1, 2, 3, 4, 5]) feature_matrix_encoded, features_encoded = encode_features(feature_matrix, features) feature_matrix_encoded_shape = feature_matrix_encoded.shape @@ -62,16 +77,16 @@ def test_to_encode_features(es): assert feature_matrix_encoded_shape != feature_matrix_encoded.shape -def test_encode_features_handles_pass_columns(es): - f1 = IdentityFeature(es["log"]["product_id"]) - f2 = IdentityFeature(es["log"]["value"]) +def test_encode_features_handles_pass_columns(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) + f2 = IdentityFeature(pd_es["log"]["value"]) features = [f1, f2] cutoff_time = pd.DataFrame({'instance_id': range(6), - 'time': es['log'].df['datetime'][0:6], + 'time': pd_es['log'].df['datetime'][0:6], 'label': [i % 2 for i in range(6)]}, columns=["instance_id", "time", "label"]) - feature_matrix = calculate_feature_matrix(features, es, cutoff_time) + feature_matrix = calculate_feature_matrix(features, pd_es, cutoff_time) assert 'label' in feature_matrix.columns @@ -90,17 +105,17 @@ def test_encode_features_handles_pass_columns(es): assert 'label' in feature_matrix_encoded.columns -def test_encode_features_catches_features_mismatch(es): - f1 = IdentityFeature(es["log"]["product_id"]) - f2 = IdentityFeature(es["log"]["value"]) - f3 = IdentityFeature(es["log"]["session_id"]) +def test_encode_features_catches_features_mismatch(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) + f2 = IdentityFeature(pd_es["log"]["value"]) + f3 = IdentityFeature(pd_es["log"]["session_id"]) features = [f1, f2] cutoff_time = pd.DataFrame({'instance_id': range(6), - 'time': es['log'].df['datetime'][0:6], + 'time': pd_es['log'].df['datetime'][0:6], 'label': [i % 2 for i in range(6)]}, columns=["instance_id", "time", "label"]) - feature_matrix = calculate_feature_matrix(features, es, cutoff_time) + feature_matrix = calculate_feature_matrix(features, pd_es, cutoff_time) assert 'label' in feature_matrix.columns @@ -113,9 +128,9 @@ def test_encode_unknown_features(): # Dataframe with categorical column with "unknown" string df = pd.DataFrame({'category': ['unknown', 'b', 'c', 'd', 'e']}) - es = EntitySet('test') - es.entity_from_dataframe(entity_id='a', dataframe=df, index='index', make_index=True) - features, feature_defs = dfs(entityset=es, target_entity='a') + pd_es = EntitySet('test') + pd_es.entity_from_dataframe(entity_id='a', dataframe=df, index='index', make_index=True) + features, feature_defs = dfs(entityset=pd_es, target_entity='a') # Specify unknown token for replacement features_enc, feature_defs_enc = encode_features(features, feature_defs, @@ -124,11 +139,11 @@ def test_encode_unknown_features(): 'category = c', 'category = b', 'category is unknown'] -def test_encode_features_topn(es): - topn = Feature(es['log']['product_id'], - parent_entity=es['customers'], +def test_encode_features_topn(pd_es): + topn = Feature(pd_es['log']['product_id'], + parent_entity=pd_es['customers'], primitive=NMostCommon(n=3)) - features, feature_defs = dfs(entityset=es, + features, feature_defs = dfs(entityset=pd_es, instance_ids=[0, 1, 2], target_entity="customers", agg_primitives=[NMostCommon(n=3)]) @@ -143,9 +158,9 @@ def test_encode_features_topn(es): def test_encode_features_drop_first(): df = pd.DataFrame({'category': ['ao', 'b', 'c', 'd', 'e']}) - es = EntitySet('test') - es.entity_from_dataframe(entity_id='a', dataframe=df, index='index', make_index=True) - features, feature_defs = dfs(entityset=es, target_entity='a') + pd_es = EntitySet('test') + pd_es.entity_from_dataframe(entity_id='a', dataframe=df, index='index', make_index=True) + features, feature_defs = dfs(entityset=pd_es, target_entity='a') features_enc, feature_defs_enc = encode_features(features, feature_defs, drop_first=True, include_unknown=False) assert len(features_enc.columns) == 4 @@ -156,13 +171,13 @@ def test_encode_features_drop_first(): assert len(features_enc.columns) == 2 -def test_encode_features_handles_dictionary_input(es): - f1 = IdentityFeature(es["log"]["product_id"]) - f2 = IdentityFeature(es["log"]["purchased"]) - f3 = IdentityFeature(es["log"]["session_id"]) +def test_encode_features_handles_dictionary_input(pd_es): + f1 = IdentityFeature(pd_es["log"]["product_id"]) + f2 = IdentityFeature(pd_es["log"]["purchased"]) + f3 = IdentityFeature(pd_es["log"]["session_id"]) features = [f1, f2, f3] - feature_matrix = calculate_feature_matrix(features, es, instance_ids=range(16)) + feature_matrix = calculate_feature_matrix(features, pd_es, instance_ids=range(16)) feature_matrix_encoded, features_encoded = encode_features(feature_matrix, features) true_values = ['product_id = coke zero', 'product_id = toothpaste', 'product_id = car', 'product_id = brown bag', 'product_id = taco clock', 'product_id = Haribo sugar-free gummy bears', diff --git a/featuretools/tests/utils_tests/test_entity_utils.py b/featuretools/tests/utils_tests/test_entity_utils.py index 669ed12f58..1f8b073e7b 100644 --- a/featuretools/tests/utils_tests/test_entity_utils.py +++ b/featuretools/tests/utils_tests/test_entity_utils.py @@ -1,4 +1,8 @@ +import copy + +import dask.dataframe as dd import pandas as pd +import pytest import featuretools as ft from featuretools import variable_types as vtypes @@ -10,6 +14,24 @@ ) +@pytest.fixture +def pd_mock_customer_es(): + return ft.demo.load_mock_customer(return_entityset=True, random_seed=0) + + +@pytest.fixture +def dask_mock_customer_es(pd_mock_customer_es): + dask_es = copy.deepcopy(pd_mock_customer_es) + for entity in dask_es.entities: + entity.df = dd.from_pandas(entity.df.reset_index(drop=True), npartitions=2) + return dask_es + + +@pytest.fixture(params=['pd_mock_customer_es', 'dask_mock_customer_es']) +def mock_customer_es(request): + return request.getfixturevalue(request.param) + + def test_infer_variable_types(): df = pd.DataFrame({'id': [0, 1, 2], @@ -149,17 +171,16 @@ def test_convert_variable_data(): assert df['date'].dtype.name in vtypes.PandasTypes._pandas_datetimes -def test_get_linked_vars(): - es = ft.demo.load_mock_customer(return_entityset=True) +def test_get_linked_vars(mock_customer_es): - transactions_linked_vars = get_linked_vars(es['transactions']) + transactions_linked_vars = get_linked_vars(mock_customer_es['transactions']) assert transactions_linked_vars == ['product_id', 'session_id'] - products_linked_vars = get_linked_vars(es['products']) + products_linked_vars = get_linked_vars(mock_customer_es['products']) assert products_linked_vars == ['product_id'] - sessions_linked_vars = get_linked_vars(es['sessions']) + sessions_linked_vars = get_linked_vars(mock_customer_es['sessions']) assert sessions_linked_vars == ['session_id', 'customer_id'] - customers_linked_vars = get_linked_vars(es['customers']) + customers_linked_vars = get_linked_vars(mock_customer_es['customers']) assert customers_linked_vars == ['customer_id'] diff --git a/featuretools/tests/utils_tests/test_entry_point.py b/featuretools/tests/utils_tests/test_entry_point.py index a6420b8f61..53d7efd3f0 100644 --- a/featuretools/tests/utils_tests/test_entry_point.py +++ b/featuretools/tests/utils_tests/test_entry_point.py @@ -1,9 +1,34 @@ +import dask.dataframe as dd import pandas as pd import pytest from featuretools import dfs +@pytest.fixture +def pd_entry_point_dfs(): + cards_df = pd.DataFrame({"id": [1, 2, 3, 4, 5]}) + transactions_df = pd.DataFrame({ + "id": [1, 2, 3, 4, 5, 6], + "card_id": [1, 2, 1, 3, 4, 5], + "transaction_time": [10, 12, 13, 20, 21, 20], + "fraud": [True, False, True, False, True, True] + }) + return cards_df, transactions_df + + +@pytest.fixture +def dask_entry_point_dfs(pd_entry_point_dfs): + cards_df = dd.from_pandas(pd_entry_point_dfs[0], npartitions=2) + transactions_df = dd.from_pandas(pd_entry_point_dfs[1], npartitions=2) + return cards_df, transactions_df + + +@pytest.fixture(params=['pd_entry_point_dfs', 'dask_entry_point_dfs']) +def entry_points_dfs(request): + return request.getfixturevalue(request.param) + + class MockEntryPoint(object): def on_call(self, kwargs): self.kwargs = kwargs @@ -53,7 +78,9 @@ def test_entry_point_error(es, monkeypatch): assert isinstance(entry_point.error, KeyError) -def test_entry_point_detect_arg(monkeypatch): +def test_entry_point_detect_arg(monkeypatch, entry_points_dfs): + cards_df = entry_points_dfs[0] + transactions_df = entry_points_dfs[1] cards_df = pd.DataFrame({"id": [1, 2, 3, 4, 5]}) transactions_df = pd.DataFrame({ "id": [1, 2, 3, 4, 5, 6], diff --git a/featuretools/tests/utils_tests/test_primitives_utils.py b/featuretools/tests/utils_tests/test_primitives_utils.py index ae4d5d1faf..02f8006fab 100644 --- a/featuretools/tests/utils_tests/test_primitives_utils.py +++ b/featuretools/tests/utils_tests/test_primitives_utils.py @@ -39,6 +39,7 @@ def test_list_primitives_order(): actual_desc = _get_descriptions([primitive])[0] if actual_desc: assert actual_desc == row['description'] + assert row['dask_compatible'] == primitive.dask_compatible types = df['type'].values assert 'aggregation' in types diff --git a/featuretools/utils/entity_utils.py b/featuretools/utils/entity_utils.py index 091d6c8c27..fd3e0103bf 100644 --- a/featuretools/utils/entity_utils.py +++ b/featuretools/utils/entity_utils.py @@ -1,5 +1,6 @@ from datetime import datetime +import dask.dataframe as dd import numpy as np import pandas as pd import pandas.api.types as pdtypes @@ -29,7 +30,10 @@ def infer_variable_types(df, link_vars, variable_types, time_index, secondary_ti for variable in df.columns: if variable in variable_types: continue - + elif isinstance(df, dd.DataFrame): + msg = 'Variable types cannot be inferred from Dask DataFrames, ' \ + 'use variable_types to provide type metadata for entity' + raise ValueError(msg) elif variable in vids_to_assume_datetime: if col_is_datetime(df[variable]): inferred_type = vtypes.Datetime @@ -128,23 +132,31 @@ def convert_all_variable_data(df, variable_types): def convert_variable_data(df, column_id, new_type, **kwargs): """Convert dataframe's variable to different type. """ - if df[column_id].empty: + empty = df[column_id].empty if isinstance(df, pd.DataFrame) else False + if empty: return df if new_type == vtypes.Numeric: - orig_nonnull = df[column_id].dropna().shape[0] - df[column_id] = pd.to_numeric(df[column_id], errors='coerce') - # This will convert strings to nans - # If column contained all strings, then we should - # just raise an error, because that shouldn't have - # been converted to numeric - nonnull = df[column_id].dropna().shape[0] - if nonnull == 0 and orig_nonnull != 0: - raise TypeError("Attempted to convert all string column {} to numeric".format(column_id)) + if isinstance(df, dd.DataFrame): + df[column_id] = dd.to_numeric(df[column_id], errors='coerce') + else: + orig_nonnull = df[column_id].dropna().shape[0] + df[column_id] = pd.to_numeric(df[column_id], errors='coerce') + # This will convert strings to nans + # If column contained all strings, then we should + # just raise an error, because that shouldn't have + # been converted to numeric + nonnull = df[column_id].dropna().shape[0] + if nonnull == 0 and orig_nonnull != 0: + raise TypeError("Attempted to convert all string column {} to numeric".format(column_id)) elif issubclass(new_type, vtypes.Datetime): format = kwargs.get("format", None) # TODO: if float convert to int? - df[column_id] = pd.to_datetime(df[column_id], format=format, - infer_datetime_format=True) + if isinstance(df, dd.DataFrame): + df[column_id] = dd.to_datetime(df[column_id], format=format, + infer_datetime_format=True) + else: + df[column_id] = pd.to_datetime(df[column_id], format=format, + infer_datetime_format=True) elif new_type == vtypes.Boolean: map_dict = {kwargs.get("true_val", True): True, kwargs.get("false_val", False): False, @@ -171,9 +183,10 @@ def get_linked_vars(entity): def col_is_datetime(col): - # check if dtype is datetime + # check if dtype is datetime - use .head() when getting first value + # in case column is a dask Series if (col.dtype.name.find('datetime') > -1 or - (len(col) and isinstance(col.iloc[0], datetime))): + (len(col) and isinstance(col.head(1).iloc[0], datetime))): return True # if it can be casted to numeric, it's not a datetime diff --git a/featuretools/utils/wrangle.py b/featuretools/utils/wrangle.py index fa785cd990..cc8c3bdce4 100644 --- a/featuretools/utils/wrangle.py +++ b/featuretools/utils/wrangle.py @@ -110,9 +110,11 @@ def _check_time_type(time): def _dataframes_equal(df1, df2): # ^ means XOR - if df1.empty ^ df2.empty: + df1_empty = bool(len(df1)) + df2_empty = bool(len(df2)) + if df1_empty ^ df2_empty: return False - elif not df1.empty and not df2.empty: + elif not df1_empty and not df2_empty: if not set(df1.columns) == set(df2.columns): return False diff --git a/requirements.txt b/requirements.txt index 48cc99ee67..1fe1fd20f1 100644 --- a/requirements.txt +++ b/requirements.txt @@ -5,6 +5,6 @@ tqdm>=4.32.0 pyyaml>=3.12 cloudpickle>=0.4.0 distributed>=1.24.2 -dask>=1.1.0 +dask[dataframe]>=1.1.0 psutil>=5.4.8 click>=7.0.0 diff --git a/test-requirements.txt b/test-requirements.txt index 2f227a701f..e636a26a2d 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -8,3 +8,4 @@ graphviz>=0.8.4 moto>=1.3.13 smart-open>=1.8.4 boto3>=1.10.45 +composeml>=0.2.0 \ No newline at end of file