Skip to content

Commit

Permalink
[SPARK-49943][PS] Remove timestamp_ntz_to_long from PythonSQLUtils
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?
Remove `timestamp_ntz_to_long` from `PythonSQLUtils`

### Why are the changes needed?
we no longer need to add internal functions in `PythonSQLUtils` for PySpark Classic

### Does this PR introduce _any_ user-facing change?
no

### How was this patch tested?
existing tests

### Was this patch authored or co-authored using generative AI tooling?
no

Closes #48437 from zhengruifeng/fun_cat_nzt.

Authored-by: Ruifeng Zheng <ruifengz@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@apache.org>
  • Loading branch information
zhengruifeng committed Oct 13, 2024
1 parent 54fd408 commit 1abfd49
Show file tree
Hide file tree
Showing 3 changed files with 6 additions and 7 deletions.
6 changes: 2 additions & 4 deletions python/pyspark/pandas/data_type_ops/datetime_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
)
from pyspark.sql.utils import pyspark_column_op
from pyspark.pandas._typing import Dtype, IndexOpsLike, SeriesOrIndex
from pyspark.pandas.spark import functions as SF
from pyspark.pandas.base import IndexOpsMixin
from pyspark.pandas.data_type_ops.base import (
DataTypeOps,
Expand Down Expand Up @@ -150,10 +151,7 @@ class DatetimeNTZOps(DatetimeOps):
"""

def _cast_spark_column_timestamp_to_long(self, scol: Column) -> Column:
from pyspark import SparkContext

jvm = SparkContext._active_spark_context._jvm
return Column(jvm.PythonSQLUtils.castTimestampNTZToLong(scol._jc))
return SF.timestamp_ntz_to_long(scol)

def astype(self, index_ops: IndexOpsLike, dtype: Union[str, type, Dtype]) -> IndexOpsLike:
dtype, spark_type = pandas_on_spark_type(dtype)
Expand Down
4 changes: 4 additions & 0 deletions python/pyspark/pandas/spark/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ def _invoke_internal_function_over_columns(name: str, *cols: "ColumnOrName") ->
return Column(sc._jvm.PythonSQLUtils.internalFn(name, _to_seq(sc, cols, _to_java_column)))


def timestamp_ntz_to_long(col: Column) -> Column:
return _invoke_internal_function_over_columns("timestamp_ntz_to_long", col)


def product(col: Column, dropna: bool) -> Column:
return _invoke_internal_function_over_columns("pandas_product", col, F.lit(dropna))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,9 +143,6 @@ private[sql] object PythonSQLUtils extends Logging {
}
}

def castTimestampNTZToLong(c: Column): Column =
Column.internalFn("timestamp_ntz_to_long", c)

def unresolvedNamedLambdaVariable(name: String): Column =
Column(internal.UnresolvedNamedLambdaVariable.apply(name))

Expand Down

0 comments on commit 1abfd49

Please sign in to comment.