I tried to use the “append” mode, and it worked. Over 3.8 million rows in 64s. 
I would assume that now I can use the “overwrite” mode on existing data. Now, I 
have to find answers to these questions. What would happen if I “append” to the 
data in the Kudu table if the data already exists? What would happen if I 
“overwrite” existing data when the DataFrame has data in it that does not exist 
in the Kudu table? I need to evaluate the best way to simulate the UPSERT 
behavior in HBase because this is what our use case is.

Thanks,
Ben


> On Jun 14, 2016, at 5:05 PM, Benjamin Kim <bbuil...@gmail.com> wrote:
> 
> Hi,
> 
> Now, I’m getting this error when trying to write to the table.
> 
> import scala.collection.JavaConverters._
> val key_seq = Seq(“my_id")
> val key_list = List(“my_id”).asJava
> kuduContext.createTable(tableName, df.schema, key_seq, new 
> CreateTableOptions().setNumReplicas(1).addHashPartitions(key_list, 100))
> 
> df.write
>     .options(Map("kudu.master" -> kuduMaster,"kudu.table" -> tableName))
>     .mode("overwrite")
>     .kudu
> 
> java.lang.RuntimeException: failed to write 1000 rows from DataFrame to Kudu; 
> sample errors: Not found: key not found (error 0)Not found: key not found 
> (error 0)Not found: key not found (error 0)Not found: key not found (error 
> 0)Not found: key not found (error 0)
> 
> Does the key field need to be first in the DataFrame?
> 
> Thanks,
> Ben
> 
>> On Jun 14, 2016, at 4:28 PM, Dan Burkert <d...@cloudera.com 
>> <mailto:d...@cloudera.com>> wrote:
>> 
>> 
>> 
>> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim <bbuil...@gmail.com 
>> <mailto:bbuil...@gmail.com>> wrote:
>> Dan,
>> 
>> Thanks! It got further. Now, how do I set the Primary Key to be a column(s) 
>> in the DataFrame and set the partitioning? Is it like this?
>> 
>> kuduContext.createTable(tableName, df.schema, Seq(“my_id"), new 
>> CreateTableOptions().setNumReplicas(1).addHashPartitions(“my_id"))
>> 
>> java.lang.IllegalArgumentException: Table partitioning must be specified 
>> using setRangePartitionColumns or addHashPartitions
>> 
>> Yep.  The `Seq("my_id")` part of that call is specifying the set of primary 
>> key columns, so in this case you have specified the single PK column 
>> "my_id".  The `addHashPartitions` call adds hash partitioning to the table, 
>> in this case over the column "my_id" (which is good, it must be over one or 
>> more PK columns, so in this case "my_id" is the one and only valid 
>> combination).  However, the call to `addHashPartition` also takes the number 
>> of buckets as the second param.  You shouldn't get the 
>> IllegalArgumentException as long as you are specifying either 
>> `addHashPartitions` or `setRangePartitionColumns`.
>> 
>> - Dan
>>  
>> 
>> Thanks,
>> Ben
>> 
>> 
>>> On Jun 14, 2016, at 4:07 PM, Dan Burkert <d...@cloudera.com 
>>> <mailto:d...@cloudera.com>> wrote:
>>> 
>>> Looks like we're missing an import statement in that example.  Could you 
>>> try:
>>> 
>>> import org.kududb.client._
>>> and try again?
>>> 
>>> - Dan
>>> 
>>> On Tue, Jun 14, 2016 at 4:01 PM, Benjamin Kim <bbuil...@gmail.com 
>>> <mailto:bbuil...@gmail.com>> wrote:
>>> I encountered an error trying to create a table based on the documentation 
>>> from a DataFrame.
>>> 
>>> <console>:49: error: not found: type CreateTableOptions
>>>               kuduContext.createTable(tableName, df.schema, Seq("key"), new 
>>> CreateTableOptions().setNumReplicas(1))
>>> 
>>> Is there something I’m missing?
>>> 
>>> Thanks,
>>> Ben
>>> 
>>>> On Jun 14, 2016, at 3:00 PM, Jean-Daniel Cryans <jdcry...@apache.org 
>>>> <mailto:jdcry...@apache.org>> wrote:
>>>> 
>>>> It's only in Cloudera's maven repo: 
>>>> https://repository.cloudera.com/cloudera/cloudera-repos/org/kududb/kudu-spark_2.10/0.9.0/
>>>>  
>>>> <https://repository.cloudera.com/cloudera/cloudera-repos/org/kududb/kudu-spark_2.10/0.9.0/>
>>>> 
>>>> J-D
>>>> 
>>>> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim <bbuil...@gmail.com 
>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>> Hi J-D,
>>>> 
>>>> I installed Kudu 0.9.0 using CM, but I can’t find the kudu-spark jar for 
>>>> spark-shell to use. Can you show me where to find it?
>>>> 
>>>> Thanks,
>>>> Ben
>>>> 
>>>> 
>>>>> On Jun 8, 2016, at 1:19 PM, Jean-Daniel Cryans <jdcry...@apache.org 
>>>>> <mailto:jdcry...@apache.org>> wrote:
>>>>> 
>>>>> What's in this doc is what's gonna get released: 
>>>>> https://github.com/cloudera/kudu/blob/master/docs/developing.adoc#kudu-integration-with-spark
>>>>>  
>>>>> <https://github.com/cloudera/kudu/blob/master/docs/developing.adoc#kudu-integration-with-spark>
>>>>> 
>>>>> J-D
>>>>> 
>>>>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim <bbuil...@gmail.com 
>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>> Will this be documented with examples once 0.9.0 comes out?
>>>>> 
>>>>> Thanks,
>>>>> Ben
>>>>> 
>>>>> 
>>>>>> On May 28, 2016, at 3:22 PM, Jean-Daniel Cryans <jdcry...@apache.org 
>>>>>> <mailto:jdcry...@apache.org>> wrote:
>>>>>> 
>>>>>> It will be in 0.9.0.
>>>>>> 
>>>>>> J-D
>>>>>> 
>>>>>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim <bbuil...@gmail.com 
>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>> Hi Chris,
>>>>>> 
>>>>>> Will all this effort be rolled into 0.9.0 and be ready for use?
>>>>>> 
>>>>>> Thanks,
>>>>>> Ben
>>>>>> 
>>>>>> 
>>>>>>> On May 18, 2016, at 9:01 AM, Chris George <christopher.geo...@rms.com 
>>>>>>> <mailto:christopher.geo...@rms.com>> wrote:
>>>>>>> 
>>>>>>> There is some code in review that needs some more refinement.
>>>>>>> It will allow upsert/insert from a dataframe using the datasource api. 
>>>>>>> It will also allow the creation and deletion of tables from a dataframe
>>>>>>> http://gerrit.cloudera.org:8080/#/c/2992/ 
>>>>>>> <http://gerrit.cloudera.org:8080/#/c/2992/>
>>>>>>> 
>>>>>>> Example usages will look something like:
>>>>>>> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
>>>>>>> <http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc>
>>>>>>> 
>>>>>>> -Chris George
>>>>>>> 
>>>>>>> 
>>>>>>> On 5/18/16, 9:45 AM, "Benjamin Kim" <bbuil...@gmail.com 
>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>> 
>>>>>>> Can someone tell me what the state is of this Spark work?
>>>>>>> 
>>>>>>> Also, does anyone have any sample code on how to update/insert data in 
>>>>>>> Kudu using DataFrames?
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> Ben
>>>>>>> 
>>>>>>> 
>>>>>>>> On Apr 13, 2016, at 8:22 AM, Chris George <christopher.geo...@rms.com 
>>>>>>>> <mailto:christopher.geo...@rms.com>> wrote:
>>>>>>>> 
>>>>>>>> SparkSQL cannot support these type of statements but we may be able to 
>>>>>>>> implement similar functionality through the api.
>>>>>>>> -Chris
>>>>>>>> 
>>>>>>>> On 4/12/16, 5:19 PM, "Benjamin Kim" <bbuil...@gmail.com 
>>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>> 
>>>>>>>> It would be nice to adhere to the SQL:2003 standard for an “upsert” if 
>>>>>>>> it were to be implemented.
>>>>>>>> 
>>>>>>>> MERGE INTO table_name USING table_reference ON (condition)
>>>>>>>>  WHEN MATCHED THEN
>>>>>>>>  UPDATE SET column1 = value1 [, column2 = value2 ...]
>>>>>>>>  WHEN NOT MATCHED THEN
>>>>>>>>  INSERT (column1 [, column2 ...]) VALUES (value1 [, value2 …])
>>>>>>>> 
>>>>>>>> Cheers,
>>>>>>>> Ben
>>>>>>>> 
>>>>>>>>> On Apr 11, 2016, at 12:21 PM, Chris George 
>>>>>>>>> <christopher.geo...@rms.com <mailto:christopher.geo...@rms.com>> 
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>> I have a wip kuduRDD that I made a few months ago. I pushed it into 
>>>>>>>>> gerrit if you want to take a look. 
>>>>>>>>> http://gerrit.cloudera.org:8080/#/c/2754/ 
>>>>>>>>> <http://gerrit.cloudera.org:8080/#/c/2754/>
>>>>>>>>> It does pushdown predicates which the existing input formatter based 
>>>>>>>>> rdd does not.
>>>>>>>>> 
>>>>>>>>> Within the next two weeks I’m planning to implement a datasource for 
>>>>>>>>> spark that will have pushdown predicates and insertion/update 
>>>>>>>>> functionality (need to look more at cassandra and the hbase 
>>>>>>>>> datasource for best way to do this) I agree that server side upsert 
>>>>>>>>> would be helpful.
>>>>>>>>> Having a datasource would give us useful data frames and also make 
>>>>>>>>> spark sql usable for kudu.
>>>>>>>>> 
>>>>>>>>> My reasoning for having a spark datasource and not using Impala is: 
>>>>>>>>> 1. We have had trouble getting impala to run fast with high 
>>>>>>>>> concurrency when compared to spark 2. We interact with datasources 
>>>>>>>>> which do not integrate with impala. 3. We have custom sql query 
>>>>>>>>> planners for extended sql functionality.
>>>>>>>>> 
>>>>>>>>> -Chris George
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On 4/11/16, 12:22 PM, "Jean-Daniel Cryans" <jdcry...@apache.org 
>>>>>>>>> <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>> 
>>>>>>>>> You guys make a convincing point, although on the upsert side we'll 
>>>>>>>>> need more support from the servers. Right now all you can do is an 
>>>>>>>>> INSERT then, if you get a dup key, do an UPDATE. I guess we could at 
>>>>>>>>> least add an API on the client side that would manage it, but it 
>>>>>>>>> wouldn't be atomic.
>>>>>>>>> 
>>>>>>>>> J-D
>>>>>>>>> 
>>>>>>>>> On Mon, Apr 11, 2016 at 9:34 AM, Mark Hamstra 
>>>>>>>>> <m...@clearstorydata.com <mailto:m...@clearstorydata.com>> wrote:
>>>>>>>>> It's pretty simple, actually.  I need to support versioned datasets 
>>>>>>>>> in a Spark SQL environment.  Instead of a hack on top of a Parquet 
>>>>>>>>> data store, I'm hoping (among other reasons) to be able to use Kudu's 
>>>>>>>>> write and timestamp-based read operations to support not only 
>>>>>>>>> appending data, but also updating existing data, and even some schema 
>>>>>>>>> migration.  The most typical use case is a dataset that is updated 
>>>>>>>>> periodically (e.g., weekly or monthly) in which the the preliminary 
>>>>>>>>> data in the previous window (week or month) is updated with values 
>>>>>>>>> that are expected to remain unchanged from then on, and a new set of 
>>>>>>>>> preliminary values for the current window need to be added/appended.
>>>>>>>>> 
>>>>>>>>> Using Kudu's Java API and developing additional functionality on top 
>>>>>>>>> of what Kudu has to offer isn't too much to ask, but the ease of 
>>>>>>>>> integration with Spark SQL will gate how quickly we would move to 
>>>>>>>>> using Kudu and how seriously we'd look at alternatives before making 
>>>>>>>>> that decision. 
>>>>>>>>> 
>>>>>>>>> On Mon, Apr 11, 2016 at 8:14 AM, Jean-Daniel Cryans 
>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>> Mark,
>>>>>>>>> 
>>>>>>>>> Thanks for taking some time to reply in this thread, glad it caught 
>>>>>>>>> the attention of other folks!
>>>>>>>>> 
>>>>>>>>> On Sun, Apr 10, 2016 at 12:33 PM, Mark Hamstra 
>>>>>>>>> <m...@clearstorydata.com <mailto:m...@clearstorydata.com>> wrote:
>>>>>>>>> Do they care being able to insert into Kudu with SparkSQL
>>>>>>>>> 
>>>>>>>>> I care about insert into Kudu with Spark SQL.  I'm currently delaying 
>>>>>>>>> a refactoring of some Spark SQL-oriented insert functionality while 
>>>>>>>>> trying to evaluate what to expect from Kudu.  Whether Kudu does a 
>>>>>>>>> good job supporting inserts with Spark SQL will be a key 
>>>>>>>>> consideration as to whether we adopt Kudu.
>>>>>>>>> 
>>>>>>>>> I'd like to know more about why SparkSQL inserts in necessary for 
>>>>>>>>> you. Is it just that you currently do it that way into some database 
>>>>>>>>> or parquet so with minimal refactoring you'd be able to use Kudu? 
>>>>>>>>> Would re-writing those SQL lines into Scala and directly use the Java 
>>>>>>>>> API's KuduSession be too much work?
>>>>>>>>> 
>>>>>>>>> Additionally, what do you expect to gain from using Kudu VS your 
>>>>>>>>> current solution? If it's not completely clear, I'd love to help you 
>>>>>>>>> think through it.
>>>>>>>>>  
>>>>>>>>> 
>>>>>>>>> On Sun, Apr 10, 2016 at 12:23 PM, Jean-Daniel Cryans 
>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>> Yup, starting to get a good idea.
>>>>>>>>> 
>>>>>>>>> What are your DS folks looking for in terms of functionality related 
>>>>>>>>> to Spark? A SparkSQL integration that's as fully featured as 
>>>>>>>>> Impala's? Do they care being able to insert into Kudu with SparkSQL 
>>>>>>>>> or just being able to query real fast? Anything more specific to 
>>>>>>>>> Spark that I'm missing?
>>>>>>>>> 
>>>>>>>>> FWIW the plan is to get to 1.0 in late Summer/early Fall. At Cloudera 
>>>>>>>>> all our resources are committed to making things happen in time, and 
>>>>>>>>> a more fully featured Spark integration isn't in our plans during 
>>>>>>>>> that period. I'm really hoping someone in the community will help 
>>>>>>>>> with Spark, the same way we got a big contribution for the Flume 
>>>>>>>>> sink. 
>>>>>>>>> 
>>>>>>>>> J-D
>>>>>>>>> 
>>>>>>>>> On Sun, Apr 10, 2016 at 11:29 AM, Benjamin Kim <bbuil...@gmail.com 
>>>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>>> Yes, we took Kudu for a test run using 0.6 and 0.7 versions. But, 
>>>>>>>>> since it’s not “production-ready”, upper management doesn’t want to 
>>>>>>>>> fully deploy it yet. They just want to keep an eye on it though. Kudu 
>>>>>>>>> was so much simpler and easier to use in every aspect compared to 
>>>>>>>>> HBase. Impala was great for the report writers and analysts to 
>>>>>>>>> experiment with for the short time it was up. But, once again, the 
>>>>>>>>> only blocker was the lack of Spark support for our Data 
>>>>>>>>> Developers/Scientists. So, production-level data population won’t 
>>>>>>>>> happen until then.
>>>>>>>>> 
>>>>>>>>> I hope this helps you get an idea where I am coming from…
>>>>>>>>> 
>>>>>>>>> Cheers,
>>>>>>>>> Ben
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> On Apr 10, 2016, at 11:08 AM, Jean-Daniel Cryans 
>>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>>> 
>>>>>>>>>> On Sun, Apr 10, 2016 at 12:30 AM, Benjamin Kim <bbuil...@gmail.com 
>>>>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>>>> J-D,
>>>>>>>>>> 
>>>>>>>>>> The main thing I hear that Cassandra is being used as an updatable 
>>>>>>>>>> hot data store to ensure that duplicates are taken care of and 
>>>>>>>>>> idempotency is maintained. Whether data was directly retrieved from 
>>>>>>>>>> Cassandra for analytics, reports, or searches, it was not clear as 
>>>>>>>>>> to what was its main use. Some also just used it for a staging area 
>>>>>>>>>> to populate downstream tables in parquet format. The last thing I 
>>>>>>>>>> heard was that CQL was terrible, so that rules out much use of 
>>>>>>>>>> direct queries against it.
>>>>>>>>>> 
>>>>>>>>>> I'm no C* expert, but I don't think CQL is meant for real analytics, 
>>>>>>>>>> just ease of use instead of plainly using the APIs. Even then, Kudu 
>>>>>>>>>> should beat it easily on big scans. Same for HBase. We've done 
>>>>>>>>>> benchmarks against the latter, not the former.
>>>>>>>>>>  
>>>>>>>>>> 
>>>>>>>>>> As for our company, we have been looking for an updatable data store 
>>>>>>>>>> for a long time that can be quickly queried directly either using 
>>>>>>>>>> Spark SQL or Impala or some other SQL engine and still handle TB or 
>>>>>>>>>> PB of data without performance degradation and many configuration 
>>>>>>>>>> headaches. For now, we are using HBase to take on this role with 
>>>>>>>>>> Phoenix as a fast way to directly query the data. I can see Kudu as 
>>>>>>>>>> the best way to fill this gap easily, especially being the closest 
>>>>>>>>>> thing to other relational databases out there in familiarity for the 
>>>>>>>>>> many SQL analytics people in our company. The other alternative 
>>>>>>>>>> would be to go with AWS Redshift for the same reasons, but it would 
>>>>>>>>>> come at a cost, of course. If we went with either solutions, Kudu or 
>>>>>>>>>> Redshift, it would get rid of the need to extract from HBase to 
>>>>>>>>>> parquet tables or export to PostgreSQL to support more of the SQL 
>>>>>>>>>> language using by analysts or the reporting software we use..
>>>>>>>>>> 
>>>>>>>>>> Ok, the usual then *smile*. Looks like we're not too far off with 
>>>>>>>>>> Kudu. Have you folks tried Kudu with Impala yet with those use cases?
>>>>>>>>>>  
>>>>>>>>>> 
>>>>>>>>>> I hope this helps.
>>>>>>>>>> 
>>>>>>>>>> It does, thanks for nice reply.
>>>>>>>>>>  
>>>>>>>>>> 
>>>>>>>>>> Cheers,
>>>>>>>>>> Ben 
>>>>>>>>>> 
>>>>>>>>>>> On Apr 9, 2016, at 2:00 PM, Jean-Daniel Cryans <jdcry...@apache.org 
>>>>>>>>>>> <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Ha first time I'm hearing about SMACK. Inside Cloudera we like to 
>>>>>>>>>>> refer to "Impala + Kudu" as Kimpala, but yeah it's not as sexy. My 
>>>>>>>>>>> colleagues who were also there did say that the hype around Spark 
>>>>>>>>>>> isn't dying down.
>>>>>>>>>>> 
>>>>>>>>>>> There's definitely an overlap in the use cases that Cassandra, 
>>>>>>>>>>> HBase, and Kudu cater to. I wouldn't go as far as saying that C* is 
>>>>>>>>>>> just an interim solution for the use case you describe.
>>>>>>>>>>> 
>>>>>>>>>>> Nothing significant happened in Kudu over the past month, it's a 
>>>>>>>>>>> storage engine so things move slowly *smile*. I'd love to see more 
>>>>>>>>>>> contributions on the Spark front. I know there's code out there 
>>>>>>>>>>> that could be integrated in kudu-spark, it just needs to land in 
>>>>>>>>>>> gerrit. I'm sure folks will happily review it.
>>>>>>>>>>> 
>>>>>>>>>>> Do you have relevant experiences you can share? I'd love to learn 
>>>>>>>>>>> more about the use cases for which you envision using Kudu as a C* 
>>>>>>>>>>> replacement.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> J-D
>>>>>>>>>>> 
>>>>>>>>>>> On Fri, Apr 8, 2016 at 12:45 PM, Benjamin Kim <bbuil...@gmail.com 
>>>>>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>>>>> Hi J-D,
>>>>>>>>>>> 
>>>>>>>>>>> My colleagues recently came back from Strata in San Jose. They told 
>>>>>>>>>>> me that everything was about Spark and there is a big buzz about 
>>>>>>>>>>> the SMACK stack (Spark, Mesos, Akka, Cassandra, Kafka). I still 
>>>>>>>>>>> think that Cassandra is just an interim solution as a low-latency, 
>>>>>>>>>>> easily queried data store. I was wondering if anything significant 
>>>>>>>>>>> happened in regards to Kudu, especially on the Spark front. Plus, 
>>>>>>>>>>> can you come up with your own proposed stack acronym to promote?
>>>>>>>>>>> 
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Ben
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> On Mar 1, 2016, at 12:20 PM, Jean-Daniel Cryans 
>>>>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> Hi Ben,
>>>>>>>>>>>> 
>>>>>>>>>>>> AFAIK no one in the dev community committed to any timeline. I 
>>>>>>>>>>>> know of one person on the Kudu Slack who's working on a better 
>>>>>>>>>>>> RDD, but that's about it.
>>>>>>>>>>>> 
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> 
>>>>>>>>>>>> J-D
>>>>>>>>>>>> 
>>>>>>>>>>>> On Tue, Mar 1, 2016 at 11:00 AM, Benjamin Kim <b...@amobee.com 
>>>>>>>>>>>> <mailto:b...@amobee.com>> wrote:
>>>>>>>>>>>> Hi J-D,
>>>>>>>>>>>> 
>>>>>>>>>>>> Quick question… Is there an ETA for KUDU-1214? I want to target a 
>>>>>>>>>>>> version of Kudu to begin real testing of Spark against it for our 
>>>>>>>>>>>> devs. At least, I can tell them what timeframe to anticipate.
>>>>>>>>>>>> 
>>>>>>>>>>>> Just curious,
>>>>>>>>>>>> Benjamin Kim
>>>>>>>>>>>> Data Solutions Architect
>>>>>>>>>>>> 
>>>>>>>>>>>> [a•mo•bee] (n.) the company defining digital marketing.
>>>>>>>>>>>> 
>>>>>>>>>>>> Mobile: +1 818 635 2900 <tel:%2B1%20818%20635%202900>
>>>>>>>>>>>> 3250 Ocean Park Blvd, Suite 200  |  Santa Monica, CA 90405  |  
>>>>>>>>>>>> www.amobee.com <http://www.amobee.com/>
>>>>>>>>>>>> 
>>>>>>>>>>>>> On Feb 24, 2016, at 3:51 PM, Jean-Daniel Cryans 
>>>>>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The DStream stuff isn't there at all. I'm not sure if it's needed 
>>>>>>>>>>>>> either.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The kuduRDD is just leveraging the MR input format, ideally we'd 
>>>>>>>>>>>>> use scans directly.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The SparkSQL stuff is there but it doesn't do any sort of 
>>>>>>>>>>>>> pushdown. It's really basic.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The goal was to provide something for others to contribute to. We 
>>>>>>>>>>>>> have some basic unit tests that others can easily extend. None of 
>>>>>>>>>>>>> us on the team are Spark experts, but we'd be really happy to 
>>>>>>>>>>>>> assist one improve the kudu-spark code.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> J-D
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Wed, Feb 24, 2016 at 3:41 PM, Benjamin Kim <bbuil...@gmail.com 
>>>>>>>>>>>>> <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>>>>>>> J-D,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> It looks like it fulfills most of the basic requirements (kudu 
>>>>>>>>>>>>> RDD, kudu DStream) in KUDU-1214. Am I right? Besides shoring up 
>>>>>>>>>>>>> more Spark SQL functionality (Dataframes) and doing the 
>>>>>>>>>>>>> documentation, what more needs to be done? Optimizations?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I believe that it’s a good place to start using Spark with Kudu 
>>>>>>>>>>>>> and compare it to HBase with Spark (not clean).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Ben
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Feb 24, 2016, at 3:10 PM, Jean-Daniel Cryans 
>>>>>>>>>>>>>> <jdcry...@apache.org <mailto:jdcry...@apache.org>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> AFAIK no one is working on it, but we did manage to get this in 
>>>>>>>>>>>>>> for 0.7.0: https://issues.cloudera.org/browse/KUDU-1321 
>>>>>>>>>>>>>> <https://issues.cloudera.org/browse/KUDU-1321>
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> It's a really simple wrapper, and yes you can use SparkSQL on 
>>>>>>>>>>>>>> Kudu, but it will require a lot more work to make it fast/useful.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hope this helps,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> J-D
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Wed, Feb 24, 2016 at 3:08 PM, Benjamin Kim 
>>>>>>>>>>>>>> <bbuil...@gmail.com <mailto:bbuil...@gmail.com>> wrote:
>>>>>>>>>>>>>> I see this KUDU-1214 
>>>>>>>>>>>>>> <https://issues.cloudera.org/browse/KUDU-1214> targeted for 
>>>>>>>>>>>>>> 0.8.0, but I see no progress on it. When this is complete, will 
>>>>>>>>>>>>>> this mean that Spark will be able to work with Kudu both 
>>>>>>>>>>>>>> programmatically and as a client via Spark SQL? Or is there more 
>>>>>>>>>>>>>> work that needs to be done on the Spark side for it to work?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Just curious.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>> Ben
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
> 

Reply via email to