From 8487b19ded6f7715288650be90813515e9297edc Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Mon, 7 Dec 2020 13:23:49 +0800 Subject: [PATCH 01/15] support reading decimal columns from parquet files Signed-off-by: sperlingxx --- integration_tests/run_pyspark_from_build.sh | 2 +- .../src/main/python/parquet_test.py | 22 +++++++++++------- .../rapids/shims/spark300/Spark300Shims.scala | 3 ++- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +++--- .../nvidia/spark/rapids/GpuParquetScan.scala | 3 ++- tests/src/test/resources/decimal-test.parquet | Bin 0 -> 18150 bytes .../spark/rapids/ParquetScanSuite.scala | 14 +++++++++++ 7 files changed, 36 insertions(+), 15 deletions(-) create mode 100644 tests/src/test/resources/decimal-test.parquet diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 22da7be6b74..8feb6b0f8dc 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -105,7 +105,7 @@ else --conf 'spark.sql.session.timeZone=UTC' \ --conf 'spark.sql.shuffle.partitions=12' \ $SPARK_SUBMIT_FLAGS \ - "$SCRIPTPATH"/runtests.py --rootdir "$SCRIPTPATH" "$SCRIPTPATH"/src/main/python \ + "$SCRIPTPATH"/runtests.py --rootdir "$SCRIPTPATH" "$SCRIPTPATH"/src/main/python/gen_decimal_parquet.py \ -v -rfExXs "$TEST_TAGS" \ --std_input_path="$SCRIPTPATH"/src/test/resources/ \ --color=yes \ diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index e6553597b33..16d79d56ae6 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -27,14 +27,20 @@ def read_parquet_df(data_path): def read_parquet_sql(data_path): return lambda spark : spark.sql('select * from parquet.`{}`'.format(data_path)) + +# LIMIT: We have to use DECIMAL64(precision>8) to test decimal reading, because we only support DECIMAL64 in current. +decimal_gens_sample = [DecimalGen(), DecimalGen(precision=10, scale=3), DecimalGen(precision=10, scale=10)] + parquet_gens_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc)), ArrayGen(byte_gen), ArrayGen(long_gen), ArrayGen(string_gen), ArrayGen(date_gen), ArrayGen(TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))), + ArrayGen(DecimalGen()), ArrayGen(ArrayGen(byte_gen)), - StructGen([['child0', ArrayGen(byte_gen)], ['child1', byte_gen], ['child2', float_gen]]), - ArrayGen(StructGen([['child0', string_gen], ['child1', double_gen], ['child2', int_gen]]))] + map_gens_sample, + StructGen([['child0', ArrayGen(byte_gen)], ['child1', byte_gen], ['child2', float_gen], ['child3', DecimalGen()]]), + ArrayGen(StructGen([['child0', string_gen], ['child1', double_gen], ['child2', int_gen]]))] + + map_gens_sample + decimal_gens_sample, pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/132'))] # test with original parquet file reader, the multi-file parallel reader for cloud, and coalesce file reader for @@ -67,7 +73,7 @@ def test_read_round_trip(spark_tmp_path, parquet_gens, read_func, reader_confs, @pytest.mark.parametrize('disable_conf', ['spark.rapids.sql.format.parquet.enabled', 'spark.rapids.sql.format.parquet.read.enabled']) def test_parquet_fallback(spark_tmp_path, read_func, disable_conf): data_gens =[string_gen, - byte_gen, short_gen, int_gen, long_gen, boolean_gen] + byte_gen, short_gen, int_gen, long_gen, boolean_gen] + decimal_gens_sample gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] gen = StructGen(gen_list, nullable=False) @@ -104,7 +110,7 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea string_gen, date_gen, # Once https://github.com/NVIDIA/spark-rapids/issues/132 is fixed replace this with # timestamp_gen - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample @pytest.mark.parametrize('parquet_gen', parquet_pred_push_gens, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @@ -195,7 +201,7 @@ def test_ts_read_fails_datetime_legacy(gen, spark_tmp_path, ts_write, ts_rebase, parquet_gens_legacy_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))], + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample, pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133')), pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133'))] @@ -221,7 +227,7 @@ def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( @@ -291,7 +297,7 @@ def test_read_merge_schema(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( @@ -316,7 +322,7 @@ def test_read_merge_schema_from_conf(spark_tmp_path, v1_enabled_list, reader_con # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala index 31b66b42a82..19cc88881b6 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala @@ -156,7 +156,8 @@ class Spark300Shims extends SparkShims { allowArray = true, allowMaps = true, allowStruct = true, - allowNesting = true) + allowNesting = true, + allowDecimal = true) // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 29a072ed267..e29896d3840 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -17,11 +17,8 @@ package com.nvidia.spark.rapids import java.time.ZoneId - import scala.reflect.ClassTag - import ai.rapids.cudf.DType - import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ @@ -41,6 +38,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.execution.datasources.v2.{AlterNamespaceSetPropertiesExec, AlterTableExec, AtomicReplaceTableExec, BatchScanExec, CreateNamespaceExec, CreateTableExec, DeleteFromTableExec, DescribeNamespaceExec, DescribeTableExec, DropNamespaceExec, DropTableExec, RefreshTableExec, RenameTableExec, ReplaceTableExec, SetCatalogAndNamespaceExec, ShowCurrentNamespaceExec, ShowNamespacesExec, ShowTablePropertiesExec, ShowTablesExec} import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python._ @@ -2035,7 +2033,8 @@ object GpuOverrides { allowMaps = true, allowArray = true, allowStruct = true, - allowNesting = true) + allowNesting = true, + allowDecimal = p.scan.isInstanceOf[ParquetScan]) override def convertToGpu(): GpuExec = GpuBatchScanExec(p.output, childScans(0).convertToGpu()) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index f4a077ae0e7..3a941f0afa2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -139,7 +139,8 @@ object GpuParquetScanBase { allowMaps = true, allowArray = true, allowStruct = true, - allowNesting = true)) { + allowNesting = true, + allowDecimal = true)) { meta.willNotWorkOnGpu(s"GpuParquetScan does not support fields of type ${field.dataType}") } } diff --git a/tests/src/test/resources/decimal-test.parquet b/tests/src/test/resources/decimal-test.parquet new file mode 100644 index 0000000000000000000000000000000000000000..7777b468274a35799fe2f276f5282343735d5a88 GIT binary patch literal 18150 zcmb7s1z45Mw(z{W*uVw>X^@f>q`Og4QAz>nE@_bNM$$k+K}ks|K|(?rL=*`Lr4&U( zK?Ff1{^6YSedpZg-upk#-H)5KXJ*ZsSvhO%dAGVM`XV%d=8ingJ$V6|6(aZ8$wZb1X1fy^dxVM^yYNW* z_ZZDqi$^B#QMwr z{Kai3MJN@_$>6~pU>#L(Nt+j>@xwfX^E>R0AmrAk<-u2zB=$E2!Zb+QNrflt`bZ_s zG1aCcTYT&jWMr%oP2N0-B+_hO`Y)@N;8};>OR&R(KA z)Q*^S>>r$zQ(IrY+5`0JMTKF*FgHM}Zf89Bl^cA$X8MhPRS~sjZf1UZ^)`vCBkes? zhESqfp$HzuA|@h@I!3dlvKN3?>s(oG1wS%=ImmIoq6_rv(f6(0l>m|{KttpEBr#)m zP_$di6KvM!7mvT#R06g~v@wI+UqRc33CsD+pD3av&v{$=En?HPAxF-aWuX3wT!ZCJ z7BUNltMxTh4?rvTvF*Ez#8`{MdQY~P+bHg|4EnoP{1|sLgS_aGG*W}Ya{s3R=Sd6* zWQ98D*I>@>Op!6AEAVb+zU&y!hTIK_TKlOIPDGr<@he(ejpzb(jz{}-3|6|hHQ3_^ zJy>Znf4=NCjrt~~?cE-hisAJ*yb@$|fHm8$8B;X0A=47&9KKWBj;q`IT*jO)N$P7O z|48^TmQW4CuNAR8CSk7Ewn{q4!x}7xPE2KT;eKp~tvyUVK>Z0J-0fb-BJ<{#Q}D{j z26BNL{H&@*_|qY~xo$f~s0cIz|5wcfgj#LzO>nXgnlo;t>t;nHp)bK|bFTA;r7~fi~Bc~gL{&eMxpSH{+gs=GpOLOxH zz&us`{8=xKtSN%V_MJft2-4>fG&1rg>v`JTiWfLVB*T+$`2PNB(odZWzjn2l04EQ# zm3&?aN$7&IHQPoZNpOwXPijjGQjw61SEF4d7`2BE=X%L8c*`NuJHObt33j&?9EOE- z$$ZxAp4F-|Vy1(yvR4O><6#8*$1B@6;M2Cd2=OL`f@JOzImlh3|L`Cn4}i;1K^sSD5r-82hPIA=t16LA!)(yn2i%uap*CT0)?kfJ^HIzFO#E znV&>aO4?*JvGg(d+`f3P2X9<@HP~>8%HaJe26BSQZoN+OwR1Sp+!}So zW-HtoDXKubZ)F&2&zterURsc_W_Oz$n8XwLnsQwT+`W(ZQYcqE`pALvz7_ZtGg(Wz zw|Gg;=%O<+mzmnrz2%1~$ujpKQjjC{d4rtW6r3SSh!el$<{E+hDsvChapo=lAW+yc z?)pQJsl-9y9(n;o>Z4J^sZ>Jps{@Voy*Wvw(b1?_=cP$d5>?o$wkQK$Kh7LW@#r9W zq_DFTGM|rX(GR$j#9OhnMwY?DIjs8nsNsW`sOf#Rs zFGJ6Do)hcn%xY`9JG6_$Bb26&HgY+?` z^Mq{s%E1Cc(IQXu^R6}N^7y >dXZMuSL%(q1Szo%iBHmsMlL;U7308a5WcBzX z16OmB*-AL-S_`FVL<09~Y-E}uSz#8#y17Y-)hFka1M}%0>m~ zKU3Vzk1YY*QNKjTciga#JbTDkon(m@GI?ar=q+L%KKI#J33!N-s*Ud156mR&ZPiAk z+S-v)n~%l+o=?PtWz>~6_k6@E^&m3qANp`x37w+z`nss`x=r6$m|L(?Ub+8=bph-Q zlubWNSVwowExuus!hoz-64N3YMIiPO>i2R8Io>-*)NB7hki=5iO!KmJ9(ZlK_$!L3?KH2wJ~HI+}m$3)(^OrwQ)iHI%T)CHs}BIpV{z*EM3_uM-CB zmrjPBkRf4b%M$h9BqQa1rt`BC5+XcLaku<9R_SH&xWmXg>36jiV8 zeTiN~YOkP?`nfkD)Fij+1U?Q|b>n-Z#KI8fv}gO&2RRbd)U&6%*-|;AW&Aeu3X>nv zcU<`UTs9Ag^Yxe4emxq(`*9wN+$iNllJRfiNA5i$rd>_@@>*{KXe0^#p1fR$%D5e$ zq0@*(m)7q)AO1jysdLS=>lzZtwo_3m_|4Bm=B#lG&#;uIp)v}|MfZj>t zb0xc`YwgKo9)ZC(jWOfm=rMv@&dD@qRZs(JNQ*IOYx`A}xpUi(4{E8&(UiZeK?j?-U;d@Ds%UQ&A84vs3NzV|(9GBdf z3EV^{qRq!LKeQ3#TDlEhU;2U`cZ!VESF9kLx%tf0_s2SN#n0rKB3}LPl>Wo7h$}5` zDUsMpzJ+dMbIg@0^7Bs=pOf7x`7ruQivim%tIA47R}OBU5v3i|Y9yj9u{=|IFiw=q z@&0RnS2KcOw}Mp@QHbgb++kQ|E{Wn@lKg@)NA&y15XMhYC&{!N?06dGsgQ7io=c={ z+8|~6KyJB+g-l$pL9_P~8z!os&~E9sKzv7Pb%^_31Zu}PdR=rT7g3wpKK=1_J!V{g z=Edc2RY--1Y0cpKEbO*a$B;f-HqhLtRlCn7f?v7LlGwatLMmQpO^?&MfSeB$KPGq} zjrJomOx~YAj-(iPe3d#RC8RzMc1`u1LC1`0AXl^H78ggvL(73S7VQYfXi=ir6 zYv*2pH%ei@by^w_qQQF&nXOhhwnD{JjemV7Yl#)Y_(UM#qEZ^2Xq$L>dCLs8dz$z~GVX`Yd8e0)YPR@xp6`1K-a>Z!rE9wzR-Ml^Zl*KwZ@ExE<~u1tRz}oT z86jh*7EH3}s>|Fqw~K7H;}?l+_K{`A!e^Nht02j=VQW#49{Y8U|8_ja68TQ~(9)ys zF6lW--!HCp$512>C!T}g>qN)SpD8=p(G1cI1U$}pydl0%Mk9K7--5_${%b@ljXKIe zF7wAgNiHdI?>kcbyaf_$sPgdB{0)>F-_9%=kUsJr?2ynqq=>wHMho&HTlrNeZ&=`0KuUNo|C;oWG@bgkO_> zZ#(PoBefX(9JV4wy576 zp!Z$N~yB^ii5V@Theo(qsQ>uIvX4Fo$6hQzK|45xbYRlIU$sM&> z{3+9q;G^50YmWLI`Ld*r+A<{igd-TJ>hR*IU;F2dGeSWyn>? zFG5cUR5%@u;3*Np>H7;d2%Ese$9+sk>^RBuN<*xt%Ri(c*8LdCGDLuI!S)UZY~8pV z;sxn-eK9wOD7hD^xnQTWv2Z2qEZ?L z<@_y@TG+yE@rWaygxKZoj6!ge@02A3(kB!;Auu_yssKR(_u&en zNyzE#?13%rb_r?-8v1hFAeG$V2NjUY?+bGD2n9ZVgNZAFAf(9buZzvNDis_)He@Kf5mpM&&|F0W#|+sV z8Q_?p1EB(fR0lLW1WJ~*

{MCT$pTqyS}_Yz>e!H&J^V9IHeWwh=b`+u3b3C&Da= z{E^Z>7ct$1!1jxX0mPs@0R#{g!cuz_5p0R@tdb(UgrF{-Gq5$XAX@-iMWlR3Eh=Gw zBsg#AY`-Mj%8p?N0{6Ecq{W3NymNgh5Bt25@>n3ga-@k5Y<)Nu1TBETp`bkh`QIA4 zhY(Q!Sxc)suoa}wfn_6{u6%$-7zkxrvZ6>jfdV(e4W|uz$jd_EH7h1?(GyzhUe`m# z5fPGM<7vx@9f)u{o0I|J#ixGTv^OUFTrO9GefaYo$xtabU+!+hB{eL*%RoR#U$TOr z=`8(A2y_HFz95>sfBCkFL7P22K0&e((Vzf1C!oy|6H+n^s}W(mwgE3+E#9bq44T+S z@8v&m@`thKL{Pt6XXiKIY8F5FjE^XKDAPb-U+t6&$JBS@@4>MZe+o7TJjpN^$i?K4 zGS>oO=6BC2*dVd5Q{yGXGtCl1gwlej9H@J8*4I>!#^s;aRUtSg49Fo!yfEz#IedPJ z@knG9jE_+;3$^_ibTkwn;|xXo`ZL)oB%+~r_Xh+`e?niwv2R7+j@)5iiA4pr(z;CV zLrFKe$#Oj)ba^BN!=YuV%%=zYR*T?qIB zbn>BS)M(aqVCzt7j|l>muR158z2VnGdg%#g2U}XMeOgs6_QaD`w8iQQDBFdn9L49J zeeo-h`@LGRW=Q1)#$gh{3$$Dmk~j*$U0s<^<$~G<;kPxF7+l0yea?ObpQauXyWy2im=F@F zfg}XVd`1yt!be+vb`Y8Uiym_w+={FB)Bp}KH?6E@ZM<*}+ zoF5k)3<)BMhMsJaOE~gO;T1t6gcwh=#Kr~i)N;;hc>HC&npyzC$oi@ll#6Wi@F_^% zw%uKA7g`hNB{^ky67K2F4MF)-mQC14;2kO2xZvs5S^g1jYW`;$b{8*fptPJ$*!)JW z4ZCEmc$PZ}JQIhzFuXPTm>;=GDlfAaq|U+QmI1dU=5Blwb}oLPXM*wwaW*%Gon0>8 z$Dmm98~w1JgwK0;PKcjhG@T9mOj$Uv6oiqqx4e+cBp&`iIJxBHFCGZ&NlEG<#?~qy z6$Kunwuj{*0GhcfHG0AiW8L4-w+=Hr5grP_cD?KkuiolsrbrI3U z%YZ1tdqWpr5<5v%iYIX1desG8cQipx5>Bj>eP<5{ItANPkfwP_Truq1 z>$EE#z_YeKj13^{7Yb-WT3_Y4dWG@2F$}TF1TXohL%4Zkj&xZh0>8?O#_|%@-K>m(*q+!`0FL%D3k%U0z*Ml!``v<=C>2O6e zBClRRpr7LqN1O;k`Vm!7UfmbgsnUt@=d4|!{C~$^z^3CxpXS?V5!{nr4MSyai8)+^ z%)VfTDTMG#A=#ddK}8 z(-7R19o2x-K1;__@sbdp-Q($koT)15`oSpF424LTk> z+ccIQPZ6ur8$c+e+Gm8;a-PpK)|0SpZ%YPEm*358S(Vtgj;uh0HsP`;zBicoQ9XmY+oUZ=Zqyp^)ct>DPr}3Xt(-k{%s8u*R8_r&;+gw}S$(#tH8$ z3QIvB)i@C{rjg!zM>S40%a-7=qdJ5d7B1@WO$%$5Kf(yn+|E3_6Wn8g7laW0qqZ@I z&<+lVAC*66;0IX#pugE5KGepIjcX_%k%lxosQ@q@!s}C041mHf328rf6xfXZv|=uY z0e@kbI+xRNM^(EWiIaF+qd^;w4UGk75x}dd@CR3oFlWwr0 zMS-#m(%%x305m3VHs?g4!C3g@1?O=T$ef9>64XM2uB8jsUb!gnN0Zc+z=D9wyp|)6 zU=3D3rC*Tiiw35$X6U&J1egpJ+71PwLGQ+c{vZl8_-ah7>pBJLCq0ih4n%?Umu1I@ z*iax^d2xC6G74P8MOu2%A>ik{Z^t`MG%#r)m`z?m0ju!#!|8nlER{Dqmi+->$Ig7O zND~05_Y6ZO8~_A4_NA9NpuwkGS>n&h0N@{Wc4l~h0t07f{VFa2KpR+NP}zWhDvHRa z)`tLi&KNHBa-sm$fQ)Pe76G(!kCP|;0Wd9oOqqQT1xkmw3-?3-`0#vIhMOD(h|bs$ zH}C_{-L=Sy09-51@e0#JK>T~z z3)fkWEbE`IUg2=9r8EQ>|4P54bQJ|2OBGsN)j$Efvdh&sCs2S}gXc9l)C+!3&kx^< zfQl=;nXvK%{l*RV#-u3<*qg6xg)#}BUENEXGr(JIK+CzJb>vY;!0HFJDo1@qi0ha== zUXJ(;?kPb3S3?z6Iqkkf0@Z837&^Q^fZ0>xy?W z1EYzKXmIj5F%!2A8pNbrSFeHk@Y~EwzEg_=?b?dQA@>0&toJhR=6ezF>ZgBn{}}*&JKy6?6+?hk+fZg#1N86y*=si6P=J!+ z=bf9g0N7fmUPy-a{_RZ7qnBg|kUHM}Dj5p{k~3-OEDHc6iv|-l2PhDC|6`}PAR07H zYyTQ6fo;QVjbSwuU}If%$bJdH=x5?`sG0)M{> zWUO*CVcao(k0uG^LV*=ARu1$906Yx9N|qZMR6AMswn6(k%k8IpcMAnJoY5B3ClPRh z%9!f5ECAcQI_(3C0EB)tcT9ud#jGW^yfNoM14DkUuReNcz{PE^DD?#ac8^S+|ABI$ z?6!{cg!c4ftj=@X82}UUKe@+J5b*8AvJoE!4f^sg(fTW(!Are@v7Fy1u&+i{!sUnp z6E=$EpM?O3_2i@+_eTMy{7JQA8z_J~*~i|)kAT;|NO7Zw2*}cs{-kLSz?YQ!j`4IE$fu6Xo&5C~1Qb|d>+0yx;1xk{SI83u zuAtua27~u)NQD9)UiUY+2*LQMXOnaG6AJ8kGQPL7fbr~Ve^xOq+z(3KDFgb|aB}=HK{4q6 z-|p=O4xm7NO(-gI69JYRwhf_>yrgs}G(# zf5mq(#@nA=?g21yp``Vc7o-#Qk?^Av0C$FM$K??e(3$K_Je&sLybu?0$VCK~ud z(1QYU#^RjEuAxD3w6+ohjEgRH$|A@F0zPY@dyQed!=#?D>mr3Y(AmiEErlrX^u5A@ zLpD4&Ps=~vybA!j&3z`V8h}(P)ugNeXiqI97Sg42LuR^`Q_QrQ~a3BDqL$Ioc z=kbm;Wpi>H0`hDFr9%I}^QgDdln=^PFW#rS9qL6F)x^ek37%J~3G^!DXb{rzQk}05 z0pc1D5(Rk>uyu#YM05{;f&NftpG*X#Z3uk*V}u6bx4rwthyl15oH%+_69q&P3OGer z0g$`)ESOXZ0rEHAVInaA5N^Ned&h}@@&@aqTyg}=l-dxtmIH7=mFM&r`cv?iY@F#j z+-Kh1@CVWecymo<_+l^`T;HV*nuGF6E~^#eAB1tK@a3b79(eAL7QA672EY!W2~0=v zP^0}R3-p5*rZ@c3?;&7t_mZ6R3jmnH7Vb`YQtEeynfIa*QMjXz)@BGQj=X#`mWW#Xx%t4c2yvL;?Sp zchtc!&hNTJprYWqnP&OdmW2V}!&3%!9*23zM}3Kq<1qgDTrTLni-6|DHHk=X6o?^j zq!T4ZfEdksrruQml+@GL>7D~{oxXnURwwkMVLqZ%MHuJO+4O9y05IPCJ%>qw@$@2V zJ_`DwalO8XGQ1Pe(e0lOfqAahNG*p1Jg-`|nVxgt0I2G!?Vg4C;}bj^(>v&w@8bNG zh8pr?M1wY(KPHr3D5IJB)X$S2ARH(17}*>r1>T!yo7bwsb9>)$ zSk)c@H)hBM>aW23HXLVL?VE%Ta90U+VqRMQBvg}?Gkx)Y@GH|*ef*zaW4&x`djl1 z5l;={I)~eRWESS5?gB^?{q+ybi=^7bI5f~8gSIN_87TrdJ8ZTF;W<|3B&7M# z9tFxNzTtntdg;dw;q7x%n4dFYNoiqR*cxw^Nuoi3ZJi2L?l8>jxirt+ErNOUt06@; zG63*6_C*o6j-0wv+AAFd2rI92y!Z;wflycD`_R91yTWQ-z;nJyd_6qM9O`M#DK;+y z1;(CMpH6^rD2goRTYMe@Tz*iB$wI$pYYx94p9b@_^JcrZ7T|gCEwlDKJQo}k8cwc8 z|Jz@v91D%S>QF%V3kTbAsK1*ZuG4nF`9&$dcZfyNz@ehN+ymAxGDIZkok#F|Bm44c z8~S&B=&QcgV3>aZnid6`CWZgFL!^fn_5XN<)8qf8=m=UCSabikUO4)WB}4>*f}`hd z@4-J|wD5xfe0#z-+fjin4X^9j(N?f|RB9hRk_(^B{l*$FfA+(1+90P>HyNr~M&ieH zY&35O-g(yKPvw4xe5G(u`kN;AK%I>%SzOc~6BD}g(nHSABWjB(Ce4aiCbYRf(3oz! z&J@nGXjYxF6a0g(zp~pYpsjBmeQW!ZM$wa)f+wPKBGtN+qP|xn8`e92CB1X~{gTmJ zef1e#%{>}>eD+haG3v(h%vc)CI7#Y?({-o%hQ#~}MO8nyuxkC?n`LErA7kG|CCbuu zjND=AqQt@bRI?GHrGA zmwkV#v)>PT2lcfK`VCD;+S7wnSM9~-;_lwh4vOvB6n^A&n6ueDCiH>5?)wGh4uZh; zdq*u+;-#KQ=u`^2&qxN(llclx!7+C8{taDcYI$_upwqRu??8|5E1A~D1IJSR zhKXi}fmaI5jZsN=*j+q^8qJF&q&@wl{S3~a=osoNE`2NRuQYjwq>(dJo%>F^HQF}Y z6xI{JvYhBpE%ZA~(SSZBjr#6{oYHW>h}z{2&)y9^;~!}^`mXUM;ha=Gv^()V-`3PQ zpYJIf%Y0S;(>J{5DRo}Sjcg^1!?p6QeckM@B_X9FsH~q<(77xmaKO4 zeV5xED3vvjsON4?_0+EpMfDb0sZG92WoOKyS94_gQf=>k&-|>BtgC;0DDC6Y)%ftqD&gE}-Nq(fKQ@**{GT>+YJKo~wKP z%Q|U?cd@A;{da^NGN>TxUG~SYIg8PSHuex#GQdA)1LK&RuY z>G7eUZ}fsU*1ebNT8DqjMs%mXCD}_E$~*4QCLT};4Au=4Jy-0nf2LpkX)htmC&o`# zf;U{d$CUh&nYdO+fk@CJ3E3q>{`ah-euua_55~ow)eUz<{(N&=w#rseIyTt3Fk`#i zLPARM!p~(z&aI^=mq+Y6-u({aq_mIyGLjktts64+COf3%FHT*qk(BEYv1Xj}_q+M( zCHIB69{0gRv#$FMrsZ6Z!4s`_DST;3V_v9m7Pfx#;&YncZ|$#(_vKmUYVD-5A4gxN zjLN7A+M}h|~?6zEjn)3TitQ(JRA13NDC|0njrLeq^ z`zhvz9_#QI#h#3HKb9pzzKsqRB`UkP#QCsM0IiiS`~C2H?iO8)qjzNW#BT?z4t76` zF6m@;$^4$Qk0`bIne<|2&ZlRuMf|`Cylvi5H~Xb3)9;hV|FcG~Zq!^_IGW^Q!8CBW zuP2<*P?!BuRcG}k*xmk;qx+OM?G?ncVG_9~$c>9?m7Pqr4%A1b&VH_q&T@1KH+5G5+SF;o{Y5%^|IQld17f&Oi5 zCsll7QKpWVmdll4x&E3W4)v%_j&%73QqIw5(Wgw_ezbAdEuxP6+&B7l_hP|fP#MGE z)0e||5Pt;;s0e<64xe_<8R!dUKJLjR6mHVqGGW%`jjY<2WV#%<-c8|i7v}2M3-5tIHSLeR*?bo-zBpYzdFjs;uGZBMqx~=R zud7DhonsoU5a+Y=CT;Xqk-S9XpUm-Qx#GMjPW<`}CF&eZvyUCc*@sS4LnJ6$VZDgd zVz~~^xzX27S&9z+mw&7NbvU91TpEUr7OQaXhIbz)+%Me~cYU2c-Zs@AmC>;loqS;a zA)r2fu+Lp$+N-6sKY#5fTT%ojW~V+QyC$8Je^5>19z)7K0hQ`zl`xYNnBm*F9eBy6NJ-wxWKbTWelmwq!-2*!Y<-ZPC@JMdp~XdgMn~B?j4eb7AC4;wAeV0C6?B{>h_XR?~98Ku@31r z{83Zws&u2Da~?!~in^-q`x2A-(S*UVEN4zj4#F9lbC>t!%ZcKykv)-)2j@+9OfElHTiln@VD(PeIZi^}$vlyGx2@ z0nwwRSh9>h&4AbKUsk4s7!*1K{?h9(@z;irO5|@mIi%M`MSLYjPER~RuM!qkOz30` zlD?i&tcZFSBvKh+;+Of*#zvR;vciv(MXQ**PFY@TUfO!hLh%xHX_3+Bzm65VvwSNn zqs*}-$ELFJM9S))-%am?XXtEoysvuOn8!HLebw(l`HRKTmuFi-H~Rb|K1JqM2{|i8 z3+(w*N{Lxd{0Wge(R8BuSU{|x`c~Y7mID9c?q^nXF~7fO_<-|6HNT`1`4_(S{*ooz{W&R5>C3uJf5EE$n0eO1*0`zzD@pV$xrPq7rzH6PRMd@<%`CEvIcxfx4%cUP?={aRl1#W?6UT;%USBNnO0HVwb+y9WHI(YAjbN`m>!A^`a?_qI z?}~@u6@SNO&I2;#ip3V9*lEiMlA)jg@4zvih3^s`Ywe{wMYlYL=QnH7p%}SW_u5Dv zKba0F50+9wlihd>UdV}4AHO*K_-17V4~I76ffx%S%w~@JG|QkIxKtce9kEhU^Xt=y zOk;|~fG3{OBJ$cyFc!&-Rc~&F&0Xq5o5^zV8R3Gc{jctNTW?lS~{4 zEA4;2yvv)vG`|?rdhf4!a-KwMA8!+2dQX?(iRF|5)1!drua=wKteng$41cJ1@zX1n zg}T&Du0CX88RB5An7;m?`1IM&uDbdfUeVs(SKo@Zq454mb|m6gZ)}>=l|EPyn)GII z$Wbv!6J9F3U-f3Qp-z=TdE9mN33UkXomCo{-jK&jve!(JE*D{ehs>UOe7iwImsgUhIGN>316QjbHjR=4LNUl%Q zT7}UhQ)PC5O{Bs*+#j?{A> z=<{`*$$PrN+v z`7F$!s!^n`kKsE0;p~9qa&4q?&d}#}$On}P@rH8s4{1*=(fdCLZbM8Xj>$sTeSGe#~|oe)0DcpJF2Q zfHLc$mtI5I?B)AhZL0jud|NViO;tMB!&+6hT6*n%uFF4}dRe`-_Vh|5mO{&w$W4Ee zhX40yst<=>*^;6S#~(?l;bDEvjUNVVZj+~7HFApA#C_L9#As%(zR`Ja_N?yhao_Ya zjFcJ_uQ*R}+l=%_Y43WBaT81X>OEoL{nUu2cbfK#$&&E5^U9nP*>faOE@x$FnV+!p zc+o6vTi}Wio_+|86b*ZV#e^lNe{#^@_I-R!X9id}1E18p^q(oR59_ z1=AZ{e)WdIcYR}O*7#2$GA-fyGQaYh;(zxl-{eUe_V8N2@yzf?Gcy2*=;7- z)VO0f<{Qfy)|K>=t!4UZmYmLzmSSTy3G%0U6q)quA6XjHE;@3*z<&5CWXQX~Ezd-# zqSr_|y8C9K`M~m@wK)cW&TxA8H{y_1ocBLkFnm-W+mJsC;nCH%8GIl8+LarY;QL3H z-m35oe=oZi4h!^M3;5*k&Xc3(e2;!Tdvq@jd)w&|0lWWEiQ^uKr@T4y%oZVK z4^Y$Q`q-xvXEM*MHuWVf|0FtC>&2HF&vEDEEKo}>OxP=bWDc7)A9Goh7F9(DnVpg3 zK0FLR9Cz3tH@FZ9@BShVl@vc+K6h2*YyPLo3^C#j^$vS_&Lc*S-hWPZXl|Td9X_Yk z{W#%ZkFu>WYNDImV&~T*B8?+DZ!}}$NsICwAgyrPbdu&2U0$~*VRWyD8ELj! zJ=}*cAMg4Uv{v33T;O1)Q>RfVDjYCst;N0E-4H*{sGstl>m?mW;R9BW^e73zQ@r_D zM{f5vj;h()=dYcdiBHqZzQ`y)kgdEirj>hvuIe1A^=nV=w7{Iyvcy0;9?>yA?o1b( z`vz6C`bPDPjPE!#wtF~4gPwl;5Ej>3<@zl^a%x><&8;+aP^!aMjAB+OnjP*N@&+Q} zJ6U$7yt{5V&Oa-b+2?rVSKi7)%k`6k>UQ^qH*t&bo0ABoKf}a=qkOB&M{`PlGz&4H zAFS8OX|hF+s{S;cSWtfaYj`Slgt7e9n{9~S8Q#Ef_C(l8RK>!e`&IhQV>$WN2{&XV ztWVAI>P1pIF|D2dayzdm?#|GOf*XLmzd=T_n7IBxo<8D9NqQ!|s+@DE^P!p60{=h0 z(7+K4Ka*YKr#=^&ZMgo$tVBe(+0|Xp zL|elo*P3rZlpf1SQg$d(POd>J+== zMA-aSOI?Gf9NEl1YBwDdc8|9gtQ!@C7_yf?&Dz_%HrakWUgo)>Yq_7@*(KTDVfj6N z%?@Xcw;OVPk_`fVcLuC`x&uzdO};R)=$L$j_WvedRU+qg@1y&wlFM`3;^N%xK8^el z^*E7;(36XdvB5Q+iae?i=+1WA8C(otx0M|JPPbp8H@YW@oh514MTTcy{xRB z|S5M_&Gu z4pkiC@$oz3qTbC|;`8h9uHm14?Q!U}J$$0v*b2*>n6ODiN7?qjjjeNtsJVELAn_YwVTlxb05+GsP|gjhV1 z-QP>-x8t+38*^p~d7c8f+%gPCH!~XE$Zx^>2lzw{_VX#)trEJUJyuu>7Q*QrN3^?n zi^kQZ=Tp=NW`4Ci4%t7}lF$stl*|M7_=rLVvPFJ=)iK^kxqzbQ0`kLat3Tr_0xael69yH9G< zW`e15YtMB9b|g**!!-^yS#;L@M0awZcv0;O%mTG@uCAp)&*+b(woH$`jeXmCv{z*- zn_DEMMs>05!fok6zR6EToB0aA9=25Qi>1Zz@$U%57P=kLdOg?VE+Rks5C2>nj}Ewz zA0@+JHv5`Lom5x!J&|>kWm)8@N(15u0|w?J{JZoqj4BCE)1eWh=+kJ(1=J?@y{W zaBe8)4Ggbv3pEvfzQ3OJ$|ZKs@*lu^8u;W0;3hez@HXL|!CNz`tM!a{V3$ZWy?ZPBj}$Me?V7eo+Fk@Y?6bMMLj7Xl*xB#^2=J0J$f*`d^Z*|DZ!#o^d2qz#u?R@8RtzXzghYH7WQHr3(7ld;2(hxYJ|B1ce1f z06pJ-!|?HP6@>ctx@PYy=b9J@0afNu0fV}|6U$i-7 zgF{0F{)2|d|3E`nK;-{MMD%|kA|W6m`hVJq*uROmdbm6OUzia8_f!X059=d&{eJ~u z9t!Bs{Y&(x-MrcUr7JcOK@q4cHeQ=+&aQTB*3vR|4$=-XGGd}OlJ*Xgl44?Z!ji(0 mGWHU}wlX$SGE%}e_BM7Rd< frame.select(col("*")) } + + // Column schema of decimal-test.parquet is: [_c0: decimal(18, 0), _c1: decimal(10, 10), + // _c2: decimal(15, 12), _c3: int64, _c4: float] + // LIMIT: Because we only support DECIMAL64, we only support reading decimal columns whose + // physical storage type are INT64 in current. + testSparkResultsAreEqual("Test Parquet decimal", + frameFromParquet("decimal-test.parquet"), + new SparkConf().set("spark.sql.sources.useV1SourceList", "")) { + frame => frame.select(col("*")) + } } From f60689744cf83eb3cc1263af9b64b7cef74e93f9 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Mon, 7 Dec 2020 13:25:27 +0800 Subject: [PATCH 02/15] fix Signed-off-by: sperlingxx --- integration_tests/run_pyspark_from_build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 8feb6b0f8dc..22da7be6b74 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -105,7 +105,7 @@ else --conf 'spark.sql.session.timeZone=UTC' \ --conf 'spark.sql.shuffle.partitions=12' \ $SPARK_SUBMIT_FLAGS \ - "$SCRIPTPATH"/runtests.py --rootdir "$SCRIPTPATH" "$SCRIPTPATH"/src/main/python/gen_decimal_parquet.py \ + "$SCRIPTPATH"/runtests.py --rootdir "$SCRIPTPATH" "$SCRIPTPATH"/src/main/python \ -v -rfExXs "$TEST_TAGS" \ --std_input_path="$SCRIPTPATH"/src/test/resources/ \ --color=yes \ From ecb6b8d62b288a62969daf41cfe89d30aeab1793 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Mon, 7 Dec 2020 14:03:54 +0800 Subject: [PATCH 03/15] fix typo --- .../src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index e29896d3840..b5b852cdaac 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -17,8 +17,11 @@ package com.nvidia.spark.rapids import java.time.ZoneId + import scala.reflect.ClassTag + import ai.rapids.cudf.DType + import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ From c3afe24f9d8affff57db57ccd3e1d2167673cf0e Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Tue, 8 Dec 2020 15:05:13 +0800 Subject: [PATCH 04/15] support reading decimal32 Signed-off-by: sperlingxx --- .../src/main/python/parquet_test.py | 26 ++++++------- .../nvidia/spark/rapids/GpuParquetScan.scala | 36 ++++++++++++++++-- .../spark/rapids/HostColumnarToGpu.scala | 3 +- tests/src/test/resources/decimal-test.parquet | Bin 18150 -> 20133 bytes .../spark/rapids/ParquetScanSuite.scala | 6 +-- 5 files changed, 50 insertions(+), 21 deletions(-) diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 16d79d56ae6..1665dccb442 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -28,8 +28,8 @@ def read_parquet_sql(data_path): return lambda spark : spark.sql('select * from parquet.`{}`'.format(data_path)) -# LIMIT: We have to use DECIMAL64(precision>8) to test decimal reading, because we only support DECIMAL64 in current. -decimal_gens_sample = [DecimalGen(), DecimalGen(precision=10, scale=3), DecimalGen(precision=10, scale=10)] +decimal_gens = [DecimalGen(), DecimalGen(precision=7, scale=3), DecimalGen(precision=10, scale=10), + DecimalGen(precision=9, scale=0), DecimalGen(precision=18, scale=15)] parquet_gens_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, @@ -40,8 +40,8 @@ def read_parquet_sql(data_path): ArrayGen(ArrayGen(byte_gen)), StructGen([['child0', ArrayGen(byte_gen)], ['child1', byte_gen], ['child2', float_gen], ['child3', DecimalGen()]]), ArrayGen(StructGen([['child0', string_gen], ['child1', double_gen], ['child2', int_gen]]))] + - map_gens_sample + decimal_gens_sample, - pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/132'))] + map_gens_sample + decimal_gens, + pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/132'))] # test with original parquet file reader, the multi-file parallel reader for cloud, and coalesce file reader for # non-cloud @@ -72,8 +72,8 @@ def test_read_round_trip(spark_tmp_path, parquet_gens, read_func, reader_confs, @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @pytest.mark.parametrize('disable_conf', ['spark.rapids.sql.format.parquet.enabled', 'spark.rapids.sql.format.parquet.read.enabled']) def test_parquet_fallback(spark_tmp_path, read_func, disable_conf): - data_gens =[string_gen, - byte_gen, short_gen, int_gen, long_gen, boolean_gen] + decimal_gens_sample + data_gens = [string_gen, + byte_gen, short_gen, int_gen, long_gen, boolean_gen] + decimal_gens gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] gen = StructGen(gen_list, nullable=False) @@ -110,7 +110,7 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea string_gen, date_gen, # Once https://github.com/NVIDIA/spark-rapids/issues/132 is fixed replace this with # timestamp_gen - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens @pytest.mark.parametrize('parquet_gen', parquet_pred_push_gens, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @@ -201,9 +201,9 @@ def test_ts_read_fails_datetime_legacy(gen, spark_tmp_path, ts_write, ts_rebase, parquet_gens_legacy_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample, - pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133')), - pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133'))] + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens, + pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133')), + pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133'))] @pytest.mark.parametrize('parquet_gens', parquet_gens_legacy_list, ids=idfn) @pytest.mark.parametrize('reader_confs', reader_opt_confs) @@ -227,7 +227,7 @@ def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( @@ -297,7 +297,7 @@ def test_read_merge_schema(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( @@ -322,7 +322,7 @@ def test_read_merge_schema_from_conf(spark_tmp_path, v1_enabled_list, reader_con # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens_sample + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 0ac417613fa..ab840eb7100 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -198,6 +198,33 @@ object GpuParquetScanBase { meta.willNotWorkOnGpu(s"$other is not a supported read rebase mode") } } + + private[rapids] def convertDecimal32Columns(t: Table): Table = { + val containDecimal32Column = (0 until t.getNumberOfColumns).exists { i => + t.getColumn(i).getType.getTypeId == DType.DTypeEnum.DECIMAL32 + } + // return input table if there exists no DECIMAL32 columns + if (!containDecimal32Column) return t + + val columns = (0 until t.getNumberOfColumns).map { i => + t.getColumn(i).getType match { + case tpe if tpe.getTypeId == DType.DTypeEnum.DECIMAL32 => + t.getColumn(i).castTo(DType.create(DType.DTypeEnum.DECIMAL64, tpe.getScale)) + case _ => + t.getColumn(i) + } + } + val ret = new Table(columns: _*) + // clean temporary column vectors produced by castTo + (0 until t.getNumberOfColumns).foreach { + case i if t.getColumn(i).getType.getTypeId == DType.DTypeEnum.DECIMAL32 => + columns(i).close() + case _ => + } + // clean original table + t.close() + ret + } } /** @@ -658,13 +685,16 @@ abstract class FileParquetPartitionReaderBase( inputTable: Table, filePath: String, clippedSchema: MessageType): Table = { - if (readDataSchema.length > inputTable.getNumberOfColumns) { + // Convert Decimal32 columns to Decimal64, because spark-rapids only supports Decimal64. + val inTable = GpuParquetScanBase.convertDecimal32Columns(inputTable) + + if (readDataSchema.length > inTable.getNumberOfColumns) { // Spark+Parquet schema evolution is relatively simple with only adding/removing columns // To type casting or anyting like that val clippedGroups = clippedSchema.asGroupType() val newColumns = new Array[ColumnVector](readDataSchema.length) try { - withResource(inputTable) { table => + withResource(inTable) { table => var readAt = 0 (0 until readDataSchema.length).foreach(writeAt => { val readField = readDataSchema(writeAt) @@ -687,7 +717,7 @@ abstract class FileParquetPartitionReaderBase( newColumns.safeClose() } } else { - inputTable + inTable } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala index 8f36537233a..3c9343aea5f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala @@ -127,7 +127,8 @@ object HostColumnarToGpu { if (cv.isNullAt(i)) { b.appendNull() } else { - b.append(cv.getDecimal(i, DType.DECIMAL64_MAX_PRECISION, dt.scale).toUnscaledLong) + // The precision here matters for cpu column vectors (such as OnHeapColumnVector). + b.append(cv.getDecimal(i, dt.precision, dt.scale).toUnscaledLong) } } } else { diff --git a/tests/src/test/resources/decimal-test.parquet b/tests/src/test/resources/decimal-test.parquet index 7777b468274a35799fe2f276f5282343735d5a88..2d029c704bef76898bdf114e0878c911ca6f1321 100644 GIT binary patch literal 20133 zcmZ6z2RzkZ_&EN&?{(dad+m_DN%o2mSs@u^Cabdd%*+niWsk}zQbs~XqNJ2c8bYE$ zL_$%N_#gegpa1Xs`rVhe$NRjWbIx;~{haf@%=RCa+W|Z7si7aq9kWFAH}wt@x3aD{ zX>035_087z6b=#wr8dTfeBS}!1CRS51k^G7CL2cFoLuC`Xg2@G#IRw6iEd^B8wtP3 zO+&SEi~f!{M)>B)kMLO0C5&)?{T0Oso0}#Weu53dpGG}4Ps?JT$7(bJZrmi-u|DYK ztiA!SHqZmLOSqB4pTbMzbMe|n#qGHPoOrEQJx;047Buy{#MZ*Tj1qnCPW&{__ks4W zH~K%&Gg1}I@l~yJZ_xEiUl=OVF`(68;}&0cO(&jVtBN!!DI(W%y<{LUvcUQozR-K+ zj@a!tbMNjxNP=TIw~j|F4-h1U31K0no)G@zjQQGRD?V}ksQ$j-AW#_f)-zmvPArdI zvAlD&ktY7?nYEW(uc5Os+HPM!4$f-r`jLU_6$G%JU~TOXCY;*)b6wPG57}2XW!)gP ziJWa(Se8?mO0sOUakbt!LHo&k(XqNJf#~J`!p5U$jMCOlZiGUhsX0aode-^~>&#Fu zZ|cdXEL3=v%_d0`SSB7uUr`by96kPK;fcQ~rQ_n)!c)0-Y1!5bq;>8my zTcFotr!c-tB)!(A-k`gmj_;bNNI&>6kS3tFba}&Vp2p(6vI)s9q6e0U#m)qD$H1Mua>5H}pEvd(5J$NLR#MXV_jRr0VzZKfA zI$X#=cVc&@N?MpSt&^O-;p6+3)L|d?f5tSWpb7U&!8z#f8_S1-*jO= zq-BXg9i)kwmj}akwBTM@Mz~F0y9ZXBvg??n80<@n8Ll4UBX!N}h+{1_!U9h7XQy`# z;BEZ`bm|16Vzi3U2RAg*dW z(}&S7L#_=(x_M{&)EHlq+t%M6A`weyLQtON@s6w^=CU=Jkh` z6+#N6;+I=BMdC-Xmw^SjfwaCf-}V>>-Z`B~CGNLn8)3akIJVmOa-Y?4inXlVM#~RB z{GJh}?!?Bcn4Jw#_DOq?%=k;E~gs?apP(p*b&sG1d2eoY(L zk)G3*ba(Gua6L*b6}498^UE5SBI_YyJd%pfaaXZ=B|}fpa=N6vE6$qw-1!$b4c_Ha zCBx+}is!s>PyJSE6$0g9^#m>@^Ixm~5nML4dY_&s;RqL@%TZ{I(iFnfPw|4|Z zW-mb06Dlf_h7omRMM!}0_Ygwn-UH2_Yc=Rp#mcQdU6i78R6CYxq0dPr=6cL|QBiMKVhBXK`C zP;&<~nr=)k)ZHTmiMK5rV@e<=Cz{_r#x;T^V)wpX=@Y^{>aS+#-{^)S1NK`Y^;5Ki zU)-QiJ&)R+uhU_}`WIg0%8@6#ZCZ(=qsO?+3-d``M~=y@jvLWc>7Bm(VHb|978_sn zjK7xF*ur@K)%~4hQ;%>xFB>y_#fsn0nqCtcli$krCwHDDw5s1N{Fv5G@m91Q)TN1_ zqONRi5KnER>g#&tph&qzmlA3#vx~EjMCGbsI<4nQefmkk)3M%@G_`H!l}2?Rsl)oh zyuLbFU_S*%i$mMZX&5drWv=uDit#TCTkBqiBdbdHTBTjr|Z}734D@= zEN{Liw<(I(7jdKzDSkxaP%k_6f4+Qk?PqXKPcOMCNmzoP!iG~j9T>*ePNqcuj4d+Y zrJ=qiamKTh<~CtR5=UaWXfUbR#3ntZUW!y3t`%{GC!WerSLf2^o;)1wktQM4-Kok;{_}A4c1xIEzCZ0mLFWD5 zl~5|KQ+1P?C1z0AI>PqDxrMNxT1wTeJx=OT>!)*ga1nwtu14^BcM<7=4m1jDUZEY6 z)-%k$`V-sA>ap6{e1%q1USp-f;VK0D&}PW+>n1)*IMZUqWq#7Lf^<<}41;)<%*Z0fJD z;3mSPS6na0Lmc~Omf2m;s72$~9KUINjcip`JwzCQP$;^S(m+CT>^sR7t016&2;{o?|wdqhxM=|7xRCGg#Uw4$lSp zS{QhiX7|gJkYZSd<&wMbr2?oc!|B2VCwhF8ak6yXlDimNQ*OuW^>{ z4{{ZKpV5u|rT_Rxw;l6J%s7Z8C1aWIc8hkj-zE5PMW5=k_(B){^6sgp1ALgLvCH0y z#nUj*`BQ5$c#2f+e$!gX?J8k@h`u-5?-}{VSs{8J_ItFEr&yFYWZ0=FY6Y1s4IZ>8 zN&5GpcX2pTLwgeRc5IB!t0h6g&#-gxmL{0B(Y>lKV|L_y9uwrzzin8jD>);9UEnsktuXS9R;+KWIuI`5m1yFT@SH0j^N zmX)}?+~hpdRsz1}=3kGk9~ez}T=TvtDhaUf+&8tB%!~hX?&!J2leBm;y`g}>G#_nJ z-OP|d{0TZz+e>chGaOXXD<0`PboS%U7+CFDYv7}qyTszB^4AmhpFXdSo3~idU!(O= z$#Lu2=Sh;(J**W%KSI@%SN-6f${1O}?V}GjS(NzYs^X0UqxCpHn>e1}gb2c?w|O7( z*7%67OnR~1F9{^+a;^s17fF;P&3%oTdTn%S%kRE1v2fzM=?o5D}v~ zV=a?Ceo=)iu9xx3tL_`XQz0Sld;$01 zcoJp5f=>M3YJFTIuBhumv^UQ2Fh}UI;y&_!awpio&zO3CgvQ`&h!A)4eqs%^!CTJ4 zVS@5T+0jBrYpidJkC(ep4x@%32R<5qe8_$B&vs^7I)#ji8wyL^m&%ZPam(oF$7&p>&A@AVH7lBD!bsp4ft zJ>u37Zf5hVDCn!h<0m$+a6dyYn+vP^5xzz!n=wpeP>x)nn*TFuhTp_h4eSm|rbMc< z$gX{n#Y#uNcxdZihl?+hrT;FNV#-6iSo~ZDX;^dBZoZW!!LjsLiIw_QQ2B$`@w)qN z@=cDa=_8XBbVM;;BYEp3C=K_ly5-wVxJcO{zje`p=r?-3O+3?yj=}H3%nP3uV&hKd z;&gi%Y(KqX`w^BmXO}xi@N697dc*fRDc82NZVFD@6F7tzn{9+z zXn13N&6{h1HhRG5%+#YaeAU*;Qo4qlw624>DO@=>>HOAYnBN}QMdRi)IxhrenABJY zkD6B)bx1(Z}O_l4F48dXzu!xg(V5UcNZ^5zUqRz z%7K=4J+xGK5qiW}8=<}Q0HiW_Cc=J0~L zq&&qc%=Z?Jx&g*<1CtW$dq~O%o#Xw>c$ebaSn-UMwMF|}9oNHDM5N5+%Ti$X8G?UL z()?G}8Jzi}62;(imLyr`^9Np)Ptx&DKJ_-1)T4VgnH`mOSe|x2X|HaN`YL&fJ6A8h z6r&-kiO1E(f1!uP+6VL@@KnEXQE zpo;Tw+d_=G$B-?LfYHC=miX<@q%Qhv6GOiw(8Y@`ZVbPnh+%vy@i_d33TjYC)$Sx( zorN5T+h%}4LL^)|Fk3NHP8;6Kg}!QUugTM!BjKa|1RnHnBNX#Y-1+QM0|`zL(oNDI(^Dh=|b^l+n%0(f}~NZ{p! z!*aYR45;P8?FY25aD5NxhvYzV8V+(sb>Y;i zDKz;6!O=7=_~RP{`0^xBA7TOT7i5TF+66ZZSwZZ939!xZf8S(RYu{Hp-Y z597dES{8n3IK!(D0>p~N!Dsn2pnw}l+uMMOwh*X2_6KPT2N0@Lgl2jfi{jFM0yXxx&|vGG~)j~k2iFN@qQ0TFO7lK$NbO~EehdzXTi^Jh=QJ4~*q#E-jsTSrP)l84uY~)D&Vu! zK&~hV2A{mWI+F{5! z?+qXN^}(EM_D`DIlDz-N0jPuafXnen$laL?jyK5g%P|xlUKfVKcl#mB>hONAB~vJ* z2!hfHYRKJx6c(?T!Ppr#7}KMIo==B?&%_s2t`TAOJ|4dF0i4;x4;*7eD2d{O(J2D( zpHv5TWd;}>+6!U7BEZhYlMuBcLsgkd4{7#R!0~_+Xw=BSxvmRlObSpFc?`ZOu)r-3 z9Z(%Q4SuUZFm^}}>c3dQR|yiFVGxAb@8ZDZbRP6ABSG6X5FVMJ{gjUfp>rk>=ymu0 zlb-coZa5ZA1K0))G(O^hgRVPZ9$B+jw=SFyr2y7x213jG;B^Ew&>cDnJQM)s!rh=E zBma*-sc$miKKps#jkg3hmWlx%q|@g8IHq|C`EYj zBnsXHr+{&}FepVR1M>_4sFDZ}bd4Iy&zQrEZXpzL#X-Re5msM{03fYZna6{3Bn?Cc zhr?29D4aMI1yqlY0(TS<5|l%b+}&Ytk2Gk!-U)6~DG*(K2%^ZcP~CU}PP~Z#1e>nTmm+y%pfe#0V=)u!TOvlB)pS_k)saq%1Z_uy^sa=kl>!L9i%xifk%H1dVHk)ug(HdP5F>s1AFivY5(KwPf}*t-e9GPjsTOol zGph>^Ls@}ALmL8HkX#lMEMoMPOW=5foek zV1AhmUb_gw#J#isa6NVDz=RReD3~rC>;>It(Kkhu|K;5qS7~;iaT2 zu+p%>D|`+pZsH+pXdmdUD#5@{esI=~24?z}MdETlj#k`p8Z%;D2b z0Cy^B;A+=}BV3w5S4IXlodaO&EeuVHTEO^29rUAwU`r_n*2kh@EpsoV8(D+bOUv5AgkRLb`^U-b3_z$ubG3(I|^`d8v#u*74U}a0%^Iukh+EgsXSwN5TgY2 z#|7b%sz0dR3WL{jUXaZi1SrnuqD^nP{~hOz(5Ck%Itv_H2-@`KUb*&nYim2+Kjtwo zK>&YR8pHqO*pB3aD5PUFOAe9z7~%HjUW~Bviy5_dEB>GOzp)qXif{hvW<{X;^r9qc z-QEdjL>DKzxDhJf35v_7ZH|QoUwMroxLi4Hv<-b;+rT#LoZ^wgILKdXN#PiYteY_Q z5F<%y8elTpsK6F=6DTVEbN-OlIz9X~e|IZ$D6pA*354 zf=3WAJ$K>?8c5mlh%*|t^d2KOy32}W^g_U+wNM>%C-r~82(sj(U9Y$ieh+7DE2cso zE8gToW8F-5=s=Afxebd5TvH#|g}U9=*{z7cRHHK;0NY)VB0?f5Tzp3mCr`3w5z&*f zCzr0FUjD%=QJi{0OgKJZBL-X{rM{01X3 zLe_ePpIuR};-!FfL@>i+(Pnf`?2lz$!$+U?$AM~hfIJyq;Eb!x;Bf1lMP!@y6qzcM(5hsV_9)61k(_~J$jb=5K z7Mw>sUd(?Kj}WI$r@TUqyH;z?qY;nMTtKV;9Vz(*(VV?n^A5GeJai>IC*xyV@=@Cp zE@nCG5;=pO*9~=k$kkhjI;T`{eM3NhEx8m4;B$KVJeoR=HXQM8>3YT(#zAUg4#e0Q zp{63f7%k2q zep1hwH5Y%<$3cI+Z1R-Ep&B&sWMg8II9XN3YY8#_BV&l37MYuSWLxVM^R%fjGZNhb z6`_aR$#X#fwN>726-G;cclw$dn(E*mdu>F<^m^&GOav{7dC1lurkIG0gBnje@oXFHwv$7P=$l~1_!wdM@xBDainz1GM-YnAj)h*NeDMJtBK~9--+4MT z$ht7KG9s4xbL?JpSGtgW2!TBw9^6REG|l%N(U9NFfA%4i1ebuF-l!ra@QZq-=TGkcqJEi!bZ4Z1P(#LDp|M0{ z#r$e+L}dLcWn)JJFMO@fMil7A#M+|9$rHIp5N2BM&!M^B}SlRweC_E|z{W5+smFrxGnt zr^%JO+nh7+BNC9@YQC81#gcB%%OF)_?YW1Pg!G*~023oaI$wLAKUvz`bq&)ZsmJgo zwUMXa4a8#tbt*{1I|r^zbC7p7-O51-{R!v3A|lim zT4xX*6S3nw!eb}dU3MbBm5gJ-V#&`Q?G{6iY+o4^A;cGTo|0&ZE9Q7)wIVOrLXb%o zxXXzkjPkto9|+_0Q@;VUQq?yk+>Z&0yeIE-;a)|H@P83s3YDTTGsBHC8@nGN$Md7#UA;A zWUOX(bGnf9sKTCLMmBBNez85nu$LYpXf5YjUk|B2zLyY7Hsp`5M6BSz@rgV~n$X>n zt&F1ZJu&hzGI`;W@HE5 z&|gS(<2LzS5vAo9g9-0Rh7~Z~PZ~;5yR1TP+WSrbJ^$F;fyZn}c_n|C`$?oaDgl=; z@+YeiQ8dDWx`+@)GUp(d_a@k4DkQm7GW^^ajV*V3>@0u*$2uRbeA&HZH11zFYw;j(LrbpyBNc{_6`V7cFEEuy9 zbCkY5ibG(9@tX=-QQ^6t1QqfgP5$2qpVm|8GC%p~o>DJFv2y75V6-9{mkj1M8glhy z)Da}+GxwB|-jjnX?IX}+2aG!T5k_b#PhSf8ZKA_@)RuNO_d063l%RAug|uw$hqO25 z;8Zx025zT05)`?0R)~lIqo2fg%%41euW6rv4Jn}E3iAjJN%7zc;S#x{sVW-dmop=O z^cYY?l-`xfG(r6pR08RdBxpbVp`g3EQVRn#U1e^SFB)w;Nm3X!p76EW&YP+|4ONk5 zo)-PO1Knj0KBh(}`?3sY5eS;(;KXFn;QJd+zC^)>aicEjIoVV7fHaz^b|=RM8b-c7 z3h|Pp5`V!3$P1RAx3k4!$%A;*{V2oN{b+fLYWqc;$Tdq^3uvC6$z*oSo$PTjPZ;$w zV>QvMAvIx^+uqbsn$;eKhx|jY>yhIa=MU`@Cnb!s5)8<8q@f|iWcQ6!6$CuywY$(H zFVb(%p-I%n=~B@oEAsVyKyK`fyMZ1Zm6Dc6pQzY~mY55fWMO&V4CFVZoa&=2#M$f9mZ?glIll zJj&$>AsJWg2Suv;aBBEM` zNWTM-;o_9LA5qBN5{d{V5mJ9nZ^F`0nC7OBCd^jmP|hUA#6L%yBg(_i5QwA+YyS^Rru@;3X;e*aDSa) z#0=DH0-a(}WQUN~d<)wl9u*^->O0P8{O@8!92Fxm52{XQ;J1sBSXH~Yz>V!5Sy`e;|9eG81^tfcMja-U(RB=42)Q?s z*@c6z!#QB5fCn?x#LD~`98?Jphu1FxG|vWjn-FqPa5f4)Xv=F`JzA48uU7tF4)ogMkTkoMQl@SLn^| z=j2^@@cYh2WY)#d5xMee-3CCOR@AsDH4f@CvY)Z}AXZ@DdJM8rdHg^rHDUsPil)NssQ8ZW?rZ}r5cf;k?PCvCEexp3ei zl5@J65fAmxS(s-wFtC19ap6ZU4qkML1%9!?!~5os2WCEE&=TyZ@%k|iICaujv%D~f ztG#wTnH>+}-(F_GOC0!I7g}D|Lj1U4Mcaf@T{Abc{2v$wjDHTN?4{tqi}b37fejBU zeESSTui@Z+&10*udVrBm4kir)q%Y>pvUiaFI=NL&ecFe?!;kDKdsC2JWH(4h>EY0Z zZcAysDh9=~H-EXE!9d#V!s&5OJV5>WRyR99Jo)e9Da8L9&2QZw@590GshghzlrSh+ zR2R>p18{kD@vzBmJahy-wbmyB2o2D1iJIc!B^6oM$qWavoAI`ym1tf@-kwEzJnRyD z#8i)VPWCOC=#1~cgI!bscUe6K6_46ppvu(x@#D4b4=fEK-+`zL5~x6A$eV5 z?JwAAiwD)$zG=ooIPi8awUP3{!*7pL@k|8_R{H(BKhFWE$z|MI(!e04gG*;x2SC7U zy7i+t22l)=6?(%sQ272-X}J#Rp$|Lvu2g_`l~*{p(3V?%>H+HmF?g6O#|Gb3Vc-&R zPIov4gLBgk0gn1eKkXkdpnYXf+atjFO%2gOXM8qe4Fmod0gWr)0BG;K-S@KvDD#%E z(iB1c+ZT_N3Ib>q^a@^FMSj5DxvxtDVAEJbTV@^u=L^g6P5TkQ^c3WuzQbT(A$YO5 z7~o?(uV;NK2A;>v-wq)9$1*>gzsmr4af$X+yf+Rg`5po%2pIg>OMj6GUO8H+pRKBQm;pFqJ+Y@tOw$wgaM=oi~?zaaVk zXD7e;RfGpTF}Of@%3SY;!LbVq70ML=`~iG-UZVs4oJOhp5s2P|r%s1W#BdPd%UmQr zg~6#Got*}X02ixgaBloK(0r_{%(@A{MZyjh#v(h*{}w?M0*H`7?0YDDI71CSqWSY*731IyFK7mLkt zV60vf{!1MPQ&-Zi_EPX5CTv{$4Q=zAb3Ht#hW4UZ4qnYmx`Knj>JZm}n;6{XQOryN|cNEU{qVAlp6op&vlS z-RG^#NgV98R~hGPMB8ykq;8d40Z598+z8vmpncPME;&4+Rv1hxG+GMbJ-Jdb8evmOR(OgYNV#{tAQ z1FqH~`#5`ureiz{2d`F24s@4eU|(Ol`}%h5{-00o#bDJB$UmRb93vh=^7>Ej7EV%f zbP>I`>!f(!>_YKa)Bf9KJ!FUXzH4ly;Xx&G!f93pgRN9=^`cA^$Ch5ls2StnJ-h1l zpGbdhf6{pGT7-l3WrxSd7EwGYUr5>cmiw`_LJM!Yu;hx8% zy+U|cy)856IfKDb;`gY%$j_T62ld`Tem3~^-?b^k?<8qfZ!HW5=dXUCzS4X(vKN)7U!!T$Upt5H|IuvYNy~}#CZk}MrI>Jvle*x zLak|4$$|%&hQM}aLmW_ltGtL$#z9~YAL$`81`jeP4zaf(e{45ry9e1jQ{SiKJ`5<| zFi43~LviBflGW@jWEXS9BK6EbbaYU@QNZ1T!C*`#wUY>dXcgW345H&i^26-MD1JQZ zzmfUn1_rNe?tSc)$6yzUDZGaT`MrP~ovtVVZU0?{)~5jWK4jM$jKF~qLGMr{io1#Q zt?cqhpYBD^U7SKMZA(AuQ1v*D`~b(zujhL(U}$J@>p}9Z%jP#ff#Sr3+3K!JBFbwh z6$ckWkzQ==&!&5V{PN5Zh8U#R>VkAUcF14oAC3GwiyWwntyRWjTa;&+*jV-TT_}dp|KE|FZuw&EvgzFrkqxMQ?@xz1C@8Szn}2Zc8WX4+8ulo7^qQ zMR7iO2fIQb246kAT;xOW&^t*d_L~<2s&h$f=rj^I4oc8Epg3&sROPfg$_GRw^G0~3 zF%W*!`90zbicf#tp6I$@kTXUvc@xFY==4OM*LE1_M@G^J+2Vlex&rNyDwGFkn0GXN zLV2rjsZZKl9Awc%%_kyx`Z$-JbwDpTaZ6r=3S*QGiLcb0|))zXS433xccyH5o3rB4tO8rmuo)< zAR5+-J}34fa0-LnqD@m1oHo3`qi2!-!TsHK1h8I2S@WCZl3SJV8YWlmW=q7 z!zC}vREg4}ch?#4h~N8^yu|ZBTD6dz0x&xUfgulJQ?;nfP!) zQbzeuLnz~0h7*u5h=hMjJfpAPX?V}FBuu=T%63UDAVGj2v zCjkTvuSimDAU`9R@W}(kb8*Q(pH)!Y)Xf&vOZ|c36U|HaWIQ@TleTIn`{3aE7SjaP zZB$1!sp$8i{6+2YK+8PRn<(*%qw%R2NSPd}aX@w&{79WWXB>kW3sHu%C@+iLEwS%2 zvV-6@Z6fXu27Os~q6IZ@kj*Z4uNT>G-XY;1=I2nniF?IE6^8uS+shVzP+o!RCuS|m z3Lh37+L`}56eZfRGCHc4x0>^iEE?;Xd3{*ukhqb2>V~qOQH&KBn@y$qch7J#lPa3XIBZp5b@}!aZ z1akbuO_OcI{*fcs#Ds#|sR~@gOM5&Vo+&mY382Ki5|^>%jE98#*IJ^GeTpq%U z3LtT&rQaxU%iXz^`X&LjM}DdOor(;Jwq%7#6%X0MbTa}&7&w}%#poahKO@##Ie-G~ z)x9^Awpam_0uTE1A-diA{QfqhMCy&#)F*V@1Xg=&OI!>wm=iQuPeTmS-)hqHj>ZGE zf1PLSVcd4P-Kf6fy88brw>i+)UH*MyIFQ8uePY?j)7ZhzlpW6>Sm>Z3~@quZQ}>HfQY5Qr*u`sA*=zo@iv zzgIFI?o4JnH)eX=AV)g?Zrdql-+cPn%10{S3`F~zj(dM)Q>)(!Sw6SJZq<9Na+3FF zrNI26?Kl%T)wU*qPow9Syqa8t*|5AZi6MFGQQGnNNe=;K%S%y|kgk$r+RCCoUU6Oy zJvB0Nvm;C|N$cqA+fMcx3vsl5|7VeC4d1F6JV*&26=;?om@r(fU5Yv-@(uT>SaD%z zsZ6G)Z>zB5+%GZqvp2q06pf!fAFc6#0V<-)HS`X2b*OBHTDaNLl6^1zE@~=lXw4nb z;PaDIr~Pggvhm$D-HW<=2*WzpJE^Peg9k2RY7XAD=Bl?33oA8FG<31aZ|Z+;$o2Dh zL-+TU`>|2s$$;gje+5isBV;ts&K_*IBcM2B7BrCQIs1oaG$pBsRYv}Gzc)4vdK$@_D(fv0t?V=eD9+9K@`W|=b^zQbNa zTkgs2CKvNsB;)XR#Ac-j!qWtw*_DW?|2?w|=DQXshIAWWuZ0(9Uj0u~&NBBmWE6!e z*33$2J}{LC6uvup;Y{!hznXW6=sV#9tC1V88ftbtOU+^}d99}TBrJxOkHUSRgt2zL zy);>XwsGwqPuhFEDNd9K{rmjHjHVX9t#!dGiVGpQ%ipy1HO&X!-8)iGiDDyx_i}iY z4>O$1Z@wDSCv?Iq?dyF@ z$eZ`x`t3z?zf>l6XxE&JPq|4N&4>Hc2KMsvTU&%*{4w>91K%GNtGlfyl7gt{uYR0p z_xin7*RqjP^ZJvKq-UHG%vru2`}X2Rc%CG95R_N;VTb`OJ9Wy&26ta0@IH81oJHf} z{OzOv2l`8lgMyzn%UxKb<%#hII+cl~adZSs)&?ohv5$aY}yJQs&TA`%=ppyS{O|M04kn ziBX`a&OzYJ3-D=e7x?g+_`H_VH4|>I;Agj#$-#u%Jsfsx-_e}LGj;O&H<|EG z;#6uyP;^Au6{=49HU*lp|1eytwgM_b44U2L(d zZQ1$drcT%~{)hILjSQC0OA1}@y(GK!HT9B8q}JlVajb>D`H=aJCm>L^1_DcC(KLoN7WvS0*=9g5(8MtsXntJ>#dA99Qjb-{!NyUlf==4 zr=v^P_Zj7IxK1f_D354G3kiP2Hk?9O7^dl&Pm24_*HrRZOy*k_?8uLmCsyV>ZVJof z%MUWq#Ii0MG6}qN`=wMaVDd}tVWaSaOv?}F+)~z9?o{{*@VN{gvuN>iJ+nD0eD>1r zz8=x1vPuF$k8&eF@89B+C_6H^9C}jcxUb>8j^q11d-wJ|QGfQ9dinb5ibC4%HIMc@ zhm*2=(@TXn1}SEO)P3J&jSkVPy)(4;bdr2y8@t=B_0C8TPigUIY1tHh=4avwE4jZX zBlGWx^KtwP5%;aF?DUqthc{M~CB_?jltu$OY95w< zjqg5y&fSGio!iT7H!%33Wy@>f$)f0dbgPpk-2s?A^Xq~y=e4vhXStC>x{nRc2b`Re z2pTF{eZ%#zeap7-ZJETZ%j#{a@-C}>pRKs0<4@lv84Hd(5pPbE%X3%!eLjfgVc1e! zjV|@1{yLP>N3*JyBvmsF=?;1P>bw0H0#i<<)i-JNbBjh@-ekFWPr{+Vbws?f-^jVx zF6PuDy#u8?GfE}*w>!~IHlP2hY!F+&j=OQbuUV+_nWF1M(^D?7)u+r32b8#Yedm*O zzrgqMuja6@5DgWrQ|j5aysQ`Xeb0te+>HSAOjr7evAYk&;Y%4+ciXI9G!O7UBw*DN zDZjAKz7@CkfHS8igQ3XuL5X(h>ZgXwk2FcflWZMttQu~l(v)U<5-3_%jbOv=g{?5de)ya5hCP;3aV%nqN{5r9X@$|%b58CPTPTy}Q z*A`JlJeHZrG?|Sf{Bk&ZD()TqiRWd%=k05kt=1K*_6gj4RJ^u1)8X;y^9OaeuElRY z+Ksw>N81dZe^|aAv$(VTUclc^uUI^PX^ckOF?<_*Uff>Rp{;l3uC3HZzNf77g&hX_ z9%jIz*NtnA(n3KUYkenNU-kG`H}c2_n(-)o=HgT7{F|e%GMOQz-d5PQQA^txTKzo# z_V*NW{**=0*nn_@)9*XBCz%|>*_AaGX!mwTa(wC?R7;I%9;rUpsY)@`QF|Xx%__Gj z#V)^8&3WWqWT5M#Mw75CUQcX%Y^_xF6;;kHtD2q3uEJLFHS4~e0v5+y%~$PSnA^K_ zR`R@Wy0E&y;oaUDRZtMPslj^Dclvr0X$^QsZd@>+i#}m#DY&Of;4|JU_3@>V$&r<(+HETEVzT`9)a9o{M_T_7cmGn7`eP<9IB&e% zl&V6l6I5Q-p%g0rtIWpF?0L^p)~)}Xd+?|JgaU)Xkfv`tex=idd-c(}42+Kt90{y# zpKpKq((o;v*43PVx^wRW@=h(BI`&`OjkTSQl+8GE&*+E#z1 zbh$3;*Qt1&FWzQ{4+JFKU4DGXg6}-@skD={ie5S2*O|KH4m}W%BYkp-HY|=kzC)Ys zYjeBc{CmCqhj<(keRbS78DIT;7|XFg@z;knV;9OY;;cbdCVTCcYzygm4nL1nFclv@}8ZFvWjG(`m0l|zgN)Xi{il~ z=R}9!tq(QcZ3mO>&)A#JXXrf|szoj<>(1R{G$(R(0@t_NTvoGOIX)_F_2QmcrUyQ+u9v0k&*jQk&Y zz|NX$%DNlcHXb!w;h*QN+XCv4CwZg1@k}gbPp!Q)&&eE9_A=M(KmXx3aK;E9De_@Y z${#iQ<@x1adhc#ayA_4bxS)in8k2MWpVfII-q9a@Ho;;a8rIn)mGCo9@{=!j@`3Wc zH)2=P3`7hhM>u*L%B(NMe3^WCpiA*CEyG^HhSg!GElb5eH(xN*#y^!w@jWq4)inO) zTkcEC&Z0mA?h0q`1I3rWDh7WoUN%^pnF(>7&wDB96g3lMw_^wG%nQ*=5AL2f9ucf;1U{dwew@InMNB!+pw^9UU+AF-VbRDUt_?dQZ6W4~N!aLP;R@D=Y{HNxs zK40I}9rB({gW*~_{sFD5A$xvr@>YBF)nn}NV#$J?)2+JB_9k~Xf6!g=&dnPD3euR3wJdn)>>2~nO{vdzr1>y zxi#qHLHA=m1!s4i-t%LBK

    Aws@C^HxsNzh5tJBW5(0)ZTqP-YLyC?O0U5*{%L^ zJ|*8phc%sYwl#VsrSpkEyi(x9{E4vq_s)_sRIP^dMP&;rlA&VRDsp?R7&~q99XX^s zL(esSbl0Y)-4no(Rf3Kl;<@N?3%W z!pq#Hx|1t3O5a3J*zBN^h|rn;T3g#|p?p4==ODvaDJ^wSKIf+ho5bDMg&OlhIkZZ% zqvn1z9dK-{oV^r3F}l`KA@?y*Xgv=5*$KyMe14CHUc5tRKpm0Z|4B?gmd|-`WbOCN zGa*%Tqkx#D$+kkx16rLuj=!BujNXuQc}0k78HKO=3nQn;8Si*xzWu;3^1#;mkb~jN zDVBngzI3&u)yU9pGNaU(6uC5dsI;a-Lm|Y!+Qu+g)z`6^?KMyPwEXn@hbyeCnhKMq z>Yp`O63l32F5SE9M89AmbAf&D$FPDbEgH!iqU+i(x}SB5_z72C&^LQ?_L{1$dd#Jh zibp=yx*IXqIYw0;R4t9+ADWnaC{T3v;rGhxJL`sN*IE1ra=A`3%fzM6DKgpIi46H| z6ZYOa`quTT;|AV=MGppTt;`n`JcWhK)M#IPeCgZ0r4&_jznGEoY4Bz>Thyc0fZ&1G z!&Bi>FJ>y+-`49KSf~9imYyF_GcB93H~5j&PL?f^r4>WX#WrOQ=Hx6{){~E(u3I{G zW+<~sYS=K7vQSk$#fyIacU4UV&=bRfe#dUHDT@EEJEZGP4DM)C7tsHL_|A&Tr{HMJxvtYf%o*jD0+8{1#9+5TS=%;-XI)J||97>^+YA`4rDJC$4@ zyy0V27Yv`y=gchm>prHINl@Kt`#vCkQ!06&PmDWvanHN> z)V&;@mw)Rt$vvJ93Mgwl*>a?v=fn$>ao4j(hP}-xn;T56tMaKYYo(GJ%01CmG{lBH zzB`w^tL@}7o%E2cq({$e#hy0CRu;G!1BC&(F{+w)i zl&m(`Pv1X+8!(wBul^IKM&JOAL3d&moa(JULs zlLA)+eQy0jD><+KgI(t>BS4-8p{lb~fI&ajb8Dg98)wqXrzjwsifPd(MHp5cF zJbxEXe?g#>X_t8M+9PxMywKWCv5ekt-iC8S=W|1<3+Xx;sDe{>%S)cz(f{VQ;-Q)! z?J_&@mlnrdO^g$Ubbku(U=8^SKe-#<)R=DzZs%#A>Qi+psl_{<9^cxE*{a9K(8kLg zFJ*G)`g(9a_d?Z~{Kr(9YP?sq!}KKc3OzQ}Ib!9eHRSk5O1=-Pk-ogY@zY18jA^RD zj%RW7mVT3)oy1a0WMI;o^=$TF_YylLclU7N9M4ww=r1)`dx&D+-3eNc64mbp`ObjACBWypKjxYo_lT@-oSdj_}QlN z{GiUE8q*2$vV-sFhTj&lkP0WT`Ok_Q*ZLN4i8qV7eQ-4Mhb6%~pMxR^VvvYf{_cC3(cbVv2 zuE)C0Z}a48zwLV?o0D}mp9O{V<^M)4m&boCl(yQ+?7!&s;y_B2+xvK_FWx>Sg2Ah` z`#bhqISI2D=YKuxq!y~&BKaV{f0S#A@dwB>H_yGTg+*C4!oJZb}%Y9u^R@e zCr?@a*1aux+ZfFi!`GiRgJ|gfnvlNv*D{UC{QkA)Gu@8H0?&Bb#H(-P%(a%%PgY5m zKE?&VKRnVOo&&b{NyXgLWkR#`cHo(3 za22_r@9AcI@JCipf$^&Cu^VnYi53T#h#q!(kF@SnIGIpwoJ!eexXCX1d!e;x=u7dX zF|&K7tnDvfkQBH-v1^@~b1_VjczhXmQ$YRQsddZtw{K~^|1b#zm&6OsqV4F|7$%c} zLO-3)&pZp)`B^v;bt6oZL-T$=r6`wO8f2HL(+se1`2Tj=sEB{Y0@ERCy<*{LQkB0^r~u^F1O0znNgf3Vqe%c&)g!owz*z-$3_PJ$F}ndbmppA zO8UlX#zNE3e(u{7>KM}n_(0+JW9O0GATyn=I`G=0>3F6t%uWZ1gW^%?;L5*se68o6 zp2&N6*82C)$fGK$pKcd2=9QYXg3Nlz^hV-jt8{Npr17ib2Omw<-pDW6ZE}|JM6+t> z_x(}~3v4cWG+ePvIXQKjZH!)@HOO3v`(dgvi)VvcG7Sa$YBY3iGq`%?SPH812+t{PzE@_3KP-v}kI7UpYT_?E;59<4(!os8`!X zYTjIFu9$#O!j>Z`>S$1d>*WU8Jk-)nxg@1pCj@TQeB%g=ch-Td17_u-!u27PZ; z>Epuu3eEhM2mOlQYBT@!)KYX4KL5HpottMqcnB~r@@316*s5EgG;lxi%dC}rll;US z96k1)di)vW2T*vOu-Nvq=Bk*?gSTbRE2H#lIJSC5obx;I%8t)DYpzm4Wm-SK;Bzb8 zS6%wRJry=W6+Xn+4->-=JWs~UEJ-`Oiqxi9ha z;P)*3Rm~rHZ26tfD@!b8>UNgLwb*8St^}5~4fdb5$rx~zc3GU=X_36cQu5s0?#&;s z?dnY1dVZp`VEih{n1y!ED+~hmOCx%b7S-q5gD`h z_tz|(x^dE(TUxwnpl~UEp3n9A^ahQmZr)yA_kU$-h4xP1Qj-Jqs-~LJGmZwmcp$_DZQd;-roCu^?B7O7`GUBwJMB9Rr>b=NgO54^`Ss5Z z?eo_=*0#gu|5Zn~&A(MGCTgq}WsZpO%&x}5M@zil3-4bH&EpQ zwg6NZM42FbLm;0`ln*X%1muf|iop5CK)!^i9Gq{$Aj$#JY04n>f?ZNZGDnR=MwCU; zl!J{y5^RGg6Hr2ImJJZ8aoFHEqe@Z+=nSAMfbpib$VLX}X1D={5ChK0i(QZh%GsEq zZtO+Y`6(L3Y9olwH`ZdajMP3w+n8bBDuir8Pah+S5ylWBuGxy+vsD91+L&Qq;fHL- ztJA1vm_W?fU@Nx9Rt+d=L&CHLvbobbnPfyc5CLNfG53hP*b#X(prj3`$%dohhGdS| zoK2t@VUX03kjO7e*Gnu&1f~i-a01aQOD!tS%+HfxHP$oGGh~p^#86ROn4ij}OQtCjK+b3szT zBnK3Ss!K^t&dg2B(J-{oG0+4G=auHX^@f>q`Og4QAz>nE@_bNM$$k+K}ks|K|(?rL=*`Lr4&U( zK?Ff1{^6YSedpZg-upk#-H)5KXJ*ZsSvhO%dAGVM`XV%d=8ingJ$V6|6(aZ8$wZb1X1fy^dxVM^yYNW* z_ZZDqi$^B#QMwr z{Kai3MJN@_$>6~pU>#L(Nt+j>@xwfX^E>R0AmrAk<-u2zB=$E2!Zb+QNrflt`bZ_s zG1aCcTYT&jWMr%oP2N0-B+_hO`Y)@N;8};>OR&R(KA z)Q*^S>>r$zQ(IrY+5`0JMTKF*FgHM}Zf89Bl^cA$X8MhPRS~sjZf1UZ^)`vCBkes? zhESqfp$HzuA|@h@I!3dlvKN3?>s(oG1wS%=ImmIoq6_rv(f6(0l>m|{KttpEBr#)m zP_$di6KvM!7mvT#R06g~v@wI+UqRc33CsD+pD3av&v{$=En?HPAxF-aWuX3wT!ZCJ z7BUNltMxTh4?rvTvF*Ez#8`{MdQY~P+bHg|4EnoP{1|sLgS_aGG*W}Ya{s3R=Sd6* zWQ98D*I>@>Op!6AEAVb+zU&y!hTIK_TKlOIPDGr<@he(ejpzb(jz{}-3|6|hHQ3_^ zJy>Znf4=NCjrt~~?cE-hisAJ*yb@$|fHm8$8B;X0A=47&9KKWBj;q`IT*jO)N$P7O z|48^TmQW4CuNAR8CSk7Ewn{q4!x}7xPE2KT;eKp~tvyUVK>Z0J-0fb-BJ<{#Q}D{j z26BNL{H&@*_|qY~xo$f~s0cIz|5wcfgj#LzO>nXgnlo;t>t;nHp)bK|bFTA;r7~fi~Bc~gL{&eMxpSH{+gs=GpOLOxH zz&us`{8=xKtSN%V_MJft2-4>fG&1rg>v`JTiWfLVB*T+$`2PNB(odZWzjn2l04EQ# zm3&?aN$7&IHQPoZNpOwXPijjGQjw61SEF4d7`2BE=X%L8c*`NuJHObt33j&?9EOE- z$$ZxAp4F-|Vy1(yvR4O><6#8*$1B@6;M2Cd2=OL`f@JOzImlh3|L`Cn4}i;1K^sSD5r-82hPIA=t16LA!)(yn2i%uap*CT0)?kfJ^HIzFO#E znV&>aO4?*JvGg(d+`f3P2X9<@HP~>8%HaJe26BSQZoN+OwR1Sp+!}So zW-HtoDXKubZ)F&2&zterURsc_W_Oz$n8XwLnsQwT+`W(ZQYcqE`pALvz7_ZtGg(Wz zw|Gg;=%O<+mzmnrz2%1~$ujpKQjjC{d4rtW6r3SSh!el$<{E+hDsvChapo=lAW+yc z?)pQJsl-9y9(n;o>Z4J^sZ>Jps{@Voy*Wvw(b1?_=cP$d5>?o$wkQK$Kh7LW@#r9W zq_DFTGM|rX(GR$j#9OhnMwY?DIjs8nsNsW`sOf#Rs zFGJ6Do)hcn%xY`9JG6_$Bb26&HgY+?` z^Mq{s%E1Cc(IQXu^R6}N^7y >dXZMuSL%(q1Szo%iBHmsMlL;U7308a5WcBzX z16OmB*-AL-S_`FVL<09~Y-E}uSz#8#y17Y-)hFka1M}%0>m~ zKU3Vzk1YY*QNKjTciga#JbTDkon(m@GI?ar=q+L%KKI#J33!N-s*Ud156mR&ZPiAk z+S-v)n~%l+o=?PtWz>~6_k6@E^&m3qANp`x37w+z`nss`x=r6$m|L(?Ub+8=bph-Q zlubWNSVwowExuus!hoz-64N3YMIiPO>i2R8Io>-*)NB7hki=5iO!KmJ9(ZlK_$!L3?KH2wJ~HI+}m$3)(^OrwQ)iHI%T)CHs}BIpV{z*EM3_uM-CB zmrjPBkRf4b%M$h9BqQa1rt`BC5+XcLaku<9R_SH&xWmXg>36jiV8 zeTiN~YOkP?`nfkD)Fij+1U?Q|b>n-Z#KI8fv}gO&2RRbd)U&6%*-|;AW&Aeu3X>nv zcU<`UTs9Ag^Yxe4emxq(`*9wN+$iNllJRfiNA5i$rd>_@@>*{KXe0^#p1fR$%D5e$ zq0@*(m)7q)AO1jysdLS=>lzZtwo_3m_|4Bm=B#lG&#;uIp)v}|MfZj>t zb0xc`YwgKo9)ZC(jWOfm=rMv@&dD@qRZs(JNQ*IOYx`A}xpUi(4{E8&(UiZeK?j?-U;d@Ds%UQ&A84vs3NzV|(9GBdf z3EV^{qRq!LKeQ3#TDlEhU;2U`cZ!VESF9kLx%tf0_s2SN#n0rKB3}LPl>Wo7h$}5` zDUsMpzJ+dMbIg@0^7Bs=pOf7x`7ruQivim%tIA47R}OBU5v3i|Y9yj9u{=|IFiw=q z@&0RnS2KcOw}Mp@QHbgb++kQ|E{Wn@lKg@)NA&y15XMhYC&{!N?06dGsgQ7io=c={ z+8|~6KyJB+g-l$pL9_P~8z!os&~E9sKzv7Pb%^_31Zu}PdR=rT7g3wpKK=1_J!V{g z=Edc2RY--1Y0cpKEbO*a$B;f-HqhLtRlCn7f?v7LlGwatLMmQpO^?&MfSeB$KPGq} zjrJomOx~YAj-(iPe3d#RC8RzMc1`u1LC1`0AXl^H78ggvL(73S7VQYfXi=ir6 zYv*2pH%ei@by^w_qQQF&nXOhhwnD{JjemV7Yl#)Y_(UM#qEZ^2Xq$L>dCLs8dz$z~GVX`Yd8e0)YPR@xp6`1K-a>Z!rE9wzR-Ml^Zl*KwZ@ExE<~u1tRz}oT z86jh*7EH3}s>|Fqw~K7H;}?l+_K{`A!e^Nht02j=VQW#49{Y8U|8_ja68TQ~(9)ys zF6lW--!HCp$512>C!T}g>qN)SpD8=p(G1cI1U$}pydl0%Mk9K7--5_${%b@ljXKIe zF7wAgNiHdI?>kcbyaf_$sPgdB{0)>F-_9%=kUsJr?2ynqq=>wHMho&HTlrNeZ&=`0KuUNo|C;oWG@bgkO_> zZ#(PoBefX(9JV4wy576 zp!Z$N~yB^ii5V@Theo(qsQ>uIvX4Fo$6hQzK|45xbYRlIU$sM&> z{3+9q;G^50YmWLI`Ld*r+A<{igd-TJ>hR*IU;F2dGeSWyn>? zFG5cUR5%@u;3*Np>H7;d2%Ese$9+sk>^RBuN<*xt%Ri(c*8LdCGDLuI!S)UZY~8pV z;sxn-eK9wOD7hD^xnQTWv2Z2qEZ?L z<@_y@TG+yE@rWaygxKZoj6!ge@02A3(kB!;Auu_yssKR(_u&en zNyzE#?13%rb_r?-8v1hFAeG$V2NjUY?+bGD2n9ZVgNZAFAf(9buZzvNDis_)He@Kf5mpM&&|F0W#|+sV z8Q_?p1EB(fR0lLW1WJ~*

    {MCT$pTqyS}_Yz>e!H&J^V9IHeWwh=b`+u3b3C&Da= z{E^Z>7ct$1!1jxX0mPs@0R#{g!cuz_5p0R@tdb(UgrF{-Gq5$XAX@-iMWlR3Eh=Gw zBsg#AY`-Mj%8p?N0{6Ecq{W3NymNgh5Bt25@>n3ga-@k5Y<)Nu1TBETp`bkh`QIA4 zhY(Q!Sxc)suoa}wfn_6{u6%$-7zkxrvZ6>jfdV(e4W|uz$jd_EH7h1?(GyzhUe`m# z5fPGM<7vx@9f)u{o0I|J#ixGTv^OUFTrO9GefaYo$xtabU+!+hB{eL*%RoR#U$TOr z=`8(A2y_HFz95>sfBCkFL7P22K0&e((Vzf1C!oy|6H+n^s}W(mwgE3+E#9bq44T+S z@8v&m@`thKL{Pt6XXiKIY8F5FjE^XKDAPb-U+t6&$JBS@@4>MZe+o7TJjpN^$i?K4 zGS>oO=6BC2*dVd5Q{yGXGtCl1gwlej9H@J8*4I>!#^s;aRUtSg49Fo!yfEz#IedPJ z@knG9jE_+;3$^_ibTkwn;|xXo`ZL)oB%+~r_Xh+`e?niwv2R7+j@)5iiA4pr(z;CV zLrFKe$#Oj)ba^BN!=YuV%%=zYR*T?qIB zbn>BS)M(aqVCzt7j|l>muR158z2VnGdg%#g2U}XMeOgs6_QaD`w8iQQDBFdn9L49J zeeo-h`@LGRW=Q1)#$gh{3$$Dmk~j*$U0s<^<$~G<;kPxF7+l0yea?ObpQauXyWy2im=F@F zfg}XVd`1yt!be+vb`Y8Uiym_w+={FB)Bp}KH?6E@ZM<*}+ zoF5k)3<)BMhMsJaOE~gO;T1t6gcwh=#Kr~i)N;;hc>HC&npyzC$oi@ll#6Wi@F_^% zw%uKA7g`hNB{^ky67K2F4MF)-mQC14;2kO2xZvs5S^g1jYW`;$b{8*fptPJ$*!)JW z4ZCEmc$PZ}JQIhzFuXPTm>;=GDlfAaq|U+QmI1dU=5Blwb}oLPXM*wwaW*%Gon0>8 z$Dmm98~w1JgwK0;PKcjhG@T9mOj$Uv6oiqqx4e+cBp&`iIJxBHFCGZ&NlEG<#?~qy z6$Kunwuj{*0GhcfHG0AiW8L4-w+=Hr5grP_cD?KkuiolsrbrI3U z%YZ1tdqWpr5<5v%iYIX1desG8cQipx5>Bj>eP<5{ItANPkfwP_Truq1 z>$EE#z_YeKj13^{7Yb-WT3_Y4dWG@2F$}TF1TXohL%4Zkj&xZh0>8?O#_|%@-K>m(*q+!`0FL%D3k%U0z*Ml!``v<=C>2O6e zBClRRpr7LqN1O;k`Vm!7UfmbgsnUt@=d4|!{C~$^z^3CxpXS?V5!{nr4MSyai8)+^ z%)VfTDTMG#A=#ddK}8 z(-7R19o2x-K1;__@sbdp-Q($koT)15`oSpF424LTk> z+ccIQPZ6ur8$c+e+Gm8;a-PpK)|0SpZ%YPEm*358S(Vtgj;uh0HsP`;zBicoQ9XmY+oUZ=Zqyp^)ct>DPr}3Xt(-k{%s8u*R8_r&;+gw}S$(#tH8$ z3QIvB)i@C{rjg!zM>S40%a-7=qdJ5d7B1@WO$%$5Kf(yn+|E3_6Wn8g7laW0qqZ@I z&<+lVAC*66;0IX#pugE5KGepIjcX_%k%lxosQ@q@!s}C041mHf328rf6xfXZv|=uY z0e@kbI+xRNM^(EWiIaF+qd^;w4UGk75x}dd@CR3oFlWwr0 zMS-#m(%%x305m3VHs?g4!C3g@1?O=T$ef9>64XM2uB8jsUb!gnN0Zc+z=D9wyp|)6 zU=3D3rC*Tiiw35$X6U&J1egpJ+71PwLGQ+c{vZl8_-ah7>pBJLCq0ih4n%?Umu1I@ z*iax^d2xC6G74P8MOu2%A>ik{Z^t`MG%#r)m`z?m0ju!#!|8nlER{Dqmi+->$Ig7O zND~05_Y6ZO8~_A4_NA9NpuwkGS>n&h0N@{Wc4l~h0t07f{VFa2KpR+NP}zWhDvHRa z)`tLi&KNHBa-sm$fQ)Pe76G(!kCP|;0Wd9oOqqQT1xkmw3-?3-`0#vIhMOD(h|bs$ zH}C_{-L=Sy09-51@e0#JK>T~z z3)fkWEbE`IUg2=9r8EQ>|4P54bQJ|2OBGsN)j$Efvdh&sCs2S}gXc9l)C+!3&kx^< zfQl=;nXvK%{l*RV#-u3<*qg6xg)#}BUENEXGr(JIK+CzJb>vY;!0HFJDo1@qi0ha== zUXJ(;?kPb3S3?z6Iqkkf0@Z837&^Q^fZ0>xy?W z1EYzKXmIj5F%!2A8pNbrSFeHk@Y~EwzEg_=?b?dQA@>0&toJhR=6ezF>ZgBn{}}*&JKy6?6+?hk+fZg#1N86y*=si6P=J!+ z=bf9g0N7fmUPy-a{_RZ7qnBg|kUHM}Dj5p{k~3-OEDHc6iv|-l2PhDC|6`}PAR07H zYyTQ6fo;QVjbSwuU}If%$bJdH=x5?`sG0)M{> zWUO*CVcao(k0uG^LV*=ARu1$906Yx9N|qZMR6AMswn6(k%k8IpcMAnJoY5B3ClPRh z%9!f5ECAcQI_(3C0EB)tcT9ud#jGW^yfNoM14DkUuReNcz{PE^DD?#ac8^S+|ABI$ z?6!{cg!c4ftj=@X82}UUKe@+J5b*8AvJoE!4f^sg(fTW(!Are@v7Fy1u&+i{!sUnp z6E=$EpM?O3_2i@+_eTMy{7JQA8z_J~*~i|)kAT;|NO7Zw2*}cs{-kLSz?YQ!j`4IE$fu6Xo&5C~1Qb|d>+0yx;1xk{SI83u zuAtua27~u)NQD9)UiUY+2*LQMXOnaG6AJ8kGQPL7fbr~Ve^xOq+z(3KDFgb|aB}=HK{4q6 z-|p=O4xm7NO(-gI69JYRwhf_>yrgs}G(# zf5mq(#@nA=?g21yp``Vc7o-#Qk?^Av0C$FM$K??e(3$K_Je&sLybu?0$VCK~ud z(1QYU#^RjEuAxD3w6+ohjEgRH$|A@F0zPY@dyQed!=#?D>mr3Y(AmiEErlrX^u5A@ zLpD4&Ps=~vybA!j&3z`V8h}(P)ugNeXiqI97Sg42LuR^`Q_QrQ~a3BDqL$Ioc z=kbm;Wpi>H0`hDFr9%I}^QgDdln=^PFW#rS9qL6F)x^ek37%J~3G^!DXb{rzQk}05 z0pc1D5(Rk>uyu#YM05{;f&NftpG*X#Z3uk*V}u6bx4rwthyl15oH%+_69q&P3OGer z0g$`)ESOXZ0rEHAVInaA5N^Ned&h}@@&@aqTyg}=l-dxtmIH7=mFM&r`cv?iY@F#j z+-Kh1@CVWecymo<_+l^`T;HV*nuGF6E~^#eAB1tK@a3b79(eAL7QA672EY!W2~0=v zP^0}R3-p5*rZ@c3?;&7t_mZ6R3jmnH7Vb`YQtEeynfIa*QMjXz)@BGQj=X#`mWW#Xx%t4c2yvL;?Sp zchtc!&hNTJprYWqnP&OdmW2V}!&3%!9*23zM}3Kq<1qgDTrTLni-6|DHHk=X6o?^j zq!T4ZfEdksrruQml+@GL>7D~{oxXnURwwkMVLqZ%MHuJO+4O9y05IPCJ%>qw@$@2V zJ_`DwalO8XGQ1Pe(e0lOfqAahNG*p1Jg-`|nVxgt0I2G!?Vg4C;}bj^(>v&w@8bNG zh8pr?M1wY(KPHr3D5IJB)X$S2ARH(17}*>r1>T!yo7bwsb9>)$ zSk)c@H)hBM>aW23HXLVL?VE%Ta90U+VqRMQBvg}?Gkx)Y@GH|*ef*zaW4&x`djl1 z5l;={I)~eRWESS5?gB^?{q+ybi=^7bI5f~8gSIN_87TrdJ8ZTF;W<|3B&7M# z9tFxNzTtntdg;dw;q7x%n4dFYNoiqR*cxw^Nuoi3ZJi2L?l8>jxirt+ErNOUt06@; zG63*6_C*o6j-0wv+AAFd2rI92y!Z;wflycD`_R91yTWQ-z;nJyd_6qM9O`M#DK;+y z1;(CMpH6^rD2goRTYMe@Tz*iB$wI$pYYx94p9b@_^JcrZ7T|gCEwlDKJQo}k8cwc8 z|Jz@v91D%S>QF%V3kTbAsK1*ZuG4nF`9&$dcZfyNz@ehN+ymAxGDIZkok#F|Bm44c z8~S&B=&QcgV3>aZnid6`CWZgFL!^fn_5XN<)8qf8=m=UCSabikUO4)WB}4>*f}`hd z@4-J|wD5xfe0#z-+fjin4X^9j(N?f|RB9hRk_(^B{l*$FfA+(1+90P>HyNr~M&ieH zY&35O-g(yKPvw4xe5G(u`kN;AK%I>%SzOc~6BD}g(nHSABWjB(Ce4aiCbYRf(3oz! z&J@nGXjYxF6a0g(zp~pYpsjBmeQW!ZM$wa)f+wPKBGtN+qP|xn8`e92CB1X~{gTmJ zef1e#%{>}>eD+haG3v(h%vc)CI7#Y?({-o%hQ#~}MO8nyuxkC?n`LErA7kG|CCbuu zjND=AqQt@bRI?GHrGA zmwkV#v)>PT2lcfK`VCD;+S7wnSM9~-;_lwh4vOvB6n^A&n6ueDCiH>5?)wGh4uZh; zdq*u+;-#KQ=u`^2&qxN(llclx!7+C8{taDcYI$_upwqRu??8|5E1A~D1IJSR zhKXi}fmaI5jZsN=*j+q^8qJF&q&@wl{S3~a=osoNE`2NRuQYjwq>(dJo%>F^HQF}Y z6xI{JvYhBpE%ZA~(SSZBjr#6{oYHW>h}z{2&)y9^;~!}^`mXUM;ha=Gv^()V-`3PQ zpYJIf%Y0S;(>J{5DRo}Sjcg^1!?p6QeckM@B_X9FsH~q<(77xmaKO4 zeV5xED3vvjsON4?_0+EpMfDb0sZG92WoOKyS94_gQf=>k&-|>BtgC;0DDC6Y)%ftqD&gE}-Nq(fKQ@**{GT>+YJKo~wKP z%Q|U?cd@A;{da^NGN>TxUG~SYIg8PSHuex#GQdA)1LK&RuY z>G7eUZ}fsU*1ebNT8DqjMs%mXCD}_E$~*4QCLT};4Au=4Jy-0nf2LpkX)htmC&o`# zf;U{d$CUh&nYdO+fk@CJ3E3q>{`ah-euua_55~ow)eUz<{(N&=w#rseIyTt3Fk`#i zLPARM!p~(z&aI^=mq+Y6-u({aq_mIyGLjktts64+COf3%FHT*qk(BEYv1Xj}_q+M( zCHIB69{0gRv#$FMrsZ6Z!4s`_DST;3V_v9m7Pfx#;&YncZ|$#(_vKmUYVD-5A4gxN zjLN7A+M}h|~?6zEjn)3TitQ(JRA13NDC|0njrLeq^ z`zhvz9_#QI#h#3HKb9pzzKsqRB`UkP#QCsM0IiiS`~C2H?iO8)qjzNW#BT?z4t76` zF6m@;$^4$Qk0`bIne<|2&ZlRuMf|`Cylvi5H~Xb3)9;hV|FcG~Zq!^_IGW^Q!8CBW zuP2<*P?!BuRcG}k*xmk;qx+OM?G?ncVG_9~$c>9?m7Pqr4%A1b&VH_q&T@1KH+5G5+SF;o{Y5%^|IQld17f&Oi5 zCsll7QKpWVmdll4x&E3W4)v%_j&%73QqIw5(Wgw_ezbAdEuxP6+&B7l_hP|fP#MGE z)0e||5Pt;;s0e<64xe_<8R!dUKJLjR6mHVqGGW%`jjY<2WV#%<-c8|i7v}2M3-5tIHSLeR*?bo-zBpYzdFjs;uGZBMqx~=R zud7DhonsoU5a+Y=CT;Xqk-S9XpUm-Qx#GMjPW<`}CF&eZvyUCc*@sS4LnJ6$VZDgd zVz~~^xzX27S&9z+mw&7NbvU91TpEUr7OQaXhIbz)+%Me~cYU2c-Zs@AmC>;loqS;a zA)r2fu+Lp$+N-6sKY#5fTT%ojW~V+QyC$8Je^5>19z)7K0hQ`zl`xYNnBm*F9eBy6NJ-wxWKbTWelmwq!-2*!Y<-ZPC@JMdp~XdgMn~B?j4eb7AC4;wAeV0C6?B{>h_XR?~98Ku@31r z{83Zws&u2Da~?!~in^-q`x2A-(S*UVEN4zj4#F9lbC>t!%ZcKykv)-)2j@+9OfElHTiln@VD(PeIZi^}$vlyGx2@ z0nwwRSh9>h&4AbKUsk4s7!*1K{?h9(@z;irO5|@mIi%M`MSLYjPER~RuM!qkOz30` zlD?i&tcZFSBvKh+;+Of*#zvR;vciv(MXQ**PFY@TUfO!hLh%xHX_3+Bzm65VvwSNn zqs*}-$ELFJM9S))-%am?XXtEoysvuOn8!HLebw(l`HRKTmuFi-H~Rb|K1JqM2{|i8 z3+(w*N{Lxd{0Wge(R8BuSU{|x`c~Y7mID9c?q^nXF~7fO_<-|6HNT`1`4_(S{*ooz{W&R5>C3uJf5EE$n0eO1*0`zzD@pV$xrPq7rzH6PRMd@<%`CEvIcxfx4%cUP?={aRl1#W?6UT;%USBNnO0HVwb+y9WHI(YAjbN`m>!A^`a?_qI z?}~@u6@SNO&I2;#ip3V9*lEiMlA)jg@4zvih3^s`Ywe{wMYlYL=QnH7p%}SW_u5Dv zKba0F50+9wlihd>UdV}4AHO*K_-17V4~I76ffx%S%w~@JG|QkIxKtce9kEhU^Xt=y zOk;|~fG3{OBJ$cyFc!&-Rc~&F&0Xq5o5^zV8R3Gc{jctNTW?lS~{4 zEA4;2yvv)vG`|?rdhf4!a-KwMA8!+2dQX?(iRF|5)1!drua=wKteng$41cJ1@zX1n zg}T&Du0CX88RB5An7;m?`1IM&uDbdfUeVs(SKo@Zq454mb|m6gZ)}>=l|EPyn)GII z$Wbv!6J9F3U-f3Qp-z=TdE9mN33UkXomCo{-jK&jve!(JE*D{ehs>UOe7iwImsgUhIGN>316QjbHjR=4LNUl%Q zT7}UhQ)PC5O{Bs*+#j?{A> z=<{`*$$PrN+v z`7F$!s!^n`kKsE0;p~9qa&4q?&d}#}$On}P@rH8s4{1*=(fdCLZbM8Xj>$sTeSGe#~|oe)0DcpJF2Q zfHLc$mtI5I?B)AhZL0jud|NViO;tMB!&+6hT6*n%uFF4}dRe`-_Vh|5mO{&w$W4Ee zhX40yst<=>*^;6S#~(?l;bDEvjUNVVZj+~7HFApA#C_L9#As%(zR`Ja_N?yhao_Ya zjFcJ_uQ*R}+l=%_Y43WBaT81X>OEoL{nUu2cbfK#$&&E5^U9nP*>faOE@x$FnV+!p zc+o6vTi}Wio_+|86b*ZV#e^lNe{#^@_I-R!X9id}1E18p^q(oR59_ z1=AZ{e)WdIcYR}O*7#2$GA-fyGQaYh;(zxl-{eUe_V8N2@yzf?Gcy2*=;7- z)VO0f<{Qfy)|K>=t!4UZmYmLzmSSTy3G%0U6q)quA6XjHE;@3*z<&5CWXQX~Ezd-# zqSr_|y8C9K`M~m@wK)cW&TxA8H{y_1ocBLkFnm-W+mJsC;nCH%8GIl8+LarY;QL3H z-m35oe=oZi4h!^M3;5*k&Xc3(e2;!Tdvq@jd)w&|0lWWEiQ^uKr@T4y%oZVK z4^Y$Q`q-xvXEM*MHuWVf|0FtC>&2HF&vEDEEKo}>OxP=bWDc7)A9Goh7F9(DnVpg3 zK0FLR9Cz3tH@FZ9@BShVl@vc+K6h2*YyPLo3^C#j^$vS_&Lc*S-hWPZXl|Td9X_Yk z{W#%ZkFu>WYNDImV&~T*B8?+DZ!}}$NsICwAgyrPbdu&2U0$~*VRWyD8ELj! zJ=}*cAMg4Uv{v33T;O1)Q>RfVDjYCst;N0E-4H*{sGstl>m?mW;R9BW^e73zQ@r_D zM{f5vj;h()=dYcdiBHqZzQ`y)kgdEirj>hvuIe1A^=nV=w7{Iyvcy0;9?>yA?o1b( z`vz6C`bPDPjPE!#wtF~4gPwl;5Ej>3<@zl^a%x><&8;+aP^!aMjAB+OnjP*N@&+Q} zJ6U$7yt{5V&Oa-b+2?rVSKi7)%k`6k>UQ^qH*t&bo0ABoKf}a=qkOB&M{`PlGz&4H zAFS8OX|hF+s{S;cSWtfaYj`Slgt7e9n{9~S8Q#Ef_C(l8RK>!e`&IhQV>$WN2{&XV ztWVAI>P1pIF|D2dayzdm?#|GOf*XLmzd=T_n7IBxo<8D9NqQ!|s+@DE^P!p60{=h0 z(7+K4Ka*YKr#=^&ZMgo$tVBe(+0|Xp zL|elo*P3rZlpf1SQg$d(POd>J+== zMA-aSOI?Gf9NEl1YBwDdc8|9gtQ!@C7_yf?&Dz_%HrakWUgo)>Yq_7@*(KTDVfj6N z%?@Xcw;OVPk_`fVcLuC`x&uzdO};R)=$L$j_WvedRU+qg@1y&wlFM`3;^N%xK8^el z^*E7;(36XdvB5Q+iae?i=+1WA8C(otx0M|JPPbp8H@YW@oh514MTTcy{xRB z|S5M_&Gu z4pkiC@$oz3qTbC|;`8h9uHm14?Q!U}J$$0v*b2*>n6ODiN7?qjjjeNtsJVELAn_YwVTlxb05+GsP|gjhV1 z-QP>-x8t+38*^p~d7c8f+%gPCH!~XE$Zx^>2lzw{_VX#)trEJUJyuu>7Q*QrN3^?n zi^kQZ=Tp=NW`4Ci4%t7}lF$stl*|M7_=rLVvPFJ=)iK^kxqzbQ0`kLat3Tr_0xael69yH9G< zW`e15YtMB9b|g**!!-^yS#;L@M0awZcv0;O%mTG@uCAp)&*+b(woH$`jeXmCv{z*- zn_DEMMs>05!fok6zR6EToB0aA9=25Qi>1Zz@$U%57P=kLdOg?VE+Rks5C2>nj}Ewz zA0@+JHv5`Lom5x!J&|>kWm)8@N(15u0|w?J{JZoqj4BCE)1eWh=+kJ(1=J?@y{W zaBe8)4Ggbv3pEvfzQ3OJ$|ZKs@*lu^8u;W0;3hez@HXL|!CNz`tM!a{V3$ZWy?ZPBj}$Me?V7eo+Fk@Y?6bMMLj7Xl*xB#^2=J0J$f*`d^Z*|DZ!#o^d2qz#u?R@8RtzXzghYH7WQHr3(7ld;2(hxYJ|B1ce1f z06pJ-!|?HP6@>ctx@PYy=b9J@0afNu0fV}|6U$i-7 zgF{0F{)2|d|3E`nK;-{MMD%|kA|W6m`hVJq*uROmdbm6OUzia8_f!X059=d&{eJ~u z9t!Bs{Y&(x-MrcUr7JcOK@q4cHeQ=+&aQTB*3vR|4$=-XGGd}OlJ*Xgl44?Z!ji(0 mGWHU}wlX$SGE%}e_BM7Rd< frame.select(col("*")) } - // Column schema of decimal-test.parquet is: [_c0: decimal(18, 0), _c1: decimal(10, 10), - // _c2: decimal(15, 12), _c3: int64, _c4: float] - // LIMIT: Because we only support DECIMAL64, we only support reading decimal columns whose - // physical storage type are INT64 in current. + // Column schema of decimal-test.parquet is: [_c0: decimal(18, 0), _c1: decimal(7, 3), + // _c2: decimal(10, 10), _c3: decimal(15, 12), _c4: int64, _c5: float] testSparkResultsAreEqual("Test Parquet decimal", frameFromParquet("decimal-test.parquet"), new SparkConf().set("spark.sql.sources.useV1SourceList", "")) { From 8e2fdd067257f994f746d42a998a747d9a943b4e Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Tue, 8 Dec 2020 15:59:40 +0800 Subject: [PATCH 05/15] adapt decimalTypeEnabled conf --- .../src/main/python/parquet_test.py | 24 ++++++++++--------- .../rapids/shims/spark300/Spark300Shims.scala | 2 +- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +++--- .../nvidia/spark/rapids/GpuParquetScan.scala | 2 +- 4 files changed, 18 insertions(+), 17 deletions(-) diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 1665dccb442..4ad2f62046c 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -28,6 +28,7 @@ def read_parquet_sql(data_path): return lambda spark : spark.sql('select * from parquet.`{}`'.format(data_path)) +# Override decimal_gens because decimal with negative scale is unsupported in parquet reading decimal_gens = [DecimalGen(), DecimalGen(precision=7, scale=3), DecimalGen(precision=10, scale=10), DecimalGen(precision=9, scale=0), DecimalGen(precision=18, scale=15)] @@ -45,11 +46,12 @@ def read_parquet_sql(data_path): # test with original parquet file reader, the multi-file parallel reader for cloud, and coalesce file reader for # non-cloud -original_parquet_file_reader_conf={'spark.rapids.sql.format.parquet.reader.type': 'PERFILE'} -multithreaded_parquet_file_reader_conf={'spark.rapids.sql.format.parquet.reader.type': 'MULTITHREADED'} -coalesce_parquet_file_reader_conf={'spark.rapids.sql.format.parquet.reader.type': 'COALESCING'} +original_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'PERFILE'} +multithreaded_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'MULTITHREADED'} +coalesce_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'COALESCING'} +decimal_type_enable_conf = {'spark.rapids.sql.decimalType.enabled': 'true'} reader_opt_confs = [original_parquet_file_reader_conf, multithreaded_parquet_file_reader_conf, - coalesce_parquet_file_reader_conf] + coalesce_parquet_file_reader_conf, decimal_type_enable_conf] @pytest.mark.parametrize('parquet_gens', parquet_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @@ -74,7 +76,7 @@ def test_read_round_trip(spark_tmp_path, parquet_gens, read_func, reader_confs, def test_parquet_fallback(spark_tmp_path, read_func, disable_conf): data_gens = [string_gen, byte_gen, short_gen, int_gen, long_gen, boolean_gen] + decimal_gens - + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] gen = StructGen(gen_list, nullable=False) data_path = spark_tmp_path + '/PARQUET_DATA' @@ -109,7 +111,7 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, boolean_gen, string_gen, date_gen, # Once https://github.com/NVIDIA/spark-rapids/issues/132 is fixed replace this with - # timestamp_gen + # timestamp_gen TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens @pytest.mark.parametrize('parquet_gen', parquet_pred_push_gens, ids=idfn) @@ -405,15 +407,15 @@ def test_small_file_memory(spark_tmp_path, v1_enabled_list): _nested_pruning_schemas = [ - ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], + ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], [["a", StructGen([["c_1", StringGen()]])]]), - ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], + ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], [["a", StructGen([["c_2", LongGen()]])]]), - ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], + ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], [["a", StructGen([["c_3", ShortGen()]])]]), - ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], + ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], [["a", StructGen([["c_1", StringGen()], ["c_3", ShortGen()]])]]), - ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], + ([["a", StructGen([["c_1", StringGen()], ["c_2", LongGen()], ["c_3", ShortGen()]])]], [["a", StructGen([["c_3", ShortGen()], ["c_2", LongGen()], ["c_1", StringGen()]])]]), ([["ar", ArrayGen(StructGen([["str_1", StringGen()],["str_2", StringGen()]]))]], [["ar", ArrayGen(StructGen([["str_2", StringGen()]]))]]) diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala index 26c022c9d12..0448993f728 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala @@ -157,7 +157,7 @@ class Spark300Shims extends SparkShims { allowMaps = true, allowStruct = true, allowNesting = true, - allowDecimal = true) + allowDecimal = conf.decimalTypeEnabled) // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 688c534c3d0..fb496253c76 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -17,11 +17,8 @@ package com.nvidia.spark.rapids import java.time.ZoneId - import scala.reflect.ClassTag - import ai.rapids.cudf.DType - import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ @@ -54,6 +51,8 @@ import org.apache.spark.sql.rapids.execution.python._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import scala.language.postfixOps + /** * Base class for all ReplacementRules * @param doWrap wraps a part of the plan in a [[RapidsMeta]] for further processing. @@ -2043,7 +2042,7 @@ object GpuOverrides { allowArray = true, allowStruct = true, allowNesting = true, - allowDecimal = p.scan.isInstanceOf[ParquetScan]) + allowDecimal = conf.decimalTypeEnabled && p.scan.isInstanceOf[ParquetScan]) override def convertToGpu(): GpuExec = GpuBatchScanExec(p.output, childScans(0).convertToGpu()) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index ab840eb7100..90a0a620fe0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -140,7 +140,7 @@ object GpuParquetScanBase { allowArray = true, allowStruct = true, allowNesting = true, - allowDecimal = true)) { + allowDecimal = meta.conf.decimalTypeEnabled)) { meta.willNotWorkOnGpu(s"GpuParquetScan does not support fields of type ${field.dataType}") } } From baf29fc8258ab609e64eaaeaeb66f9c2a6a6a633 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Tue, 8 Dec 2020 16:15:16 +0800 Subject: [PATCH 06/15] fix typo --- .../src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index fb496253c76..a705c8871ce 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -17,8 +17,11 @@ package com.nvidia.spark.rapids import java.time.ZoneId + import scala.reflect.ClassTag + import ai.rapids.cudf.DType + import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions._ From 0f742848ef279640ba59e9690ced35b3c09593e3 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Tue, 8 Dec 2020 16:29:09 +0800 Subject: [PATCH 07/15] fix typo Signed-off-by: sperlingxx --- .../src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index a705c8871ce..b46150dc937 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -54,8 +54,6 @@ import org.apache.spark.sql.rapids.execution.python._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -import scala.language.postfixOps - /** * Base class for all ReplacementRules * @param doWrap wraps a part of the plan in a [[RapidsMeta]] for further processing. From 57a5e3da24e96d4ddf738e9be6608edbe34995b0 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Wed, 9 Dec 2020 10:55:48 +0800 Subject: [PATCH 08/15] enforce safeClose on gpu resources --- .../nvidia/spark/rapids/GpuParquetScan.scala | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 90a0a620fe0..268942fbaf1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -206,24 +206,24 @@ object GpuParquetScanBase { // return input table if there exists no DECIMAL32 columns if (!containDecimal32Column) return t - val columns = (0 until t.getNumberOfColumns).map { i => - t.getColumn(i).getType match { - case tpe if tpe.getTypeId == DType.DTypeEnum.DECIMAL32 => - t.getColumn(i).castTo(DType.create(DType.DTypeEnum.DECIMAL64, tpe.getScale)) - case _ => - t.getColumn(i) + val columns = new Array[ColumnVector](t.getNumberOfColumns) + try { + RebaseHelper.withResource(t) { _ => + (0 until t.getNumberOfColumns).foreach { i => + t.getColumn(i).getType match { + case tpe if tpe.getTypeId == DType.DTypeEnum.DECIMAL32 => + columns(i) = t.getColumn(i).castTo( + DType.create(DType.DTypeEnum.DECIMAL64, tpe.getScale)) + case _ => + columns(i) = t.getColumn(i).incRefCount() + } + } } + new Table(columns: _*) + } finally { + // clean temporary column vectors + columns.safeClose() } - val ret = new Table(columns: _*) - // clean temporary column vectors produced by castTo - (0 until t.getNumberOfColumns).foreach { - case i if t.getColumn(i).getType.getTypeId == DType.DTypeEnum.DECIMAL32 => - columns(i).close() - case _ => - } - // clean original table - t.close() - ret } } From 7e8c597504545fa34dd271ee3cda0fff3198ac89 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Wed, 16 Dec 2020 15:53:39 +0800 Subject: [PATCH 09/15] support reading decimal from parquet created in legacy mode --- .../src/main/python/parquet_test.py | 17 +++++++++++++++++ .../nvidia/spark/rapids/GpuParquetScan.scala | 3 +++ .../test/resources/decimal-test-legacy.parquet | Bin 0 -> 4959 bytes .../nvidia/spark/rapids/ParquetScanSuite.scala | 16 +++++++++++----- 4 files changed, 31 insertions(+), 5 deletions(-) create mode 100644 tests/src/test/resources/decimal-test-legacy.parquet diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 4ad2f62046c..84fdc6f003b 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -201,6 +201,23 @@ def test_ts_read_fails_datetime_legacy(gen, spark_tmp_path, ts_write, ts_rebase, lambda spark : readParquetCatchException(spark, data_path), conf=all_confs) + +# https://github.com/NVIDIA/spark-rapids/issues/1324 +@pytest.mark.parametrize('parquet_gens', [decimal_gens], ids=idfn) +@pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) +@pytest.mark.parametrize('reader_confs', reader_opt_confs) +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_decimal_read_legacy(spark_tmp_path, parquet_gens, read_func, reader_confs, v1_enabled_list): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] + data_path = spark_tmp_path + '/PARQUET_DATA' + with_cpu_session( + lambda spark : gen_df(spark, gen_list).write.parquet(data_path), + conf={'spark.sql.parquet.writeLegacyFormat': 'true'}) + all_confs = reader_confs.copy() + all_confs.update({'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect(read_func(data_path), conf=all_confs) + + parquet_gens_legacy_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 268942fbaf1..9737ff70d2e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1071,6 +1071,7 @@ class MultiFileParquetPartitionReader( } val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) + .enableStrictDecimalType(true) .includeColumn(readDataSchema.fieldNames:_*).build() // about to start using the GPU @@ -1460,6 +1461,7 @@ class MultiFileCloudParquetPartitionReader( } val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) + .enableStrictDecimalType(true) .includeColumn(readDataSchema.fieldNames: _*).build() // about to start using the GPU @@ -1595,6 +1597,7 @@ class ParquetPartitionReader( } val parseOpts = ParquetOptions.builder() .withTimeUnit(DType.TIMESTAMP_MICROSECONDS) + .enableStrictDecimalType(true) .includeColumn(readDataSchema.fieldNames:_*).build() // about to start using the GPU diff --git a/tests/src/test/resources/decimal-test-legacy.parquet b/tests/src/test/resources/decimal-test-legacy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..969f049848fc39da463fb9a0b9eeec2bdc560d36 GIT binary patch literal 4959 zcmb7I3p`Y7*I&=xW{hDFW(>Jb8kZQxt%Op#P#q*Hx((gi7(?vid4CB7}E?{wbte(&!)&u@M+YybCN|7Y#B{?D`59=mz=TsqKqiP4kA zwCN3qu*r0ag0Yweh1OkmA1YwvVe2jsF$GYCBAkr8~ z?7}&J^Zbq%a4t{2wqE8emTgKxjL~~|s{WC%r}k~QynpKDj3x6$9C+~?tM6F|x4#Y@ zR@W>9??cN!KdJE%k`(W0IM;t!bW~yFxN_fNs+DDWOBSa=IMIDXLxEu~mVU%0{5WIR z&rr~YoNHT@ca1=7ea+*(hn1w}4_jR>rmTg{nZcnXHEq1CYk2vt8(|O_G-r|X$WGxq zL)Hrg?FJ&nr>-~-M&-c$sjb7iPVK-_{O7ERD6Syoo@rijW_Tq@t-MU;$tn<=xspBd zd3K1{`FN|5-FiNh__UOxX|~B!*U?q(tBW0lV~*!-c(?p1ts~uY^<6(N5lDk#vCEfG zW7(}h+S+{Dq3ZKFW_v40oY+@Ky>j1@SVku!u0N5c+4|jh`_RRiT9X=**8h7sxm)XT zPu2(noL1hrn|qW=+4)$KCc97^PplIiCC|Sv5^#NKV(ybFkvATrA2MtGNlSgLy0p6; zM3z~`Dh)fW!ggO`j41l)LJ~J}eN&m6$fNyT&h5_6M9=0{r9K~|iN37Y$@Hv!4r}(8 zM0S=(gVRCn8lF-w{!=iy@Eks zTh_(YJ|&&k8>AnAjKiSnj06(7-`lLsSxb?+wl(Y0Y!yFz{ms7pt&}pnbf(~1&$eXI zui2F=@VN&l<$=O_oKK-xDLr&)>&qu_?R1yNR>ymwu8l_Y#U`!ual50@6)C9jkA7=7f#()^JwZ{LsQznq14pr0!c*v$dClP z0?(ToI7PI`Rk+B#<#l_{6pBw|{5swnQOu~%v9b0Imy+QiEhFhTPc&XAyjjR3w= z!6E$!aAUo|#%&)k$0C5*X#NnUlI}S7$OTek(Hmrd?M3?{$;XL5nTm<5o`arvM4jlAx zkXMQU`#Kj$@Ms1$Hw4s=ZUQ{X5X>FZ!HE(I*1EbtS!WH~PH-UVffH=j_zK#cJHT>Q zEBNZsV0)c0>@&Xw!ZIpg9c>OPdEFo%mo||pVBF66kt=`u0YC~};4q{LRLf1k?N|zd zDNV3vpF8Nq7s8_JMS?Q~`h0`f92}%u%!s1h5T-v6{jgge98Dn% zd^-o>pP9t=*2N+!D2d=HpXQ40pA6c9mOMZ90m16~-iWLCtOA8m3i{FYj+h0QF0z`7 zOq6ar0?FA?6O5Kol_!a7*9E;t#nKH@pf1$&2(4hXokJ&G_uB$*g`+VjNM6eXFUjzI zKY~FkP)`G0+fQ+V-@%2BO;L&qT^zR9V#PD6;|ZH^xfHyY^iY#QRTA zK&_wHXFy0V@(CtW6nz~n5Zm|-2EBG$fR1d*c~I{ka6-d-JPom_mkQ28X`fXAw7fUI zjd3o1DnWcL%UZitz{w zq@u`3MekBH8bI?velePsyX7QUt_YAp8@FcK!u+Ot%TXYAT?)`TGGAcoWnY9*Lj3w= zNYaz+4d`Ks3iQZj?E{?MP3+RrPu8OasVX7BGNV313be!ywB-GZa-chInERw4TxT2 z5`~BYmXQ>sh#w_F2S{k}6gD9U`5&Uds*>8S1;51&)?ak;9X(Z1GVE= zZp9J*S0$LoNPorc8A$dWTUEmb#TL>wois!Tckjp|M6Ju9o9x0om+8@} zXX8;(H_xjmgEKkf5&s)j|66&$ngpQ89ac8oi*6k58x(56n!nV2d}ay!Dtz70qo0B1 zq^?L=brX9ZZee~W9BnJ8TxM2)j`{@K$2H>EIn8bB--4q3W8G9X3f;WNMoI^zP)aUU z86a8z4bfy9TvP1MjEB?E0x4m}M_(8{J7m0S3P!f=+-7kQ_hZ~j-dl*6t7jPu97klK z`r?y4T`S4&Gbsg z{2Dsbok{4^Ue!@739jfDNqNF#Ljzs&Q)6YN0ou{n^Mw+0@TBua=B#`Wf9%T z;u7eN4$m*nK^;P0AR!eU?#%9E=Htr^#$1P#v8fr!tGkSG$-M0;(>>AXODX9)g;?sR ztlY*``0CdI>Q#49@5{x#CEZxlee(@LrO0dFP@<(gE*p2|bU`%M3VT}zG{9oy!?ZKr z==pi-%5%C1eL7_4VTaLX@3#LYi8Cbf?@_$4(_5_PC=Wtm=*p)2I?-gj5$CnGQGVslv+q3LAgvK+)=qG>e z?wFME$q`9dU%f=?GIgUE6pjCY5|U3M57KZdVUW5aeTcpAsyvFTs`(-`ANxAE_qtr% zl>HsHE>cP zh{ZeQ$0(auU(`3{7TG<}Uq+FL8*fZ-+0o^xYV4T3jTT&WWbJ%o_NRODC$@yNwOuF= zT^MWOe7|zzI`g;DR9_;PAA>ZIC*etn=`|2}UvDFEqTv0ukm&R`36IIM7`)m67w4$=qHgjcPZ8Pde zyWr7}?ulOHt1Spsj8u@1uM2Aa9ht5d6f8HXv{y4Keo|D$&>vg$qA1nZ^P%YXdz;p_ zt9AK*xcObxR&T5B+&!C42co1a=Mi}cn#cIil7)QYy3 zeKTq;xvaPux_n6qHgwm*9Tax`IcraI@*6aEFD(KkkyK@x!r~*61tXU;<5a1| z!`BoOx)(l^pV%-b#%HwT*}H+^D!51ElY*-JY|HM8^eD}s$A_>{TfnsxMPqvjw?ieT zBL6kb)k;k%(5`viM>FG9`!kDocQU4bW{r+*Y<`VTHRkEm>D_P=3S2p-_7Ume8^KS_ zwrKY8y>o05M0)DTR_pZN{Jf9Crhsl|^|O_eQ2;dLHSJt;zuThf!xE#CksEEXx8${? z_W6|0nT=0Zyb8!ZF(N)f3(=Gbj4*w*Sj{I?t7M4*Ou%a8=X=%q#xf}pc7k^ zv)PlFIx4uSrY}zOlG`oI#NhGR##>J6cb*xjZWY#kuA|qqYSHILDb|cY?9LK<>lN3h zN8P^o&8@UC&h&QCv211EMxt7AC7MXYHyuD##%c?1LwQZ}c zR9dp;OT3SHqf&(}daIf_MN=bVxN*7u!7T+9>U(Pk1lD~m2U3Ic(nKpHgo5H{=xe*t z8y;2~{+9pgvaTGS{jS5^W?4+jg3oGnnc#l-tWf+gfB8h_<+gk0g`AxgpFzLrGEtF2 zJd(OY*CmsV$!)U|3-sd!cx?!D-Ng6i(Zzr+PM4%(Ox$&Y-gGKaTD1v;GKp{rgiIHo zK&X?5K7q*4WhM~uNyLCam~R)^fCVi_#84aTB(Wz#}|EX2_)xS<%fiCptZ5vMdXVMra zY3w?6vw!dy`jxfMP=QYQlhtUF^%a@%k<9KhH2W_r@fQ|GhAB}3kke)|eFHo=JU<>G z2hI;=aJKLR0zG|wm?Q&^9)}A|&A(j{=r7<9BJvOB2XO*?Lpa|2Af6j9h{rsRhqHph z{P?WdtiYgvVAmj)Hp|_UFK`QF&0Z7E^5J<;oFbG);D21_#&`Af<_R>ov$XXz3D_rC zAmF(Oi0cRfzBcP8H*0oyM1=M)9NfQi%+xmcM-=*hr{L;obN^W!2LFM?NSmwwk0LSr zlSJU_<3Sw#zw==9r>ncbmp75}|LVIg0+>twsGg2@fXW}`R^f8EM7dQoT!K9XZYsQ4 zCT{Mt+)Ydj^j(bk?#9Lj25x%Bdd4PvBRy9WmzgFr^<4NaZd^?{fdat4eZZ4=0N4?K Gt^E(7xpbZY literal 0 HcmV?d00001 diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScanSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScanSuite.scala index 7af0291b970..0942b2c0346 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScanSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScanSuite.scala @@ -43,11 +43,17 @@ class ParquetScanSuite extends SparkQueryCompareTestSuite { frame => frame.select(col("*")) } - // Column schema of decimal-test.parquet is: [_c0: decimal(18, 0), _c1: decimal(7, 3), - // _c2: decimal(10, 10), _c3: decimal(15, 12), _c4: int64, _c5: float] - testSparkResultsAreEqual("Test Parquet decimal", - frameFromParquet("decimal-test.parquet"), - new SparkConf().set("spark.sql.sources.useV1SourceList", "")) { + // Column schema of decimal-test.parquet is: [c_0: decimal(18, 0), c_1: decimal(7, 3), + // c_2: decimal(10, 10), c_3: decimal(15, 12), c_4: int64, c_5: float] + testSparkResultsAreEqual("Test Parquet decimal stored as INT32/64", + frameFromParquet("decimal-test.parquet")) { + frame => frame.select(col("*")) + } + + // Column schema of decimal-test-legacy.parquet is: [c_0: decimal(18, 0), c_1: decimal(7, 3), + // c_2: decimal(10, 10), c_3: decimal(15, 12), c_4: int64, c_5: float] + testSparkResultsAreEqual("Test Parquet decimal stored as FIXED_LEN_BYTE_ARRAY", + frameFromParquet("decimal-test-legacy.parquet")) { frame => frame.select(col("*")) } } From 63ac10e335d3abd54e8e4723a52404c09a6f9a6f Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Wed, 16 Dec 2020 17:19:12 +0800 Subject: [PATCH 10/15] keep up with latest master --- docs/supported_ops.md | 16 +++++++------- .../rapids/shims/spark300/Spark300Shims.scala | 9 +------- .../nvidia/spark/rapids/GpuOverrides.scala | 21 +++++++++++-------- 3 files changed, 21 insertions(+), 25 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 966102ea9cc..1b72cc087c3 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -187,13 +187,13 @@ Accelerator supports are described below. S S* S -NS +S* S NS NS -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) NS @@ -486,13 +486,13 @@ Accelerator supports are described below. S S* S +S* NS NS NS -NS -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) NS diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala index dd56c27cb29..902f76d7ea2 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala @@ -147,15 +147,8 @@ class Spark300Shims extends SparkShims { GpuOverrides.exec[FileSourceScanExec]( "Reading data from files, often from Hive tables", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY).nested(), TypeSig.all), + TypeSig.ARRAY + TypeSig.DECIMAL).nested(), TypeSig.all), (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { - override def isSupportedType(t: DataType): Boolean = - GpuOverrides.isSupportedType(t, - allowArray = true, - allowMaps = true, - allowStruct = true, - allowNesting = true, - allowDecimal = conf.decimalTypeEnabled) // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 7c622b586f5..321018e40e4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2202,20 +2202,23 @@ object GpuOverrides { exec[BatchScanExec]( "The backend for most file input", ExecChecks( - (TypeSig.commonCudfTypes + TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY).nested(), + (TypeSig.commonCudfTypes + TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY + + TypeSig.DECIMAL).nested(), TypeSig.all), (p, conf, parent, r) => new SparkPlanMeta[BatchScanExec](p, conf, parent, r) { + override def tagPlanForGpu(): Unit = { + val hasDecimal = p.scan.readSchema().exists(_.dataType.isInstanceOf[DecimalType]) + if (hasDecimal) { + if (!(p.scan.isInstanceOf[ParquetScan] && this.conf.decimalTypeEnabled)) { + this.willNotWorkOnGpu( + "Only supports reading decimal from parquet with DECIMAL_TYPE_ENABLED = true") + } + } + } + override val childScans: scala.Seq[ScanMeta[_]] = Seq(GpuOverrides.wrapScan(p.scan, conf, Some(this))) - override def isSupportedType(t: DataType): Boolean = - GpuOverrides.isSupportedType(t, - allowMaps = true, - allowArray = true, - allowStruct = true, - allowNesting = true, - allowDecimal = conf.decimalTypeEnabled && p.scan.isInstanceOf[ParquetScan]) - override def convertToGpu(): GpuExec = GpuBatchScanExec(p.output, childScans(0).convertToGpu()) }), From 6bdeb99c69d34a5dd81e9299c0ba0841627e008d Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Wed, 16 Dec 2020 17:38:07 +0800 Subject: [PATCH 11/15] revert supported_ops.md --- docs/supported_ops.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 1b72cc087c3..966102ea9cc 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -187,13 +187,13 @@ Accelerator supports are described below. S S* S -S* +NS S NS NS -PS* (missing nested BINARY, CALENDAR, UDT) -PS* (missing nested BINARY, CALENDAR, UDT) -PS* (missing nested BINARY, CALENDAR, UDT) +PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) +PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) +PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) NS @@ -486,13 +486,13 @@ Accelerator supports are described below. S S* S -S* NS NS NS -PS* (missing nested NULL, BINARY, CALENDAR, UDT) -PS* (missing nested NULL, BINARY, CALENDAR, UDT) -PS* (missing nested NULL, BINARY, CALENDAR, UDT) +NS +PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) +PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) +PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) NS From 022d65c5450f1e7168ad32dfb36962760428c66d Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Thu, 17 Dec 2020 10:26:38 +0800 Subject: [PATCH 12/15] fix --- .../nvidia/spark/rapids/shims/spark310/Spark310Shims.scala | 2 +- .../main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala index 7cd54d0d55a..1164319b4f5 100644 --- a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala @@ -134,7 +134,7 @@ class Spark310Shims extends Spark301Shims { GpuOverrides.exec[FileSourceScanExec]( "Reading data from files, often from Hive tables", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY).nested(), TypeSig.all), + TypeSig.ARRAY + TypeSig.DECIMAL).nested(), TypeSig.all), (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { // partition filters and data filters are not run on the GPU override val childExprs: Seq[ExprMeta[_]] = Seq.empty diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 321018e40e4..6130a2d6275 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2209,9 +2209,8 @@ object GpuOverrides { override def tagPlanForGpu(): Unit = { val hasDecimal = p.scan.readSchema().exists(_.dataType.isInstanceOf[DecimalType]) if (hasDecimal) { - if (!(p.scan.isInstanceOf[ParquetScan] && this.conf.decimalTypeEnabled)) { - this.willNotWorkOnGpu( - "Only supports reading decimal from parquet with DECIMAL_TYPE_ENABLED = true") + if (!(p.scan.isInstanceOf[ParquetScan])) { + willNotWorkOnGpu("Decimals are only supported by Parquet") } } } From b37554600782bac26dd5300cdc211928129fc1d7 Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Fri, 18 Dec 2020 18:12:05 +0800 Subject: [PATCH 13/15] small fix Signed-off-by: sperlingxx --- integration_tests/src/main/python/parquet_test.py | 8 +++----- .../scala/com/nvidia/spark/rapids/GpuOverrides.scala | 9 --------- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 84fdc6f003b..04b6fcc19c6 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -49,9 +49,8 @@ def read_parquet_sql(data_path): original_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'PERFILE'} multithreaded_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'MULTITHREADED'} coalesce_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'COALESCING'} -decimal_type_enable_conf = {'spark.rapids.sql.decimalType.enabled': 'true'} reader_opt_confs = [original_parquet_file_reader_conf, multithreaded_parquet_file_reader_conf, - coalesce_parquet_file_reader_conf, decimal_type_enable_conf] + coalesce_parquet_file_reader_conf] @pytest.mark.parametrize('parquet_gens', parquet_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @@ -202,7 +201,6 @@ def test_ts_read_fails_datetime_legacy(gen, spark_tmp_path, ts_write, ts_rebase, conf=all_confs) -# https://github.com/NVIDIA/spark-rapids/issues/1324 @pytest.mark.parametrize('parquet_gens', [decimal_gens], ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs) @@ -219,8 +217,8 @@ def test_decimal_read_legacy(spark_tmp_path, parquet_gens, read_func, reader_con parquet_gens_legacy_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, - string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens, + string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), + TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc))] + decimal_gens, pytest.param([timestamp_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133')), pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133'))] diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 6130a2d6275..88a2cb513b8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2206,15 +2206,6 @@ object GpuOverrides { TypeSig.DECIMAL).nested(), TypeSig.all), (p, conf, parent, r) => new SparkPlanMeta[BatchScanExec](p, conf, parent, r) { - override def tagPlanForGpu(): Unit = { - val hasDecimal = p.scan.readSchema().exists(_.dataType.isInstanceOf[DecimalType]) - if (hasDecimal) { - if (!(p.scan.isInstanceOf[ParquetScan])) { - willNotWorkOnGpu("Decimals are only supported by Parquet") - } - } - } - override val childScans: scala.Seq[ScanMeta[_]] = Seq(GpuOverrides.wrapScan(p.scan, conf, Some(this))) From 6d333c5e098935d3592bf249bd5113b563b1c85e Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Mon, 28 Dec 2020 17:10:21 +0800 Subject: [PATCH 14/15] DecimalType TagSupport: OrcReader/CsvReader --- docs/supported_ops.md | 16 +-- .../spark/rapids/GpuBatchScanExec.scala | 6 +- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 2 +- tests/src/test/resources/decimal-test.csv | 100 ++++++++++++++++++ tests/src/test/resources/decimal-test.orc | Bin 0 -> 4943 bytes .../spark/rapids/unit/DecimalUnitTest.scala | 40 ++++++- 6 files changed, 152 insertions(+), 12 deletions(-) create mode 100644 tests/src/test/resources/decimal-test.csv create mode 100644 tests/src/test/resources/decimal-test.orc diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 966102ea9cc..1b72cc087c3 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -187,13 +187,13 @@ Accelerator supports are described below. S S* S -NS +S* S NS NS -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) +PS* (missing nested BINARY, CALENDAR, UDT) NS @@ -486,13 +486,13 @@ Accelerator supports are described below. S S* S +S* NS NS NS -NS -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) -PS* (missing nested DECIMAL, NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) +PS* (missing nested NULL, BINARY, CALENDAR, UDT) NS diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala index 01687c4a5cf..0b096ddf781 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DateType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{DateType, DecimalType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -229,6 +229,10 @@ object GpuCSVScan { } } // TODO parsedOptions.emptyValueInRead + + if (readSchema.exists(_.dataType.isInstanceOf[DecimalType])) { + meta.willNotWorkOnGpu("DecimalType is not supported") + } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index e4a0f10359a..cdb14c06aa4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -115,7 +115,7 @@ object GpuOrcScanBase { meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") } schema.foreach { field => - if (!GpuColumnVector.isNonNestedSupportedType(field.dataType)) { + if (!GpuOverrides.isSupportedType(field.dataType)) { meta.willNotWorkOnGpu(s"GpuOrcScan does not support fields of type ${field.dataType}") } } diff --git a/tests/src/test/resources/decimal-test.csv b/tests/src/test/resources/decimal-test.csv new file mode 100644 index 00000000000..1fd4646904c --- /dev/null +++ b/tests/src/test/resources/decimal-test.csv @@ -0,0 +1,100 @@ +915270249210239718,3232.792,"",771.371173049837,-431710025170174585,4.7239953E24 +50004804273312941,9263.400,0.3815050595,900.890874730220,-2697073954890740236,7.463101E-31 +189216077028719828,8094.536,0.7879423094,817.584127883600,-5133656973475552689,2.4919543E-32 +257886722013221592,7097.760,0.5551530993,263.532647425188,-3917032101531217289,5.176538E7 +223616015091255874,2021.691,0.4300789101,741.876603938996,7810001276519378488,6.673943E9 +672269487046710336,5925.644,0.5119009979,677.273069670187,"",-3.037228E-26 +565739933980371374,2082.917,0.6325578476,307.790881040462,-7858370133784586516,-1.7402571E-20 +30846051239772526,2153.584,"",428.457572476468,2224676899470531349,-3.720298E-28 +292750338676616377,913.012,0.7647953732,329.427928877243,1287002480498025462,963.7641 +863597087074282135,7771.993,0.9378547775,655.942008799151,-4594481394522420980,-5.143567E-37 +740916466628308680,3601.327,0.5253287765,293.314491660678,3663204060287939255,-7.328485E30 +543922875310806589,418.135,0.0089806121,678.725898059834,7100991336799471692,"" +907927561275971206,4357.321,0.0894175775,985.715625304095,8980271347953950594,2.036806E-11 +5365651368778670,28.443,0E-10,98.984414373371,2753461187700946189,"" +98975573402945273,5544.347,0.3277681904,780.987096259467,-2821373815087770381,"" +37654707856660412,3245.419,0.7249053714,"","",5.489915E-15 +417692898406404444,5137.551,0.0654631387,8.906002611085,-3073072854586574157,2.8559989E-5 +30008375870039084,1638.013,0.5689759921,506.427830699217,8001350543913869500,0.0010491271 +267647485487795321,711.758,0.7086510671,787.214975034435,0,-4.294404E35 +32497433138887044,1500.500,0.1632851254,111.862479713755,-2124845351861490171,-2911376.8 +563219648467491123,2533.799,0.0470860444,-999.999999999999,-6150492885085058765,-3.1956E-21 +698244577193394337,7710.613,0.1796242319,177.658429265869,-7293294899695747856,1.55815667E12 +59984767260993922,9598.853,0.9606575721,827.362102857851,7956328026836954420,-2.4386406E11 +185053572335775931,1504.547,0.5687599560,418.547722108074,-8496278226268642122,-1.1115532E-32 +406076905698949206,1584.821,0.9073638746,454.572717017033,9048728756392488113,-3.6374905E-7 +370626360548153088,6979.091,0.7359568576,847.384858615533,4440968842303227453,3.9915457E-28 +880266977576630903,3969.112,0.0536779482,32.241735406630,6086331466042394437,-2.1541503E-33 +914944620826734702,258.240,0.7421725875,413.873958949421,-7351387047220061849,-8.1657964E-36 +615190575294467984,8101.993,0.4021261498,440.941554482483,2251193521986836658,1.7728261E25 +748719415383076949,928.200,0.0701185195,307.250677745230,-255208803392541585,"" +"",8482.048,0.0875250620,889.559939012387,131095898465527538,-0.0 +541457418990604771,298.713,"",356.319195755119,8385038430064196353,-1.1713938E26 +998278945156788331,3126.578,0.0190951652,277.824073520913,5884635015085662155,2.7177546E22 +648685475062397368,955.643,0.4443410992,569.890024734869,-5773983929953240007,3.5561367E32 +128835062166543528,6830.727,"",967.668187303342,-7732708842585057603,-1.69050099E9 +979181842441473467,2627.146,0.0931873144,308.242155826852,6658806232357759343,-3.0718065E22 +373383053238313709,5880.783,0.8089926358,288.091654931223,2449340209240679641,1.8817666E-34 +369917108588895763,"",0.0784246739,419.063742180292,5469773751425577039,6.3262706E-10 +959109271119159898,5963.914,0.5131396358,"",-3786594002235441489,-1.0057181E33 +775063852272099258,0.000,0.1505213451,419.677059264876,-8824564747028895090,5.758003E-13 +495443091472185622,5536.609,0.6390268097,739.197371445563,1760002372390556825,6816057.5 +25146710634437912,3067.206,0.2161656375,215.056698028213,803782761142075607,-54089.32 +613858510238074126,957.490,0.4822075927,518.956895534893,2437026550498167082,-2.4735995E-5 +255794834605228757,899.249,0.8092193887,588.233634837773,-525199667559164571,"" +680825620982324925,5517.827,0.0065746462,193.251744967577,-5041734557262022470,-1485983.4 +714779301951357809,1711.747,0.6136524686,686.957950228997,-6839627708332633308,"" +381520952215527508,5424.226,0.2305893771,549.262462812814,-2059623267661777620,-3.7695377E-10 +669984683600664919,9136.075,0.7850895020,209.124878112725,-2710360188048040119,4.9402314E28 +182598193716752431,2503.031,0.0070287326,3.249783083502,-6633320123827272048,4.1270598E-11 +"",2596.200,0.8936980935,85.288926615679,662259464231680319,"" +40023869813864036,233.012,0.5436629243,171.799250634543,-3664974252843436884,-1.1522237E21 +710322854240541412,4019.609,"",446.549422285354,8834022966795744609,-1.00241495E-20 +841507640518767629,5834.872,0.4100687936,369.576552043052,6075221653337964625,-3.9562905E-11 +857872153237373925,4796.207,0.0756528306,575.779939808894,3893516458893827324,3.1058907E-18 +25775526817610369,548.169,0.0669838937,717.645262020503,-1,-2.9235664E-31 +25215292382715618,"",0.8464787764,632.667903196574,3221645619906578280,3.8987961E-19 +47171113226028385,8168.750,0.9055021741,35.058703401277,8776159873495597953,-3.7015653E-27 +943951645776822973,8443.481,0.3052030122,807.819184192606,-199240735297494389,2.9509407E32 +650796227970751779,6957.306,0.3522180815,680.869699320152,5264124301174436230,-8.20616E-30 +750204382327921920,"",0.9373742004,143.248267456298,-6228871387240662005,7.737559E-19 +612727546233536520,2940.034,0.5992235971,277.240099823954,9223372036854775807,-1.4193973E-5 +248544869063573270,4405.459,0.4938911635,185.838164288601,3780377771042673290,-4.794553E-20 +"",7286.797,0.0525265125,821.424386490011,8226475382227724456,-1.0 +797023704927729407,8307.766,0.3520962355,"",2370991850373366052,-9.2752095E-33 +733656328652598115,9387.470,"",292.371386489063,-772608278816658373,6.0993237 +456487310339748192,"",0.8716752095,759.109542616645,-8516846031062369157,-1.9623711E-20 +12735651342573944,463.904,0.7174409373,999.999999999999,8704401706019656847,1.6802378E27 +503142435912090290,4817.213,0.3841225840,9.127318172586,1196628311801793151,1.0 +227184013575342512,5661.152,0.3130044658,529.075843748348,-5761755906933392982,-1.2703339E27 +"",1738.648,0.1635048458,456.258113062836,1937750570385130232,-547.00464 +786474267467256790,9999.999,0.7247985984,246.846730055841,1,221027.05 +5722907039468293,1494.945,0.4833744532,521.189835443092,"",79.40845 +498893766826535453,-9999.999,0.8314235080,448.448825006248,-4757790292222279178,-3.1900585E26 +746704316885770352,7426.798,0.0697542832,933.708683277599,8123776869365306368,8367.281 +242857837293930678,72.927,0.4892817062,593.300614078262,1525346850676520527,2.4633369E-28 +267180693163965584,5567.777,0.1129542164,862.736499156329,-7943660520670333047,-2.5013383E34 +863179009301888915,"",0.5913615439,559.544347644078,-6218179828993870307,"" +628941572681954168,672.616,0.1120588881,726.724012853591,8295107456982566271,-3.601133E-26 +195085765272922078,3267.147,0.6227582353,942.184595963346,-7901787385806075056,5.208774E-15 +54795121483994580,7573.962,0.0232430822,530.957948073184,2519241821679825052,2.3091825E27 +11025911078984278,9080.678,0.9256236807,785.967159932085,3898714205093691132,-5.299004E-27 +585416995369880382,3180.974,0.6636002892,399.741020816154,8287847947794918753,-4.328721E-24 +93056155170236349,9851.699,0.4570285677,870.296415087233,7848530628017602134,1.6895455E-36 +273518414773911153,541.209,0.0030913568,794.949286358568,-9209970347163780353,9.984763E-9 +950579123902836569,2958.730,"","",-8408605963020785199,-152976.69 +294812000018063416,5947.866,0.3776552591,71.442236681325,"",1.89517168E8 +999999999999999999,4313.357,0.0052879447,534.027817103663,-1499033162028359415,3.3824432E-10 +47797026331225179,2523.889,0.0202628702,283.620687243088,9223372036854775807,-9.904269E28 +945353911554496947,9714.635,0.5445525623,577.494999825168,-4851177952341996882,5.4106984E7 +"",706.236,0.0574360283,868.681442920555,0,-9.979879E32 +362850502113699515,3648.749,0.4765617370,751.474476720116,4674971002257188808,1.8771697E32 +263209284245630819,9532.742,0.5552251822,191.831875430596,-2824608313884610115,1.6279153E-21 +553774330947428625,166.791,0.9306022422,79.427117038552,-5012912027038200187,9.0646643E10 +607563227695541003,4512.145,0.6783432188,901.316884058369,7603145021478615614,NaN +159291016019700529,5240.183,0.7258544713,350.623491123040,-1,-3.2958715E-15 +619900339634591041,7564.722,0.1887079611,994.209902879431,-6112563531377075787,-2.2229757E-33 +831245367052238890,4229.978,0.5127881028,639.349614940198,7280508167761829381,1.8915695E21 +868372001868812448,7565.051,0.1726353517,344.897092976059,1197973862304902753,-3.4028235E38 +"",8757.507,0.8479961242,429.256072226970,-503988022213926193,1.22784955E-32 +590855156246510004,1298.748,0.3022440975,961.920179785774,-4325271223339769315,-7.483853E-22 diff --git a/tests/src/test/resources/decimal-test.orc b/tests/src/test/resources/decimal-test.orc new file mode 100644 index 0000000000000000000000000000000000000000..8396738735f27c06926d476c4e1054fa7ffcafac GIT binary patch literal 4943 zcmb7I2~<!g-zrE&#xO0^S{O1ek$0iFpxGN5u@kTc8NV7$Zh%>`GS>v7T?A)AOY@HpQ&Vrr*;7q_f zzMGIh+>ZOkVG|y-BO2S;y1Ur9IJ-K#I@nvAIN8~{IJmkxIJi5xx}61o1c1{7{Hb@7 zoQMZ;-#WPC&ECGn)_ntrA`%V|f%DcxCs!wDH(M7M#M{}~Ik-7Go(0bWz%BxQ2ObL; zfj<$~iE84Qyzk@W!t zzxw4JxB9Q{`QOii>yXf(TQL}_R+bQu`eT3wjaZSc?8vuL$a47|;+8y+GgBOvBuq>~ z=hnwDC(~jhvDD|X*yy2>I1IVjCoof$D<>+`N-3O*dW{S$}7m7ZOlf;v0&~ z<{E?rmBGtvL(8XP%hbuW=8GMDUe7A?15E=JNo7qfuVSr~6EQ1Z6u&jo@yq>7s5P==yGu}-TVA~VPhWTJF> zG`BIos}9x`4MxZ_`xxsK0o`p8i=u3N+I*$5X(Wz~FYqt#PRuW9vYyos2G1l)gcg~D zZ({kyQGF)4vyt4m{ESNpk!8(Z)e}@RZU9eUpr`_|i;E1&lx$#{s0vcYG7b5fH5{+k z)A6$6dVKbfXfmZgxq;Xt(1i8Ulz!;i&ekzaZ%i;)tC3`BitD=8#7T1+c{$BlYb7C# z;{1St*J~-GLn|R!bzfC4`7vVZC6`1|i*F+5CF$TIT~)>r*2h|lXH~6{69Ili@~pTs ztt@%OJd`YsVkM4|%`%rOX3IjnW-$7$7YQ#58$v9_SyO(m+5BvCK}}v{9Z%J2tfhqv z@&zoLL3&A5T8C)Kq%Uon@68QK$7D9mM^yxbj9Sn$mEqJ1u6Y@!nHJwvG-lH0)$HB= zdPHeH#*WF7EvkDkNz(z@F4Mi<1Ei>b@SQY*~^g=YJUG`?| zO{)mZ93e+kkr_LdGnZd4jtsL1i_Dy`P}(aAq+0N%gb#GrV|f(;JWfVfp0$=-@Bd~w zHyWomuynJTU483{^4gaJ>s~(x)oMkHF1n6st&3Zb>eGV!@#2c^P?a($9m}RRB~}#V z=a@1wL)tP_RTBhNCM%&dYOWAPX{88NRBoJ^oLoRFmoog<3PiP3>ZE@VX;@vQlnjUE z6Oc0}_yz!wg9aMjEE)8ZRc{Oitq=lorF6I;eN||X5}PEupc~YIF_|rBNwOa7oAQB% zOYic!3$Q(n#c(#wF?=QhRm6vj)T!vUs6dz*K7(3ZcnKyD?};!H z%ZtI$G9j2Y--5{%4#I*6SmxD@rG|mzf&d(^eA}RW1=6+r6g(=#M0N@w=Hx4MB&7ol zh{jXUfrUDV-myxhy5m!ujP_ZSg4JORG=tTJo2bcyHkqh(*N-7BFL>I;^h#zCh}>NNqp~eDl${OgX6UtTqx=7J(~> zSJu<9l`}YzJRL0!I>#6Zh9h4lU>hprIAt0YRyXjBcz%Z|A=?Uk+t6{NM0us{u7w6pt;)qM~%=@mL>f7H+I0U~10RaQNb;!UC05J2^V9@s(CJnzE-WaqYFd#1E z6s;d?&R2M-Lb9@VDO$y}O}g;OuCOanU_Afx+804ZU|(%*&}J=7F^|dAM5KI>#+uWh z10&lLw==XA@vA7_%*K5aexe{kHm5_kwX&J>>;ryB=YfPumePj7@Q*g*hbYI;e1rFq zVBclQVV(DYU?(|QDSmHIR?&>+mA=-ahErSo4l1}cN%Zp__+*qYK<`W{5 z?M`+c>rHg*fJv+;=a#xh)gN*3t8A8)oF)V6d`y>1fP?q{LnkTkEq$ zAg$QITxe~yF!>dVO$^zi9X2W6p1V^0)UO30B;T)0sxHpX@CfW=uH9>a{vhi1T@wO ze47-QzDr=t-|hqha-AagA%e(Gs7C{e_K3i)fgWhNb(XAM+BrO~>7VX`Ys2#rn>0TU zvqOq!vYVkceouXcJ|9X*_E!xq1%SgbHElwT2-LB)+{D*WV2@s-9!upyf$HbU5qUIl zmOMfBX(@mz$zffwNdj0G-IdkiWy6>VPcM!bfZ`?=rg%Omc0^Ln$}=_S6BgYxzcK_B zwNDl%O**U?@*Cv6VmPjsUP>Q2s}oLm4>RW>`E-AEJGT-HXGOL)b~S>EB~|pZOfPss zUbWO(4k~K3sm*bGP(52y&k)sux?yrd@v@0n*+YqY5efIy6;BDMY>=(wwNz$2#t*ai(4C$A4uR0Tl1?19d1?U(dnNnJdz1WKUyu~G*XKyELizcuiR@}*oA zSpn+%2Pyo)?L7&zk<`%Baz9#mgI_(^Q7fH__TzxdomFG%Q5ux3Z<{F16hnMnKwOA| z1uas^o$d3c{Hf&mX)<*XB#ZJNnSAaCD@Dv?RjD4#e#RJ`;Y*;@Hx=yc zvP@Xe5~Uw5odQQ?ORedJLr_7WszX0e2vWG+lDYl>SX#pB4ik((ioC?|%F0bW)rHT5 zW7A+#qW94BG8s%7i>5U<`ao^TNkMbo0r0Rv8&TKY3i+k9j?tN4=1$gxu|u{|s@VbJ z$ym5AWMq7L-ZVI%%BwlJ8g39QkJ5VCkWQCV-=awXWx9~g2GJU&V0t8^xDa+nYEy`0 zX)drJ0jFV063J5V0TKb00O>SnNm>g4`XxPVSkfDo4BBZlK*4CKPp_<6`b|f8LY+F< zrJQQGH@G(VNejFC?*6N{OZKT+wsx6y)Z7R?6{ftW*kBH%_{(;DSoLv1=V;OHttI)r zZAU&H`<*73&p9>z`bK8*1Ld?P;?(M@ZRj5<9sim%uN3BnvF(VdA zE?9l>x2JOtSJPvAEMUdsGT|>s?TUtHNubBVq)lJ$);Vrr|F!!Y$73aT7Pl6fVKypT z*I#jSRNT}w=MwgOR1~qa!=lvjSm(9h4lf-AHU>WCj4%K5;p1R>1HNkOYt5(j2Y$=j zb?1t`C)VQfI*o7oW;o@kaXabN;Bff~lb>fEK`G}u_>bU&jHJV?sN%o-2DK$Q-iPC5 zw36oMi)*uGJx7+}RRfiu@S>Z|pI)B*mifKa1P+FpWY~SKJR_+YhkhVhN`%o8kx_RXz3alLFLe4XMP8yqmsRMi`Ewqr+-p9m1r zp3vVMcU*TR^nLTe^Q^$9Up1fosBy&P!+gpKi+>g!Y)bB4>V z_cpr(yICJF*RwQytvJv)!?`v^ zt8w`;SN;Aj>TKV=s@UgArXflE^Y~6YIC8DxeWyK{-|UJ%c6a*GhvNoNychX;`#j4# zdC>Y+3Sa_0@e*>2jiSzkp8n1_I_lPXTELe6t_;bwxfX1U$4SgjPfT_sfzLVLo~-$` zAxg4lqjP%T2l&m0Hi-Iblp>QA<6al{va7!st7+1j@|t%y#0YW#b@r_u#9T^I+gj!D^1vI?Y*NC%sy{-L&sk1D>4xjF)5`DYCqH>n;HO_UE1GlgZ{ga|a*}U- zk8F4HLy2(IcUOsLqaIp8MqUm+zCC zikCbK>wNb&<)kESyc#f;T0UaOToM5(VXWV? zw)34n<;;}svx1W*72lmc<=uQ#x6Q)*;TWJ>pWDv)45^=gw|aUf{pTz<|2T_R*9=Hy zmJR=or|i3ya2f;@;Ceg;hamu@RcIUzB-tVek?awKNzMqONFE5HNqf+^+Zd7^G8Q0! zBmy3b0}=lcgCx*MfJIv&(@fD;#t>-$diUUjAWSktQZPvZzkAq+wgONlR>mmOFd9jq zNE;DElUAc05L^bWi~$0MWOQUFhBN~aAsm1hh&zBn=(2=>JSY^O4dLIZ68NKt5V8MH zg$VZp5a4IM#@^o9$;sW`-qFt4-oe4u!O6|_EO@a8gWvs!C;>W}|Ft#@Q42`K%g&J{GmUT{JH*cF$RO*_-CpE8UjG@2GYCj{}dW7QsMVtKmdp0 zU_#VpYZJs3K%{~Vo4DXk&n-J~#zYJV!6-Dw8oqwT4nza?h&dqUh?o;nwK;FW;8yP> z+5rf$SUVU&yd55m)1EW6Gu{H>P>7*-Z^o1E-v5W~)q7X3-@IY_;NI2yKiEFHasR>h zcYh?J9c}Gw?d?20-f`ja7hJe|{3RJ4|0cuXugLKD&&lxk|B>PG7i8STc>J$qph2g% nnh!$7q0rMOfQuk5940^{GMTUi0*rSCLv~ooEDWDHcf{&ngU@0J literal 0 HcmV?d00001 diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala b/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala index 7bd4467a6ae..f5886b58d40 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala @@ -21,10 +21,16 @@ import java.math.RoundingMode import scala.util.Random import ai.rapids.cudf.{ColumnVector, DType, HostColumnVector} -import com.nvidia.spark.rapids.{GpuAlias, GpuColumnVector, GpuIsNotNull, GpuIsNull, GpuLiteral, GpuOverrides, GpuScalar, GpuUnitTests, HostColumnarToGpu, RapidsConf} +import com.nvidia.spark.rapids.{GpuAlias, GpuBatchScanExec, GpuColumnVector, GpuIsNotNull, GpuIsNull, GpuLiteral, GpuOverrides, GpuScalar, GpuUnitTests, HostColumnarToGpu, RapidsConf} +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Literal} -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, LongType, StructField, StructType} class DecimalUnitTest extends GpuUnitTests { Random.setSeed(1234L) @@ -260,4 +266,34 @@ class DecimalUnitTest extends GpuUnitTests { } } } + + test("test type checking of Scans") { + val conf = new SparkConf().set(RapidsConf.DECIMAL_TYPE_ENABLED.key, "true") + .set(RapidsConf.TEST_ALLOWED_NONGPU.key, "BatchScanExec,ColumnarToRowExec,FileSourceScanExec") + val decimalCsvStruct = StructType(Array( + StructField("c_0", DecimalType(18, 0), true), + StructField("c_1", DecimalType(7, 3), true), + StructField("c_2", DecimalType(10, 10), true), + StructField("c_3", DecimalType(15, 12), true), + StructField("c_4", LongType, true), + StructField("c_5", IntegerType, true))) + + withGpuSparkSession((ss: SparkSession) => { + var rootPlan = frameFromOrc("decimal-test.orc")(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[FileSourceScanExec])) + rootPlan = fromCsvDf("decimal-test.csv", decimalCsvStruct)(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[FileSourceScanExec])) + rootPlan = frameFromParquet("decimal-test.parquet")(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuFileSourceScanExec])) + }, conf) + + withGpuSparkSession((ss: SparkSession) => { + var rootPlan = frameFromOrc("decimal-test.orc")(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[BatchScanExec])) + rootPlan = fromCsvDf("decimal-test.csv", decimalCsvStruct)(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[BatchScanExec])) + rootPlan = frameFromParquet("decimal-test.parquet")(ss).queryExecution.executedPlan + assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuBatchScanExec])) + }, conf.set(SQLConf.USE_V1_SOURCE_LIST.key, "")) + } } From a2d6bee406f6fde9b362b34e55cca83b0a47f7ef Mon Sep 17 00:00:00 2001 From: sperlingxx Date: Tue, 5 Jan 2021 12:52:50 +0800 Subject: [PATCH 15/15] revert --- docs/supported_ops.md | 8 ++++---- .../main/scala/com/nvidia/spark/rapids/TypeChecks.scala | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 1b72cc087c3..91e4c9ae3a6 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16592,13 +16592,13 @@ dates or timestamps, or for a lack of type coercion support. S S S -NS +S NS -PS (missing nested DECIMAL, BINARY) -PS (missing nested DECIMAL, BINARY) -PS (missing nested DECIMAL, BINARY) +PS (missing nested BINARY) +PS (missing nested BINARY) +PS (missing nested BINARY) Output diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 231f4a51021..c0badbed029 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1409,13 +1409,13 @@ object SupportedOpsDocs { println("S") // DATE println("S") // TIMESTAMP println("S") // STRING - println("NS") // DECIMAL + println("S") // DECIMAL println("") // NULL println("NS") // BINARY println("") // CALENDAR - println("PS (missing nested DECIMAL, BINARY)") // ARRAY - println("PS (missing nested DECIMAL, BINARY)") // MAP - println("PS (missing nested DECIMAL, BINARY)") // STRUCT + println("PS (missing nested BINARY)") // ARRAY + println("PS (missing nested BINARY)") // MAP + println("PS (missing nested BINARY)") // STRUCT println("") println("") println("Output")