From 98bf3aecb6ccace8bf820bc45f905f309ab823b3 Mon Sep 17 00:00:00 2001 From: Ben Rutter Date: Fri, 24 May 2024 16:30:41 +0100 Subject: [PATCH 1/4] catching single node configurations --- dask_databricks/databrickscluster.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dask_databricks/databrickscluster.py b/dask_databricks/databrickscluster.py index 3b064b8..8a9d5a7 100644 --- a/dask_databricks/databrickscluster.py +++ b/dask_databricks/databrickscluster.py @@ -29,6 +29,13 @@ def __init__( "Unable to find expected environment variable SPARK_LOCAL_IP. " "Are you running this on a Databricks driver node?" ) + if os.getenv("MASTER") and "local[" in os.getenv("MASTER"): + raise EnvironmentError( + "You appear to be running dask-databricks on a " + "single-node cluster. Dask requires at least one worker node " + "in order to function as expected." + + ) try: name = spark.conf.get("spark.databricks.clusterUsageTags.clusterId") except AttributeError: From e90ceee80a0c24aeded1abd56b907784b999c859 Mon Sep 17 00:00:00 2001 From: Ben Rutter Date: Fri, 24 May 2024 16:59:18 +0100 Subject: [PATCH 2/4] test added --- dask_databricks/tests/test_databricks.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dask_databricks/tests/test_databricks.py b/dask_databricks/tests/test_databricks.py index 62e56c8..c0e09ae 100644 --- a/dask_databricks/tests/test_databricks.py +++ b/dask_databricks/tests/test_databricks.py @@ -38,6 +38,14 @@ def test_databricks_cluster_raises_key_error_when_initialised_outside_of_databri with pytest.raises(KeyError): DatabricksCluster() +def test_databricks_cluster_raises_environment_error_when_master_variable_implies_single_node( + monkeypatch, + set_spark_local_ip, + dask_cluster, +): + monkeypatch.setenv("MASTER", "local[8]") + with pytest.raises(EnvironmentError): + DatabricksCluster() def test_databricks_cluster_create(set_spark_local_ip, dask_cluster): cluster = DatabricksCluster() From d284014cd86484ac9fa9142af824061b3ee5fc5a Mon Sep 17 00:00:00 2001 From: Ben Rutter Date: Mon, 27 May 2024 14:05:54 +0100 Subject: [PATCH 3/4] environ.get over getenv --- dask_databricks/databrickscluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dask_databricks/databrickscluster.py b/dask_databricks/databrickscluster.py index 8a9d5a7..3be2904 100644 --- a/dask_databricks/databrickscluster.py +++ b/dask_databricks/databrickscluster.py @@ -23,13 +23,13 @@ def __init__( loop: Optional[IOLoop] = None, asynchronous: bool = False, ): - self.spark_local_ip = os.getenv("SPARK_LOCAL_IP") + self.spark_local_ip = os.environ.get("SPARK_LOCAL_IP") if self.spark_local_ip is None: raise KeyError( "Unable to find expected environment variable SPARK_LOCAL_IP. " "Are you running this on a Databricks driver node?" ) - if os.getenv("MASTER") and "local[" in os.getenv("MASTER"): + if os.environ.get("MASTER") and "local[" in os.environ.get("MASTER"): raise EnvironmentError( "You appear to be running dask-databricks on a " "single-node cluster. Dask requires at least one worker node " From a627eedfe3a24e1f63c0b73ff628c59aa73ef4e1 Mon Sep 17 00:00:00 2001 From: Ben Rutter Date: Mon, 27 May 2024 14:07:45 +0100 Subject: [PATCH 4/4] updated error message --- dask_databricks/databrickscluster.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dask_databricks/databrickscluster.py b/dask_databricks/databrickscluster.py index 3be2904..f02a16c 100644 --- a/dask_databricks/databrickscluster.py +++ b/dask_databricks/databrickscluster.py @@ -31,9 +31,9 @@ def __init__( ) if os.environ.get("MASTER") and "local[" in os.environ.get("MASTER"): raise EnvironmentError( - "You appear to be running dask-databricks on a " - "single-node cluster. Dask requires at least one worker node " - "in order to function as expected." + "You appear to be trying to run a multi-node Dask cluster on a " + "single-node databricks cluster. Maybe you want " + "`dask.distributed.LocalCluster().get_client()` instead" ) try: