public boolean hasNext() { while (curFileIndex < filePaths.size()) { if (curReader.hasNext()) { return true; } curReader.close(); curFileIndex++; if (curFileIndex < filePaths.size()) { curReader = HdfsReaderFactory.getHdfsReader(readerType, systemStreamPartition); curReader.open(filePaths.get(curFileIndex), "0"); } } return false; }
private void init(String offset) { if (curReader != null) { curReader.close(); curReader = null; } curFileIndex = getCurFileIndex(offset); if (curFileIndex >= filePaths.size()) { throw new SamzaException( String.format("Invalid file index %d. Number of files is %d", curFileIndex, filePaths.size())); } curSingleFileOffset = getCurSingleFileOffset(offset); curReader = HdfsReaderFactory.getHdfsReader(readerType, systemStreamPartition); curReader.open(filePaths.get(curFileIndex), curSingleFileOffset); }
@Test public void testFileReopen() throws Exception { SystemStreamPartition ssp = new SystemStreamPartition("hdfs", "testStream", new Partition(0)); SingleFileHdfsReader reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, "0"); int index = 0; for (;index < NUM_EVENTS / 2; index++) { GenericRecord record = (GenericRecord) reader.readNext().getMessage(); Assert.assertEquals(index, record.get(FIELD_1)); Assert.assertEquals("string_" + index, record.get(FIELD_2).toString()); } String offset = reader.nextOffset(); reader.close(); reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, offset); for (;index < NUM_EVENTS; index++) { GenericRecord record = (GenericRecord) reader.readNext().getMessage(); Assert.assertEquals(index, record.get(FIELD_1)); Assert.assertEquals("string_" + index, record.get(FIELD_2).toString()); } Assert.assertEquals(NUM_EVENTS, index); reader.close(); }
@Test public void testSequentialRead() throws Exception { SystemStreamPartition ssp = new SystemStreamPartition("hdfs", "testStream", new Partition(0)); SingleFileHdfsReader reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, "0"); int index = 0; while (reader.hasNext()) { GenericRecord record = (GenericRecord) reader.readNext().getMessage(); Assert.assertEquals(index, record.get(FIELD_1)); Assert.assertEquals("string_" + index, record.get(FIELD_2).toString()); index++; } Assert.assertEquals(NUM_EVENTS, index); reader.close(); }
public IncomingMessageEnvelope readNext() { if (!hasNext()) { LOG.warn("Attempting to read more data when there aren't any. ssp=" + systemStreamPartition); return null; } // record the next offset before we read, so when the read fails and we reconnect, // we seek to the same offset that we try below curSingleFileOffset = curReader.nextOffset(); IncomingMessageEnvelope messageEnvelope = curReader.readNext(); // Copy everything except for the offset. Turn the single-file style offset into a multi-file one return new IncomingMessageEnvelope(messageEnvelope.getSystemStreamPartition(), getCurOffset(), messageEnvelope.getKey(), messageEnvelope.getMessage(), messageEnvelope.getSize(), messageEnvelope.getEventTime(), messageEnvelope.getArrivalTime()); }
public void close() { LOG.info(String.format("MiltiFileHdfsReader shutdown requested for %s. Current offset = %s", systemStreamPartition, getCurOffset())); if (curReader != null) { curReader.close(); } }
@Test public void testRandomRead() throws Exception { SystemStreamPartition ssp = new SystemStreamPartition("hdfs", "testStream", new Partition(0)); SingleFileHdfsReader reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, "0"); for (int i = 0;i < NUM_EVENTS / 2; i++) { reader.readNext(); } String offset = reader.nextOffset(); IncomingMessageEnvelope envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record1 = (GenericRecord) envelope.getMessage(); for (int i = 0; i < 5; i++) reader.readNext(); // seek to the offset within the same reader reader.seek(offset); Assert.assertEquals(offset, reader.nextOffset()); envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record2 = (GenericRecord) envelope.getMessage(); Assert.assertEquals(record1, record2); reader.close(); // open a new reader and initialize it with the offset reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, offset); envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record3 = (GenericRecord) envelope.getMessage(); Assert.assertEquals(record1, record3); reader.close(); }