public static SingleFileHdfsReader getHdfsReader(ReaderType readerType, SystemStreamPartition systemStreamPartition) { switch (readerType) { case AVRO: return new AvroFileHdfsReader(systemStreamPartition); default: throw new SamzaException("Unsupported reader type: " + readerType); } }
@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(); }
@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(); }