From issues-return-186798-archive-asf-public=cust-asf.ponee.io@spark.apache.org Sat Mar 10 22:20:06 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id F40FD180651 for ; Sat, 10 Mar 2018 22:20:05 +0100 (CET) Received: (qmail 11339 invoked by uid 500); 10 Mar 2018 21:20:05 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 11291 invoked by uid 99); 10 Mar 2018 21:20:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 10 Mar 2018 21:20:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 7F88DC09C5 for ; Sat, 10 Mar 2018 21:20:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.511 X-Spam-Level: X-Spam-Status: No, score=-109.511 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id vjj7QrYi7gRD for ; Sat, 10 Mar 2018 21:20:03 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 7A3AA5F30B for ; Sat, 10 Mar 2018 21:20:02 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 45521E0112 for ; Sat, 10 Mar 2018 21:20:01 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 0DB3021458 for ; Sat, 10 Mar 2018 21:20:00 +0000 (UTC) Date: Sat, 10 Mar 2018 21:20:00 +0000 (UTC) From: "Stu (Michael Stewart) (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (SPARK-23645) pandas_udf can not be called with keyword arguments MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-23645?page=3Dcom.atlassi= an.jira.plugin.system.issuetabpanels:all-tabpanel ] Stu (Michael Stewart) updated SPARK-23645: ------------------------------------------ Description:=20 pandas_udf (all python udfs(?))=C2=A0do not accept keyword arguments becaus= e `pyspark/sql/udf.py` class `UserDefinedFunction`=C2=A0has=C2=A0__call__, = and also wrapper utility=C2=A0methods, that only accept=C2=A0args and not k= wargs: @ line 168: {code:java} ... def __call__(self, *cols): judf =3D self._judf sc =3D SparkContext._active_spark_context return Column(judf.apply(_to_seq(sc, cols, _to_java_column))) # This function is for improving the online help system in the interactive = interpreter. # For example, the built-in help / pydoc.help. It wraps the UDF with the do= cstring and # argument annotation. (See: SPARK-19161) def _wrapped(self): """ Wrap this udf with a function and attach docstring from func """ # It is possible for a callable instance without __name__ attribute or/= and # __module__ attribute to be wrapped here. For example, functools.parti= al. In this case, # we should avoid wrapping the attributes from the wrapped function to = the wrapper # function. So, we take out these attribute names from the default name= s to set and # then manually assign it after being wrapped. assignments =3D tuple( a for a in functools.WRAPPER_ASSIGNMENTS if a !=3D '__name__' and a= !=3D '__module__') @functools.wraps(self.func, assigned=3Dassignments) def wrapper(*args): return self(*args) ...{code} as seen in: {code:java} from pyspark.sql import SparkSession from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit spark =3D SparkSession.builder.getOrCreate() df =3D spark.range(12).withColumn('b', col('id') * 2) def ok(a,b): return a*b df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')('id','b')).sh= ow()=C2=A0 # no problems df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')(a=3D'id',b=3D= 'b')).show() # fail with ~no stacktrace thanks to wrapper helper{code} =C2=A0 =C2=A0 *discourse*: it isn't=C2=A0difficult to=C2=A0swap back in the kwargs,=C2=A0= allowing the UDF to be called=C2=A0as such, but the=C2=A0cols tuple that ge= ts passed in the call method: {code:java} _to_seq(sc, cols, _to_java_column{code} =C2=A0has to be in the right order based on the functions defined argument = inputs, or the function will return incorrect results. so, the challenge he= re is to: (a) make sure to reconstruct the proper order of the full args/kwargs --> args first, and then kwargs (not in the order passed but in the order r= equested by the fn) (b) handle python2 and python3 `inspect` module inconsistencies=C2=A0 was: pandas_udf (all python udfs(?))=C2=A0do not accept keyword arguments becaus= e `pyspark/sql/udf.py` class `UserDefinedFunction`=C2=A0has=C2=A0__call__, = and also wrapper utility=C2=A0methods, that only accept=C2=A0args and not k= wargs: @ line 168: {code:java} ... def __call__(self, *cols): judf =3D self._judf sc =3D SparkContext._active_spark_context return Column(judf.apply(_to_seq(sc, cols, _to_java_column))) # This function is for improving the online help system in the interactive = interpreter. # For example, the built-in help / pydoc.help. It wraps the UDF with the do= cstring and # argument annotation. (See: SPARK-19161) def _wrapped(self): """ Wrap this udf with a function and attach docstring from func """ # It is possible for a callable instance without __name__ attribute or/= and # __module__ attribute to be wrapped here. For example, functools.parti= al. In this case, # we should avoid wrapping the attributes from the wrapped function to = the wrapper # function. So, we take out these attribute names from the default name= s to set and # then manually assign it after being wrapped. assignments =3D tuple( a for a in functools.WRAPPER_ASSIGNMENTS if a !=3D '__name__' and a= !=3D '__module__') @functools.wraps(self.func, assigned=3Dassignments) def wrapper(*args): return self(*args) ...{code} as seen in: {code:java} from pyspark.sql import SparkSession from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit spark =3D SparkSession.builder.getOrCreate() df =3D spark.range(12).withColumn('b', col('id') * 2) def ok(a,b): return a*b df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')('id','b')).sh= ow()=C2=A0 # no problems df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')(a=3D'id',b=3D= 'b')).show() # fail with ~no stacktrace thanks to wrapper helper{code} discourse: it isn't=C2=A0difficult to=C2=A0swap back in the kwargs,=C2=A0al= lowing the UDF to be called=C2=A0as such, but the=C2=A0cols tuple that gets= passed in the call method: {code:java} _to_seq(sc, cols, _to_java_column{code} =C2=A0has to be in the right order based on the functions defined argument = inputs, or the function will return incorrect results.=C2=A0 > pandas_udf can not be called with keyword arguments > --------------------------------------------------- > > Key: SPARK-23645 > URL: https://issues.apache.org/jira/browse/SPARK-23645 > Project: Spark > Issue Type: Improvement > Components: PySpark > Affects Versions: 2.3.0 > Environment: python 3.6 | pyspark 2.3.0 |=C2=A0Using Scala versio= n 2.11.8, OpenJDK 64-Bit Server VM, 1.8.0_141 > Reporter: Stu (Michael Stewart) > Priority: Minor > > pandas_udf (all python udfs(?))=C2=A0do not accept keyword arguments beca= use `pyspark/sql/udf.py` class `UserDefinedFunction`=C2=A0has=C2=A0__call__= , and also wrapper utility=C2=A0methods, that only accept=C2=A0args and not= kwargs: > @ line 168: > {code:java} > ... > def __call__(self, *cols): > judf =3D self._judf > sc =3D SparkContext._active_spark_context > return Column(judf.apply(_to_seq(sc, cols, _to_java_column))) > # This function is for improving the online help system in the interactiv= e interpreter. > # For example, the built-in help / pydoc.help. It wraps the UDF with the = docstring and > # argument annotation. (See: SPARK-19161) > def _wrapped(self): > """ > Wrap this udf with a function and attach docstring from func > """ > # It is possible for a callable instance without __name__ attribute o= r/and > # __module__ attribute to be wrapped here. For example, functools.par= tial. In this case, > # we should avoid wrapping the attributes from the wrapped function t= o the wrapper > # function. So, we take out these attribute names from the default na= mes to set and > # then manually assign it after being wrapped. > assignments =3D tuple( > a for a in functools.WRAPPER_ASSIGNMENTS if a !=3D '__name__' and= a !=3D '__module__') > @functools.wraps(self.func, assigned=3Dassignments) > def wrapper(*args): > return self(*args) > ...{code} > as seen in: > {code:java} > from pyspark.sql import SparkSession > from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit > spark =3D SparkSession.builder.getOrCreate() > df =3D spark.range(12).withColumn('b', col('id') * 2) > def ok(a,b): return a*b > df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')('id','b')).= show()=C2=A0 # no problems > df.withColumn('ok', pandas_udf(f=3Dok, returnType=3D'bigint')(a=3D'id',b= =3D'b')).show() # fail with ~no stacktrace thanks to wrapper helper{code} > =C2=A0 > =C2=A0 > *discourse*: it isn't=C2=A0difficult to=C2=A0swap back in the kwargs,=C2= =A0allowing the UDF to be called=C2=A0as such, but the=C2=A0cols tuple that= gets passed in the call method: > {code:java} > _to_seq(sc, cols, _to_java_column{code} > =C2=A0has to be in the right order based on the functions defined argumen= t inputs, or the function will return incorrect results. so, the challenge = here is to: > (a) make sure to reconstruct the proper order of the full args/kwargs > --> args first, and then kwargs (not in the order passed but in the order= requested by the fn) > (b) handle python2 and python3 `inspect` module inconsistencies=C2=A0 -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org