Re: Performance Question

2016-06-14 Thread Benjamin Kim
Hi Todd,

Now that Kudu 0.9.0 is out. I have done some tests. Already, I am impressed. 
Compared to HBase, read and write performance are better. Write performance has 
the greatest improvement (> 4x), while read is > 1.5x. Albeit, these are only 
preliminary tests. Do you know of a way to really do some conclusive tests? I 
want to see if I can match your results on my 50 node cluster.

Thanks,
Ben

> On May 30, 2016, at 10:33 AM, Todd Lipcon  wrote:
> 
> On Sat, May 28, 2016 at 7:12 AM, Benjamin Kim  > wrote:
> Todd,
> 
> It sounds like Kudu can possibly top or match those numbers put out by 
> Aerospike. Do you have any performance statistics published or any 
> instructions as to measure them myself as good way to test? In addition, this 
> will be a test using Spark, so should I wait for Kudu version 0.9.0 where 
> support will be built in?
> 
> We don't have a lot of benchmarks published yet, especially on the write 
> side. I've found that thorough cross-system benchmarks are very difficult to 
> do fairly and accurately, and often times users end up misguided if they pay 
> too much attention to them :) So, given a finite number of developers working 
> on Kudu, I think we've tended to spend more time on the project itself and 
> less time focusing on "competition". I'm sure there are use cases where Kudu 
> will beat out Aerospike, and probably use cases where Aerospike will beat 
> Kudu as well.
> 
> From my perspective, it would be great if you can share some details of your 
> workload, especially if there are some areas you're finding Kudu lacking. 
> Maybe we can spot some easy code changes we could make to improve 
> performance, or suggest a tuning variable you could change.
> 
> -Todd
> 
> 
>> On May 27, 2016, at 9:19 PM, Todd Lipcon > > wrote:
>> 
>> On Fri, May 27, 2016 at 8:20 PM, Benjamin Kim > > wrote:
>> Hi Mike,
>> 
>> First of all, thanks for the link. It looks like an interesting read. I 
>> checked that Aerospike is currently at version 3.8.2.3, and in the article, 
>> they are evaluating version 3.5.4. The main thing that impressed me was 
>> their claim that they can beat Cassandra and HBase by 8x for writing and 25x 
>> for reading. Their big claim to fame is that Aerospike can write 1M records 
>> per second with only 50 nodes. I wanted to see if this is real.
>> 
>> 1M records per second on 50 nodes is pretty doable by Kudu as well, 
>> depending on the size of your records and the insertion order. I've been 
>> playing with a ~70 node cluster recently and seen 1M+ writes/second 
>> sustained, and bursting above 4M. These are 1KB rows with 11 columns, and 
>> with pretty old HDD-only nodes. I think newer flash-based nodes could do 
>> better.
>>  
>> 
>> To answer your questions, we have a DMP with user profiles with many 
>> attributes. We create segmentation information off of these attributes to 
>> classify them. Then, we can target advertising appropriately for our sales 
>> department. Much of the data processing is for applying models on all or if 
>> not most of every profile’s attributes to find similarities (nearest 
>> neighbor/clustering) over a large number of rows when batch processing or a 
>> small subset of rows for quick online scoring. So, our use case is a typical 
>> advanced analytics scenario. We have tried HBase, but it doesn’t work well 
>> for these types of analytics.
>> 
>> I read, that Aerospike in the release notes, they did do many improvements 
>> for batch and scan operations.
>> 
>> I wonder what your thoughts are for using Kudu for this.
>> 
>> Sounds like a good Kudu use case to me. I've heard great things about 
>> Aerospike for the low latency random access portion, but I've also heard 
>> that it's _very_ expensive, and not particularly suited to the columnar scan 
>> workload. Lastly, I think the Apache license of Kudu is much more appealing 
>> than the AGPL3 used by Aerospike. But, that's not really a direct answer to 
>> the performance question :)
>>  
>> 
>> Thanks,
>> Ben
>> 
>> 
>>> On May 27, 2016, at 6:21 PM, Mike Percy >> > wrote:
>>> 
>>> Have you considered whether you have a scan heavy or a random access heavy 
>>> workload? Have you considered whether you always access / update a whole 
>>> row vs only a partial row? Kudu is a column store so has some awesome 
>>> performance characteristics when you are doing a lot of scanning of just a 
>>> couple of columns.
>>> 
>>> I don't know the answer to your question but if your concern is performance 
>>> then I would be interested in seeing comparisons from a perf perspective on 
>>> certain workloads.
>>> 
>>> Finally, a year ago Aerospike did quite poorly in a Jepsen test: 
>>> https://aphyr.com/posts/324-jepsen-aerospike 
>>> 
>>> 
>>> I wonder if they have addressed any of those issues.
>>> 
>>> Mike
>>> 
>>> On Friday, M

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
Ah, that makes more sense when you put it that way.

Thanks,
Ben

> On Jun 14, 2016, at 6:08 PM, Dan Burkert  wrote:
> 
> I'm not sure exactly what the semantics will be, but at least one of them 
> will be upsert.  These modes come from spark, and they were really designed 
> for file-backed storage and not table storage.  We may want to do append = 
> upsert, and overwrite = truncate + insert.  I think that may match the normal 
> spark semantics more closely.
> 
> - Dan
> 
> On Tue, Jun 14, 2016 at 6:00 PM, Benjamin Kim  > wrote:
> Dan,
> 
> Thanks for the information. That would mean both “append” and “overwrite” 
> modes would be combined or not needed in the future.
> 
> Cheers,
> Ben
> 
>> On Jun 14, 2016, at 5:57 PM, Dan Burkert > > wrote:
>> 
>> Right now append uses an update Kudu operation, which requires the row 
>> already be present in the table. Overwrite maps to insert.  Kudu very 
>> recently got upsert support baked in, but it hasn't yet been integrated into 
>> the Spark connector.  So pretty soon these sharp edges will get a lot 
>> better, since upsert is the way to go for most spark workloads.
>> 
>> - Dan
>> 
>> On Tue, Jun 14, 2016 at 5:41 PM, Benjamin Kim > > wrote:
>> 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 >> > 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 >>> > wrote:
 
 
 
 On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim >>> > 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  > 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  > wrote:
> I encountered an error trying to create a table based on the 
> documentation from a DataFrame.
> 
> :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

Re: Spark on Kudu

2016-06-14 Thread Dan Burkert
I'm not sure exactly what the semantics will be, but at least one of them
will be upsert.  These modes come from spark, and they were really designed
for file-backed storage and not table storage.  We may want to do append =
upsert, and overwrite = truncate + insert.  I think that may match the
normal spark semantics more closely.

- Dan

On Tue, Jun 14, 2016 at 6:00 PM, Benjamin Kim  wrote:

> Dan,
>
> Thanks for the information. That would mean both “append” and “overwrite”
> modes would be combined or not needed in the future.
>
> Cheers,
> Ben
>
> On Jun 14, 2016, at 5:57 PM, Dan Burkert  wrote:
>
> Right now append uses an update Kudu operation, which requires the row
> already be present in the table. Overwrite maps to insert.  Kudu very
> recently got upsert support baked in, but it hasn't yet been integrated
> into the Spark connector.  So pretty soon these sharp edges will get a lot
> better, since upsert is the way to go for most spark workloads.
>
> - Dan
>
> On Tue, Jun 14, 2016 at 5:41 PM, Benjamin Kim  wrote:
>
>> 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  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  wrote:
>>
>>
>>
>> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim  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  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 
>>> wrote:
>>>
 I encountered an error trying to create a table based on the
 documentation from a DataFrame.

 :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 
 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/

 J-D

 On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim 
 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 
> wrote:
>
> What's in this doc is what's go

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
Dan,

Thanks for the information. That would mean both “append” and “overwrite” modes 
would be combined or not needed in the future.

Cheers,
Ben

> On Jun 14, 2016, at 5:57 PM, Dan Burkert  wrote:
> 
> Right now append uses an update Kudu operation, which requires the row 
> already be present in the table. Overwrite maps to insert.  Kudu very 
> recently got upsert support baked in, but it hasn't yet been integrated into 
> the Spark connector.  So pretty soon these sharp edges will get a lot better, 
> since upsert is the way to go for most spark workloads.
> 
> - Dan
> 
> On Tue, Jun 14, 2016 at 5:41 PM, Benjamin Kim  > wrote:
> 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 > > 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 >> > wrote:
>>> 
>>> 
>>> 
>>> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim >> > 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 >>> > 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 >>> > wrote:
 I encountered an error trying to create a table based on the documentation 
 from a DataFrame.
 
 :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  > 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/
>  
> 
> 
> J-D
> 
> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim  > 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 > > wrote:
>> 
>>

Re: Spark on Kudu

2016-06-14 Thread Dan Burkert
Right now append uses an update Kudu operation, which requires the row
already be present in the table. Overwrite maps to insert.  Kudu very
recently got upsert support baked in, but it hasn't yet been integrated
into the Spark connector.  So pretty soon these sharp edges will get a lot
better, since upsert is the way to go for most spark workloads.

- Dan

On Tue, Jun 14, 2016 at 5:41 PM, Benjamin Kim  wrote:

> 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  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  wrote:
>
>
>
> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim  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  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  wrote:
>>
>>> I encountered an error trying to create a table based on the
>>> documentation from a DataFrame.
>>>
>>> :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 
>>> 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/
>>>
>>> J-D
>>>
>>> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim 
>>> 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 
 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

 J-D

 On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim 
 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 
> wrote:
>
> It will be in 0.9.0.
>
> J-D
>
> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim 
> 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 
>> wrote:
>>
>> There is some code in review 

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
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  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 > > wrote:
>> 
>> 
>> 
>> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim > > 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 >> > 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 >> > wrote:
>>> I encountered an error trying to create a table based on the documentation 
>>> from a DataFrame.
>>> 
>>> :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 >>> > 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/
  
 
 
 J-D
 
 On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim >>> > 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  > 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
>  
> 
> 
> J-D
> 
> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim  > 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 > > wrote:
>> 
>> It will be in 0.9.0.
>> 
>> J-D
>> 
>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim > > wrote:
>> Hi Chris,
>> 
>> Will all this effort be rolled into 0.9.0 and be ready f

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
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  wrote:
> 
> 
> 
> On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim  > 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 > > 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 > > wrote:
>> I encountered an error trying to create a table based on the documentation 
>> from a DataFrame.
>> 
>> :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 >> > 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/
>>>  
>>> 
>>> 
>>> J-D
>>> 
>>> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim >> > 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 >>> > 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
  
 
 
 J-D
 
 On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim >>> > 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  > wrote:
> 
> It will be in 0.9.0.
> 
> J-D
> 
> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim  > 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 > > 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/ 
>> 
>> 
>> Example usages will look something like:
>> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
>> 
>> 
>> -Chris George
>> 
>> 
>> On 5/18/16, 9:45 AM, "Benjamin Kim" > 

Re: Spark on Kudu

2016-06-14 Thread Dan Burkert
On Tue, Jun 14, 2016 at 4:20 PM, Benjamin Kim  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  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  wrote:
>
>> I encountered an error trying to create a table based on the
>> documentation from a DataFrame.
>>
>> :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 
>> 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/
>>
>> J-D
>>
>> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim  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 
>>> 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
>>>
>>> J-D
>>>
>>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim  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 
 wrote:

 It will be in 0.9.0.

 J-D

 On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim 
 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 
> 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/
>
> Example usages will look something like:
> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc
>
> -Chris George
>
>
> On 5/18/16, 9:45 AM, "Benjamin Kim"  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 
> 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"  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 
> 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/
> 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 w

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
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

Thanks,
Ben


> On Jun 14, 2016, at 4:07 PM, Dan Burkert  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  > wrote:
> I encountered an error trying to create a table based on the documentation 
> from a DataFrame.
> 
> :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 > > 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/
>>  
>> 
>> 
>> J-D
>> 
>> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim > > 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 >> > 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
>>>  
>>> 
>>> 
>>> J-D
>>> 
>>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim >> > 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 >>> > wrote:
 
 It will be in 0.9.0.
 
 J-D
 
 On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim >>> > 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  > 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/ 
> 
> 
> Example usages will look something like:
> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
> 
> 
> -Chris George
> 
> 
> On 5/18/16, 9:45 AM, "Benjamin Kim"  > 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 > > 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" > > 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 >> > 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/ 
>>> 
>>> 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 
>>> fo

Re: Spark on Kudu

2016-06-14 Thread Dan Burkert
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  wrote:

> I encountered an error trying to create a table based on the documentation
> from a DataFrame.
>
> :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 
> 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/
>
> J-D
>
> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim  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 
>> 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
>>
>> J-D
>>
>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim  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 
>>> wrote:
>>>
>>> It will be in 0.9.0.
>>>
>>> J-D
>>>
>>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim 
>>> 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 
 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/

 Example usages will look something like:
 http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc

 -Chris George


 On 5/18/16, 9:45 AM, "Benjamin Kim"  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 
 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"  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 
 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/
 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"  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 
 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 d

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
I encountered an error trying to create a table based on the documentation from 
a DataFrame.

: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  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/
>  
> 
> 
> J-D
> 
> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim  > 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 > > 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
>>  
>> 
>> 
>> J-D
>> 
>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim > > 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 >> > wrote:
>>> 
>>> It will be in 0.9.0.
>>> 
>>> J-D
>>> 
>>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim >> > 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 >>> > 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/ 
 
 
 Example usages will look something like:
 http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
 
 
 -Chris George
 
 
 On 5/18/16, 9:45 AM, "Benjamin Kim" >>> > 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  > 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"  > 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 > > 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/ 
>> 
>> 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" > > 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

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
Thank you.

> On Jun 14, 2016, at 3:00 PM, Jean-Daniel Cryans  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/
>  
> 
> 
> J-D
> 
> On Tue, Jun 14, 2016 at 2:59 PM, Benjamin Kim  > 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 > > 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
>>  
>> 
>> 
>> J-D
>> 
>> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim > > 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 >> > wrote:
>>> 
>>> It will be in 0.9.0.
>>> 
>>> J-D
>>> 
>>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim >> > 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 >>> > 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/ 
 
 
 Example usages will look something like:
 http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
 
 
 -Chris George
 
 
 On 5/18/16, 9:45 AM, "Benjamin Kim" >>> > 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  > 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"  > 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 > > 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/ 
>> 
>> 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" > > 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 > > wrote:
>> It's pretty simple, act

Re: Spark on Kudu

2016-06-14 Thread Jean-Daniel Cryans
It's only in Cloudera's maven repo:
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  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 
> 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
>
> J-D
>
> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim  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 
>> wrote:
>>
>> It will be in 0.9.0.
>>
>> J-D
>>
>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim  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 
>>> 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/
>>>
>>> Example usages will look something like:
>>> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc
>>>
>>> -Chris George
>>>
>>>
>>> On 5/18/16, 9:45 AM, "Benjamin Kim"  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 
>>> 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"  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 
>>> 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/
>>> 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"  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 
>>> 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> wrote:
>>

Re: Spark on Kudu

2016-06-14 Thread Benjamin Kim
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  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
>  
> 
> 
> J-D
> 
> On Tue, Jun 7, 2016 at 8:52 PM, Benjamin Kim  > 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 > > wrote:
>> 
>> It will be in 0.9.0.
>> 
>> J-D
>> 
>> On Sat, May 28, 2016 at 8:31 AM, Benjamin Kim > > 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 >> > 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/ 
>>> 
>>> 
>>> Example usages will look something like:
>>> http://gerrit.cloudera.org:8080/#/c/2992/5/docs/developing.adoc 
>>> 
>>> 
>>> -Chris George
>>> 
>>> 
>>> On 5/18/16, 9:45 AM, "Benjamin Kim" >> > 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 >>> > 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" >>> > 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  > 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/ 
> 
> 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"  > 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  > 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 r

Fw: Re: kudu-user - Google Groups: Message Pending [{IOvAyOaAr6f8MSoCaXQwADHiJQdpgUvR0}]

2016-06-14 Thread 445484...@qq.com

Google Groups: Message Pending [{IOvAyOaAr6f8MSoCaXQwADHiJQdpgUvR0}]


Jerry Fan
445484...@qq.com
 
From: Mike Percy
Date: 2016-06-14 12:06
To: 445484859
Subject: Re: kudu-user - Google Groups: Message Pending 
[{IOvAyOaAr6f8MSoCaXQwADHiJQdpgUvR0}]
Hi, please send this message to user@kudu.incubator.apache.org

This mailing list has been retired.

Thanks,
Mike

--
Mike Percy
Software Engineer, Cloudera 


On Mon, Jun 13, 2016 at 7:35 PM, kudu-user  
wrote:
A message has been sent to the kudu-user group and is awaiting approval. We've 
included this message for your review.
The message requires moderation because the user does not have permission to 
post.
You can approve or reject this message or you can approve this message by 
replying to this email.
Start your own group, visit the help center, or report abuse. 


-- Forwarded message --
From: 445484...@qq.com
To: kudu-user 
Cc: 
Date: Mon, 13 Jun 2016 19:35:26 -0700 (PDT)
Subject: all tablet servers down when run the 6th query of TPC-H
the query is:
select 
sum(l_extendedprice*l_discount) as revenue
from 
lineitem_new
where 
l_shipdate >= '1994-01-01'
and l_shipdate < '1995-01-01'
and l_discount >= 0.05 and l_discount <= 0.07
and l_quantity < 24;

When i use impala to run this query, all tablet servers will be down.

And get warnings:
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=207, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.038s
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=96, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.023s
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=38, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.028s
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=61, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.083s
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=56, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.045s
Unable to open scanner: Timed out: GetTableLocations(lineitem_new, bucket=227, 
int64 l_orderkey=-9223372036854775808, int32 l_linenumber=-2147483648, 1) 
failed: GetTableLocations RPC to 10.128.3.99:7051 timed out after 0.056s

and in the error log:
F0614 09:39:49.236127  1207 key_util.cc:229] Unable to handle type 11 in row 
keysF0614 09:39:49.236134  1204 key_util.cc:229] Unable to handle type 11 in 
row keysF0614 09:39:49.236117  1194 key_util.cc:229] Unable to handle type 11 
in row keysF0614 09:39:49.236143  1212 key_util.cc:229] Unable to handle type 
11 in row keysF0F0614 09:39:49.236163  1203 key_util.cc:229] 6Unable to handle 
type 14 1^@ in row keys9:39:49.236150  1201 key_util.cc:229] Unable to handle 
type 11 in row keysF0614 09:39:49.236173  1205 key_util.cc:229] Unable to 
handle type 11 in row keysF0614 09:39:49.236189  1210 key_util.cc:229] Unable 
to handle type 11 in row keysF0614 09:39:49.236202  1193 key_util.cc:229] 
Unable to handle type 11 in row keysF0614 09:39:49.236213  1198 
key_util.cc:229] Unable to handle type 11 in row keysF0614 09:39:49.236232  
1211 key_util.cc:229] Unable to handle type 11 in row keysF0614 09:39:49.236246 
 1192 key_util.cc:229] Unable to handle type 11 in row keysF0614 
09:39:49.236251  1208 key_util.cc:229] Unable to handle type 11 in row 
keysF0614 09:39:49.236258  1209 key_util.cc:229] Unable to handle type 11 in 
row keys
*** Check failure stack trace: ***
@   0x79b53d  google::LogMessage::Fail()
@   0x79d43d  google::LogMessage::SendToLog()
@   0x79b079  google::LogMessage::Flush()
@   0x79dedf  google::LogMessageFatal::~LogMessageFatal()
@  0x16b0b21  kudu::key_util::IncrementCell()
@  0x169a8af  kudu::ColumnPredicate::InclusiveRange()
@   0x75513f  kudu::tserver::SetupScanSpec()
@   0x758c59  
kudu::tserver::TabletServiceImpl::HandleNewScanRequest()
@   0x75b73b  kudu::tserver::TabletServiceImpl::Scan()
@   0x7e935c  kudu::tserver::TabletServerServiceIf::Handle()

the table is:
CREATE TABLE lineitem_new ( 
 l_orderkey BIGINT, 
 l_linenumber INT,
 l_partkey BIGINT,
 l_suppkey BIGINT,
 l_quantity DOUBLE,
 l_extendedprice DOUBLE,
 l_discount DOUBLE,
 l_tax DOUBLE,
 l_returnflag STRING,
 l_linestatus STRING
 l_shipdate STRING,
 l_commitdate STRING,
 l_receiptdate STRING,
 l_shipinstruct STRING, 
 l_shipmode STRING,
 l_comment STRING)
TBLPROPERTIES (
'kudu.master_addresses'='host4.local:7051', 
'kudu.key_columns'