Is there a jira about the design of the new file format?

Sent from my iPhone

On Jan 24, 2011, at 5:40 PM, yongqiang he <heyongqiang...@gmail.com> wrote:

> 4M is a good number based on a lot of experiments. Increase the number
> will reduce the file size, but the saving will increase very slow
> after the block size goes beyond 4M, but the perf/memory will increase
> a lot.
> 
> FYI, we are trying a new columnar file format. And it seems 4M is no
> longer the magic number for this new file format. But still do not
> have some number now.
> 
> On Mon, Jan 24, 2011 at 5:03 PM, Edward Capriolo <edlinuxg...@gmail.com> 
> wrote:
>> On Mon, Jan 24, 2011 at 7:51 PM, yongqiang he <heyongqiang...@gmail.com> 
>> wrote:
>>> Yes. It only support block compression. (No record level compression 
>>> support.)
>>> You can use the config 'hive.io.rcfile.record.buffer.size' to specify
>>> the block size (before compression). The default is 4MB.
>>> 
>>> Thanks
>>> Yongqiang
>>> On Mon, Jan 24, 2011 at 4:44 PM, Edward Capriolo <edlinuxg...@gmail.com> 
>>> wrote:
>>>> On Mon, Jan 24, 2011 at 4:42 PM, Edward Capriolo <edlinuxg...@gmail.com> 
>>>> wrote:
>>>>> On Mon, Jan 24, 2011 at 4:14 PM, yongqiang he <heyongqiang...@gmail.com> 
>>>>> wrote:
>>>>>> How did you upload the data to the new table?
>>>>>> You can get the data compressed by doing a insert overwrite to the
>>>>>> destination table with setting "hive.exec.compress.output" to true.
>>>>>> 
>>>>>> Thanks
>>>>>> Yongqiang
>>>>>> On Mon, Jan 24, 2011 at 12:30 PM, Edward Capriolo 
>>>>>> <edlinuxg...@gmail.com> wrote:
>>>>>>> I am trying to explore some use case that I believe are perfect for
>>>>>>> the columnarSerDe, tables with 100+ columns where only one or two are
>>>>>>> selected in a particular query.
>>>>>>> 
>>>>>>> CREATE TABLE (....)
>>>>>>> ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"
>>>>>>>   STORED AS RCFile ;
>>>>>>> 
>>>>>>> My issue is my data from our source table, with gzip sequence files,
>>>>>>> is much smaller then the ColumnarSerDe table and as a result any
>>>>>>> performance gains are lost.
>>>>>>> 
>>>>>>> Any ideas?
>>>>>>> 
>>>>>>> Thank you,
>>>>>>> Edward
>>>>>>> 
>>>>>> 
>>>>> 
>>>>> Thank you! That was a RTFM question.
>>>>> 
>>>>>  set hive.exec.dynamic.partition.mode=nonstrict;
>>>>> set hive.exec.compress.output=true;
>>>>> set 
>>>>> mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec;
>>>>> 
>>>>> I was unclear about  'STORED AS RCFile' since normally you would need
>>>>> to use ' STORED AS SEQUENCEFILE'
>>>>> 
>>>>> However 
>>>>> http://hive.apache.org/docs/r0.6.0/api/org/apache/hadoop/hive/ql/io/RCFile.html
>>>>> explains this well. RCFILE is a special type of sequence file.
>>>>> 
>>>>> I did get it working. Looks good compression for my table was smaller
>>>>> then using GZIP BLOCK Sequence file. Query time was slightly better in
>>>>> limited testing. Cool stuff.
>>>>> 
>>>>> Edward
>>>>> 
>>>> 
>>>> Do rcfiles support a blocksize for compression like other compressed
>>>> sequence files?
>>>> 
>>> 
>> 
>> Great. Do you have any suggestions or hints on how to tune this. Any
>> information on what the ceiling or the floor might be?
>> 
>> Edward
>> 

Reply via email to