You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by sumit ghosh <su...@yahoo.com> on 2013/04/18 00:44:04 UTC

Reading and Writing Sequencefile using Hadoop 2.0 Apis

 I am looking for an example which is using the new Hadoop 2.0 API to read and write Sequence Files. Effectively I need to know how to use these functions:
createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
The Old definition is not working for me:
SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass()); 
 
Similarly I need to know what will be the code for reading the Sequence file, as the follwoing is deprecated:
SequenceFile.Reader(fs, path, conf);  
Thanks,
Sumit

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Azuryy Yu <az...@gmail.com>.
you can use if even if it's depracated.

I can find in
the org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.java,

  @Override
  public void initialize(InputSplit split,
                         TaskAttemptContext context
                         ) throws IOException, InterruptedException {
    FileSplit fileSplit = (FileSplit) split;
    conf = context.getConfiguration();
    Path path = fileSplit.getPath();
    FileSystem fs = path.getFileSystem(conf);
    this.in = new SequenceFile.Reader(fs, path, conf);
    this.end = fileSplit.getStart() + fileSplit.getLength();

    if (fileSplit.getStart() > in.getPosition()) {
      in.sync(fileSplit.getStart());                  // sync to start
    }

    this.start = in.getPosition();
    more = start < end;
  }


On Thu, Apr 18, 2013 at 6:44 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Azuryy Yu <az...@gmail.com>.
you can use if even if it's depracated.

I can find in
the org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.java,

  @Override
  public void initialize(InputSplit split,
                         TaskAttemptContext context
                         ) throws IOException, InterruptedException {
    FileSplit fileSplit = (FileSplit) split;
    conf = context.getConfiguration();
    Path path = fileSplit.getPath();
    FileSystem fs = path.getFileSystem(conf);
    this.in = new SequenceFile.Reader(fs, path, conf);
    this.end = fileSplit.getStart() + fileSplit.getLength();

    if (fileSplit.getStart() > in.getPosition()) {
      in.sync(fileSplit.getStart());                  // sync to start
    }

    this.start = in.getPosition();
    more = start < end;
  }


On Thu, Apr 18, 2013 at 6:44 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by sumit ghosh <su...@yahoo.com>.
Hi,
 
Looks like it still points to the old API. The following worked for me - 
http://stackoverflow.com/questions/16070587/reading-and-writing-sequencefile-using-hadoop-2-0-apis
 
    String uri = args[0];
    Configuration conf = new Configuration();
    Path path = new Path( uri);

    IntWritable key = new IntWritable();
    Text value = new Text();

    CompressionCodec Codec = new GzipCodec();
    SequenceFile.Writer writer = null;
    Option optPath = SequenceFile.Writer.file(path);
    Option optKey = SequenceFile.Writer.keyClass(key.getClass());
    Option optVal = SequenceFile.Writer.valueClass(value.getClass());
    Option optCom = SequenceFile.Writer.compression(CompressionType.RECORD,  Codec);

        writer = SequenceFile.createWriter( conf, optPath, optKey, optVal, optCom);

 
Thanks,
Sumit


________________________________
From: Harsh J <ha...@cloudera.com>
To: "<us...@hadoop.apache.org>" <us...@hadoop.apache.org>; sumit ghosh <su...@yahoo.com> 
Sent: Wednesday, 17 April 2013 9:52 PM
Subject: Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Sumit, 

I believe we've answered this one before, so you may find http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.
On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote: 
 I am looking for an example which is using the new Hadoop 2.0 API to read and write Sequence Files. Effectively I need to know how to use these functions:
>createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>The Old definition is not working for me:
>SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass()); 
>
>Similarly I need to know what will be the code for reading the Sequence file, as the follwoing is deprecated:
>SequenceFile.Reader(fs, path, conf);  
>Thanks,
>Sumit-- Harsh J 

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by sumit ghosh <su...@yahoo.com>.
Hi,
 
Looks like it still points to the old API. The following worked for me - 
http://stackoverflow.com/questions/16070587/reading-and-writing-sequencefile-using-hadoop-2-0-apis
 
    String uri = args[0];
    Configuration conf = new Configuration();
    Path path = new Path( uri);

    IntWritable key = new IntWritable();
    Text value = new Text();

    CompressionCodec Codec = new GzipCodec();
    SequenceFile.Writer writer = null;
    Option optPath = SequenceFile.Writer.file(path);
    Option optKey = SequenceFile.Writer.keyClass(key.getClass());
    Option optVal = SequenceFile.Writer.valueClass(value.getClass());
    Option optCom = SequenceFile.Writer.compression(CompressionType.RECORD,  Codec);

        writer = SequenceFile.createWriter( conf, optPath, optKey, optVal, optCom);

 
Thanks,
Sumit


________________________________
From: Harsh J <ha...@cloudera.com>
To: "<us...@hadoop.apache.org>" <us...@hadoop.apache.org>; sumit ghosh <su...@yahoo.com> 
Sent: Wednesday, 17 April 2013 9:52 PM
Subject: Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Sumit, 

I believe we've answered this one before, so you may find http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.
On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote: 
 I am looking for an example which is using the new Hadoop 2.0 API to read and write Sequence Files. Effectively I need to know how to use these functions:
>createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>The Old definition is not working for me:
>SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass()); 
>
>Similarly I need to know what will be the code for reading the Sequence file, as the follwoing is deprecated:
>SequenceFile.Reader(fs, path, conf);  
>Thanks,
>Sumit-- Harsh J 

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by sumit ghosh <su...@yahoo.com>.
Hi,
 
Looks like it still points to the old API. The following worked for me - 
http://stackoverflow.com/questions/16070587/reading-and-writing-sequencefile-using-hadoop-2-0-apis
 
    String uri = args[0];
    Configuration conf = new Configuration();
    Path path = new Path( uri);

    IntWritable key = new IntWritable();
    Text value = new Text();

    CompressionCodec Codec = new GzipCodec();
    SequenceFile.Writer writer = null;
    Option optPath = SequenceFile.Writer.file(path);
    Option optKey = SequenceFile.Writer.keyClass(key.getClass());
    Option optVal = SequenceFile.Writer.valueClass(value.getClass());
    Option optCom = SequenceFile.Writer.compression(CompressionType.RECORD,  Codec);

        writer = SequenceFile.createWriter( conf, optPath, optKey, optVal, optCom);

 
Thanks,
Sumit


________________________________
From: Harsh J <ha...@cloudera.com>
To: "<us...@hadoop.apache.org>" <us...@hadoop.apache.org>; sumit ghosh <su...@yahoo.com> 
Sent: Wednesday, 17 April 2013 9:52 PM
Subject: Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Sumit, 

I believe we've answered this one before, so you may find http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.
On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote: 
 I am looking for an example which is using the new Hadoop 2.0 API to read and write Sequence Files. Effectively I need to know how to use these functions:
>createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>The Old definition is not working for me:
>SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass()); 
>
>Similarly I need to know what will be the code for reading the Sequence file, as the follwoing is deprecated:
>SequenceFile.Reader(fs, path, conf);  
>Thanks,
>Sumit-- Harsh J 

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by sumit ghosh <su...@yahoo.com>.
Hi,
 
Looks like it still points to the old API. The following worked for me - 
http://stackoverflow.com/questions/16070587/reading-and-writing-sequencefile-using-hadoop-2-0-apis
 
    String uri = args[0];
    Configuration conf = new Configuration();
    Path path = new Path( uri);

    IntWritable key = new IntWritable();
    Text value = new Text();

    CompressionCodec Codec = new GzipCodec();
    SequenceFile.Writer writer = null;
    Option optPath = SequenceFile.Writer.file(path);
    Option optKey = SequenceFile.Writer.keyClass(key.getClass());
    Option optVal = SequenceFile.Writer.valueClass(value.getClass());
    Option optCom = SequenceFile.Writer.compression(CompressionType.RECORD,  Codec);

        writer = SequenceFile.createWriter( conf, optPath, optKey, optVal, optCom);

 
Thanks,
Sumit


________________________________
From: Harsh J <ha...@cloudera.com>
To: "<us...@hadoop.apache.org>" <us...@hadoop.apache.org>; sumit ghosh <su...@yahoo.com> 
Sent: Wednesday, 17 April 2013 9:52 PM
Subject: Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Sumit, 

I believe we've answered this one before, so you may find http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.
On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote: 
 I am looking for an example which is using the new Hadoop 2.0 API to read and write Sequence Files. Effectively I need to know how to use these functions:
>createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>The Old definition is not working for me:
>SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass()); 
>
>Similarly I need to know what will be the code for reading the Sequence file, as the follwoing is deprecated:
>SequenceFile.Reader(fs, path, conf);  
>Thanks,
>Sumit-- Harsh J 

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Harsh J <ha...@cloudera.com>.
Sumit,

I believe we've answered this one before, so you may find
http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.


On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>



-- 
Harsh J

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Harsh J <ha...@cloudera.com>.
Sumit,

I believe we've answered this one before, so you may find
http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.


On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>



-- 
Harsh J

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Harsh J <ha...@cloudera.com>.
Sumit,

I believe we've answered this one before, so you may find
http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.


On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>



-- 
Harsh J

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Harsh J <ha...@cloudera.com>.
Sumit,

I believe we've answered this one before, so you may find
http://search-hadoop.com/m/xp2w02A8bqw1 helpful too.


On Thu, Apr 18, 2013 at 4:14 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>



-- 
Harsh J

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Azuryy Yu <az...@gmail.com>.
you can use if even if it's depracated.

I can find in
the org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.java,

  @Override
  public void initialize(InputSplit split,
                         TaskAttemptContext context
                         ) throws IOException, InterruptedException {
    FileSplit fileSplit = (FileSplit) split;
    conf = context.getConfiguration();
    Path path = fileSplit.getPath();
    FileSystem fs = path.getFileSystem(conf);
    this.in = new SequenceFile.Reader(fs, path, conf);
    this.end = fileSplit.getStart() + fileSplit.getLength();

    if (fileSplit.getStart() > in.getPosition()) {
      in.sync(fileSplit.getStart());                  // sync to start
    }

    this.start = in.getPosition();
    more = start < end;
  }


On Thu, Apr 18, 2013 at 6:44 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>

Re: Reading and Writing Sequencefile using Hadoop 2.0 Apis

Posted by Azuryy Yu <az...@gmail.com>.
you can use if even if it's depracated.

I can find in
the org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.java,

  @Override
  public void initialize(InputSplit split,
                         TaskAttemptContext context
                         ) throws IOException, InterruptedException {
    FileSplit fileSplit = (FileSplit) split;
    conf = context.getConfiguration();
    Path path = fileSplit.getPath();
    FileSystem fs = path.getFileSystem(conf);
    this.in = new SequenceFile.Reader(fs, path, conf);
    this.end = fileSplit.getStart() + fileSplit.getLength();

    if (fileSplit.getStart() > in.getPosition()) {
      in.sync(fileSplit.getStart());                  // sync to start
    }

    this.start = in.getPosition();
    more = start < end;
  }


On Thu, Apr 18, 2013 at 6:44 AM, sumit ghosh <su...@yahoo.com> wrote:

>    **
>   I am looking for an example which is using the new Hadoop 2.0 API to
> read and write Sequence Files. Effectively I need to know how to use these
> functions:
>
> createWriter(Configuration conf, org.apache.hadoop.io.SequenceFile.Writer.Option... opts)
>
> The Old definition is not working for me:
>
> SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
>
>
> Similarly I need to know what will be the code for reading the Sequence
> file, as the follwoing is deprecated:
>
> SequenceFile.Reader(fs, path, conf);
>
> Thanks,
> Sumit
>