Hi Juho, I think you should be able to use the Thrift serialization stuff that I've been working on in https://issues.apache.org/jira/browse/HADOOP-3787 - at least as a basis. Since you are not using sequence files, you will need to write an InputFormat (probably one that extends FileInputFormat) and an associated RecordReader that knows how to break the input into logical records. See SequenceFileInputFormat for the kind of thing. Also, since you store the Thrift type's name in the data, you can use a variant of ThriftDeserializer that first reads the type name and instantiates an instance of the type before reading its fields from the stream.
Hope this helps. Tom On Wed, Sep 3, 2008 at 8:32 AM, Juho Mäkinen <[EMAIL PROTECTED]> wrote: > Thanks Jeff. I believe that you mean the serde module inside hadoop > (hadoop-core-trunk\src\contrib\hive\serde)? > I'm currently looking into it, but it seems to lack a lot of useful > documentation so it'll take me some time to figure it out (all > additional info is appreciated). > > I've already put some effort into this and designed a partial > sollution for my log analysis which so far seems ok to me. As I don't > know the details of serde yet, I'm not sure if this is the way I > should go, or should I change my implementation and plans so that I > could use serve (if it makes my job easier). I'm not yet interested in > HIVE, but I'd like to keep the option open in the future, so that I > could easily run hive on my datas (so that I would not need to > transform my datas to hive if I choose to use it in the future). > > Currently I've come up with the following design: > 1) Each log event type has it's own thrift structure. The structure is > compiled into php code. The log entry creators creates and populates > the structure php object with data and sends it to be stored > 2) Log sender object receiveres this object ($tbase) and serializes it > using TBinaryTransport, adds the structure name to the beginning and > sends the byte array to loc receiver using UDP. The following code > does this: > > $this->transport = new TResetableMemoryBuffer(); // a TMemoryBuffer > with a reset() method > $this->protocol = new TBinaryProtocol($this->transport); > $this->transport->open(); > > $this->transport->reset(); // Reset the memory buffer array > $this->protocol->writeByte(1); // version 1: we have the TBase name in string > $this->protocol->writeString($tbase->getName()); // Name of the structure > $tbase->write($this->protocol); // Serialize our thrift structure to > the memory buffer > > $this->sendBytes($this->transport->getBuffer()); > > 3) Log receiver reads the structure name and stores the byte array > (without the version byte and structure name) into HDFS file > "/events/<insert structure name here>/<week > number>/<timestamp>.datafile" > > My plan is that I could read the stored entries using MapReduce, > deserialize them into java objects (the map-reducer would need to have > the thrift compiled structures available) and use the structures > directly in Map operations. (How) can serde help me with this part? > Should I modify my plans so that I could use HIVE directly in the > future? How Hive stores the thrift serialized log data into HDFS? > > - Juho Mäkinen > > On Wed, Sep 3, 2008 at 7:37 AM, Jeff Hammerbacher > <[EMAIL PROTECTED]> wrote: >> Hey Juho, >> >> You should check out Hive >> (https://issues.apache.org/jira/browse/HADOOP-3601), which was just >> committed to the Hadoop trunk today. It's what we use at Facebook to >> query our collection of Thrift-serialized logfiles. Inside of the Hive >> code, you'll find a pure-Java (using JavaCC) parser for >> Thrift-serialized data structures. >> >> Regards, >> Jeff >> >> On Tue, Sep 2, 2008 at 6:57 AM, Stuart Sierra <[EMAIL PROTECTED]> wrote: >>> On Tue, Sep 2, 2008 at 3:53 AM, Juho Mäkinen <[EMAIL PROTECTED]> wrote: >>>> What's the current status of Thrift with Hadoop? Is there any >>>> documentation online or even some code in the SVN which I could look >>>> into? >>> >>> I think you have two choices: 1) wrap your Thrift code in a class that >>> implements Writable, or 2) use Thrift to serialize your data to byte >>> arrays and store them as BytesWritable. >>> -Stuart >>> >> >
