Sorry for the delay in getting back on this one but we have been investigating other angles.
We are using Tableau with Drill, so we have to create views in Drill to access Parquet files even though the Parquet files hold the schema. When we create the views we have to CAST the columns to the correct data type (BIGINT, FLOAT, TIMESTAMP etc), so the updates you suggested for parquet-avro to add support for TIMESTAMP are not as urgent as we originally thought since we have to cast everything anyway. Cheers — Chris > On 15 Oct 2015, at 18:46, Ryan Blue <[email protected]> wrote: > > I'm attaching the patch that I put together for decimal. > > This includes: > * Decimal schema translation from Avro to Parquet > - Need to add date, time, timestamp > - Need to add Parquet to Avro support > * Read-side support for any Avro logical type > * Special write-side support for decimal > - This was added to fixed and bytes > - We should try to make this more generic > > This was based on a branch of Avro with logical type support, so where Avro > classes are missing we would need to copy them into Parquet. The only other > fix is that Schema doesn't have a getLogicalType accessor until 1.8.0, so we > will need to use this instead: > > LogicalTypes.fromSchemaIgnoreInvalid(schema) > > I think it is pretty straight-forward, but if you have questions let me know. > > rb > > On 10/15/2015 10:24 AM, Chris Mathews wrote: >> Hi Ryan >> >> Thanks for this - it sounds just what we need. >> >> How do we go about doing a trial of the local copies with our code ? >> It would be good to check this all out now if 1.8.0 is delayed for a while ? >> >> contact me by https://drillers.slack.com/messages/dev/team/cmathews/ to >> discuss. >> >> Cheers — Chris >> >>> On 15 Oct 2015, at 17:59, Julien Le Dem <[email protected]> wrote: >>> >>> thanks Ryan! >>> (cc parquet dev list as well) >>> >>> On Thu, Oct 15, 2015 at 9:46 AM, Ryan Blue <[email protected]> wrote: >>> >>>> Hi Chris, >>>> >>>> Avro does have support for dates, but it hasn't been released yet because >>>> 1.8.0 was blocked on license issues (AVRO-1722). I have a branch with >>>> preliminary parquet-avro support for Decimal (which uses the same Avro >>>> construct) if you would like to take a look at it. >>>> >>>> What we can do in the mean time, before Avro's logical type support is >>>> released, is to copy the logical types and conversions into parquet-avro >>>> and use local copies until we can use the correct upstream ones. If you're >>>> interested, ping me and we can get it working. >>>> >>>> rb >>>> >>>> On 10/15/2015 09:24 AM, Julien Le Dem wrote: >>>> >>>>> Hi Chris, >>>>> You could probably contribute some sort of type annotation to >>>>> parquet-avro so that it produces the data type in the Parquet schema. >>>>> This class generates a Parquet schema from the Avro schema: >>>>> >>>>> https://github.com/apache/parquet-mr/blob/master/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java >>>>> It looks like Avro has some annotation support but I have never used it. >>>>> http://avro.apache.org/docs/1.4.1/idl.html#minutiae_annotations >>>>> (CC'ing Ryan, who knows the avro integration much better) >>>>> >>>>> On Thu, Oct 15, 2015 at 8:45 AM, Chris Mathews <[email protected] >>>>> <mailto:[email protected]>> wrote: >>>>> >>>>> Thank you Jacques - yes this is exactly the issue I am having. >>>>> >>>>> We are currently using Avro to define schemas for our Parquet files, >>>>> and as you correctly point out there is no way of defining date >>>>> types in Avro. Due to the volumes of data we are dealing with, >>>>> using CTAS is not an option for us as we need to create the Parquet >>>>> files on the fly rather than create intermediate files for use as >>>>> input to the CTAS. >>>>> >>>>> Since Avro is the restriction here we need to come up with some way >>>>> of defining our Parquet schemas so the date types can be defined >>>>> natively; and as you say there will be no need for any casting again. >>>>> >>>>> What we really need is some way of creating a schema for use in a >>>>> Java app, similar to the way Avro works using JSON, for ease of >>>>> implementation but without the restrictions. >>>>> >>>>> We are still investigating this approach, but for now we are casting ! >>>>> >>>>> Cheers — Chris >>>>> >>>>> >>>>>> On 15 Oct 2015, at 16:19, Jacques Nadeau <[email protected] >>>>> <mailto:[email protected]>> wrote: >>>>>> >>>>>> A little clarification here: >>>>>> >>>>>> Parquet has native support for date types. Drill does too. >>>>> However, since >>>>>> Avro does not, there is no way that I know of to write a Parquet >>>>> file via >>>>>> the Avro adapter that will not require a cast. If you did a CTAS >>>>> in Drill >>>>>> and cast the data types correctly in the CTAS, then Drill will >>>>> output >>>>>> Parquet files that never need casting again. >>>>>> >>>>>> -- >>>>>> Jacques Nadeau >>>>>> CTO and Co-Founder, Dremio >>>>>> >>>>>> On Thu, Oct 15, 2015 at 7:31 AM, Stefán Baxter >>>>> <[email protected] <mailto:[email protected]>> >>>>>> wrote: >>>>>> >>>>>>> Hi Chris, >>>>>>> >>>>>>> I understand now, thank you. >>>>>>> >>>>>>> What threw me off was that, in our standard use-case, we are not >>>>> using cast >>>>>>> for our TIMESTAMP_MILLIS fields and I thought we were getting >>>>> them directly >>>>>>> formatted from Parquet but then I overlooked our UDF that is >>>>> handling the >>>>>>> casting... sorry :). >>>>>>> >>>>>>> Thank you for taking the time! >>>>>>> >>>>>>> Regards, >>>>>>> -Stefan >>>>>>> >>>>>>> >>>>>>> On Thu, Oct 15, 2015 at 2:21 PM, Chris Mathews <[email protected] >>>>> <mailto:[email protected]>> wrote: >>>>>>> >>>>>>>> Hi Stefan >>>>>>>> >>>>>>>> I am not sure I fully understand your question 'why you don't >>>>> seem to be >>>>>>>> storing your dates in Parquet Date files.' >>>>>>>> >>>>>>>> As far as I am aware all date types in Parquet (ie: DATE, >>>>> TIME_MILLIS, >>>>>>>> TIMESTAMP_MILLIS) are all stored as either in int32 or int64 >>>>> annotated >>>>>>>> types. The only other option is to store them as strings (or >>>>> VARCHAR) and >>>>>>>> interpret them as required when selecting from the Parquet files. >>>>>>>> >>>>>>>> Please let me know if I have understood your question correctly >>>>> or not. >>>>>>>> >>>>>>>> What I have not acheived yet is to use Avro schema definitions >>>>> (via JSON) >>>>>>>> to define a TIMESTAMP type, which is why we have gone the route >>>>> of >>>>>>> defining >>>>>>>> a VIEW for each Parquet file. By doing this we reduce the amount >>>>> of >>>>>>> casting >>>>>>>> we have to do when building the query since the VIEW >>>>> effectively does all >>>>>>>> the casting for us behind the scenes. >>>>>>>> >>>>>>>> We are currently looking at possibility of defining Parquet >>>>> schemas >>>>>>>> directly (using java) without going the Avro route; in other >>>>> words >>>>>>> produce >>>>>>>> a parser from JSON to Parquet, similar to the Avro parser but >>>>> supporting >>>>>>>> some other logical types. >>>>>>>> >>>>>>>> Some background to our drill trials: >>>>>>>> >>>>>>>> We are generating billions of columns from machine generated >>>>> data every >>>>>>>> day. There a quite a number of different types of machine >>>>> generating this >>>>>>>> data and the format of the data varies between machines. Some >>>>> produce >>>>>>>> string format dates/timestamps and others numeric (unix epoch >>>>> style), and >>>>>>>> we have to normalise this data to a common format; for dates >>>>> this format >>>>>>> is >>>>>>>> the Parquet TIMESTAMP_MILLIS type because we need to use >>>>> millisecond >>>>>>>> granularity when available. >>>>>>>> >>>>>>>> quote from Parquet docs: >>>>>>>> >>>>>>>> "TIMESTAMP_MILLIS Logical date and time. Annotates an int64 >>>>> that stores >>>>>>>> the number of milliseconds from the Unix epoch, 00:00:00.000 on >>>>> 1 January >>>>>>>> 1970, UTC." >>>>>>>> >>>>>>>> This type corresponds nicely to the SQL type TIMESTAMP (which >>>>> is why we >>>>>>>> cast). >>>>>>>> >>>>>>>> Again, hope this helps. >>>>>>>> >>>>>>>> Cheers -- Chris >>>>>>>> >>>>>>>>> On 15 Oct 2015, at 14:46, Stefán Baxter >>>>> <[email protected] <mailto:[email protected]>> >>>>>>>> wrote: >>>>>>>>> >>>>>>>>> Thank you Chris, this clarifies a whole lot :). >>>>>>>>> >>>>>>>>> I wanted to try to avoid the cast in the CTAS on the way from >>>>> Avro to >>>>>>>>> Parquet (not possible) and then avoid casting as much as >>>>> possible when >>>>>>>>> selecting from the Parquet files. >>>>>>>>> >>>>>>>>> What is still unclear to me is why you don't seem to be >>>>> storing your >>>>>>>> dates >>>>>>>>> in Parquet Date files. >>>>>>>>> >>>>>>>>> Can you please elaborate a bit on the pros/cons? >>>>>>>>> >>>>>>>>> Regards, >>>>>>>>> -Stefan >>>>>>>>> >>>>>>>>> On Thu, Oct 15, 2015 at 10:59 AM, Chris Mathews >>>>> <[email protected] <mailto:[email protected]>> >>>>> >>>>>>> wrote: >>>>>>>>> >>>>>>>>>> Hello Stefan >>>>>>>>>> >>>>>>>>>> We use Avro to define our schemas for Parquet files, and we >>>>> find that >>>>>>>>>> using long for dates and converting the dates to long using >>>>>>> milliseconds >>>>>>>>>> works. We then CAST the long to a TIMESTAMP on the way out >>>>> during the >>>>>>>>>> SELECT statement (or by using a VIEW). >>>>>>>>>> >>>>>>>>>> example java snippet: >>>>>>>>>> >>>>>>>>>> // various date and time formats >>>>>>>>>> public static final String FORMAT_Z_TIMESTAMP = "yyyy-MM-dd >>>>>>>> HH:mm:ss.SSS"; >>>>>>>>>> public static final String FORMAT_DATETIME = "yyyy-MM-dd >>>>> HH:mm:ss"; >>>>>>>>>> public static final String FORMAT_DATE = "yyyy-MM-dd"; >>>>>>>>>> >>>>>>>>>> … >>>>>>>>>> >>>>>>>>>> // parser for each format >>>>>>>>>> public final SimpleDateFormat sdf_z_timestamp = new >>>>> SimpleDateFormat( >>>>>>>>>> FORMAT_Z_TIMESTAMP ); >>>>>>>>>> public final SimpleDateFormat sdf_datetime = new >>>>> SimpleDateFormat( >>>>>>>>>> FORMAT_DATETIME ); >>>>>>>>>> public final SimpleDateFormat sdf_date = new >>>>> SimpleDateFormat( >>>>>>>>>> FORMAT_DATE ); >>>>>>>>>> >>>>>>>>>> … >>>>>>>>>> >>>>>>>>>> // choose parser based on column name / string format >>>>>>>>>> public SimpleDateFormat sdf = >>>>> (NAME_Z_TIMESTAMP.equals(name())) ? >>>>>>>>>> sdf_z_timestamp >>>>>>>>>> : (NAME_DATETIME.equals(name())) ? >>>>>>>>>> sdf_datetime >>>>>>>>>> : (NAME_DATE.equals(name())) ? >>>>>>>> sdf_date >>>>>>>>>> : null; >>>>>>>>>> … >>>>>>>>>> >>>>>>>>>> Date date = sdf.parse(str); >>>>>>>>>> long millis = date.getTime(); >>>>>>>>>> Object value = new java.lang.Long(millis); >>>>>>>>>> >>>>>>>>>> We then use something like >>>>>>>>>> >>>>>>>>>> AvroParquetWriter<GenericRecord> writer = new >>>>>>>>>> AvroParquetWriter<>(hdfs_path, schema); >>>>>>>>>> GenericRecord data = new GenericData.Record(schema); >>>>>>>>>> data.put( name(), value); >>>>>>>>>> writer.write(data); >>>>>>>>>> >>>>>>>>>> to write the records out directly to a Parquet file. >>>>>>>>>> >>>>>>>>>> example schema: >>>>>>>>>> >>>>>>>>>> { >>>>>>>>>> "type": "record", >>>>>>>>>> "name": "timestamp_test", >>>>>>>>>> "doc": "Avro -> Parquet long to timestamp test", >>>>>>>>>> "fields": >>>>>>>>>> [ >>>>>>>>>> { "name": "z_timestamp", "type": "long" } >>>>>>>>>> ,{ "name": "datetime", "type": "long" } >>>>>>>>>> ,{ "name": "date", "type": "long" } >>>>>>>>>> ,{ "name": "granularity", "type": "long" } >>>>>>>>>> ] >>>>>>>>>> } >>>>>>>>>> >>>>>>>>>> Then to get the data back we either define a VIEW, or cast >>>>> during the >>>>>>>>>> SELECT statement. >>>>>>>>>> >>>>>>>>>> example view: >>>>>>>>>> >>>>>>>>>> use hdfs.cjm; >>>>>>>>>> >>>>>>>>>> create or replace view TIMESTAMP_TEST_VIEW as >>>>>>>>>> SELECT >>>>>>>>>> CAST(`z_timestamp` AS TIMESTAMP) AS `z_timestamp` >>>>>>>>>> ,CAST(`datetime` AS TIMESTAMP) AS `datetime` >>>>>>>>>> ,CAST(`date` AS DATE) AS `date` >>>>>>>>>> ,CAST(`granularity` AS BIGINT) AS `granularity` >>>>>>>>>> >>>>>>>>>> FROM hdfs.cjm.TIMESTAMP_TEST; >>>>>>>>>> >>>>>>>>>> Then execute the following to get results: >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> 0: jdbc:drill:> select z_timestamp, `datetime`, `date`, >>>>> granularity >>>>>>> from >>>>>>>>>> TIMESTAMP_TEST limit 1; >>>>>>>>>> >>>>> +----------------+----------------+----------------+--------------+ >>>>>>>>>> | z_timestamp | datetime | date | >>>>> granularity | >>>>>>>>>> >>>>> +----------------+----------------+----------------+--------------+ >>>>>>>>>> | 1429592511991 | 1429520400000 | 1421625600000 | 3600 >>>>> | >>>>>>>>>> >>>>> +----------------+----------------+----------------+--------------+ >>>>>>>>>> 1 row selected (2.593 seconds) >>>>>>>>>> >>>>>>>>>> 0: jdbc:drill:> select z_timestamp, `datetime`, `date`, >>>>> granularity >>>>>>> from >>>>>>>>>> TIMESTAMP_TEST_VIEW limit 1; >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> | z_timestamp | datetime | >>>>> date | >>>>>>>>>> granularity | >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> | 2015-04-22 05:16:22.173 | 2015-04-21 12:00:00.0 | >>>>> 2015-01-20 | >>>>>>> 3600 >>>>>>>>>> | >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> 1 row selected (3.282 seconds) >>>>>>>>>> >>>>>>>>>> 0: jdbc:drill:> SELECT >>>>>>>>>> . . . . . . . > CAST(`z_timestamp` AS TIMESTAMP) AS >>>>> `z_timestamp` >>>>>>>>>> . . . . . . . > ,CAST(`datetime` AS TIMESTAMP) AS `datetime` >>>>>>>>>> . . . . . . . > ,CAST(`date` AS DATE) AS `date` >>>>>>>>>> . . . . . . . > ,CAST(`granularity` AS BIGINT) AS >>>>> `granularity` >>>>>>>>>> . . . . . . . > from TIMESTAMP_TEST limit 1; >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> | z_timestamp | datetime | >>>>> date | >>>>>>>>>> granularity | >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> | 2015-04-22 05:16:22.173 | 2015-04-21 09:00:00.0 | >>>>> 2015-01-20 | >>>>>>> 3600 >>>>>>>>>> | >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> +--------------------------+------------------------+-------------+--------------+ >>>>>>>>>> 1 row selected (3.071 seconds) >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> Hope this helps. >>>>>>>>>> >>>>>>>>>> Cheers — Chris >>>>>>>>>> >>>>>>>>>>> On 14 Oct 2015, at 16:07, Stefán Baxter >>>>> <[email protected] <mailto:[email protected]>> >>>>>>>>>> wrote: >>>>>>>>>>> >>>>>>>>>>> Hi, >>>>>>>>>>> >>>>>>>>>>> What is the best practice when working with dates in a >>>>> Avro/Parquet >>>>>>>>>>> scenario? >>>>>>>>>>> >>>>>>>>>>> Avro does not support dates directly (only longs) and I'm >>>>> wondering >>>>>>> how >>>>>>>>>> the >>>>>>>>>>> get persisted in Parquet. >>>>>>>>>>> >>>>>>>>>>> Perhaps Parquet does not distinguish between long and >>>>> date-long in >>>>>>> any >>>>>>>>>>> significant way. >>>>>>>>>>> >>>>>>>>>>> Regards, >>>>>>>>>>> -Stefan >>>>>>>>>> >>>>>>>>>> >>>>>>>> >>>>>>>> >>>>>>> >>>>> >>>>> >>>>> >>>>> >>>>> -- >>>>> Julien >>>>> >>>> >>>> >>>> -- >>>> Ryan Blue >>>> Software Engineer >>>> Cloudera, Inc. >>>> >>> >>> >>> >>> -- >>> Julien >> > > > -- > Ryan Blue > Software Engineer > Cloudera, Inc. > <0001-Add-simple-decimal-support-to-parquet-avro.patch>
