@Override public void configure(Configuration parameters) { // pass parameters to FileInputFormat super.configure(parameters); // read own parameters this.recordLength = parameters.getInteger(RECORDLENGTH_PARAMETER_KEY, 0); if (recordLength < 1) { throw new IllegalArgumentException("The record length parameter must be set and larger than 0."); } }
@Override public void open(FileInputSplit split) throws IOException { // open input split using FileInputFormat super.open(split); // adjust the stream positions for boundary splits int recordOffset = (int) (this.splitStart % this.recordLength); if(recordOffset != 0) { // move start to next boundary super.stream.seek(this.splitStart + recordOffset); } this.streamPos = this.splitStart + recordOffset; this.streamEnd = this.splitStart + this.splitLength; this.streamEnd += this.streamEnd % this.recordLength; // adjust readBufferSize this.readBufferSize += this.recordLength - (this.readBufferSize % this.recordLength); if (this.readBuffer == null || this.readBuffer.length != this.readBufferSize) { this.readBuffer = new byte[this.readBufferSize]; } this.readBufferLimit = 0; this.readBufferPos = 0; this.exhausted = false; fillReadBuffer(); }
/** * {@inheritDoc} * @throws IOException */ @Override public FileBaseStatistics getStatistics(BaseStatistics cachedStats) throws IOException { final FileBaseStatistics stats = super.getStatistics(cachedStats); return stats == null ? null : new FileBaseStatistics(stats.getLastModificationTime(), stats.getTotalInputSize(), this.recordLength); }
/** * Set minNumSplits to number of files. */ @Override public FileInputSplit[] createInputSplits(int minNumSplits) throws IOException { int numAvroFiles = 0; final Path path = this.filePath; // get all the files that are involved in the splits final FileSystem fs = path.getFileSystem(); final FileStatus pathFile = fs.getFileStatus(path); if (!acceptFile(pathFile)) { throw new IOException("The given file does not pass the file-filter"); } if (pathFile.isDir()) { // input is directory. list all contained files final FileStatus[] dir = fs.listStatus(path); for (int i = 0; i < dir.length; i++) { if (!dir[i].isDir() && acceptFile(dir[i])) { numAvroFiles++; } } } else { numAvroFiles = 1; } return super.createInputSplits(numAvroFiles); }
@Override public void open(FileInputSplit split) throws IOException { super.open(split); DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(); SeekableInput in = new FSDataInputStreamWrapper(stream, (int) split.getLength()); LOG.info("Opening split " + split); dataFileReader = DataFileReader.openReader(in, datumReader); dataFileReader.sync(split.getStart()); }
@Override public void open(FileInputSplit split) throws IOException { super.open(split); this.wrapper = InstantiationUtil.instantiate(avroWrapperTypeClass, AvroBaseValue.class); DatumReader<E> datumReader; if (org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(avroValueType)) { datumReader = new SpecificDatumReader<E>(avroValueType); } else { datumReader = new ReflectDatumReader<E>(avroValueType); } LOG.info("Opening split " + split); SeekableInput in = new FSDataInputStreamWrapper(stream, (int) split.getLength()); dataFileReader = DataFileReader.openReader(in, datumReader); dataFileReader.sync(split.getStart()); reuseAvroValue = null; }