Return-Path: X-Original-To: apmail-avro-user-archive@www.apache.org Delivered-To: apmail-avro-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E12FB10908 for ; Thu, 3 Oct 2013 01:50:52 +0000 (UTC) Received: (qmail 60906 invoked by uid 500); 3 Oct 2013 01:50:52 -0000 Delivered-To: apmail-avro-user-archive@avro.apache.org Received: (qmail 60822 invoked by uid 500); 3 Oct 2013 01:50:51 -0000 Mailing-List: contact user-help@avro.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@avro.apache.org Delivered-To: mailing list user@avro.apache.org Received: (qmail 60814 invoked by uid 99); 3 Oct 2013 01:50:51 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Oct 2013 01:50:51 +0000 X-ASF-Spam-Status: No, hits=2.5 required=5.0 tests=FREEMAIL_REPLY,HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of phearm@gmail.com designates 209.85.219.53 as permitted sender) Received: from [209.85.219.53] (HELO mail-oa0-f53.google.com) (209.85.219.53) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Oct 2013 01:50:43 +0000 Received: by mail-oa0-f53.google.com with SMTP id i7so1693793oag.26 for ; Wed, 02 Oct 2013 18:50:21 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=from:to:references:in-reply-to:subject:date:message-id:mime-version :content-type:thread-index:content-language; bh=NYpYFtYD0pB+GaAWQvXv6aTVpa5bSDTgDeEIbNPg69o=; b=xCLKeR8aYHpKWLp7kBNmuPlU03C8LAqEs2U8vV4lWUH9DYHWMlWfm8Wql8BmIvmVf7 4whhgM64JQQXh/y9WTnworQBCCcL2wgYRCb2k3RW07rFFxQWQvStoX2wd5PqQnjZUHfv IaqctBynu1ZFTJBIAdgkr4efVLmkUdLk2m3GucSr5VVmeCXuuEG/RiMmljN1b2c3QP1c AdHTNH2VBOLTQJP8sF3fioh7/6NEtMFINJzdpcTliNWdJUMUWhY6dkOJy6KkdsnJXF4T TzsKQfcnl+RioKPxKTFFDsUWewp+uAEyjAcayvSgmZkDQaBU5MLoYbiLVDrXYPCzcxuO qKQQ== X-Received: by 10.60.60.105 with SMTP id g9mr8434587oer.8.1380765021757; Wed, 02 Oct 2013 18:50:21 -0700 (PDT) Received: from Beren (cpe-72-182-107-45.austin.res.rr.com. [72.182.107.45]) by mx.google.com with ESMTPSA id j9sm10597326oef.8.1969.12.31.16.00.00 (version=TLSv1 cipher=RC4-SHA bits=128/128); Wed, 02 Oct 2013 18:50:21 -0700 (PDT) From: Alan Paulsen To: References: ,<08a801cebe55$6324d960$296e8c20$@com> In-Reply-To: Subject: RE: Question related to AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema s) Date: Wed, 2 Oct 2013 20:50:18 -0500 Message-ID: <09fe01cebfda$ea3fd270$bebf7750$@com> MIME-Version: 1.0 Content-Type: multipart/alternative; boundary="----=_NextPart_000_09FF_01CEBFB1.0169CA70" X-Mailer: Microsoft Office Outlook 12.0 Thread-Index: Ac6+mJS8j318iT82QY+j/M7IvLlasgBQWO2w Content-Language: en-us X-Virus-Checked: Checked by ClamAV on apache.org This is a multi-part message in MIME format. ------=_NextPart_000_09FF_01CEBFB1.0169CA70 Content-Type: text/plain; charset="us-ascii" Content-Transfer-Encoding: 7bit Hi Yong, Sorry for the delay. You can specify all the schemas you will use in your driver. You can use SpecificRecord if you've generated the class files, GenericRecord can be used without any code generation. Take a look at some of the unit tests to get a better understanding on using SpecificRecords and GenericRecords. In other words, if you are generating your avro records in your reducers, you can write to your multiple outputs using the appropriate output. If you are generating the records in your mapper, you need to set the MapOutputSchema to a union of all of your schemas, in order to pass along the avro records to your reducers. Thanks, Alan From: java8964 java8964 [mailto:java8964@hotmail.com] Sent: Tuesday, October 01, 2013 6:22 AM To: user@avro.apache.org Subject: RE: Question related to AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema s) Hi, Alan: Thanks for you suggestion. I will take a look about MultipleOutput. But in this case, I still need to specify the schema in my driver, right? You mean I should use union schema in this case? But in my mapper, should I use SpecificRecord or GenericRecord? I can use (K,V) in my reducer, but in the mapper, I need the concrete Record object to serialize my data, right? Yong _____ From: phearm@gmail.com To: user@avro.apache.org Subject: RE: Question related to AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema s) Date: Mon, 30 Sep 2013 22:21:57 -0500 Hi Yong, It sounds like you might need to use AvroMultipleOutputs here. You can set all five of your output schemas in your driver, then route your message to the appropriate output in your reducer. See the following for mapred: http://avro.apache.org/docs/1.7.5/api/java/org/apache/avro/mapred/AvroMultip leOutputs.html And the following for mapreduce: http://avro.apache.org/docs/1.7.5/api/java/org/apache/avro/mapreduce/AvroMul tipleOutputs.html If your mapper is generating the Avro records, then you will probably have to set AvroJob.setMapOutputSchema to a union of all five of your schemas. Thanks, Alan From: java8964 java8964 [mailto:java8964@hotmail.com] Sent: Monday, September 30, 2013 9:37 PM To: user@avro.apache.org Subject: Question related to AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema s) Hi, I am new to user Avro. Currently, I am working on an existing project, and I want to see if using Avro makes sense. The project is to do some ETL around 5 data sets' data. The ETL logic is not complex, it will do different transferring logic for different data sets, and partition the data daily in reducer. There was one MR job to handle all 5 data sets originally. The data files have the name convention to distinguish the data sets. So in the mapper, it bases on the file name to understand what data set it is, and generate the key as "datase_name + date" to partition the data set first by data set, then daily. Now if I want to store the data in Avro format, it is straight-forward to write MR job for only one data set following a lot of online examples. I have no problem to change the MR job to store the data as Avro format for one data set. But if I still want to use one MR job for all 5 data sets, I got a problem. I tried both "SpecificRecord" and "GenericRecord", but I don't know how to solve this problem. For example, I created 5 avsc files for 5 data sets, and generate the Record object for all of them. But in the mapper/reducer, I don't want to specify any Record class, and this same mapper/reducer should be able to handle all data sets. So I try to put SpecificRecord class in my mapper/reducer, but in this case, I don't have the SpecificRecord.SCHEMA$ to use in my driver of AvroJob.setMapOutputSchema(conf, Schema), even though in my case, I really prefer the "SpecificRecord". So that makes me to try "GenericRecord". I change all my mapper and reducer to use "GenericRecord" class. But still, I don't know what schema I should use in my driver class for AvroJob.setMapOutputSchema(conf, Schema). The problem is that is there a generic abstract schema class I can use in AvroJob.setMapOutputSchema or AvroJob.setOutputSchema? My mapper class will correctly generate either "GenericRecord" or "SpecificRecord" class at runtime based on the file name, and reducer will write the correct "GenericRecord" or "SpecificRecord" object to the right output location without knowing the concrete Record object. But what stops me now is what kind of schema object I can use in AvroJob. I don't know during the driver stage what is my output schema, but the mapper/reducer will figure that out at runtime. Can I do this in Avro? Thanks Yong ------=_NextPart_000_09FF_01CEBFB1.0169CA70 Content-Type: text/html; charset="us-ascii" Content-Transfer-Encoding: quoted-printable

Hi Yong,

 

Sorry for the delay.

 

You can specify all the schemas you will use in your driver.  = You can use SpecificRecord if you've generated the class files, = GenericRecord can be used without any code generation.  Take a look = at some of the unit tests to get a better understanding on using = SpecificRecords and GenericRecords.

 

In other words, if you are generating your avro records in your = reducers, you can write to your multiple outputs using the appropriate = output.

 

If you are generating the records in your mapper, you need to set the = MapOutputSchema to a union of all of your schemas, in order to pass = along the avro records to your reducers.  =   

 

Thanks,

 

Alan

 

From:= = java8964 java8964 [mailto:java8964@hotmail.com]
Sent: = Tuesday, October 01, 2013 6:22 AM
To: = user@avro.apache.org
Subject: RE: Question related to = AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema = s)

 

Hi, = Alan:

 

=

Thanks for you suggestion. = I will take a look about = MultipleOutput.

 

=

But in this case, I still = need to specify the schema in  my driver, right? You mean I should = use union schema in this case? But in my mapper, should I use = SpecificRecord or GenericRecord? I can use (K,V) in my reducer, but in = the mapper, I need the concrete Record object to serialize my data, = right?

 

=

Yong


From: phearm@gmail.com
To: user@avro.apache.org
Subject:= RE: Question related to = AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema = s)
Date: Mon, 30 Sep 2013 22:21:57 -0500

Hi Yong,

 

It sounds like you might need to use AvroMultipleOutputs here.  =

 

You can set all five of your output schemas in your driver, then = route your message to the appropriate output in your reducer.  =

 

See the following for mapred: http://avro.apache.org/docs/1.7.5/api/java/org/apache/a= vro/mapred/AvroMultipleOutputs.html

 

=

And the following for mapreduce: http://avro.apache.org/docs/1.7.5/api/java/org/apache/a= vro/mapreduce/AvroMultipleOutputs.html

 

If your mapper is generating the Avro records, then you will probably = have to set AvroJob.setMapOutputSchema to a union of all five of your = schemas.

 

Thanks,

 

Alan

 

<= div style=3D'border:none;border-top:solid #B5C4DF 1.0pt;padding:3.0pt = 0in 0in 0in'>

From:= = java8964 java8964 [mailto:java8964@hotmail.com] =
Sent: Monday, September 30, 2013 9:37 PM
To: user@avro.apache.org
Subje= ct: Question related to = AvroJob.setMapOutputSchema(org.apache.hadoop.mapred.JobConf job, Schema = s)

=

 

=

Hi, <= /p>

 

=

I am new to user Avro. = Currently, I am working on an existing project, and I want to see if = using Avro makes sense.

 

=

The project is to do some = ETL around 5 data sets' data. The ETL logic is not complex, it will do = different transferring logic for different  data sets, and = partition the data daily in reducer.

 

=

There was one MR job to = handle all 5 data sets originally. The data files have the name = convention to distinguish the data sets. So in the mapper, it bases on = the file name to understand what data set it is, and generate the key as = "datase_name + date" to partition the data set first by data = set, then daily.

 

=

Now if I want to store the = data in Avro format, it is straight-forward to write MR job for only one = data set following a lot of online examples. I have no problem to change = the MR job to store the data as Avro format for one data = set.

 

=

But if I still want to use = one MR job for all 5 data sets, I got a = problem.

 

=

I tried both = "SpecificRecord" and "GenericRecord", but I don't = know how to solve this problem.

 

=

For example, I created 5 = avsc files for 5 data sets, and generate the Record object for all of = them. But in the mapper/reducer, I don't want to specify any Record = class, and this same mapper/reducer should be able to handle all data = sets. So I try to put SpecificRecord  class in my mapper/reducer, = but in this case, I don't have the SpecificRecord.SCHEMA$ to use in my = driver of AvroJob.setMapOutputSchema(conf, Schema), even though in my = case, I really prefer the = "SpecificRecord".

 

=

So that makes me to try = "GenericRecord". I change all my mapper and reducer to use = "GenericRecord" class. But still, I don't know what schema I = should use in my driver class for AvroJob.setMapOutputSchema(conf, = Schema). The problem is that is there a generic abstract schema class I = can use in AvroJob.setMapOutputSchema or AvroJob.setOutputSchema? My = mapper class will correctly generate either "GenericRecord" or = "SpecificRecord" class at runtime based on the file name, and = reducer will write the correct "GenericRecord" or = "SpecificRecord" object to the right output location without = knowing the concrete Record object. But what stops me now is what kind = of schema object I can use in AvroJob. I don't know during the driver = stage what is my output schema, but the mapper/reducer will figure that = out at runtime. Can I do this in = Avro?

 

=

Thanks

=

 

=

Yong

------=_NextPart_000_09FF_01CEBFB1.0169CA70--