public void testFormat() throws Exception {
    JobConf job = new JobConf(conf);
    FileSystem fs = FileSystem.getLocal(conf);
    Path dir = new Path(System.getProperty("test.build.data", ".") + "/mapred");
    Path file = new Path(dir, "test.seq");

    Reporter reporter = Reporter.NULL;

    int seed = new Random().nextInt();
    // LOG.info("seed = "+seed);
    Random random = new Random(seed);

    fs.delete(dir, true);

    FileInputFormat.setInputPaths(job, dir);

    // for a variety of lengths
    for (int length = 0; length < MAX_LENGTH; length += random.nextInt(MAX_LENGTH / 10) + 1) {

      // LOG.info("creating; entries = " + length);

      // create a file with length entries
      SequenceFile.Writer writer =
          SequenceFile.createWriter(fs, conf, file, IntWritable.class, BytesWritable.class);
      try {
        for (int i = 0; i < length; i++) {
          IntWritable key = new IntWritable(i);
          byte[] data = new byte[random.nextInt(10)];
          random.nextBytes(data);
          BytesWritable value = new BytesWritable(data);
          writer.append(key, value);
        }
      } finally {
        writer.close();
      }

      // try splitting the file in a variety of sizes
      InputFormat<IntWritable, BytesWritable> format =
          new SequenceFileInputFormat<IntWritable, BytesWritable>();
      IntWritable key = new IntWritable();
      BytesWritable value = new BytesWritable();
      for (int i = 0; i < 3; i++) {
        int numSplits = random.nextInt(MAX_LENGTH / (SequenceFile.SYNC_INTERVAL / 20)) + 1;
        // LOG.info("splitting: requesting = " + numSplits);
        InputSplit[] splits = format.getSplits(job, numSplits);
        // LOG.info("splitting: got =        " + splits.length);

        // check each split
        BitSet bits = new BitSet(length);
        for (int j = 0; j < splits.length; j++) {
          RecordReader<IntWritable, BytesWritable> reader =
              format.getRecordReader(splits[j], job, reporter);
          try {
            int count = 0;
            while (reader.next(key, value)) {
              // if (bits.get(key.get())) {
              // LOG.info("splits["+j+"]="+splits[j]+" : " +
              // key.get());
              // LOG.info("@"+reader.getPos());
              // }
              assertFalse("Key in multiple partitions.", bits.get(key.get()));
              bits.set(key.get());
              count++;
            }
            // LOG.info("splits["+j+"]="+splits[j]+" count=" +
            // count);
          } finally {
            reader.close();
          }
        }
        assertEquals("Some keys in no partition.", length, bits.cardinality());
      }
    }
  }
  public void testBinary() throws IOException, InterruptedException {
    Configuration conf = new Configuration();
    Job job = new Job(conf);

    Path outdir = new Path(System.getProperty("test.build.data", "/tmp"), "outseq");
    Random r = new Random();
    long seed = r.nextLong();
    r.setSeed(seed);

    FileOutputFormat.setOutputPath(job, outdir);

    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputKeyClass(job, IntWritable.class);
    SequenceFileAsBinaryOutputFormat.setSequenceFileOutputValueClass(job, DoubleWritable.class);

    SequenceFileAsBinaryOutputFormat.setCompressOutput(job, true);
    SequenceFileAsBinaryOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);

    BytesWritable bkey = new BytesWritable();
    BytesWritable bval = new BytesWritable();

    TaskAttemptContext context =
        MapReduceTestUtil.createDummyMapTaskAttemptContext(job.getConfiguration());
    OutputFormat<BytesWritable, BytesWritable> outputFormat =
        new SequenceFileAsBinaryOutputFormat();
    OutputCommitter committer = outputFormat.getOutputCommitter(context);
    committer.setupJob(job);
    RecordWriter<BytesWritable, BytesWritable> writer = outputFormat.getRecordWriter(context);

    IntWritable iwritable = new IntWritable();
    DoubleWritable dwritable = new DoubleWritable();
    DataOutputBuffer outbuf = new DataOutputBuffer();
    LOG.info("Creating data by SequenceFileAsBinaryOutputFormat");
    try {
      for (int i = 0; i < RECORDS; ++i) {
        iwritable = new IntWritable(r.nextInt());
        iwritable.write(outbuf);
        bkey.set(outbuf.getData(), 0, outbuf.getLength());
        outbuf.reset();
        dwritable = new DoubleWritable(r.nextDouble());
        dwritable.write(outbuf);
        bval.set(outbuf.getData(), 0, outbuf.getLength());
        outbuf.reset();
        writer.write(bkey, bval);
      }
    } finally {
      writer.close(context);
    }
    committer.commitTask(context);
    committer.commitJob(job);

    InputFormat<IntWritable, DoubleWritable> iformat =
        new SequenceFileInputFormat<IntWritable, DoubleWritable>();
    int count = 0;
    r.setSeed(seed);
    SequenceFileInputFormat.setInputPaths(job, outdir);
    LOG.info("Reading data by SequenceFileInputFormat");
    for (InputSplit split : iformat.getSplits(job)) {
      RecordReader<IntWritable, DoubleWritable> reader = iformat.createRecordReader(split, context);
      MapContext<IntWritable, DoubleWritable, BytesWritable, BytesWritable> mcontext =
          new MapContextImpl<IntWritable, DoubleWritable, BytesWritable, BytesWritable>(
              job.getConfiguration(),
              context.getTaskAttemptID(),
              reader,
              null,
              null,
              MapReduceTestUtil.createDummyReporter(),
              split);
      reader.initialize(split, mcontext);
      try {
        int sourceInt;
        double sourceDouble;
        while (reader.nextKeyValue()) {
          sourceInt = r.nextInt();
          sourceDouble = r.nextDouble();
          iwritable = reader.getCurrentKey();
          dwritable = reader.getCurrentValue();
          assertEquals(
              "Keys don't match: " + "*" + iwritable.get() + ":" + sourceInt + "*",
              sourceInt,
              iwritable.get());
          assertTrue(
              "Vals don't match: " + "*" + dwritable.get() + ":" + sourceDouble + "*",
              Double.compare(dwritable.get(), sourceDouble) == 0);
          ++count;
        }
      } finally {
        reader.close();
      }
    }
    assertEquals("Some records not found", RECORDS, count);
  }