lu-wang-dl commented on code in PR #39146:
URL: https://github.com/apache/spark/pull/39146#discussion_r1061233548


##########
python/pyspark/ml/torch/distributor.py:
##########
@@ -0,0 +1,287 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import math
+from typing import Union, Callable, Optional, Any
+import warnings
+
+from pyspark.sql import SparkSession
+from pyspark.context import SparkContext
+
+
+# Moved the util functions to this file for now
+# TODO(SPARK-41589): will move the functions and tests to an external file
+#       once we are in agreement about which functions should be in utils.py
+def get_conf_boolean(sc: SparkContext, key: str, default_value: str) -> bool:
+    """Get the conf "key" from the given spark context,
+    or return the default value if the conf is not set.
+    This expects the conf value to be a boolean or string;
+    if the value is a string, this checks for all capitalization
+    patterns of "true" and "false" to match Scala.
+
+    Parameters
+    ----------
+    sc : SparkContext
+        The SparkContext for the distributor.
+    key : str
+        string for conf name
+    default_value : str
+        default value for the conf value for the given key
+
+    Returns
+    -------
+    bool
+        Returns the boolean value that corresponds to the conf
+
+    Raises
+    ------
+    Exception
+        Thrown when the conf value is not a boolean
+    """
+    val = sc.getConf().get(key, default_value)
+    lowercase_val = val.lower()
+    if lowercase_val == "true":
+        return True
+    if lowercase_val == "false":
+        return False
+    raise Exception(
+        "get_conf_boolean expected a boolean conf "
+        "value but found value of type {} "
+        "with value: {}".format(type(val), val)
+    )
+
+
+class Distributor:

Review Comment:
   Add a doc string for this class?



##########
python/pyspark/ml/torch/distributor.py:
##########
@@ -0,0 +1,287 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import math
+from typing import Union, Callable, Optional, Any
+import warnings
+
+from pyspark.sql import SparkSession
+from pyspark.context import SparkContext
+
+
+# Moved the util functions to this file for now
+# TODO(SPARK-41589): will move the functions and tests to an external file
+#       once we are in agreement about which functions should be in utils.py
+def get_conf_boolean(sc: SparkContext, key: str, default_value: str) -> bool:
+    """Get the conf "key" from the given spark context,
+    or return the default value if the conf is not set.
+    This expects the conf value to be a boolean or string;
+    if the value is a string, this checks for all capitalization
+    patterns of "true" and "false" to match Scala.
+
+    Parameters
+    ----------
+    sc : SparkContext
+        The SparkContext for the distributor.
+    key : str
+        string for conf name
+    default_value : str
+        default value for the conf value for the given key
+
+    Returns
+    -------
+    bool
+        Returns the boolean value that corresponds to the conf
+
+    Raises
+    ------
+    Exception
+        Thrown when the conf value is not a boolean
+    """
+    val = sc.getConf().get(key, default_value)
+    lowercase_val = val.lower()
+    if lowercase_val == "true":
+        return True
+    if lowercase_val == "false":
+        return False
+    raise Exception(
+        "get_conf_boolean expected a boolean conf "
+        "value but found value of type {} "
+        "with value: {}".format(type(val), val)
+    )
+
+
+class Distributor:
+    def __init__(
+        self,
+        num_processes: int = 1,
+        local_mode: bool = True,
+        use_gpu: bool = True,
+        spark: Optional[SparkSession] = None,
+    ):
+        self.num_processes = num_processes
+        self.local_mode = local_mode
+        self.use_gpu = use_gpu
+        if spark:
+            self.spark = spark
+        else:
+            self.spark = SparkSession.builder.getOrCreate()
+        self.sc = self.spark.sparkContext
+        self.num_tasks = self._get_num_tasks()
+        self.ssl_conf = None
+
+    def _get_num_tasks(self) -> int:
+        """
+        Returns the number of Spark tasks to use for distributed training
+
+        Returns
+        -------
+            The number of Spark tasks to use for distributed training
+        """
+        if self.use_gpu:
+            key = "spark.task.resource.gpu.amount"
+            if self.sc.getConf().contains(key):
+                if gpu_amount_raw := self.sc.getConf().get(key):  # mypy 
error??
+                    task_gpu_amount = int(gpu_amount_raw)
+            else:
+                task_gpu_amount = 1  # for single node clusters
+            if task_gpu_amount < 1:
+                raise ValueError(
+                    f"The Spark conf `{key}` has a value "
+                    f"of {task_gpu_amount} but it "
+                    "should not have a value less than 1."
+                )
+            return math.ceil(self.num_processes / task_gpu_amount)
+        return self.num_processes
+
+    def _validate_input_params(self) -> None:
+        if self.num_processes <= 0:
+            raise ValueError("num_proccesses has to be a positive integer")
+
+    def _check_encryption(self) -> None:
+        """Checks to see if the user requires encrpytion of data.
+        If required, throw an exception since we don't support that.
+
+        Raises
+        ------
+        NotImplementedError
+            Thrown when the user doesn't use PyTorchDistributor
+        Exception
+            Thrown when the user requires ssl encryption
+        """
+        if not "ssl_conf":
+            raise Exception(
+                "Distributor doesn't have this functionality. Use 
PyTorchDistributor instead."
+            )
+        is_ssl_enabled = get_conf_boolean(self.sc, "spark.ssl.enabled", 
"false")
+        ignore_ssl = get_conf_boolean(self.sc, self.ssl_conf, "false")  # 
type: ignore
+        if is_ssl_enabled:
+            name = self.__class__.__name__
+            if ignore_ssl:
+                warnings.warn(
+                    f"""
+                    This cluster has TLS encryption enabled;
+                    however, {name} does not
+                    support data encryption in transit.
+                    The Spark configuration
+                    '{self.ssl_conf}' has been set to
+                    'true' to override this
+                    configuration and use {name} anyway. Please
+                    note this will cause model
+                    parameters and possibly training data to
+                    be sent between nodes unencrypted.
+                    """,
+                    RuntimeWarning,
+                )
+                return
+            raise Exception(
+                f"""
+                This cluster has TLS encryption enabled;
+                however, {name} does not support
+                data encryption in transit. To override
+                this configuration and use {name}
+                anyway, you may set '{self.ssl_conf}'
+                to 'true' in the Spark configuration. Please note this
+                will cause model parameters and possibly training
+                data to be sent between nodes unencrypted.
+                """
+            )
+
+
+class PyTorchDistributor(Distributor):
+    """
+    A class to support distributed training on PyTorch and PyTorch Lightning 
using PySpark.

Review Comment:
   Do we support `PyTorch Lightning` in our current version?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to