├── requirements.txt ├── dask_vertica ├── __init__.py ├── tests │ └── test_core.py └── core.py ├── MANIFEST.in ├── ci ├── environment-3.8.yaml └── environment-3.9.yaml ├── .github └── workflows │ ├── pre-commit.yml │ └── tests.yml ├── setup.cfg ├── .pre-commit-config.yaml ├── setup.py ├── LICENSE ├── README.md ├── .gitignore └── examples └── example.ipynb /requirements.txt: -------------------------------------------------------------------------------- 1 | dask 2 | distributed 3 | verticapy 4 | vertica-python 5 | -------------------------------------------------------------------------------- /dask_vertica/__init__.py: -------------------------------------------------------------------------------- 1 | from .core import read_vertica, to_vertica 2 | 3 | __version__ = "0.0.1" 4 | -------------------------------------------------------------------------------- /MANIFEST.in: -------------------------------------------------------------------------------- 1 | recursive-include dask_vertica *.py 2 | 3 | include setup.py 4 | include README.md 5 | include LICENSE 6 | include MANIFEST.in 7 | include requirements.txt 8 | -------------------------------------------------------------------------------- /ci/environment-3.8.yaml: -------------------------------------------------------------------------------- 1 | name: test-environment 2 | channels: 3 | - conda-forge 4 | dependencies: 5 | # Required 6 | - python=3.8 7 | - dask 8 | - distributed 9 | - pandas 10 | - pyarrow 11 | - vertica-python 12 | - verticapy 13 | # Testing 14 | - pytest 15 | -------------------------------------------------------------------------------- /ci/environment-3.9.yaml: -------------------------------------------------------------------------------- 1 | name: test-environment 2 | channels: 3 | - conda-forge 4 | dependencies: 5 | # Required 6 | - python=3.9 7 | - dask 8 | - distributed 9 | - pandas 10 | - pyarrow 11 | - vertica-python 12 | - verticapy 13 | # Testing 14 | - pytest 15 | -------------------------------------------------------------------------------- /.github/workflows/pre-commit.yml: -------------------------------------------------------------------------------- 1 | name: Linting 2 | 3 | on: 4 | push: 5 | branches: [ main ] 6 | pull_request: 7 | branches: [ main ] 8 | 9 | jobs: 10 | checks: 11 | name: "pre-commit hooks" 12 | runs-on: ubuntu-latest 13 | steps: 14 | - uses: actions/checkout@v2 15 | - uses: actions/setup-python@v2 16 | - uses: pre-commit/action@v2.0.0 17 | -------------------------------------------------------------------------------- /setup.cfg: -------------------------------------------------------------------------------- 1 | [flake8] 2 | exclude = __init__.py 3 | max-line-length = 120 4 | 5 | [isort] 6 | sections = FUTURE,STDLIB,THIRDPARTY,DASK,FIRSTPARTY,LOCALFOLDER 7 | profile = black 8 | skip_gitignore = true 9 | force_to_top = true 10 | default_section = THIRDPARTY 11 | known_first_party = dask_vertica 12 | known_dask = dask,distributed 13 | 14 | [tool:pytest] 15 | addopts = -v -rsxfE --durations=10 --color=yes 16 | -------------------------------------------------------------------------------- /.pre-commit-config.yaml: -------------------------------------------------------------------------------- 1 | repos: 2 | - repo: https://github.com/psf/black 3 | rev: 20.8b1 4 | hooks: 5 | - id: black 6 | language_version: python3 7 | exclude: versioneer.py 8 | - repo: https://github.com/keewis/blackdoc 9 | rev: v0.3.4 10 | hooks: 11 | - id: blackdoc 12 | - repo: https://gitlab.com/pycqa/flake8 13 | rev: 3.8.3 14 | hooks: 15 | - id: flake8 16 | language_version: python3 17 | - repo: https://github.com/pycqa/isort 18 | rev: 5.8.0 19 | hooks: 20 | - id: isort 21 | language_version: python3 22 | -------------------------------------------------------------------------------- /setup.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python 2 | 3 | from setuptools import setup 4 | 5 | setup( 6 | name="dask-vertica", 7 | version="0.0.1", 8 | description="Dask + Veertica intergration", 9 | license="BSD", 10 | maintainer="Paul Hobson", 11 | maintainer_email="paul@coiled.io", 12 | packages=["dask_vertica"], 13 | long_description=open("README.md").read(), 14 | long_description_content_type="text/markdown", 15 | python_requires=">=3.8", 16 | install_requires=open("requirements.txt").read().strip().split("\n"), 17 | include_package_data=True, 18 | zip_safe=False, 19 | ) 20 | -------------------------------------------------------------------------------- /.github/workflows/tests.yml: -------------------------------------------------------------------------------- 1 | name: Tests 2 | 3 | on: 4 | push: 5 | workflow_dispatch: 6 | 7 | jobs: 8 | test: 9 | runs-on: ${{ matrix.os }} 10 | defaults: 11 | run: 12 | shell: bash -l {0} 13 | strategy: 14 | fail-fast: false 15 | matrix: 16 | os: ["windows-latest", "ubuntu-latest", "macos-latest"] 17 | python-version: ["3.8", "3.9"] 18 | 19 | steps: 20 | - name: Checkout source 21 | uses: actions/checkout@v2 22 | 23 | - name: Setup Conda Environment 24 | uses: conda-incubator/setup-miniconda@v2 25 | with: 26 | miniforge-variant: Mambaforge 27 | miniforge-version: latest 28 | use-mamba: true 29 | channel-priority: strict 30 | python-version: ${{ matrix.python-version }} 31 | environment-file: ci/environment-${{ matrix.python-version }}.yaml 32 | activate-environment: test-environment 33 | auto-activate-base: false 34 | 35 | - name: Install dask_vertica 36 | run: python -m pip install -e . 37 | 38 | - name: Run tests 39 | env: 40 | VERTICA_USER: ${{ secrets.VERTICA_USER }} 41 | VERTICA_PASSWORD: ${{ secrets.VERTICA_PASSWORD }} 42 | VERTICA_PORT: ${{ secrets.VERTICA_PORT }} 43 | VERTICA_DB: ${{ secrets.VERTICA_DB }} 44 | VERTICA_HOST: ${{ secrets.VERTICA_HOST }} 45 | VERTICA_SCHEMA: ${{ secrets.VERTICA_SCHEMA }} 46 | run: pytest dask_vertica 47 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | BSD 3-Clause License 2 | 3 | Copyright (c) 2022, Coiled 4 | All rights reserved. 5 | 6 | Redistribution and use in source and binary forms, with or without 7 | modification, are permitted provided that the following conditions are met: 8 | 9 | 1. Redistributions of source code must retain the above copyright notice, this 10 | list of conditions and the following disclaimer. 11 | 12 | 2. Redistributions in binary form must reproduce the above copyright notice, 13 | this list of conditions and the following disclaimer in the documentation 14 | and/or other materials provided with the distribution. 15 | 16 | 3. Neither the name of the copyright holder nor the names of its 17 | contributors may be used to endorse or promote products derived from 18 | this software without specific prior written permission. 19 | 20 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 21 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 22 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 23 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 24 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 25 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 26 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 27 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 28 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Dask-vertica 2 | 3 | [![Tests](https://github.com/coiled/dask-vertica/actions/workflows/tests.yml/badge.svg)](https://github.com/coiled/dask-vertica/actions/workflows/tests.yml) 4 | [![Linting](https://github.com/coiled/dask-vertica/actions/workflows/pre-commit.yml/badge.svg)](https://github.com/coiled/dask-vertica/actions/workflows/pre-commit.yml) 5 | 6 | 7 | This connector is in an early experimental/testing phase. 8 | 9 | [Reach out to us](https://coiled.io/contact-us/) if you are interested in trying 10 | it out! 11 | 12 | ## Installation 13 | 14 | `dask-vertica` can be installed from source: 15 | 16 | ```zsh 17 | git clone git@github.com:coiled/dask-vertica.git 18 | cd dask-vertica 19 | pip install -e . 20 | ``` 21 | 22 | 35 | 36 | ## Usage 37 | 38 | `dask-vertica` provides `read_vertica` and `to_vertica` methods 39 | for parallel IO from vertica with Dask. 40 | 41 | ```python 42 | >>> from dask_vertica import read_vertica 43 | >>> example_query = ''' 44 | ... SELECT * 45 | ... FROM VERTICA_SAMPLE_DATA.TPCH_SF1.CUSTOMER; 46 | ... ''' 47 | >>> ddf = read_vertica( 48 | ... query=example_query, 49 | ... connection_kwargs={ 50 | ... "user": "...", 51 | ... "password": "...", 52 | ... ... 53 | ... }, 54 | ... ) 55 | ``` 56 | 57 | `read_vertica` will return a dask dataframe, so data will not be fetched until `.compute()` or `.persist()` is called. 58 | This means that you can lazily perform calculations and upload the results at the same time. 59 | Note that `to_vertica` calls `.compute()` for you, so it doesn't appear in the example below. 60 | 61 | ```python 62 | >>> from dask_vertica import to_vertica 63 | >>> means = ddf.groupby("name")["y"].mean() # lazy, no calc performed 64 | >>> to_vertica( 65 | ... means, 66 | ... name="my_table", 67 | ... connection_kwargs={ 68 | ... "user": "...", 69 | ... "password": "...", 70 | ... ..., 71 | ... }, 72 | ... ) # computes means and uploads to DB 73 | ``` 74 | 75 | See their docstrings for further API information. 76 | 77 | ## License 78 | 79 | [BSD-3](LICENSE) 80 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | dask-worker-space/ 2 | 3 | # Byte-compiled / optimized / DLL files 4 | __pycache__/ 5 | *.py[cod] 6 | *$py.class 7 | 8 | # C extensions 9 | *.so 10 | 11 | # Distribution / packaging 12 | .Python 13 | build/ 14 | develop-eggs/ 15 | dist/ 16 | downloads/ 17 | eggs/ 18 | .eggs/ 19 | lib/ 20 | lib64/ 21 | parts/ 22 | sdist/ 23 | var/ 24 | wheels/ 25 | pip-wheel-metadata/ 26 | share/python-wheels/ 27 | *.egg-info/ 28 | .installed.cfg 29 | *.egg 30 | MANIFEST 31 | 32 | # PyInstaller 33 | # Usually these files are written by a python script from a template 34 | # before PyInstaller builds the exe, so as to inject date/other infos into it. 35 | *.manifest 36 | *.spec 37 | 38 | # Installer logs 39 | pip-log.txt 40 | pip-delete-this-directory.txt 41 | 42 | # Unit test / coverage reports 43 | htmlcov/ 44 | .tox/ 45 | .nox/ 46 | .coverage 47 | .coverage.* 48 | .cache 49 | nosetests.xml 50 | coverage.xml 51 | *.cover 52 | *.py,cover 53 | .hypothesis/ 54 | .pytest_cache/ 55 | 56 | # Translations 57 | *.mo 58 | *.pot 59 | 60 | # Django stuff: 61 | *.log 62 | local_settings.py 63 | db.sqlite3 64 | db.sqlite3-journal 65 | 66 | # Flask stuff: 67 | instance/ 68 | .webassets-cache 69 | 70 | # Scrapy stuff: 71 | .scrapy 72 | 73 | # Sphinx documentation 74 | docs/_build/ 75 | 76 | # PyBuilder 77 | target/ 78 | 79 | # Jupyter Notebook 80 | .ipynb_checkpoints 81 | 82 | # IPython 83 | profile_default/ 84 | ipython_config.py 85 | 86 | # pyenv 87 | .python-version 88 | 89 | # pipenv 90 | # According to pypa/pipenv#598, it is recommended to include Pipfile.lock in version control. 91 | # However, in case of collaboration, if having platform-specific dependencies or dependencies 92 | # having no cross-platform support, pipenv may install dependencies that don't work, or not 93 | # install all needed dependencies. 94 | #Pipfile.lock 95 | 96 | # PEP 582; used by e.g. github.com/David-OConnor/pyflow 97 | __pypackages__/ 98 | 99 | # Celery stuff 100 | celerybeat-schedule 101 | celerybeat.pid 102 | 103 | # SageMath parsed files 104 | *.sage.py 105 | 106 | # Environments 107 | .env 108 | .venv 109 | env/ 110 | venv/ 111 | ENV/ 112 | env.bak/ 113 | venv.bak/ 114 | 115 | # Spyder project settings 116 | .spyderproject 117 | .spyproject 118 | 119 | # Rope project settings 120 | .ropeproject 121 | 122 | # mkdocs documentation 123 | /site 124 | 125 | # mypy 126 | .mypy_cache/ 127 | .dmypy.json 128 | dmypy.json 129 | 130 | # Pyre type checker 131 | .pyre/ 132 | 133 | # OS Files 134 | .DS_Store 135 | verticapy-connect-*.py 136 | .vscode 137 | -------------------------------------------------------------------------------- /dask_vertica/tests/test_core.py: -------------------------------------------------------------------------------- 1 | import os 2 | import sys 3 | 4 | import pandas as pd 5 | import pytest 6 | from vertica_python import connect 7 | 8 | import dask.dataframe as dd 9 | from distributed import Client 10 | 11 | from dask_vertica import read_vertica, to_vertica 12 | from dask_vertica.core import _drop_table 13 | 14 | 15 | @pytest.fixture 16 | def client(): 17 | with Client(n_workers=2, threads_per_worker=10) as client: 18 | yield client 19 | 20 | 21 | @pytest.fixture 22 | def schema(): 23 | return os.environ["VERTICA_SCHEMA"] 24 | 25 | 26 | @pytest.fixture 27 | def platform(): 28 | return f"{sys.platform}{sys.version_info.major}{sys.version_info.minor}" 29 | 30 | 31 | @pytest.fixture(scope="module") 32 | def connection_kwargs(): 33 | return dict( 34 | host=os.environ["VERTICA_HOST"], 35 | port=os.environ["VERTICA_PORT"], 36 | user=os.environ["VERTICA_USER"], 37 | password=os.environ["VERTICA_PASSWORD"], 38 | database=os.environ["VERTICA_DB"], 39 | connection_load_balance=True, 40 | session_label="py", 41 | unicode_error="strict", 42 | ) 43 | 44 | 45 | @pytest.fixture 46 | def small_df(): 47 | df = pd.DataFrame( 48 | { 49 | "name": ["Alice", "Bob", "Charlene", "Dale", "Emily"], 50 | "id": [1, 2, 3, 4, 5], 51 | "height": [178.5, 180, 160, 185.5, 170], 52 | } 53 | ) 54 | return dd.from_pandas(df, npartitions=1) 55 | 56 | 57 | @pytest.fixture 58 | def demo_ts(): 59 | return dd.demo.make_timeseries( 60 | start="2000-01-01", end="2000-12-31", freq="30s", partition_freq="1W" 61 | ).reset_index() 62 | 63 | 64 | @pytest.fixture 65 | def remove_test_tables(connection_kwargs, schema, platform): 66 | with connect(**connection_kwargs) as connection: 67 | _drop_table(connection, f"testing_small_df_{platform}", schema=schema) 68 | _drop_table(connection, f"testing_if_exists_df_{platform}", schema=schema) 69 | _drop_table(connection, f"testing_if_exists_insert_{platform}", schema=schema) 70 | 71 | yield 72 | 73 | 74 | def test_write_read_roundtrip( 75 | remove_test_tables, small_df, connection_kwargs, client, schema, platform 76 | ): 77 | table_name = f"testing_small_df_{platform}" 78 | to_vertica(small_df, connection_kwargs, name=table_name, schema=schema) 79 | 80 | ddf_out = read_vertica(connection_kwargs, table_name, npartitions=1, schema=schema) 81 | 82 | result = ddf_out.sort_values(by="id") 83 | dd.utils.assert_eq(small_df, result, check_index=False, check_divisions=False) 84 | 85 | 86 | def test_write_if_exists_error( 87 | small_df, connection_kwargs, client, schema, remove_test_tables, platform 88 | ): 89 | table_name = f"testing_if_exists_df_{platform}" 90 | to_vertica(small_df, connection_kwargs, name=table_name, schema=schema) 91 | 92 | with pytest.raises(RuntimeError): 93 | to_vertica( 94 | small_df, 95 | connection_kwargs, 96 | name=table_name, 97 | schema=schema, 98 | if_exists="error", 99 | ) 100 | 101 | df_out = read_vertica( 102 | connection_kwargs, table_name, npartitions=2, schema=schema 103 | ).compute() 104 | assert df_out.shape[0] == 5 105 | 106 | 107 | def test_write_if_exists_overwrite( 108 | small_df, connection_kwargs, client, schema, remove_test_tables, platform 109 | ): 110 | 111 | table_name = f"testing_if_exists_df_{platform}" 112 | to_vertica( 113 | small_df, 114 | connection_kwargs, 115 | name=table_name, 116 | schema=schema, 117 | if_exists="overwrite", 118 | ) 119 | to_vertica( 120 | small_df, 121 | connection_kwargs, 122 | name=table_name, 123 | schema=schema, 124 | if_exists="overwrite", 125 | ) 126 | 127 | df_out = read_vertica( 128 | connection_kwargs, table_name, npartitions=2, schema=schema 129 | ).compute() 130 | assert df_out.shape[0] == 5 131 | 132 | 133 | def test_write_if_exists_insert( 134 | small_df, connection_kwargs, client, schema, remove_test_tables, platform 135 | ): 136 | table_name = f"testing_if_exists_insert_{platform}" 137 | to_vertica( 138 | small_df, 139 | connection_kwargs, 140 | name=table_name, 141 | schema=schema, 142 | if_exists="overwrite", 143 | ) 144 | 145 | to_vertica( 146 | small_df, 147 | connection_kwargs, 148 | name=table_name, 149 | schema=schema, 150 | if_exists="insert", 151 | ) 152 | 153 | df_out = read_vertica( 154 | connection_kwargs, table_name, npartitions=2, schema=schema 155 | ).compute() 156 | assert df_out.shape[0] == 10 157 | -------------------------------------------------------------------------------- /examples/example.ipynb: -------------------------------------------------------------------------------- 1 | { 2 | "cells": [ 3 | { 4 | "cell_type": "markdown", 5 | "id": "8b1ccea9-a90d-4aa1-9df7-0dd5cbd4cfa5", 6 | "metadata": {}, 7 | "source": [ 8 | "# Integrating Vertica with Dask\n", 9 | "## Imports and basic setup" 10 | ] 11 | }, 12 | { 13 | "cell_type": "code", 14 | "execution_count": null, 15 | "id": "b779cbc2-b55e-4199-b0db-5e0bccef18a8", 16 | "metadata": {}, 17 | "outputs": [], 18 | "source": [ 19 | "import numpy as np\n", 20 | "import pandas as pd\n", 21 | "import dask\n", 22 | "import dask.dataframe as dd\n", 23 | "from distributed.client import Client\n", 24 | "\n", 25 | "import vertica_python\n", 26 | "from verticapy.vdataframe import vDataFrame\n", 27 | "from dask_vertica import (\n", 28 | " to_vertica,\n", 29 | " read_vertica,\n", 30 | ")\n", 31 | "\n", 32 | "\n", 33 | "schema = \"coiled_schema\"\n", 34 | "\n", 35 | "vdb = {\n", 36 | " 'host': 'aaa.bb.cc.ddd',\n", 37 | " 'port': 5433,\n", 38 | " 'user': '',\n", 39 | " 'password': '',\n", 40 | " 'database': '',\n", 41 | " 'connection_load_balance': True,\n", 42 | " 'session_label': 'py',\n", 43 | " 'unicode_error': 'strict'\n", 44 | "}\n", 45 | "client = Client(n_workers=4)\n", 46 | "client" 47 | ] 48 | }, 49 | { 50 | "cell_type": "markdown", 51 | "id": "f4fcdadc-6ccf-4ebb-bdec-05bcd24d0157", 52 | "metadata": {}, 53 | "source": [ 54 | "## Create a large(-ish) time series dataframe \n", 55 | "(1 year of 30-sec data)" 56 | ] 57 | }, 58 | { 59 | "cell_type": "code", 60 | "execution_count": null, 61 | "id": "76c2d784-d75c-4029-beaf-595ef313668f", 62 | "metadata": {}, 63 | "outputs": [], 64 | "source": [ 65 | "# larger DF with more partitions\n", 66 | "demo_ts = dd.demo.make_timeseries(\n", 67 | " start=\"2000-01-01\",\n", 68 | " end=\"2000-12-31\",\n", 69 | " freq=\"30s\",\n", 70 | " partition_freq=\"1W\"\n", 71 | ").reset_index() # moves the date from the index to a column\n", 72 | "\n", 73 | "demo_ts.head()" 74 | ] 75 | }, 76 | { 77 | "cell_type": "markdown", 78 | "id": "a5cb83d4-1d98-4131-8d11-d5aae7d00d32", 79 | "metadata": {}, 80 | "source": [ 81 | "## Upload the dataframe to a Vertica database\n", 82 | "(this will overwrite any existing table of the same name)" 83 | ] 84 | }, 85 | { 86 | "cell_type": "code", 87 | "execution_count": null, 88 | "id": "4a0c1895-f26e-4ced-9bb2-696881479c08", 89 | "metadata": { 90 | "tags": [] 91 | }, 92 | "outputs": [], 93 | "source": [ 94 | "to_vertica(demo_ts, vdb, \"demo_ts\", schema=schema, if_exists=\"overwrite\")" 95 | ] 96 | }, 97 | { 98 | "cell_type": "markdown", 99 | "id": "77d32fad-8fc5-40c0-aae0-3d868ea186ef", 100 | "metadata": {}, 101 | "source": [ 102 | "## Try to upload load again, this time with `if_exists = \"error\"` to prevent overwriting" 103 | ] 104 | }, 105 | { 106 | "cell_type": "code", 107 | "execution_count": null, 108 | "id": "89ff1f39-51b9-4bd7-bf69-b465864377ef", 109 | "metadata": {}, 110 | "outputs": [], 111 | "source": [ 112 | "try:\n", 113 | " to_vertica(demo_ts, vdb, \"demo_ts\", schema=schema, if_exists=\"error\")\n", 114 | "except RuntimeError:\n", 115 | " print(\"Sucessfully avoided disaster\")" 116 | ] 117 | }, 118 | { 119 | "cell_type": "markdown", 120 | "id": "d3612832-c1e0-4c7f-a5ee-56958073ea08", 121 | "metadata": {}, 122 | "source": [ 123 | "## Lazily compute monthly means for each name and upload to DB" 124 | ] 125 | }, 126 | { 127 | "cell_type": "code", 128 | "execution_count": null, 129 | "id": "1682e2c0-7456-42c2-9b39-6c336f87dae7", 130 | "metadata": {}, 131 | "outputs": [], 132 | "source": [ 133 | "means = (\n", 134 | " read_vertica(vdb, name=\"demo_ts\", npartitions=50, schema=schema)\n", 135 | " .assign(month=lambda df: df[\"timestamp\"].dt.month)\n", 136 | " .groupby([\"name\", \"month\"])[\"y\"]\n", 137 | " .mean()\n", 138 | " .reset_index()\n", 139 | ")\n", 140 | "print(type(means))\n", 141 | "to_vertica(means, vdb, \"monthly_means\", schema=schema, if_exists=\"overwrite\")" 142 | ] 143 | }, 144 | { 145 | "cell_type": "markdown", 146 | "id": "2593efaa-e53f-488e-9dae-644ef107ac52", 147 | "metadata": {}, 148 | "source": [ 149 | "## Now fetch the table we uploaded as a dataframe and do some calculations" 150 | ] 151 | }, 152 | { 153 | "cell_type": "code", 154 | "execution_count": null, 155 | "id": "fa3fc0be-1f73-4007-bee2-c71998898c6f", 156 | "metadata": { 157 | "tags": [] 158 | }, 159 | "outputs": [], 160 | "source": [ 161 | "demo_means = read_vertica(vdb, \"monthly_means\", 5, schema=schema)\n", 162 | "demo_means # dask series, uncomputed" 163 | ] 164 | }, 165 | { 166 | "cell_type": "markdown", 167 | "id": "b10b6288-be2d-4536-920e-9dd485b2fcc8", 168 | "metadata": {}, 169 | "source": [ 170 | "### Finally call `compute` on the retrieved data" 171 | ] 172 | }, 173 | { 174 | "cell_type": "code", 175 | "execution_count": null, 176 | "id": "c022ffae-3d2a-448a-8cb8-d515cc334cdd", 177 | "metadata": {}, 178 | "outputs": [], 179 | "source": [ 180 | "demo_means.compute()" 181 | ] 182 | } 183 | ], 184 | "metadata": { 185 | "kernelspec": { 186 | "display_name": "Python 3 (ipykernel)", 187 | "language": "python", 188 | "name": "python3" 189 | }, 190 | "language_info": { 191 | "codemirror_mode": { 192 | "name": "ipython", 193 | "version": 3 194 | }, 195 | "file_extension": ".py", 196 | "mimetype": "text/x-python", 197 | "name": "python", 198 | "nbconvert_exporter": "python", 199 | "pygments_lexer": "ipython3", 200 | "version": "3.9.10" 201 | } 202 | }, 203 | "nbformat": 4, 204 | "nbformat_minor": 5 205 | } 206 | -------------------------------------------------------------------------------- /dask_vertica/core.py: -------------------------------------------------------------------------------- 1 | import logging 2 | import uuid 3 | from functools import partial 4 | from typing import Dict, Optional, Union 5 | 6 | import numpy as np 7 | import pandas as pd 8 | from vertica_python import connect 9 | from vertica_python.vertica.connection import Connection 10 | from verticapy.utilities import drop, pandas_to_vertica, readSQL 11 | from verticapy.vdataframe import vDataFrame 12 | 13 | import dask 14 | import dask.dataframe as dd 15 | from dask.base import tokenize 16 | from dask.delayed import delayed 17 | from dask.utils import SerializableLock 18 | 19 | 20 | @delayed 21 | def daskdf_to_vertica( 22 | df: dd.DataFrame, 23 | connection_kwargs: Dict, 24 | name: str, 25 | schema: str = "public", 26 | relation_type: str = "insert", 27 | parse_n_lines: Optional[int] = None, 28 | ) -> None: 29 | """Upload a dask dataframe to Vertica 30 | 31 | Parameters 32 | ---------- 33 | df : dask.DataFrame 34 | connection_kwargs : dict 35 | Connection arguments used when connecting to Vertica with 36 | ``vertica_python.vertica.connection.Connection``. 37 | name : str 38 | Name of the table into which the data will be inserted. 39 | schema : str (default = "public") 40 | Schema in which the table exists 41 | relation_type : str (default = "insert") 42 | How the dataframe should be uploaded. If the table does not yet 43 | exist, pass "table". If the data should be inserted into an 44 | existing table, pass "insert" 45 | parse_n_lines : optional int 46 | This parameter is passed directly to 47 | ``verticapy.utilities.pandas_to_vertica``. 48 | 49 | """ 50 | 51 | if not parse_n_lines: 52 | parse_n_lines = df.shape[0] 53 | 54 | already_exists = _check_if_exists(connection_kwargs, name, schema=schema) 55 | 56 | logging.debug(f"daskdf_to_vertica: {schema=} | {name=}") 57 | logging.debug(f"daskdf_to_vertica: {already_exists=}") 58 | logging.debug(f"daskdf_to_vertica: {relation_type=}") 59 | 60 | with SerializableLock(token="daskdf_to_vertica"): 61 | with connect(**connection_kwargs) as connection: 62 | with connection.cursor() as cursor: 63 | tmp = f"tmp_{name}_{uuid.uuid4().hex}" 64 | 65 | logging.debug(f"daskdf_to_vertica: converting to vdf with {tmp=}") 66 | 67 | vdf = pandas_to_vertica( 68 | df, 69 | cursor=cursor, 70 | name=tmp, 71 | schema=schema, 72 | parse_n_lines=parse_n_lines, 73 | ) 74 | vdf.to_db( 75 | f'"{schema}"."{name}"', 76 | df.columns.tolist(), 77 | relation_type=relation_type, 78 | inplace=True, 79 | ) 80 | 81 | 82 | def _check_if_exists( 83 | connection_kwargs: Dict, name: str, schema: str = "public" 84 | ) -> bool: 85 | """Quick check if a Vertica table already exists in a schema 86 | 87 | Parameters 88 | ---------- 89 | connection_kwargs : dict 90 | Connection arguments used when connecting to Vertica with 91 | ``vertica_python.vertica.connection.Connection``. 92 | name : str 93 | Name of the table to check. 94 | schema : str (default = "public") 95 | Schema in which to check for the table 96 | 97 | Returns 98 | ------- 99 | exists : bool 100 | 101 | """ 102 | 103 | with connect(**connection_kwargs) as connection: 104 | with connection.cursor() as cur: 105 | table_query = f""" 106 | SELECT TABLE_SCHEMA, TABLE_NAME FROM V_CATALOG.TABLES 107 | WHERE TABLE_SCHEMA = '{schema}' 108 | """ 109 | 110 | all_vtables = readSQL(table_query, cursor=cur) 111 | does_exist = name in all_vtables["TABLE_NAME"] 112 | 113 | logging.debug(f"_check_if_exists: {all_vtables=}") 114 | logging.debug(f"_check_if_exists: {schema=} | {name=}") 115 | 116 | logging.debug(f"{(schema, name, does_exist)}") 117 | return does_exist 118 | 119 | 120 | def _drop_table(connection: Connection, name: str, schema: str = "public") -> None: 121 | """Little helper to drop a Vertica table with common parameters""" 122 | 123 | logging.debug(f"_drop_table: {schema=} | {name=}") 124 | with connection.cursor() as cur: 125 | drop(name=f'"{schema}"."{name}"', cursor=cur) 126 | 127 | 128 | def _validate_daskdf(df: Union[vDataFrame, pd.DataFrame, dd.DataFrame]) -> dd.DataFrame: 129 | if isinstance(df, vDataFrame): 130 | df = df.to_pandas() 131 | 132 | if isinstance(df, pd.DataFrame): 133 | df = dd.from_pandas(df) 134 | 135 | return df 136 | 137 | 138 | def to_vertica( 139 | df: Union[vDataFrame, pd.DataFrame, dd.DataFrame], 140 | connection_kwargs: Dict, 141 | name: str, 142 | schema: str = "public", 143 | if_exists: str = "error", 144 | ): 145 | """Write a Dask DataFrame to a Vertica table. 146 | 147 | Parameters 148 | ---------- 149 | df : dask.DataFrame 150 | Dask DataFrame to save. 151 | connection_kwargs : dict 152 | Connection arguments used when connecting to Vertica with 153 | ``vertica_python.vertica.connection.Connection``. 154 | name : str 155 | Name of the table to save to. 156 | schema : str (default = "public") 157 | Schema in which the table exists 158 | if_exists : str (default = "error") 159 | How dask_vertica should handle attempts to write to an existing 160 | table. Options are: 161 | 162 | - "error" (default): raise a Runtime error if the table exists 163 | - "overwrite": drop the existing table and start fresh 164 | - "insert": keep the existing able and insert/append new rows 165 | 166 | Examples 167 | -------- 168 | 169 | >>> from dask_vertica import to_vertica 170 | >>> df = ... # Create a Dask DataFrame 171 | >>> to_vertica( 172 | ... df, 173 | ... connection_kwargs={ 174 | ... "host": "...", 175 | ... "port": 5433, 176 | ... "database": "...", 177 | ... "user": "...", 178 | ... "password": "...", 179 | ... }, 180 | ... name="my_table", 181 | ... schema="project_schema", 182 | ... if_exists="overwrite", 183 | ... ) 184 | 185 | """ 186 | 187 | df = _validate_daskdf(df) 188 | 189 | if_exists = if_exists.lower() 190 | table_already_exists = _check_if_exists(connection_kwargs, name, schema=schema) 191 | 192 | logging.debug( 193 | f"to_vertica: {schema=} | {name=} | {if_exists=} | {table_already_exists=} ({df.npartitions=})" 194 | ) 195 | 196 | if table_already_exists: 197 | if if_exists == "error": 198 | raise RuntimeError(f"Table {name} already exists in {schema}") 199 | elif if_exists == "overwrite": 200 | with connect(**connection_kwargs) as connection: 201 | _drop_table(connection, name, schema=schema) 202 | 203 | first_relation = "insert" if if_exists in ("append", "insert") else "table" 204 | 205 | futures = [] 206 | for n, partition in enumerate(df.to_delayed()): 207 | f = daskdf_to_vertica( 208 | partition, 209 | connection_kwargs, 210 | name, 211 | relation_type=first_relation if n == 0 else "insert", 212 | schema=schema, 213 | ) 214 | 215 | # HACK: need to get the first partition computed to create the table 216 | # (empty tables -- i.e., df._meta -- raise errors) 217 | if n == 0: 218 | dask.compute(f) 219 | else: 220 | futures.append(f) 221 | 222 | dask.compute(futures) 223 | return None 224 | 225 | 226 | @delayed 227 | def _fetch_vdf_batch( 228 | connection_kwargs: Dict, 229 | name: str, 230 | partition_size: int, 231 | offset: int, 232 | schema: str = "public", 233 | ): 234 | with connect(**connection_kwargs) as connection: 235 | with connection.cursor() as cur: 236 | vdf = vDataFrame(name, cur, schema=schema).iloc( 237 | limit=partition_size, offset=offset 238 | ) 239 | 240 | return vdf.to_pandas() 241 | 242 | 243 | def read_vertica( 244 | connection_kwargs: Dict, 245 | name: str, 246 | npartitions: int, 247 | schema: str = "public", 248 | ): 249 | """Fetch data from a Vertica table 250 | 251 | Parameters 252 | ---------- 253 | connection_kwargs : dict 254 | Connection arguments used when connecting to Vertica with 255 | ``vertica_python.vertica.connection.Connection``. 256 | name : str 257 | Name of the table to check. 258 | npartitions : int 259 | The number of partitions of the resulting dask dataframe 260 | schema : str (default = "public") 261 | Schema in which to check for the table 262 | 263 | Returns 264 | ------- 265 | ddf : dd.DataFrame 266 | 267 | Examples 268 | -------- 269 | 270 | >>> from dask_vertica import read_vertica 271 | >>> read_vertica( 272 | ... df, 273 | ... connection_kwargs={ 274 | ... "host": "...", 275 | ... "port": 5433, 276 | ... "database": "...", 277 | ... "user": "...", 278 | ... "password": "...", 279 | ... }, 280 | ... "my_table", 281 | ... 52, 282 | ... schema="project_schema", 283 | ... ) 284 | 285 | """ 286 | 287 | label = "read-vertica-" 288 | output_name = label + tokenize(connection_kwargs, name, npartitions, schema) 289 | 290 | with connect(**connection_kwargs) as connection: 291 | with connection.cursor() as cur: 292 | vdf = vDataFrame(name, cur, schema=schema) 293 | n_rows = vdf.shape()[0] 294 | meta = vdf.head(1).to_pandas().drop(0) 295 | 296 | partitionsize = n_rows // npartitions 297 | 298 | _fetcher = partial( 299 | _fetch_vdf_batch, connection_kwargs, name, int(partitionsize), schema=schema 300 | ) 301 | offsets = np.arange(0, n_rows, partitionsize, dtype=int) 302 | batches = [_fetcher(int(offset)) for offset in offsets] 303 | 304 | if not batches: 305 | divisions = (None, None) 306 | else: 307 | divisions = tuple([None] * (len(batches) + 1)) 308 | 309 | dd_obj = dd.from_delayed( 310 | batches, meta=meta, divisions=divisions, verify_meta=True, prefix=output_name 311 | ) 312 | return dd_obj 313 | --------------------------------------------------------------------------------