NIFI-2764

Signed-off-by: Matt Burgess <mattyb149@apache.org>

NIFI-2764: Updated call to convertToAvroStream in TestJdbcCommon

This closes #1009
This commit is contained in:
patricker 2016-09-12 21:52:58 -06:00 committed by Matt Burgess
parent 63bda32a8b
commit 4b4e099f2e
2 changed files with 42 additions and 1 deletions

View File

@ -166,7 +166,9 @@ public class JdbcCommon {
// direct put to avro record results:
// org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
rec.put(i - 1, ((Byte) value).intValue());
} else if(value instanceof Short) {
//MS SQL returns TINYINT as a Java Short, which Avro doesn't understand.
rec.put(i - 1, ((Short) value).intValue());
} else if (value instanceof BigDecimal) {
// Avro can't handle BigDecimal as a number - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
rec.put(i - 1, value.toString());

View File

@ -450,6 +450,45 @@ public class TestJdbcCommon {
}
}
@Test
public void testConvertToAvroStreamForShort() throws SQLException, IOException {
final ResultSetMetaData metadata = mock(ResultSetMetaData.class);
when(metadata.getColumnCount()).thenReturn(1);
when(metadata.getColumnType(1)).thenReturn(Types.TINYINT);
when(metadata.getColumnName(1)).thenReturn("t_int");
when(metadata.getTableName(1)).thenReturn("table");
final ResultSet rs = mock(ResultSet.class);
when(rs.getMetaData()).thenReturn(metadata);
final AtomicInteger counter = new AtomicInteger(1);
Mockito.doAnswer(new Answer<Boolean>() {
@Override
public Boolean answer(InvocationOnMock invocation) throws Throwable {
return counter.getAndDecrement() > 0;
}
}).when(rs).next();
final short s = 25;
when(rs.getObject(Mockito.anyInt())).thenReturn(s);
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
JdbcCommon.convertToAvroStream(rs, baos, false);
final byte[] serializedBytes = baos.toByteArray();
final InputStream instream = new ByteArrayInputStream(serializedBytes);
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(instream, datumReader)) {
GenericRecord record = null;
while (dataFileReader.hasNext()) {
record = dataFileReader.next(record);
assertEquals(Short.toString(s), record.get("t_int").toString());
}
}
}
// many test use Derby as database, so ensure driver is available
@Test