Val, thanks for the review. Can I ask you to add the same comments to the ticket?
On Tue, Oct 17, 2017 at 3:20 PM, Valentin Kulichenko < valentin.kuliche...@gmail.com> wrote: > Nikolay, Anton, > > I did a high level review of the code. First of all, impressive results! > However, I have some questions/comments. > > 1. Why do we have org.apache.spark.sql.ignite package in our codebase? Can > these classes reside under org.apache.ignite.spark instead? > 2. IgniteRelationProvider contains multiple constants which I guess are > some king of config options. Can you describe the purpose of each of them? > 3. IgniteCatalog vs. IgniteExternalCatalog. Why do we have two Catalog > implementations and what is the difference? > 4. IgniteStrategy and IgniteOptimization are currently no-op. What are our > plans on implementing them? Also, what exactly is planned in > IgniteOptimization and what is its purpose? > 5. I don't like that IgniteStrategy and IgniteOptimization have to be set > manually on SQLContext each time it's created. This seems to be very error > prone. Is there any way to automate this and improve usability? > 6. What is the purpose of IgniteSparkSession? I see it's used > in IgniteCatalogExample but not in IgniteDataFrameExample, which is > confusing. > 7. To create IgniteSparkSession we first create IgniteContext. Is it really > needed? It looks like we can directly provide the configuration file; if > IgniteSparkSession really requires IgniteContext, it can create it by > itself under the hood. Actually, I think it makes sense to create a builder > similar to SparkSession.builder(), it would be good if our APIs here are > consistent with Spark APIs. > 8. Can you clarify the query syntax > inIgniteDataFrameExample#nativeSparkSqlFromCacheExample2? > 9. Do I understand correctly that IgniteCacheRelation is for the case when > we don't have SQL configured on Ignite side? I thought we decided not to > support this, no? Or this is something else? > > Thanks! > > -Val > > On Tue, Oct 17, 2017 at 4:40 AM, Anton Vinogradov < > avinogra...@gridgain.com> > wrote: > > > Sounds awesome. > > > > I'll try to review API & tests this week. > > > > Val, > > Your review still required :) > > > > On Tue, Oct 17, 2017 at 2:36 PM, Николай Ижиков <nizhikov....@gmail.com> > > wrote: > > > > > Yes > > > > > > 17 окт. 2017 г. 2:34 PM пользователь "Anton Vinogradov" < > > > avinogra...@gridgain.com> написал: > > > > > > > Nikolay, > > > > > > > > So, it will be able to start regular spark and ignite clusters and, > > using > > > > peer classloading via spark-context, perform any DataFrame request, > > > > correct? > > > > > > > > On Tue, Oct 17, 2017 at 2:25 PM, Николай Ижиков < > > nizhikov....@gmail.com> > > > > wrote: > > > > > > > > > Hello, Anton. > > > > > > > > > > An example you provide is a path to a master *local* file. > > > > > These libraries are added to the classpath for each remote node > > running > > > > > submitted job. > > > > > > > > > > Please, see documentation: > > > > > > > > > > http://spark.apache.org/docs/latest/api/java/org/apache/ > > > > > spark/SparkContext.html#addJar(java.lang.String) > > > > > http://spark.apache.org/docs/latest/api/java/org/apache/ > > > > > spark/SparkContext.html#addFile(java.lang.String) > > > > > > > > > > > > > > > 2017-10-17 13:10 GMT+03:00 Anton Vinogradov < > > avinogra...@gridgain.com > > > >: > > > > > > > > > > > Nikolay, > > > > > > > > > > > > > With Data Frame API implementation there are no requirements to > > > have > > > > > any > > > > > > > Ignite files on spark worker nodes. > > > > > > > > > > > > What do you mean? I see code like: > > > > > > > > > > > > spark.sparkContext.addJar(MAVEN_HOME + > > > > > > "/org/apache/ignite/ignite-core/2.3.0-SNAPSHOT/ignite- > > > > > > core-2.3.0-SNAPSHOT.jar") > > > > > > > > > > > > On Mon, Oct 16, 2017 at 5:22 PM, Николай Ижиков < > > > > nizhikov....@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > Hello, guys. > > > > > > > > > > > > > > I have created example application to run Ignite Data Frame on > > > > > standalone > > > > > > > Spark cluster. > > > > > > > With Data Frame API implementation there are no requirements to > > > have > > > > > any > > > > > > > Ignite files on spark worker nodes. > > > > > > > > > > > > > > I ran this application on the free dataset: ATP tennis match > > > > > statistics. > > > > > > > > > > > > > > data - https://github.com/nizhikov/atp_matches > > > > > > > app - https://github.com/nizhikov/ignite-spark-df-example > > > > > > > > > > > > > > Valentin, do you have a chance to look at my changes? > > > > > > > > > > > > > > > > > > > > > 2017-10-12 6:03 GMT+03:00 Valentin Kulichenko < > > > > > > > valentin.kuliche...@gmail.com > > > > > > > >: > > > > > > > > > > > > > > > Hi Nikolay, > > > > > > > > > > > > > > > > Sorry for delay on this, got a little swamped lately. I will > do > > > my > > > > > best > > > > > > > to > > > > > > > > review the code this week. > > > > > > > > > > > > > > > > -Val > > > > > > > > > > > > > > > > On Mon, Oct 9, 2017 at 11:48 AM, Николай Ижиков < > > > > > > nizhikov....@gmail.com> > > > > > > > > wrote: > > > > > > > > > > > > > > > >> Hello, Valentin. > > > > > > > >> > > > > > > > >> Did you have a chance to look at my changes? > > > > > > > >> > > > > > > > >> Now I think I have done almost all required features. > > > > > > > >> I want to make some performance test to ensure my > > implementation > > > > > work > > > > > > > >> properly with a significant amount of data. > > > > > > > >> And I definitely need some feedback for my changes. > > > > > > > >> > > > > > > > >> > > > > > > > >> 2017-10-09 18:45 GMT+03:00 Николай Ижиков < > > > nizhikov....@gmail.com > > > > >: > > > > > > > >> > > > > > > > >>> Hello, guys. > > > > > > > >>> > > > > > > > >>> Which version of Spark do we want to use? > > > > > > > >>> > > > > > > > >>> 1. Currently, Ignite depends on Spark 2.1.0. > > > > > > > >>> > > > > > > > >>> * Can be run on JDK 7. > > > > > > > >>> * Still supported: 2.1.2 will be released soon. > > > > > > > >>> > > > > > > > >>> 2. Latest Spark version is 2.2.0. > > > > > > > >>> > > > > > > > >>> * Can be run only on JDK 8+ > > > > > > > >>> * Released Jul 11, 2017. > > > > > > > >>> * Already supported by huge vendors(Amazon for > example). > > > > > > > >>> > > > > > > > >>> Note that in IGNITE-3084 I implement some internal Spark > API. > > > > > > > >>> So It will take some effort to switch between Spark 2.1 and > > 2.2 > > > > > > > >>> > > > > > > > >>> > > > > > > > >>> 2017-09-27 2:20 GMT+03:00 Valentin Kulichenko < > > > > > > > >>> valentin.kuliche...@gmail.com>: > > > > > > > >>> > > > > > > > >>>> I will review in the next few days. > > > > > > > >>>> > > > > > > > >>>> -Val > > > > > > > >>>> > > > > > > > >>>> On Tue, Sep 26, 2017 at 2:23 PM, Denis Magda < > > > dma...@apache.org > > > > > > > > > > > > wrote: > > > > > > > >>>> > > > > > > > >>>> > Hello Nikolay, > > > > > > > >>>> > > > > > > > > >>>> > This is good news. Finally this capability is coming to > > > > Ignite. > > > > > > > >>>> > > > > > > > > >>>> > Val, Vladimir, could you do a preliminary review? > > > > > > > >>>> > > > > > > > > >>>> > Answering on your questions. > > > > > > > >>>> > > > > > > > > >>>> > 1. Yardstick should be enough for performance > > measurements. > > > > As a > > > > > > > Spark > > > > > > > >>>> > user, I will be curious to know what’s the point of this > > > > > > > integration. > > > > > > > >>>> > Probably we need to compare Spark + Ignite and Spark + > > Hive > > > or > > > > > > > Spark + > > > > > > > >>>> > RDBMS cases. > > > > > > > >>>> > > > > > > > > >>>> > 2. If Spark community is reluctant let’s include the > > module > > > in > > > > > > > >>>> > ignite-spark integration. > > > > > > > >>>> > > > > > > > > >>>> > — > > > > > > > >>>> > Denis > > > > > > > >>>> > > > > > > > > >>>> > > On Sep 25, 2017, at 11:14 AM, Николай Ижиков < > > > > > > > >>>> nizhikov....@gmail.com> > > > > > > > >>>> > wrote: > > > > > > > >>>> > > > > > > > > > >>>> > > Hello, guys. > > > > > > > >>>> > > > > > > > > > >>>> > > Currently, I’m working on integration between Spark > and > > > > Ignite > > > > > > > [1]. > > > > > > > >>>> > > > > > > > > > >>>> > > For now, I implement following: > > > > > > > >>>> > > * Ignite DataSource implementation( > > > > IgniteRelationProvider) > > > > > > > >>>> > > * DataFrame support for Ignite SQL table. > > > > > > > >>>> > > * IgniteCatalog implementation for a transparent > > > > resolving > > > > > of > > > > > > > >>>> ignites > > > > > > > >>>> > > SQL tables. > > > > > > > >>>> > > > > > > > > > >>>> > > Implementation of it can be found in PR [2] > > > > > > > >>>> > > It would be great if someone provides feedback for a > > > > > prototype. > > > > > > > >>>> > > > > > > > > > >>>> > > I made some examples in PR so you can see how API > > suppose > > > to > > > > > be > > > > > > > >>>> used [3]. > > > > > > > >>>> > > [4]. > > > > > > > >>>> > > > > > > > > > >>>> > > I need some advice. Can you help me? > > > > > > > >>>> > > > > > > > > > >>>> > > 1. How should this PR be tested? > > > > > > > >>>> > > > > > > > > > >>>> > > Of course, I need to provide some unit tests. But what > > > about > > > > > > > >>>> scalability > > > > > > > >>>> > > tests, etc. > > > > > > > >>>> > > Maybe we need some Yardstick benchmark or similar? > > > > > > > >>>> > > What are your thoughts? > > > > > > > >>>> > > Which scenarios should I consider in the first place? > > > > > > > >>>> > > > > > > > > > >>>> > > 2. Should we provide Spark Catalog implementation > inside > > > > > Ignite > > > > > > > >>>> codebase? > > > > > > > >>>> > > > > > > > > > >>>> > > A current implementation of Spark Catalog based on > > > *internal > > > > > > Spark > > > > > > > >>>> API*. > > > > > > > >>>> > > Spark community seems not interested in making Catalog > > API > > > > > > public > > > > > > > or > > > > > > > >>>> > > including Ignite Catalog in Spark code base [5], [6]. > > > > > > > >>>> > > > > > > > > > >>>> > > *Should we include Spark internal API implementation > > > inside > > > > > > Ignite > > > > > > > >>>> code > > > > > > > >>>> > > base?* > > > > > > > >>>> > > > > > > > > > >>>> > > Or should we consider to include Catalog > implementation > > in > > > > > some > > > > > > > >>>> external > > > > > > > >>>> > > module? > > > > > > > >>>> > > That will be created and released outside Ignite?(we > > still > > > > can > > > > > > > >>>> support > > > > > > > >>>> > and > > > > > > > >>>> > > develop it inside Ignite community). > > > > > > > >>>> > > > > > > > > > >>>> > > [1] https://issues.apache.org/jira/browse/IGNITE-3084 > > > > > > > >>>> > > [2] https://github.com/apache/ignite/pull/2742 > > > > > > > >>>> > > [3] https://github.com/apache/ > > > ignite/pull/2742/files#diff- > > > > > > > >>>> > > f4ff509cef3018e221394474775e0905 > > > > > > > >>>> > > [4] https://github.com/apache/ > > > ignite/pull/2742/files#diff- > > > > > > > >>>> > > f2b670497d81e780dfd5098c5dd8a89c > > > > > > > >>>> > > [5] http://apache-spark-developers-list.1001551.n3. > > > > > > > >>>> > > nabble.com/Spark-Core-Custom- > > Catalog-Integration-between- > > > > > > > >>>> > > Apache-Ignite-and-Apache-Spark-td22452.html > > > > > > > >>>> > > [6] https://issues.apache.org/jira/browse/SPARK-17767 > > > > > > > >>>> > > > > > > > > > >>>> > > -- > > > > > > > >>>> > > Nikolay Izhikov > > > > > > > >>>> > > nizhikov....@gmail.com > > > > > > > >>>> > > > > > > > > >>>> > > > > > > > > >>>> > > > > > > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > > > >>> -- > > > > > > > >>> Nikolay Izhikov > > > > > > > >>> nizhikov....@gmail.com > > > > > > > >>> > > > > > > > >> > > > > > > > >> > > > > > > > >> > > > > > > > >> -- > > > > > > > >> Nikolay Izhikov > > > > > > > >> nizhikov....@gmail.com > > > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > Nikolay Izhikov > > > > > > > nizhikov....@gmail.com > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > Nikolay Izhikov > > > > > nizhikov....@gmail.com > > > > > > > > > > > > > > >