Hi, Wenchen. This thread seems to get enough attention. Also, I'm expecting more and more if we have this on the `master` branch because we are developing together.
> Spark SQL has many active contributors/committers and this thread doesn't get much attention yet. So, what's your ETA from now? > I think the problem here is we were discussing some very detailed things without actual code. > I'll implement my idea after the holiday and then we can have more effective discussions. > We can also do benchmarks and get some real numbers. > In the meantime, we can continue to discuss other parts of this proposal, and make a prototype if possible. I'm looking forward to seeing your PR. I hope we can conclude this thread and have at least one implementation in the `master` branch this month (February). If you need more time (one month or longer), why don't we have Ryan's suggestion in the `master` branch first and benchmark with your PR later during Apache Spark 3.2 timeframe. Bests, Dongjoon. On Tue, Feb 16, 2021 at 9:26 AM Ryan Blue <rb...@netflix.com.invalid> wrote: > Andrew, > > The proposal already includes an API for aggregate functions and I think > we would want to implement those right away. > > Processing ColumnBatch is something we can easily extend the interfaces to > support, similar to Wenchen's suggestion. The important thing right now is > to agree on some basic functionality: how to look up functions and what the > simple API should be. Like the TableCatalog interfaces, we will layer on > more support through optional interfaces like `SupportsInvoke` or > `SupportsColumnBatch`. > > On Tue, Feb 16, 2021 at 9:00 AM Andrew Melo <andrew.m...@gmail.com> wrote: > >> Hello Ryan, >> >> This proposal looks very interesting. Would future goals for this >> functionality include both support for aggregation functions, as well >> as support for processing ColumnBatch-es (instead of Row/InternalRow)? >> >> Thanks >> Andrew >> >> On Mon, Feb 15, 2021 at 12:44 PM Ryan Blue <rb...@netflix.com.invalid> >> wrote: >> > >> > Thanks for the positive feedback, everyone. It sounds like there is a >> clear path forward for calling functions. Even without a prototype, the >> `invoke` plans show that Wenchen's suggested optimization can be done, and >> incorporating it as an optional extension to this proposal solves many of >> the unknowns. >> > >> > With that area now understood, is there any discussion about other >> parts of the proposal, besides the function call interface? >> > >> > On Fri, Feb 12, 2021 at 10:40 PM Chao Sun <sunc...@apache.org> wrote: >> >> >> >> This is an important feature which can unblock several other projects >> including bucket join support for DataSource v2, complete support for >> enforcing DataSource v2 distribution requirements on the write path, etc. I >> like Ryan's proposals which look simple and elegant, with nice support on >> function overloading and variadic arguments. On the other hand, I think >> Wenchen made a very good point about performance. Overall, I'm excited to >> see active discussions on this topic and believe the community will come to >> a proposal with the best of both sides. >> >> >> >> Chao >> >> >> >> On Fri, Feb 12, 2021 at 7:58 PM Hyukjin Kwon <gurwls...@gmail.com> >> wrote: >> >>> >> >>> +1 for Liang-chi's. >> >>> >> >>> Thanks Ryan and Wenchen for leading this. >> >>> >> >>> >> >>> 2021년 2월 13일 (토) 오후 12:18, Liang-Chi Hsieh <vii...@gmail.com>님이 작성: >> >>>> >> >>>> Basically I think the proposal makes sense to me and I'd like to >> support the >> >>>> SPIP as it looks like we have strong need for the important feature. >> >>>> >> >>>> Thanks Ryan for working on this and I do also look forward to >> Wenchen's >> >>>> implementation. Thanks for the discussion too. >> >>>> >> >>>> Actually I think the SupportsInvoke proposed by Ryan looks a good >> >>>> alternative to me. Besides Wenchen's alternative implementation, is >> there a >> >>>> chance we also have the SupportsInvoke for comparison? >> >>>> >> >>>> >> >>>> John Zhuge wrote >> >>>> > Excited to see our Spark community rallying behind this important >> feature! >> >>>> > >> >>>> > The proposal lays a solid foundation of minimal feature set with >> careful >> >>>> > considerations for future optimizations and extensions. Can't wait >> to see >> >>>> > it leading to more advanced functionalities like views with shared >> custom >> >>>> > functions, function pushdown, lambda, etc. It has already borne >> fruit from >> >>>> > the constructive collaborations in this thread. Looking forward to >> >>>> > Wenchen's prototype and further discussions including the >> SupportsInvoke >> >>>> > extension proposed by Ryan. >> >>>> > >> >>>> > >> >>>> > On Fri, Feb 12, 2021 at 4:35 PM Owen O'Malley < >> >>>> >> >>>> > owen.omalley@ >> >>>> >> >>>> > > >> >>>> > wrote: >> >>>> > >> >>>> >> I think this proposal is a very good thing giving Spark a >> standard way of >> >>>> >> getting to and calling UDFs. >> >>>> >> >> >>>> >> I like having the ScalarFunction as the API to call the UDFs. It >> is >> >>>> >> simple, yet covers all of the polymorphic type cases well. I >> think it >> >>>> >> would >> >>>> >> also simplify using the functions in other contexts like pushing >> down >> >>>> >> filters into the ORC & Parquet readers although there are a lot of >> >>>> >> details >> >>>> >> that would need to be considered there. >> >>>> >> >> >>>> >> .. Owen >> >>>> >> >> >>>> >> >> >>>> >> On Fri, Feb 12, 2021 at 11:07 PM Erik Krogen < >> >>>> >> >>>> > ekrogen@.com >> >>>> >> >>>> > > >> >>>> >> wrote: >> >>>> >> >> >>>> >>> I agree that there is a strong need for a FunctionCatalog within >> Spark >> >>>> >>> to >> >>>> >>> provide support for shareable UDFs, as well as make movement >> towards >> >>>> >>> more >> >>>> >>> advanced functionality like views which themselves depend on >> UDFs, so I >> >>>> >>> support this SPIP wholeheartedly. >> >>>> >>> >> >>>> >>> I find both of the proposed UDF APIs to be sufficiently >> user-friendly >> >>>> >>> and >> >>>> >>> extensible. I generally think Wenchen's proposal is easier for a >> user to >> >>>> >>> work with in the common case, but has greater potential for >> confusing >> >>>> >>> and >> >>>> >>> hard-to-debug behavior due to use of reflective method signature >> >>>> >>> searches. >> >>>> >>> The merits on both sides can hopefully be more properly examined >> with >> >>>> >>> code, >> >>>> >>> so I look forward to seeing an implementation of Wenchen's ideas >> to >> >>>> >>> provide >> >>>> >>> a more concrete comparison. I am optimistic that we will not let >> the >> >>>> >>> debate >> >>>> >>> over this point unreasonably stall the SPIP from making progress. >> >>>> >>> >> >>>> >>> Thank you to both Wenchen and Ryan for your detailed >> consideration and >> >>>> >>> evaluation of these ideas! >> >>>> >>> ------------------------------ >> >>>> >>> *From:* Dongjoon Hyun < >> >>>> >> >>>> > dongjoon.hyun@ >> >>>> >> >>>> > > >> >>>> >>> *Sent:* Wednesday, February 10, 2021 6:06 PM >> >>>> >>> *To:* Ryan Blue < >> >>>> >> >>>> > blue@ >> >>>> >> >>>> > > >> >>>> >>> *Cc:* Holden Karau < >> >>>> >> >>>> > holden@ >> >>>> >> >>>> > >; Hyukjin Kwon < >> >>>> >>> >> >>>> >> >>>> > gurwls223@ >> >>>> >> >>>> >>; Spark Dev List < >> >>>> >> >>>> > dev@.apache >> >>>> >> >>>> > >; Wenchen Fan >> >>>> >>> < >> >>>> >> >>>> > cloud0fan@ >> >>>> >> >>>> > > >> >>>> >>> *Subject:* Re: [DISCUSS] SPIP: FunctionCatalog >> >>>> >>> >> >>>> >>> BTW, I forgot to add my opinion explicitly in this thread >> because I was >> >>>> >>> on the PR before this thread. >> >>>> >>> >> >>>> >>> 1. The `FunctionCatalog API` PR was made on May 9, 2019 and has >> been >> >>>> >>> there for almost two years. >> >>>> >>> 2. I already gave my +1 on that PR last Saturday because I >> agreed with >> >>>> >>> the latest updated design docs and AS-IS PR. >> >>>> >>> >> >>>> >>> And, the rest of the progress in this thread is also very >> satisfying to >> >>>> >>> me. >> >>>> >>> (e.g. Ryan's extension suggestion and Wenchen's alternative) >> >>>> >>> >> >>>> >>> To All: >> >>>> >>> Please take a look at the design doc and the PR, and give us some >> >>>> >>> opinions. >> >>>> >>> We really need your participation in order to make DSv2 more >> complete. >> >>>> >>> This will unblock other DSv2 features, too. >> >>>> >>> >> >>>> >>> Bests, >> >>>> >>> Dongjoon. >> >>>> >>> >> >>>> >>> >> >>>> >>> >> >>>> >>> On Wed, Feb 10, 2021 at 10:58 AM Dongjoon Hyun < >> >>>> >> >>>> > dongjoon.hyun@ >> >>>> >> >>>> > > >> >>>> >>> wrote: >> >>>> >>> >> >>>> >>> Hi, Ryan. >> >>>> >>> >> >>>> >>> We didn't move past anything (both yours and Wenchen's). What >> Wenchen >> >>>> >>> suggested is double-checking the alternatives with the >> implementation to >> >>>> >>> give more momentum to our discussion. >> >>>> >>> >> >>>> >>> Your new suggestion about optional extention also sounds like a >> new >> >>>> >>> reasonable alternative to me. >> >>>> >>> >> >>>> >>> We are still discussing this topic together and I hope we can >> make a >> >>>> >>> conclude at this time (for Apache Spark 3.2) without being >> stucked like >> >>>> >>> last time. >> >>>> >>> >> >>>> >>> I really appreciate your leadership in this dicsussion and the >> moving >> >>>> >>> direction of this discussion looks constructive to me. Let's >> give some >> >>>> >>> time >> >>>> >>> to the alternatives. >> >>>> >>> >> >>>> >>> Bests, >> >>>> >>> Dongjoon. >> >>>> >>> >> >>>> >>> On Wed, Feb 10, 2021 at 10:14 AM Ryan Blue < >> >>>> >> >>>> > blue@ >> >>>> >> >>>> > > wrote: >> >>>> >>> >> >>>> >>> I don’t think we should so quickly move past the drawbacks of >> this >> >>>> >>> approach. The problems are significant enough that using invoke >> is not >> >>>> >>> sufficient on its own. But, I think we can add it as an optional >> >>>> >>> extension >> >>>> >>> to shore up the weaknesses. >> >>>> >>> >> >>>> >>> Here’s a summary of the drawbacks: >> >>>> >>> >> >>>> >>> - Magic function signatures are error-prone >> >>>> >>> - Spark would need considerable code to help users find what >> went >> >>>> >>> wrong >> >>>> >>> - Spark would likely need to coerce arguments (e.g., String, >> >>>> >>> Option[Int]) for usability >> >>>> >>> - It is unclear how Spark will find the Java Method to call >> >>>> >>> - Use cases that require varargs fall back to casting; users >> will >> >>>> >>> also get this wrong (cast to String instead of UTF8String) >> >>>> >>> - The non-codegen path is significantly slower >> >>>> >>> >> >>>> >>> The benefit of invoke is to avoid moving data into a row, like >> this: >> >>>> >>> >> >>>> >>> -- using invoke >> >>>> >>> int result = udfFunction(x, y) >> >>>> >>> >> >>>> >>> -- using row >> >>>> >>> udfRow.update(0, x); -- actual: values[0] = x; >> >>>> >>> udfRow.update(1, y); >> >>>> >>> int result = udfFunction(udfRow); >> >>>> >>> >> >>>> >>> And, again, that won’t actually help much in cases that require >> varargs. >> >>>> >>> >> >>>> >>> I suggest we add a new marker trait for BoundMethod called >> >>>> >>> SupportsInvoke. >> >>>> >>> If that interface is implemented, then Spark will look for a >> method that >> >>>> >>> matches the expected signature based on the bound input type. If >> it >> >>>> >>> isn’t >> >>>> >>> found, Spark can print a warning and fall back to the >> InternalRow call: >> >>>> >>> “Cannot find udfFunction(int, int)”. >> >>>> >>> >> >>>> >>> This approach allows the invoke optimization, but solves many of >> the >> >>>> >>> problems: >> >>>> >>> >> >>>> >>> - The method to invoke is found using the proposed load and >> bind >> >>>> >>> approach >> >>>> >>> - Magic function signatures are optional and do not cause >> runtime >> >>>> >>> failures >> >>>> >>> - Because this is an optional optimization, Spark can be more >> strict >> >>>> >>> about types >> >>>> >>> - Varargs cases can still use rows >> >>>> >>> - Non-codegen can use an evaluation method rather than >> falling back >> >>>> >>> to slow Java reflection >> >>>> >>> >> >>>> >>> This seems like a good extension to me; this provides a plan for >> >>>> >>> optimizing the UDF call to avoid building a row, while the >> existing >> >>>> >>> proposal covers the other cases well and addresses how to locate >> these >> >>>> >>> function calls. >> >>>> >>> >> >>>> >>> This also highlights that the approach used in DSv2 and this >> proposal is >> >>>> >>> working: start small and use extensions to layer on more complex >> >>>> >>> support. >> >>>> >>> >> >>>> >>> On Wed, Feb 10, 2021 at 9:04 AM Dongjoon Hyun < >> >>>> >> >>>> > dongjoon.hyun@ >> >>>> >> >>>> > > >> >>>> >>> wrote: >> >>>> >>> >> >>>> >>> Thank you all for making a giant move forward for Apache Spark >> 3.2.0. >> >>>> >>> I'm really looking forward to seeing Wenchen's implementation. >> >>>> >>> That would be greatly helpful to make a decision! >> >>>> >>> >> >>>> >>> > I'll implement my idea after the holiday and then we can have >> >>>> >>> more effective discussions. We can also do benchmarks and get >> some real >> >>>> >>> numbers. >> >>>> >>> > FYI: the Presto UDF API >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fprestodb.io%2Fdocs%2Fcurrent%2Fdevelop%2Ffunctions.html&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067978066%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=iMWmHqqXPcT7EK%2Bovyzhy%2BZpU6Llih%2BwdZD53wvobmc%3D&reserved=0> >> ; >> >>>> >>> also >> >>>> >>> takes individual parameters instead of the row parameter. I >> think this >> >>>> >>> direction at least worth a try so that we can see the performance >> >>>> >>> difference. It's also mentioned in the design doc as an >> alternative >> >>>> >>> (Trino). >> >>>> >>> >> >>>> >>> Bests, >> >>>> >>> Dongjoon. >> >>>> >>> >> >>>> >>> >> >>>> >>> On Tue, Feb 9, 2021 at 10:18 PM Wenchen Fan < >> >>>> >> >>>> > cloud0fan@ >> >>>> >> >>>> > > wrote: >> >>>> >>> >> >>>> >>> FYI: the Presto UDF API >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fprestodb.io%2Fdocs%2Fcurrent%2Fdevelop%2Ffunctions.html&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067988024%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=ZSBCR7yx3PpwL4KY9V73JG42Z02ZodqkjxC0LweHt1g%3D&reserved=0> >> ; >> >>>> >>> also takes individual parameters instead of the row parameter. I >> think >> >>>> >>> this >> >>>> >>> direction at least worth a try so that we can see the performance >> >>>> >>> difference. It's also mentioned in the design doc as an >> alternative >> >>>> >>> (Trino). >> >>>> >>> >> >>>> >>> On Wed, Feb 10, 2021 at 10:18 AM Wenchen Fan < >> >>>> >> >>>> > cloud0fan@ >> >>>> >> >>>> > > wrote: >> >>>> >>> >> >>>> >>> Hi Holden, >> >>>> >>> >> >>>> >>> As Hyukjin said, following existing designs is not the principle >> of DS >> >>>> >>> v2 >> >>>> >>> API design. We should make sure the DS v2 API makes sense. AFAIK >> we >> >>>> >>> didn't >> >>>> >>> fully follow the catalog API design from Hive and I believe Ryan >> also >> >>>> >>> agrees with it. >> >>>> >>> >> >>>> >>> I think the problem here is we were discussing some very >> detailed things >> >>>> >>> without actual code. I'll implement my idea after the holiday >> and then >> >>>> >>> we >> >>>> >>> can have more effective discussions. We can also do benchmarks >> and get >> >>>> >>> some >> >>>> >>> real numbers. >> >>>> >>> >> >>>> >>> In the meantime, we can continue to discuss other parts of this >> >>>> >>> proposal, >> >>>> >>> and make a prototype if possible. Spark SQL has many active >> >>>> >>> contributors/committers and this thread doesn't get much >> attention yet. >> >>>> >>> >> >>>> >>> On Wed, Feb 10, 2021 at 6:17 AM Hyukjin Kwon < >> >>>> >> >>>> > gurwls223@ >> >>>> >> >>>> > > wrote: >> >>>> >>> >> >>>> >>> Just dropping a few lines. I remember that one of the goals in >> DSv2 is >> >>>> >>> to >> >>>> >>> correct the mistakes we made in the current Spark codes. >> >>>> >>> It would not have much point if we will happen to just follow >> and mimic >> >>>> >>> what Spark currently does. It might just end up with another >> copy of >> >>>> >>> Spark >> >>>> >>> APIs, e.g. Expression (internal) APIs. I sincerely would like to >> avoid >> >>>> >>> this >> >>>> >>> I do believe we have been stuck mainly due to trying to come up >> with a >> >>>> >>> better design. We already have an ugly picture of the current >> Spark APIs >> >>>> >>> to >> >>>> >>> draw a better bigger picture. >> >>>> >>> >> >>>> >>> >> >>>> >>> 2021년 2월 10일 (수) 오전 3:28, Holden Karau < >> >>>> >> >>>> > holden@ >> >>>> >> >>>> > >님이 작성: >> >>>> >>> >> >>>> >>> I think this proposal is a good set of trade-offs and has >> existed in the >> >>>> >>> community for a long period of time. I especially appreciate how >> the >> >>>> >>> design >> >>>> >>> is focused on a minimal useful component, with future >> optimizations >> >>>> >>> considered from a point of view of making sure it's flexible, >> but actual >> >>>> >>> concrete decisions left for the future once we see how this API >> is used. >> >>>> >>> I >> >>>> >>> think if we try and optimize everything right out of the gate, >> we'll >> >>>> >>> quickly get stuck (again) and not make any progress. >> >>>> >>> >> >>>> >>> On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue < >> >>>> >> >>>> > blue@ >> >>>> >> >>>> > > wrote: >> >>>> >>> >> >>>> >>> Hi everyone, >> >>>> >>> >> >>>> >>> I'd like to start a discussion for adding a FunctionCatalog >> interface to >> >>>> >>> catalog plugins. This will allow catalogs to expose functions to >> Spark, >> >>>> >>> similar to how the TableCatalog interface allows a catalog to >> expose >> >>>> >>> tables. The proposal doc is available here: >> >>>> >>> >> https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fdocs.google.com%2Fdocument%2Fd%2F1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U%2Fedit&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067988024%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=Kyth8%2FhNUZ6GXG2FsgcknZ7t7s0%2BpxnDMPyxvsxLLqE%3D&reserved=0> >> ; >> >>>> >>> >> >>>> >>> Here's a high-level summary of some of the main design choices: >> >>>> >>> * Adds the ability to list and load functions, not to create or >> modify >> >>>> >>> them in an external catalog >> >>>> >>> * Supports scalar, aggregate, and partial aggregate functions >> >>>> >>> * Uses load and bind steps for better error messages and simpler >> >>>> >>> implementations >> >>>> >>> * Like the DSv2 table read and write APIs, it uses InternalRow >> to pass >> >>>> >>> data >> >>>> >>> * Can be extended using mix-in interfaces to add vectorization, >> codegen, >> >>>> >>> and other future features >> >>>> >>> >> >>>> >>> There is also a PR with the proposed API: >> >>>> >>> https://github.com/apache/spark/pull/24559/files >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Fspark%2Fpull%2F24559%2Ffiles&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067988024%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=t3ZCqffdsrmCY3X%2FT8x1oMjMcNUiQ0wQNk%2ByAXQx1Io%3D&reserved=0> >> ; >> >>>> >>> >> >>>> >>> Let's discuss the proposal here rather than on that PR, to get >> better >> >>>> >>> visibility. Also, please take the time to read the proposal >> first. That >> >>>> >>> really helps clear up misconceptions. >> >>>> >>> >> >>>> >>> >> >>>> >>> >> >>>> >>> -- >> >>>> >>> Ryan Blue >> >>>> >>> >> >>>> >>> >> >>>> >>> >> >>>> >>> -- >> >>>> >>> Twitter: https://twitter.com/holdenkarau >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Ftwitter.com%2Fholdenkarau&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067997978%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=fVfSPIyazuUYv8VLfNu%2BUIHdc3ePM1AAKKH%2BlnIicF8%3D&reserved=0> >> ; >> >>>> >>> Books (Learning Spark, High Performance Spark, etc.): >> >>>> >>> https://amzn.to/2MaRAG9 >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Famzn.to%2F2MaRAG9&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060067997978%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=NbRl9kK%2B6Wy0jWmDnztYp3JCPNLuJvmFsLHUrXzEhlk%3D&reserved=0> >> ; >> >>>> >>> YouTube Live Streams: https://www.youtube.com/user/holdenkarau >> >>>> >>> < >> https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fwww.youtube.com%2Fuser%2Fholdenkarau&data=04%7C01%7Cekrogen%40linkedin.com%7C0ccf8c15abd74dfc974f08d8ce31ae4d%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637486060068007935%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=OWXOBELzO3hBa2JI%2FOSBZ3oNyLq0yr%2FGXMkNn7bqYDM%3D&reserved=0> >> ; >> >>>> >>> >> >>>> >>> -- >> >>>> >>> Ryan Blue >> >>>> >>> >> >>>> >>> >> >>>> > >> >>>> > -- >> >>>> > John Zhuge >> >>>> >> >>>> >> >>>> >> >>>> >> >>>> >> >>>> -- >> >>>> Sent from: >> http://apache-spark-developers-list.1001551.n3.nabble.com/ >> >>>> >> >>>> --------------------------------------------------------------------- >> >>>> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org >> >>>> >> > >> > >> > -- >> > Ryan Blue >> > Software Engineer >> > Netflix >> >> --------------------------------------------------------------------- >> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org >> >> > > -- > Ryan Blue > Software Engineer > Netflix >