/** {@inheritDoc} */ @Override public Object convert(AvroWrapper<?> input) { return input.datum(); }
/** {@inheritDoc} */ @Override public T deserialize(T avroWrapperToReuse) throws IOException { // Create a new Avro wrapper if there isn't one to reuse. if (null == avroWrapperToReuse) { avroWrapperToReuse = createAvroWrapper(); } // Deserialize the Avro datum from the input stream. avroWrapperToReuse.datum(mAvroDatumReader.read(avroWrapperToReuse.datum(), mAvroDecoder)); return avroWrapperToReuse; }
/** {@inheritDoc} */ @Override public void serialize(AvroWrapper<T> avroWrapper) throws IOException { mAvroDatumWriter.write(avroWrapper.datum(), mAvroEncoder); // This would be a lot faster if the Serializer interface had a flush() method and the // Hadoop framework called it when needed. For now, we'll have to flush on every record. mAvroEncoder.flush(); }
public void write(AvroWrapper<T> wrapper, NullWritable ignore) throws IOException { writer.append(wrapper.datum()); } public void close(Reporter reporter) throws IOException {
public boolean next(AvroWrapper<Utf8> key, NullWritable value) throws IOException { boolean success = lineRecordReader.next(currentKeyHolder, currentValueHolder); if (success) { key.datum(new Utf8(currentValueHolder.getBytes()) .setLength(currentValueHolder.getLength())); } else { key.datum(null); } return success; }
public boolean next(AvroWrapper<T> wrapper, NullWritable ignore) throws IOException { if (!reader.hasNext()) return false; wrapper.datum(reader.next()); row++; return true; }
@Override public void map(AvroWrapper<IN> wrapper, NullWritable value, OutputCollector<KO,VO> collector, Reporter reporter) throws IOException { if (this.out == null) this.out = new MapCollector<>(collector, isMapOnly); mapper.map(wrapper.datum(), out, reporter); }
public void write(AvroWrapper<T> wrapper, NullWritable ignore) throws IOException { writer.write(wrapper.datum()); if (writer.sizeEstimate() >= blockSize) // block full flush(); } public void close(Reporter reporter) throws IOException {
@Override public void map(AvroWrapper<IN> wrapper, NullWritable value, OutputCollector<KO, VO> collector, Reporter reporter) throws IOException { if (mapper == null) { TaggedInputSplit is = (TaggedInputSplit) reporter.getInputSplit(); Class<? extends AvroMapper> mapperClass = is.getMapperClass(); mapper = (AvroMapper<IN,OUT>) ReflectionUtils.newInstance(mapperClass, conf); } if (out == null) out = new MapCollector<OUT,K,V,KO,VO>(collector, isMapOnly); mapper.map(wrapper.datum(), out, reporter); } }
public void collect(OUT datum) throws IOException { if (isMapOnly) { wrapper.datum(datum); collector.collect((KO)wrapper, (VO)NullWritable.get()); } else { // split a pair Pair<K,V> pair = (Pair<K,V>)datum; keyWrapper.datum(pair.key()); valueWrapper.datum(pair.value()); collector.collect((KO)keyWrapper, (VO)valueWrapper); } } }
public void testProjectionNoreducer(String inputDirectory) throws Exception { JobConf job = new JobConf(); long onel = 1; Schema readerSchema = Schema.create(Schema.Type.STRING); AvroJob.setInputSchema(job, readerSchema); Path inputPath = new Path(inputDirectory + "/myavro2-m-00000.avro"); FileStatus fileStatus = FileSystem.get(job).getFileStatus(inputPath); FileSplit fileSplit = new FileSplit(inputPath, 0, fileStatus.getLen(), job); AvroRecordReader<Utf8> recordReader = new AvroRecordReader<>(job, fileSplit); AvroWrapper<Utf8> inputPair = new AvroWrapper<>(null); NullWritable ignore = NullWritable.get(); while (recordReader.next(inputPair, ignore)) { long testl = Long.parseLong(inputPair.datum().toString().split(":")[2].replace("}", "").trim()); Assert.assertEquals(onel, testl); } } }
@SuppressWarnings("deprecation") // Test for a different schema output public void testProjection1(String inputDirectory) throws Exception { JobConf job = new JobConf(); Schema readerSchema = Schema.create(Schema.Type.STRING); AvroJob.setInputSchema(job, readerSchema); Path inputPath = new Path(inputDirectory + "/myavro1-r-00000.avro"); FileStatus fileStatus = FileSystem.get(job).getFileStatus(inputPath); FileSplit fileSplit = new FileSplit(inputPath, 0, fileStatus.getLen(), job); AvroWrapper<Utf8> inputPair = new AvroWrapper<>(null); NullWritable ignore = NullWritable.get(); AvroRecordReader<Utf8> recordReader = new AvroRecordReader<>(job, fileSplit); long sumOfCounts = 0; long numOfCounts = 0; while (recordReader.next(inputPair, ignore)) { sumOfCounts += Long.parseLong(inputPair.datum().toString().split(":")[2].replace("}", "").trim()); numOfCounts++; } Assert.assertEquals(numOfCounts, WordCountUtil.COUNTS.size()); long actualSumOfCounts = 0; for (Long count : WordCountUtil.COUNTS.values()) { actualSumOfCounts += count; } Assert.assertEquals(sumOfCounts, actualSumOfCounts); }
@SuppressWarnings("deprecation") // Test for a different schema output public void testProjectionNewMethodsTwo(String inputDirectory) throws Exception { JobConf job = new JobConf(); Schema readerSchema = Schema.create(Schema.Type.STRING); AvroJob.setInputSchema(job, readerSchema); Path inputPath = new Path(inputDirectory + "/testavrofile1-r-00000.avro"); FileStatus fileStatus = FileSystem.get(job).getFileStatus(inputPath); FileSplit fileSplit = new FileSplit(inputPath, 0, fileStatus.getLen(), job); AvroWrapper<Utf8> inputPair = new AvroWrapper<>(null); NullWritable ignore = NullWritable.get(); AvroRecordReader<Utf8> recordReader = new AvroRecordReader<>(job, fileSplit); long sumOfCounts = 0; long numOfCounts = 0; while (recordReader.next(inputPair, ignore)) { sumOfCounts += Long.parseLong(inputPair.datum().toString().split(":")[2].replace("}", "").trim()); numOfCounts++; } Assert.assertEquals(numOfCounts, WordCountUtil.COUNTS.size()); long actualSumOfCounts = 0; for (Long count : WordCountUtil.COUNTS.values()) { actualSumOfCounts += count; } Assert.assertEquals(sumOfCounts, actualSumOfCounts); }