Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mike Metzger
If I understand your question correctly, the current implementation doesn't
allow a starting value, but it's easy enough to pull off with something
like:

val startval = 1
df.withColumn('id', monotonicallyIncreasingId + startval)

Two points - your test shows what happens with a single partition. With
multiple partitions, the id values will inherently be much higher (due to
the partition id being the upper 31 bits of the value.)

The other note is that the startval in this case would need to be
communicated along with the job.  It may be worth defining it as a
broadcast variable and referencing it that way so there's less cluster
communication involved.  Honestly I doubt there's a lot of variance with
this small of a value but it's a good habit to get into.

Thanks

Mike

On Fri, Aug 5, 2016 at 11:33 AM, Mich Talebzadeh 
wrote:

> Thanks Mike for this.
>
> This is Scala. As expected it adds the id column to the end of the column
> list starting from 0 0
>
> scala> val df = ll_18740868.withColumn("id", 
> monotonically_increasing_id()).show
> (2)
> +---+---+-+-+---
> ---+---++---+---+
> |transactiondate|transactiontype| sortcode|accountnumber|transac
> tiondescription|debitamount|creditamount|balance| id|
> +---+---+-+-+---
> ---+---++---+---+
> | 2009-12-31|CPT|'30-64-72| 18740868|  LTSB STH
> KENSINGT...|   90.0|null|  400.0|  0|
> | 2009-12-31|CPT|'30-64-72| 18740868|  LTSB CHELSEA
> (309...|   10.0|null|  490.0|  1|
> +---+---+-+-+---
> ---+---++---+---+
>
> Can one provide the starting value say 1?
>
> Cheers
>
>
> Dr Mich Talebzadeh
>
>
>
> LinkedIn * 
> https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
> *
>
>
>
> http://talebzadehmich.wordpress.com
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
> On 5 August 2016 at 16:45, Mike Metzger 
> wrote:
>
>> You can use the monotonically_increasing_id method to generate guaranteed
>> unique (but not necessarily consecutive) IDs.  Calling something like:
>>
>> df.withColumn("id", monotonically_increasing_id())
>>
>> You don't mention which language you're using but you'll need to pull in
>> the sql.functions library.
>>
>> Mike
>>
>> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>>
>> Ayan - basically i have a dataset with structure, where bid are unique
>> string values
>>
>> bid: String
>> val : integer
>>
>> I need unique int values for these string bid''s to do some processing in
>> the dataset
>>
>> like
>>
>> id:int   (unique integer id for each bid)
>> bid:String
>> val:integer
>>
>>
>>
>> -Tony
>>
>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>>
>>> Hi
>>>
>>> Can you explain a little further?
>>>
>>> best
>>> Ayan
>>>
>>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
>>> wrote:
>>>
 I have a row with structure like

 identifier: String
 value: int

 All identifier are unique and I want to generate a unique long id for
 the data and get a row object back for further processing.

 I understand using the zipWithUniqueId function on RDD, but that would
 mean first converting to RDD and then joining back the RDD and dataset

 What is the best way to do this ?

 -Tony


>>>
>>>
>>> --
>>> Best Regards,
>>> Ayan Guha
>>>
>>
>>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mike Metzger
Should be pretty much the same code for Scala -

import java.util.UUID
UUID.randomUUID

If you need it as a UDF, just wrap it accordingly.

Mike

On Fri, Aug 5, 2016 at 11:38 AM, Mich Talebzadeh 
wrote:

> On the same token can one generate  a UUID like below in Hive
>
> hive> select reflect("java.util.UUID", "randomUUID");
> OK
> 587b1665-b578-4124-8bf9-8b17ccb01fe7
>
> thx
>
> Dr Mich Talebzadeh
>
>
>
> LinkedIn * 
> https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
> *
>
>
>
> http://talebzadehmich.wordpress.com
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
> On 5 August 2016 at 17:34, Mike Metzger 
> wrote:
>
>> Tony -
>>
>>From my testing this is built with performance in mind.  It's a 64-bit
>> value split between the partition id (upper 31 bits ~1billion) and the id
>> counter within a partition (lower 33 bits ~8 billion).  There shouldn't be
>> any added communication between the executors and the driver for that.
>>
>> I've been toying with an implementation that allows you to specify the
>> split for better control along with a start value.
>>
>> Thanks
>>
>> Mike
>>
>> On Aug 5, 2016, at 11:07 AM, Tony Lane  wrote:
>>
>> Mike.
>>
>> I have figured how to do this .  Thanks for the suggestion. It works
>> great.  I am trying to figure out the performance impact of this.
>>
>> thanks again
>>
>>
>> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>>
>>> @mike  - this looks great. How can i do this in java ?   what is the
>>> performance implication on a large dataset  ?
>>>
>>> @sonal  - I can't have a collision in the values.
>>>
>>> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger >> > wrote:
>>>
 You can use the monotonically_increasing_id method to generate
 guaranteed unique (but not necessarily consecutive) IDs.  Calling something
 like:

 df.withColumn("id", monotonically_increasing_id())

 You don't mention which language you're using but you'll need to pull
 in the sql.functions library.

 Mike

 On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:

 Ayan - basically i have a dataset with structure, where bid are unique
 string values

 bid: String
 val : integer

 I need unique int values for these string bid''s to do some processing
 in the dataset

 like

 id:int   (unique integer id for each bid)
 bid:String
 val:integer



 -Tony

 On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:

> Hi
>
> Can you explain a little further?
>
> best
> Ayan
>
> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
> wrote:
>
>> I have a row with structure like
>>
>> identifier: String
>> value: int
>>
>> All identifier are unique and I want to generate a unique long id for
>> the data and get a row object back for further processing.
>>
>> I understand using the zipWithUniqueId function on RDD, but that
>> would mean first converting to RDD and then joining back the RDD and 
>> dataset
>>
>> What is the best way to do this ?
>>
>> -Tony
>>
>>
>
>
> --
> Best Regards,
> Ayan Guha
>


>>>
>>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mich Talebzadeh
This is a UDF written for Hive to monolithically increment a column by 1

http://svn.apache.org/repos/asf/hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/udf/UDFRowSequence.java


package org.apache.hadoop.hive.contrib.udf;
import org.apache.hadoop.hive.ql.exec.Description;
import org.apache.hadoop.hive.ql.exec.UDF;
import org.apache.hadoop.hive.ql.udf.UDFType;
import org.apache.hadoop.io.LongWritable;
/**
 * UDFRowSequence.
 */
@Description(name = "row_sequence",
value = "_FUNC_() - Returns a generated row sequence number starting
from 1")
@UDFType(deterministic = false, stateful = true)
public class UDFRowSequence extends UDF
{
  private LongWritable result = new LongWritable();
  public UDFRowSequence() {
result.set(0);
  }
  public LongWritable evaluate() {
result.set(result.get() + 1);
return result;
  }
}
// End UDFRowSequence.java

Is there equivalent of this one for Spark and in Scala as well

Thanks



Dr Mich Talebzadeh



LinkedIn * 
https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
*



http://talebzadehmich.wordpress.com


*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.



On 5 August 2016 at 17:38, Mich Talebzadeh 
wrote:

> On the same token can one generate  a UUID like below in Hive
>
> hive> select reflect("java.util.UUID", "randomUUID");
> OK
> 587b1665-b578-4124-8bf9-8b17ccb01fe7
>
> thx
>
> Dr Mich Talebzadeh
>
>
>
> LinkedIn * 
> https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
> *
>
>
>
> http://talebzadehmich.wordpress.com
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
> On 5 August 2016 at 17:34, Mike Metzger 
> wrote:
>
>> Tony -
>>
>>From my testing this is built with performance in mind.  It's a 64-bit
>> value split between the partition id (upper 31 bits ~1billion) and the id
>> counter within a partition (lower 33 bits ~8 billion).  There shouldn't be
>> any added communication between the executors and the driver for that.
>>
>> I've been toying with an implementation that allows you to specify the
>> split for better control along with a start value.
>>
>> Thanks
>>
>> Mike
>>
>> On Aug 5, 2016, at 11:07 AM, Tony Lane  wrote:
>>
>> Mike.
>>
>> I have figured how to do this .  Thanks for the suggestion. It works
>> great.  I am trying to figure out the performance impact of this.
>>
>> thanks again
>>
>>
>> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>>
>>> @mike  - this looks great. How can i do this in java ?   what is the
>>> performance implication on a large dataset  ?
>>>
>>> @sonal  - I can't have a collision in the values.
>>>
>>> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger >> > wrote:
>>>
 You can use the monotonically_increasing_id method to generate
 guaranteed unique (but not necessarily consecutive) IDs.  Calling something
 like:

 df.withColumn("id", monotonically_increasing_id())

 You don't mention which language you're using but you'll need to pull
 in the sql.functions library.

 Mike

 On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:

 Ayan - basically i have a dataset with structure, where bid are unique
 string values

 bid: String
 val : integer

 I need unique int values for these string bid''s to do some processing
 in the dataset

 like

 id:int   (unique integer id for each bid)
 bid:String
 val:integer



 -Tony

 On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:

> Hi
>
> Can you explain a little further?
>
> best
> Ayan
>
> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
> wrote:
>
>> I have a row with structure like
>>
>> identifier: String
>> value: int
>>
>> All identifier are unique and I want to generate a unique long id for
>> the data and get a row object back for further processing.
>>
>> I understand using the zipWithUniqueId function on RDD, but that
>> would mean first converting to RDD and then 

Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mich Talebzadeh
On the same token can one generate  a UUID like below in Hive

hive> select reflect("java.util.UUID", "randomUUID");
OK
587b1665-b578-4124-8bf9-8b17ccb01fe7

thx

Dr Mich Talebzadeh



LinkedIn * 
https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
*



http://talebzadehmich.wordpress.com


*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.



On 5 August 2016 at 17:34, Mike Metzger  wrote:

> Tony -
>
>From my testing this is built with performance in mind.  It's a 64-bit
> value split between the partition id (upper 31 bits ~1billion) and the id
> counter within a partition (lower 33 bits ~8 billion).  There shouldn't be
> any added communication between the executors and the driver for that.
>
> I've been toying with an implementation that allows you to specify the
> split for better control along with a start value.
>
> Thanks
>
> Mike
>
> On Aug 5, 2016, at 11:07 AM, Tony Lane  wrote:
>
> Mike.
>
> I have figured how to do this .  Thanks for the suggestion. It works
> great.  I am trying to figure out the performance impact of this.
>
> thanks again
>
>
> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>
>> @mike  - this looks great. How can i do this in java ?   what is the
>> performance implication on a large dataset  ?
>>
>> @sonal  - I can't have a collision in the values.
>>
>> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger 
>> wrote:
>>
>>> You can use the monotonically_increasing_id method to generate
>>> guaranteed unique (but not necessarily consecutive) IDs.  Calling something
>>> like:
>>>
>>> df.withColumn("id", monotonically_increasing_id())
>>>
>>> You don't mention which language you're using but you'll need to pull in
>>> the sql.functions library.
>>>
>>> Mike
>>>
>>> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>>>
>>> Ayan - basically i have a dataset with structure, where bid are unique
>>> string values
>>>
>>> bid: String
>>> val : integer
>>>
>>> I need unique int values for these string bid''s to do some processing
>>> in the dataset
>>>
>>> like
>>>
>>> id:int   (unique integer id for each bid)
>>> bid:String
>>> val:integer
>>>
>>>
>>>
>>> -Tony
>>>
>>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>>>
 Hi

 Can you explain a little further?

 best
 Ayan

 On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
 wrote:

> I have a row with structure like
>
> identifier: String
> value: int
>
> All identifier are unique and I want to generate a unique long id for
> the data and get a row object back for further processing.
>
> I understand using the zipWithUniqueId function on RDD, but that would
> mean first converting to RDD and then joining back the RDD and dataset
>
> What is the best way to do this ?
>
> -Tony
>
>


 --
 Best Regards,
 Ayan Guha

>>>
>>>
>>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mike Metzger
Not that I've seen, at least not in any worker independent way.  To guarantee 
consecutive values you'd have to create a udf or some such that provided a new 
row id.  This probably isn't an issue on small data sets but would cause a lot 
of added communication on larger clusters / datasets.

Mike

> On Aug 5, 2016, at 11:21 AM, janardhan shetty  wrote:
> 
> Mike,
> 
> Any suggestions on doing it for consequitive id's?
> 
>> On Aug 5, 2016 9:08 AM, "Tony Lane"  wrote:
>> Mike.
>> 
>> I have figured how to do this .  Thanks for the suggestion. It works great.  
>> I am trying to figure out the performance impact of this. 
>> 
>> thanks again
>> 
>> 
>>> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>>> @mike  - this looks great. How can i do this in java ?   what is the 
>>> performance implication on a large dataset  ? 
>>> 
>>> @sonal  - I can't have a collision in the values. 
>>> 
 On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger  
 wrote:
 You can use the monotonically_increasing_id method to generate guaranteed 
 unique (but not necessarily consecutive) IDs.  Calling something like:
 
 df.withColumn("id", monotonically_increasing_id())
 
 You don't mention which language you're using but you'll need to pull in 
 the sql.functions library.
 
 Mike
 
> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
> 
> Ayan - basically i have a dataset with structure, where bid are unique 
> string values
> 
> bid: String
> val : integer
> 
> I need unique int values for these string bid''s to do some processing in 
> the dataset
> 
> like 
> 
> id:int   (unique integer id for each bid)
> bid:String
> val:integer
> 
> 
> 
> -Tony
> 
>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>> Hi
>> 
>> Can you explain a little further? 
>> 
>> best
>> Ayan
>> 
>>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane  
>>> wrote:
>>> I have a row with structure like
>>> 
>>> identifier: String
>>> value: int
>>> 
>>> All identifier are unique and I want to generate a unique long id for 
>>> the data and get a row object back for further processing. 
>>> 
>>> I understand using the zipWithUniqueId function on RDD, but that would 
>>> mean first converting to RDD and then joining back the RDD and dataset
>>> 
>>> What is the best way to do this ? 
>>> 
>>> -Tony 
>> 
>> 
>> 
>> -- 
>> Best Regards,
>> Ayan Guha


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mike Metzger
Tony -

   From my testing this is built with performance in mind.  It's a 64-bit value 
split between the partition id (upper 31 bits ~1billion) and the id counter 
within a partition (lower 33 bits ~8 billion).  There shouldn't be any added 
communication between the executors and the driver for that.

I've been toying with an implementation that allows you to specify the split 
for better control along with a start value. 

Thanks

Mike

> On Aug 5, 2016, at 11:07 AM, Tony Lane  wrote:
> 
> Mike.
> 
> I have figured how to do this .  Thanks for the suggestion. It works great.  
> I am trying to figure out the performance impact of this. 
> 
> thanks again
> 
> 
>> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>> @mike  - this looks great. How can i do this in java ?   what is the 
>> performance implication on a large dataset  ? 
>> 
>> @sonal  - I can't have a collision in the values. 
>> 
>>> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger  
>>> wrote:
>>> You can use the monotonically_increasing_id method to generate guaranteed 
>>> unique (but not necessarily consecutive) IDs.  Calling something like:
>>> 
>>> df.withColumn("id", monotonically_increasing_id())
>>> 
>>> You don't mention which language you're using but you'll need to pull in 
>>> the sql.functions library.
>>> 
>>> Mike
>>> 
 On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
 
 Ayan - basically i have a dataset with structure, where bid are unique 
 string values
 
 bid: String
 val : integer
 
 I need unique int values for these string bid''s to do some processing in 
 the dataset
 
 like 
 
 id:int   (unique integer id for each bid)
 bid:String
 val:integer
 
 
 
 -Tony
 
> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
> Hi
> 
> Can you explain a little further? 
> 
> best
> Ayan
> 
>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane  
>> wrote:
>> I have a row with structure like
>> 
>> identifier: String
>> value: int
>> 
>> All identifier are unique and I want to generate a unique long id for 
>> the data and get a row object back for further processing. 
>> 
>> I understand using the zipWithUniqueId function on RDD, but that would 
>> mean first converting to RDD and then joining back the RDD and dataset
>> 
>> What is the best way to do this ? 
>> 
>> -Tony 
> 
> 
> 
> -- 
> Best Regards,
> Ayan Guha
> 


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mich Talebzadeh
Thanks Mike for this.

This is Scala. As expected it adds the id column to the end of the column
list starting from 0 0

scala> val df = ll_18740868.withColumn("id",
monotonically_increasing_id()).show
(2)
+---+---+-+-+---
---+---++---+---+
|transactiondate|transactiontype| sortcode|accountnumber|
transactiondescription|debitamount|creditamount|balance| id|
+---+---+-+-+---
---+---++---+---+
| 2009-12-31|CPT|'30-64-72| 18740868|  LTSB STH
KENSINGT...|   90.0|null|  400.0|  0|
| 2009-12-31|CPT|'30-64-72| 18740868|  LTSB CHELSEA
(309...|   10.0|null|  490.0|  1|
+---+---+-+-+---
---+---++---+---+

Can one provide the starting value say 1?

Cheers


Dr Mich Talebzadeh



LinkedIn * 
https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
*



http://talebzadehmich.wordpress.com


*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.



On 5 August 2016 at 16:45, Mike Metzger  wrote:

> You can use the monotonically_increasing_id method to generate guaranteed
> unique (but not necessarily consecutive) IDs.  Calling something like:
>
> df.withColumn("id", monotonically_increasing_id())
>
> You don't mention which language you're using but you'll need to pull in
> the sql.functions library.
>
> Mike
>
> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>
> Ayan - basically i have a dataset with structure, where bid are unique
> string values
>
> bid: String
> val : integer
>
> I need unique int values for these string bid''s to do some processing in
> the dataset
>
> like
>
> id:int   (unique integer id for each bid)
> bid:String
> val:integer
>
>
>
> -Tony
>
> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>
>> Hi
>>
>> Can you explain a little further?
>>
>> best
>> Ayan
>>
>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
>> wrote:
>>
>>> I have a row with structure like
>>>
>>> identifier: String
>>> value: int
>>>
>>> All identifier are unique and I want to generate a unique long id for
>>> the data and get a row object back for further processing.
>>>
>>> I understand using the zipWithUniqueId function on RDD, but that would
>>> mean first converting to RDD and then joining back the RDD and dataset
>>>
>>> What is the best way to do this ?
>>>
>>> -Tony
>>>
>>>
>>
>>
>> --
>> Best Regards,
>> Ayan Guha
>>
>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread janardhan shetty
Mike,

Any suggestions on doing it for consequitive id's?
On Aug 5, 2016 9:08 AM, "Tony Lane"  wrote:

> Mike.
>
> I have figured how to do this .  Thanks for the suggestion. It works
> great.  I am trying to figure out the performance impact of this.
>
> thanks again
>
>
> On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:
>
>> @mike  - this looks great. How can i do this in java ?   what is the
>> performance implication on a large dataset  ?
>>
>> @sonal  - I can't have a collision in the values.
>>
>> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger 
>> wrote:
>>
>>> You can use the monotonically_increasing_id method to generate
>>> guaranteed unique (but not necessarily consecutive) IDs.  Calling something
>>> like:
>>>
>>> df.withColumn("id", monotonically_increasing_id())
>>>
>>> You don't mention which language you're using but you'll need to pull in
>>> the sql.functions library.
>>>
>>> Mike
>>>
>>> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>>>
>>> Ayan - basically i have a dataset with structure, where bid are unique
>>> string values
>>>
>>> bid: String
>>> val : integer
>>>
>>> I need unique int values for these string bid''s to do some processing
>>> in the dataset
>>>
>>> like
>>>
>>> id:int   (unique integer id for each bid)
>>> bid:String
>>> val:integer
>>>
>>>
>>>
>>> -Tony
>>>
>>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>>>
 Hi

 Can you explain a little further?

 best
 Ayan

 On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
 wrote:

> I have a row with structure like
>
> identifier: String
> value: int
>
> All identifier are unique and I want to generate a unique long id for
> the data and get a row object back for further processing.
>
> I understand using the zipWithUniqueId function on RDD, but that would
> mean first converting to RDD and then joining back the RDD and dataset
>
> What is the best way to do this ?
>
> -Tony
>
>


 --
 Best Regards,
 Ayan Guha

>>>
>>>
>>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Tony Lane
Mike.

I have figured how to do this .  Thanks for the suggestion. It works
great.  I am trying to figure out the performance impact of this.

thanks again


On Fri, Aug 5, 2016 at 9:25 PM, Tony Lane  wrote:

> @mike  - this looks great. How can i do this in java ?   what is the
> performance implication on a large dataset  ?
>
> @sonal  - I can't have a collision in the values.
>
> On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger 
> wrote:
>
>> You can use the monotonically_increasing_id method to generate guaranteed
>> unique (but not necessarily consecutive) IDs.  Calling something like:
>>
>> df.withColumn("id", monotonically_increasing_id())
>>
>> You don't mention which language you're using but you'll need to pull in
>> the sql.functions library.
>>
>> Mike
>>
>> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>>
>> Ayan - basically i have a dataset with structure, where bid are unique
>> string values
>>
>> bid: String
>> val : integer
>>
>> I need unique int values for these string bid''s to do some processing in
>> the dataset
>>
>> like
>>
>> id:int   (unique integer id for each bid)
>> bid:String
>> val:integer
>>
>>
>>
>> -Tony
>>
>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>>
>>> Hi
>>>
>>> Can you explain a little further?
>>>
>>> best
>>> Ayan
>>>
>>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
>>> wrote:
>>>
 I have a row with structure like

 identifier: String
 value: int

 All identifier are unique and I want to generate a unique long id for
 the data and get a row object back for further processing.

 I understand using the zipWithUniqueId function on RDD, but that would
 mean first converting to RDD and then joining back the RDD and dataset

 What is the best way to do this ?

 -Tony


>>>
>>>
>>> --
>>> Best Regards,
>>> Ayan Guha
>>>
>>
>>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Tony Lane
@mike  - this looks great. How can i do this in java ?   what is the
performance implication on a large dataset  ?

@sonal  - I can't have a collision in the values.

On Fri, Aug 5, 2016 at 9:15 PM, Mike Metzger 
wrote:

> You can use the monotonically_increasing_id method to generate guaranteed
> unique (but not necessarily consecutive) IDs.  Calling something like:
>
> df.withColumn("id", monotonically_increasing_id())
>
> You don't mention which language you're using but you'll need to pull in
> the sql.functions library.
>
> Mike
>
> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
>
> Ayan - basically i have a dataset with structure, where bid are unique
> string values
>
> bid: String
> val : integer
>
> I need unique int values for these string bid''s to do some processing in
> the dataset
>
> like
>
> id:int   (unique integer id for each bid)
> bid:String
> val:integer
>
>
>
> -Tony
>
> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>
>> Hi
>>
>> Can you explain a little further?
>>
>> best
>> Ayan
>>
>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
>> wrote:
>>
>>> I have a row with structure like
>>>
>>> identifier: String
>>> value: int
>>>
>>> All identifier are unique and I want to generate a unique long id for
>>> the data and get a row object back for further processing.
>>>
>>> I understand using the zipWithUniqueId function on RDD, but that would
>>> mean first converting to RDD and then joining back the RDD and dataset
>>>
>>> What is the best way to do this ?
>>>
>>> -Tony
>>>
>>>
>>
>>
>> --
>> Best Regards,
>> Ayan Guha
>>
>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Mike Metzger
You can use the monotonically_increasing_id method to generate guaranteed 
unique (but not necessarily consecutive) IDs.  Calling something like:

df.withColumn("id", monotonically_increasing_id())

You don't mention which language you're using but you'll need to pull in the 
sql.functions library.

Mike

> On Aug 5, 2016, at 9:11 AM, Tony Lane  wrote:
> 
> Ayan - basically i have a dataset with structure, where bid are unique string 
> values
> 
> bid: String
> val : integer
> 
> I need unique int values for these string bid''s to do some processing in the 
> dataset
> 
> like 
> 
> id:int   (unique integer id for each bid)
> bid:String
> val:integer
> 
> 
> 
> -Tony
> 
>> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>> Hi
>> 
>> Can you explain a little further? 
>> 
>> best
>> Ayan
>> 
>>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane  wrote:
>>> I have a row with structure like
>>> 
>>> identifier: String
>>> value: int
>>> 
>>> All identifier are unique and I want to generate a unique long id for the 
>>> data and get a row object back for further processing. 
>>> 
>>> I understand using the zipWithUniqueId function on RDD, but that would mean 
>>> first converting to RDD and then joining back the RDD and dataset
>>> 
>>> What is the best way to do this ? 
>>> 
>>> -Tony 
>> 
>> 
>> 
>> -- 
>> Best Regards,
>> Ayan Guha
> 


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Sonal Goyal
Hi Tony,

Would hash on the bid work for you?

hash(cols: Column

*): Column
[image:
Permalink]


Calculates the hash code of given columns, and returns the result as an int
column.
Annotations@varargs()Since

2.0

Best Regards,
Sonal
Founder, Nube Technologies 
Reifier at Strata Hadoop World 
Reifier at Spark Summit 2015






On Fri, Aug 5, 2016 at 7:41 PM, Tony Lane  wrote:

> Ayan - basically i have a dataset with structure, where bid are unique
> string values
>
> bid: String
> val : integer
>
> I need unique int values for these string bid''s to do some processing in
> the dataset
>
> like
>
> id:int   (unique integer id for each bid)
> bid:String
> val:integer
>
>
>
> -Tony
>
> On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:
>
>> Hi
>>
>> Can you explain a little further?
>>
>> best
>> Ayan
>>
>> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane 
>> wrote:
>>
>>> I have a row with structure like
>>>
>>> identifier: String
>>> value: int
>>>
>>> All identifier are unique and I want to generate a unique long id for
>>> the data and get a row object back for further processing.
>>>
>>> I understand using the zipWithUniqueId function on RDD, but that would
>>> mean first converting to RDD and then joining back the RDD and dataset
>>>
>>> What is the best way to do this ?
>>>
>>> -Tony
>>>
>>>
>>
>>
>> --
>> Best Regards,
>> Ayan Guha
>>
>
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Tony Lane
Ayan - basically i have a dataset with structure, where bid are unique
string values

bid: String
val : integer

I need unique int values for these string bid''s to do some processing in
the dataset

like

id:int   (unique integer id for each bid)
bid:String
val:integer



-Tony

On Fri, Aug 5, 2016 at 6:35 PM, ayan guha  wrote:

> Hi
>
> Can you explain a little further?
>
> best
> Ayan
>
> On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane  wrote:
>
>> I have a row with structure like
>>
>> identifier: String
>> value: int
>>
>> All identifier are unique and I want to generate a unique long id for the
>> data and get a row object back for further processing.
>>
>> I understand using the zipWithUniqueId function on RDD, but that would
>> mean first converting to RDD and then joining back the RDD and dataset
>>
>> What is the best way to do this ?
>>
>> -Tony
>>
>>
>
>
> --
> Best Regards,
> Ayan Guha
>


Re: Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread ayan guha
Hi

Can you explain a little further?

best
Ayan

On Fri, Aug 5, 2016 at 10:14 PM, Tony Lane  wrote:

> I have a row with structure like
>
> identifier: String
> value: int
>
> All identifier are unique and I want to generate a unique long id for the
> data and get a row object back for further processing.
>
> I understand using the zipWithUniqueId function on RDD, but that would
> mean first converting to RDD and then joining back the RDD and dataset
>
> What is the best way to do this ?
>
> -Tony
>
>


-- 
Best Regards,
Ayan Guha


Generating unique id for a column in Row without breaking into RDD and joining back

2016-08-05 Thread Tony Lane
I have a row with structure like

identifier: String
value: int

All identifier are unique and I want to generate a unique long id for the
data and get a row object back for further processing.

I understand using the zipWithUniqueId function on RDD, but that would mean
first converting to RDD and then joining back the RDD and dataset

What is the best way to do this ?

-Tony