[ https://issues.apache.org/jira/browse/SPARK-40311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Apache Spark reassigned SPARK-40311: ------------------------------------ Assignee: (was: Apache Spark) > Introduce withColumnsRenamed > ---------------------------- > > Key: SPARK-40311 > URL: https://issues.apache.org/jira/browse/SPARK-40311 > Project: Spark > Issue Type: Improvement > Components: PySpark, SparkR, SQL > Affects Versions: 3.0.3, 3.1.3, 3.3.0, 3.2.2 > Reporter: Santosh Pingale > Priority: Minor > > Add a scala, pyspark, R dataframe API that can rename multiple columns in a > single command. This is mostly a performance related optimisations where > users iteratively perform `withColumnRenamed`. With 100s columns and multiple > iterations, there are cases where either driver will blow up or users will > receive a StackOverflowError. > {code:java} > import datetime > import numpy as np > import pandas as pd > num_rows = 2 > num_columns = 100 > data = np.zeros((num_rows, num_columns)) > columns = map(str, range(num_columns)) > raw = spark.createDataFrame(pd.DataFrame(data, columns=columns)) > a = datetime.datetime.now() > {col, f"prefix_{col}" for col in raw.columns} > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > b = datetime.datetime.now() > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > c = datetime.datetime.now() > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > d = datetime.datetime.now() > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > e = datetime.datetime.now() > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > f = datetime.datetime.now() > for col in raw.columns: > raw = raw.withColumnRenamed(col, f"prefix_{col}") > g = datetime.datetime.now() > g-a > datetime.timedelta(seconds=12, microseconds=480021) {code} > {code:java} > import datetime > import numpy as np > import pandas as pd > num_rows = 2 > num_columns = 100 > data = np.zeros((num_rows, num_columns)) > columns = map(str, range(num_columns)) > raw = spark.createDataFrame(pd.DataFrame(data, columns=columns)) > a = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > b = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > c = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > d = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > e = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > f = datetime.datetime.now() > raw = DataFrame(raw._jdf.withColumnsRenamed({col: f"prefix_{col}" for col in > raw.columns}), spark) > g = datetime.datetime.now() > g-a > datetime.timedelta(microseconds=632116) {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org