YARN-5097. NPE in Separator.joinEncoded() (Vrushali C via sjlee)

This commit is contained in:
Sangjin Lee 2016-05-25 15:49:08 -07:00
parent a1b6d7456f
commit 831a3ffd6e
2 changed files with 66 additions and 0 deletions

View File

@ -482,6 +482,63 @@ public class TestHBaseTimelineStorage {
}
}
@Test
public void testWriteNullApplicationToHBase() throws Exception {
TimelineEntities te = new TimelineEntities();
ApplicationEntity entity = new ApplicationEntity();
String appId = "application_1000178881110_2002";
entity.setId(appId);
long cTime = 1425016501000L;
entity.setCreatedTime(cTime);
// add the info map in Timeline Entity
Map<String, Object> infoMap = new HashMap<String, Object>();
infoMap.put("infoMapKey1", "infoMapValue1");
infoMap.put("infoMapKey2", 10);
entity.addInfo(infoMap);
te.addEntity(entity);
HBaseTimelineWriterImpl hbi = null;
try {
Configuration c1 = util.getConfiguration();
hbi = new HBaseTimelineWriterImpl(c1);
hbi.init(c1);
hbi.start();
String cluster = "cluster_check_null_application";
String user = "user1check_null_application";
//set the flow name to null
String flow = null;
String flowVersion = "AB7822C10F1111";
long runid = 1002345678919L;
hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
hbi.stop();
// retrieve the row
Scan scan = new Scan();
scan.setStartRow(Bytes.toBytes(cluster));
Connection conn = ConnectionFactory.createConnection(c1);
ResultScanner resultScanner = new ApplicationTable()
.getResultScanner(c1, conn, scan);
assertTrue(resultScanner != null);
// try to iterate over results
int count = 0;
for (Result rr = resultScanner.next(); rr != null;
rr = resultScanner.next()) {
count++;
}
// there should be no rows written
// no exceptions thrown during write
assertEquals(0, count);
} finally {
if (hbi != null) {
hbi.stop();
hbi.close();
}
}
}
@Test
public void testWriteApplicationToHBase() throws Exception {
TimelineEntities te = new TimelineEntities();

View File

@ -118,6 +118,15 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
TimelineEntities data) throws IOException {
TimelineWriteResponse putStatus = new TimelineWriteResponse();
// defensive coding to avoid NPE during row key construction
if ((flowName == null) || (appId == null) || (clusterId == null)
|| (userId == null)) {
LOG.warn("Found null for one of: flowName=" + flowName + " appId=" + appId
+ " userId=" + userId + " clusterId=" + clusterId
+ " . Not proceeding with writing to hbase");
return putStatus;
}
for (TimelineEntity te : data.getEntities()) {
// a set can have at most 1 null