YARN-9822.TimelineCollectorWebService#putEntities blocked when ATSV2 HBase is down (#4492)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
This commit is contained in:
parent
43112bd472
commit
a177232ebc
@ -39,12 +39,12 @@ public class TimelineHealth {
|
|||||||
* Timline health status.
|
* Timline health status.
|
||||||
*
|
*
|
||||||
* RUNNING - Service is up and running
|
* RUNNING - Service is up and running
|
||||||
* READER_CONNECTION_FAULURE - isConnectionAlive() of reader implementation
|
* CONNECTION_FAULURE - isConnectionAlive() of reader / writer implementation
|
||||||
* reported an error
|
* reported an error
|
||||||
*/
|
*/
|
||||||
public enum TimelineHealthStatus {
|
public enum TimelineHealthStatus {
|
||||||
RUNNING,
|
RUNNING,
|
||||||
READER_CONNECTION_FAILURE
|
CONNECTION_FAILURE
|
||||||
}
|
}
|
||||||
|
|
||||||
private TimelineHealthStatus healthStatus;
|
private TimelineHealthStatus healthStatus;
|
||||||
|
@ -108,7 +108,7 @@ public TimelineHealth getHealthStatus() {
|
|||||||
"");
|
"");
|
||||||
} else {
|
} else {
|
||||||
return new TimelineHealth(
|
return new TimelineHealth(
|
||||||
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
|
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
|
||||||
"Timeline store reader not initialized.");
|
"Timeline store reader not initialized.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -25,6 +25,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
|
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
|
||||||
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
||||||
import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
|
import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
|
import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.documentstore.lib.DocumentStoreVendor;
|
import org.apache.hadoop.yarn.server.timelineservice.documentstore.lib.DocumentStoreVendor;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack;
|
||||||
@ -151,6 +152,11 @@ public TimelineWriteResponse write(TimelineCollectorContext context,
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TimelineHealth getHealthStatus() {
|
||||||
|
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, "");
|
||||||
|
}
|
||||||
|
|
||||||
private void appendSubAppUserIfExists(TimelineCollectorContext context,
|
private void appendSubAppUserIfExists(TimelineCollectorContext context,
|
||||||
String subApplicationUser) {
|
String subApplicationUser) {
|
||||||
String userId = context.getUserId();
|
String userId = context.getUserId();
|
||||||
|
@ -115,7 +115,7 @@ public TimelineHealth getHealthStatus() {
|
|||||||
"");
|
"");
|
||||||
} catch (IOException e){
|
} catch (IOException e){
|
||||||
return new TimelineHealth(
|
return new TimelineHealth(
|
||||||
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
|
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
|
||||||
"HBase connection is down");
|
"HBase connection is down");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.AbstractService;
|
import org.apache.hadoop.service.AbstractService;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.SubApplicationEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.SubApplicationEntity;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
@ -604,6 +605,19 @@ public TimelineWriteResponse aggregate(TimelineEntity data,
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TimelineHealth getHealthStatus() {
|
||||||
|
try {
|
||||||
|
storageMonitor.checkStorageIsUp();
|
||||||
|
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING,
|
||||||
|
"");
|
||||||
|
} catch (IOException e){
|
||||||
|
return new TimelineHealth(
|
||||||
|
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
|
||||||
|
"HBase connection is down");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* (non-Javadoc)
|
* (non-Javadoc)
|
||||||
*
|
*
|
||||||
|
@ -34,6 +34,7 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.CompositeService;
|
import org.apache.hadoop.service.CompositeService;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
@ -71,6 +72,9 @@ public abstract class TimelineCollector extends CompositeService {
|
|||||||
|
|
||||||
private volatile boolean isStopped = false;
|
private volatile boolean isStopped = false;
|
||||||
|
|
||||||
|
private int maxWriteRetries;
|
||||||
|
private long writeRetryInterval;
|
||||||
|
|
||||||
public TimelineCollector(String name) {
|
public TimelineCollector(String name) {
|
||||||
super(name);
|
super(name);
|
||||||
}
|
}
|
||||||
@ -86,6 +90,13 @@ protected void serviceInit(Configuration conf) throws Exception {
|
|||||||
new ArrayBlockingQueue<>(capacity));
|
new ArrayBlockingQueue<>(capacity));
|
||||||
pool.setRejectedExecutionHandler(
|
pool.setRejectedExecutionHandler(
|
||||||
new ThreadPoolExecutor.DiscardOldestPolicy());
|
new ThreadPoolExecutor.DiscardOldestPolicy());
|
||||||
|
|
||||||
|
maxWriteRetries =
|
||||||
|
conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
|
||||||
|
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
|
||||||
|
writeRetryInterval = conf.getLong(
|
||||||
|
YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
||||||
|
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -153,7 +164,10 @@ public TimelineWriteResponse putEntities(TimelineEntities entities,
|
|||||||
UserGroupInformation callerUgi) throws IOException {
|
UserGroupInformation callerUgi) throws IOException {
|
||||||
LOG.debug("putEntities(entities={}, callerUgi={})", entities, callerUgi);
|
LOG.debug("putEntities(entities={}, callerUgi={})", entities, callerUgi);
|
||||||
|
|
||||||
TimelineWriteResponse response;
|
TimelineWriteResponse response = null;
|
||||||
|
try {
|
||||||
|
boolean isStorageUp = checkRetryWithSleep();
|
||||||
|
if (isStorageUp) {
|
||||||
// synchronize on the writer object so that no other threads can
|
// synchronize on the writer object so that no other threads can
|
||||||
// flush the writer buffer concurrently and swallow any exception
|
// flush the writer buffer concurrently and swallow any exception
|
||||||
// caused by the timeline enitites that are being put here.
|
// caused by the timeline enitites that are being put here.
|
||||||
@ -161,10 +175,43 @@ public TimelineWriteResponse putEntities(TimelineEntities entities,
|
|||||||
response = writeTimelineEntities(entities, callerUgi);
|
response = writeTimelineEntities(entities, callerUgi);
|
||||||
flushBufferedTimelineEntities();
|
flushBufferedTimelineEntities();
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
String msg = String.format("Failed to putEntities(" +
|
||||||
|
"entities=%s, callerUgi=%s) as Timeline Storage is Down",
|
||||||
|
entities, callerUgi);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
String msg = String.format("Interrupted while retrying to putEntities(" +
|
||||||
|
"entities=%s, callerUgi=%s)", entities, callerUgi);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
|
||||||
return response;
|
return response;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private boolean checkRetryWithSleep() throws InterruptedException {
|
||||||
|
int retries = maxWriteRetries;
|
||||||
|
while (retries > 0) {
|
||||||
|
TimelineHealth timelineHealth = writer.getHealthStatus();
|
||||||
|
if (timelineHealth.getHealthStatus().equals(
|
||||||
|
TimelineHealth.TimelineHealthStatus.RUNNING)) {
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
try {
|
||||||
|
Thread.sleep(writeRetryInterval);
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
retries--;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add or update an domain. If the domain already exists, only the owner
|
* Add or update an domain. If the domain already exists, only the owner
|
||||||
* and the admin can update it.
|
* and the admin can update it.
|
||||||
@ -179,12 +226,26 @@ public TimelineWriteResponse putDomain(TimelineDomain domain,
|
|||||||
UserGroupInformation callerUgi) throws IOException {
|
UserGroupInformation callerUgi) throws IOException {
|
||||||
LOG.debug("putDomain(domain={}, callerUgi={})", domain, callerUgi);
|
LOG.debug("putDomain(domain={}, callerUgi={})", domain, callerUgi);
|
||||||
|
|
||||||
TimelineWriteResponse response;
|
TimelineWriteResponse response = null;
|
||||||
|
try {
|
||||||
|
boolean isStorageUp = checkRetryWithSleep();
|
||||||
|
if (isStorageUp) {
|
||||||
synchronized (writer) {
|
synchronized (writer) {
|
||||||
final TimelineCollectorContext context = getTimelineEntityContext();
|
final TimelineCollectorContext context = getTimelineEntityContext();
|
||||||
response = writer.write(context, domain);
|
response = writer.write(context, domain);
|
||||||
flushBufferedTimelineEntities();
|
flushBufferedTimelineEntities();
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
String msg = String.format("Failed to putDomain(" +
|
||||||
|
"domain=%s, callerUgi=%s) as Timeline Storage is Down",
|
||||||
|
domain, callerUgi);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
String msg = String.format("Interrupted while retrying to putDomain(" +
|
||||||
|
"domain=%s, callerUgi=%s)", domain, callerUgi);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
|
||||||
return response;
|
return response;
|
||||||
}
|
}
|
||||||
|
@ -454,7 +454,7 @@ public TimelineHealth getHealthStatus() {
|
|||||||
fs.exists(rootPath);
|
fs.exists(rootPath);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
return new TimelineHealth(
|
return new TimelineHealth(
|
||||||
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
|
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
|
||||||
e.getMessage()
|
e.getMessage()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -31,6 +31,7 @@
|
|||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.AbstractService;
|
import org.apache.hadoop.service.AbstractService;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
@ -194,6 +195,20 @@ public void flush() throws IOException {
|
|||||||
// no op
|
// no op
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TimelineHealth getHealthStatus() {
|
||||||
|
try {
|
||||||
|
fs.exists(rootPath);
|
||||||
|
} catch (IOException e) {
|
||||||
|
return new TimelineHealth(
|
||||||
|
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
|
||||||
|
e.getMessage()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING,
|
||||||
|
"");
|
||||||
|
}
|
||||||
|
|
||||||
private void mkdirs(Path... paths) throws IOException, InterruptedException {
|
private void mkdirs(Path... paths) throws IOException, InterruptedException {
|
||||||
for (Path path: paths) {
|
for (Path path: paths) {
|
||||||
if (!existsWithRetries(path)) {
|
if (!existsWithRetries(path)) {
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
|
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.AbstractService;
|
import org.apache.hadoop.service.AbstractService;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
@ -77,4 +78,10 @@ public TimelineWriteResponse aggregate(TimelineEntity data,
|
|||||||
public void flush() throws IOException {
|
public void flush() throws IOException {
|
||||||
LOG.debug("NoOpTimelineWriter is configured. Ignoring flush call");
|
LOG.debug("NoOpTimelineWriter is configured. Ignoring flush call");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TimelineHealth getHealthStatus() {
|
||||||
|
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING,
|
||||||
|
"NoOpTimelineWriter is configured. ");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -23,6 +23,7 @@
|
|||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.Service;
|
import org.apache.hadoop.service.Service;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
@ -95,4 +96,13 @@ TimelineWriteResponse aggregate(TimelineEntity data,
|
|||||||
* entities to the backend storage.
|
* entities to the backend storage.
|
||||||
*/
|
*/
|
||||||
void flush() throws IOException;
|
void flush() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if writer connection is working properly.
|
||||||
|
*
|
||||||
|
* @return True if writer connection works as expected, false otherwise.
|
||||||
|
*/
|
||||||
|
TimelineHealth getHealthStatus();
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
@ -21,6 +21,7 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.util.Sets;
|
import org.apache.hadoop.util.Sets;
|
||||||
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
@ -155,7 +156,17 @@ public void testAggregation() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
public void testPutEntity() throws IOException {
|
public void testPutEntity() throws IOException {
|
||||||
TimelineWriter writer = mock(TimelineWriter.class);
|
TimelineWriter writer = mock(TimelineWriter.class);
|
||||||
|
TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth.
|
||||||
|
TimelineHealthStatus.RUNNING, "");
|
||||||
|
when(writer.getHealthStatus()).thenReturn(timelineHealth);
|
||||||
|
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5);
|
||||||
|
conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
||||||
|
500L);
|
||||||
|
|
||||||
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
||||||
|
collector.init(conf);
|
||||||
|
|
||||||
TimelineEntities entities = generateTestEntities(1, 1);
|
TimelineEntities entities = generateTestEntities(1, 1);
|
||||||
collector.putEntities(
|
collector.putEntities(
|
||||||
@ -166,6 +177,36 @@ public void testPutEntity() throws IOException {
|
|||||||
verify(writer, times(1)).flush();
|
verify(writer, times(1)).flush();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPutEntityWithStorageDown() throws IOException {
|
||||||
|
TimelineWriter writer = mock(TimelineWriter.class);
|
||||||
|
TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth.
|
||||||
|
TimelineHealthStatus.CONNECTION_FAILURE, "");
|
||||||
|
when(writer.getHealthStatus()).thenReturn(timelineHealth);
|
||||||
|
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5);
|
||||||
|
conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
||||||
|
500L);
|
||||||
|
|
||||||
|
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
||||||
|
collector.init(conf);
|
||||||
|
|
||||||
|
TimelineEntities entities = generateTestEntities(1, 1);
|
||||||
|
boolean exceptionCaught = false;
|
||||||
|
try {
|
||||||
|
collector.putEntities(entities, UserGroupInformation.
|
||||||
|
createRemoteUser("test-user"));
|
||||||
|
} catch (Exception e) {
|
||||||
|
if (e.getMessage().contains("Failed to putEntities")) {
|
||||||
|
exceptionCaught = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assertTrue("TimelineCollector putEntity failed to " +
|
||||||
|
"handle storage down", exceptionCaught);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test TimelineCollector's interaction with TimelineWriter upon
|
* Test TimelineCollector's interaction with TimelineWriter upon
|
||||||
* putEntityAsync() calls.
|
* putEntityAsync() calls.
|
||||||
@ -222,7 +263,17 @@ public void testAsyncEntityDiscard() throws Exception {
|
|||||||
*/
|
*/
|
||||||
@Test public void testPutDomain() throws IOException {
|
@Test public void testPutDomain() throws IOException {
|
||||||
TimelineWriter writer = mock(TimelineWriter.class);
|
TimelineWriter writer = mock(TimelineWriter.class);
|
||||||
|
TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth.
|
||||||
|
TimelineHealthStatus.RUNNING, "");
|
||||||
|
when(writer.getHealthStatus()).thenReturn(timelineHealth);
|
||||||
|
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5);
|
||||||
|
conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
||||||
|
500L);
|
||||||
|
|
||||||
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
TimelineCollector collector = new TimelineCollectorForTest(writer);
|
||||||
|
collector.init(conf);
|
||||||
|
|
||||||
TimelineDomain domain =
|
TimelineDomain domain =
|
||||||
generateDomain("id", "desc", "owner", "reader1,reader2", "writer", 0L,
|
generateDomain("id", "desc", "owner", "reader1,reader2", "writer", 0L,
|
||||||
@ -287,8 +338,19 @@ public TimelineCollectorContext getTimelineEntityContext() {
|
|||||||
1L, ApplicationId.newInstance(ts, 1).toString());
|
1L, ApplicationId.newInstance(ts, 1).toString());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
collector.init(new Configuration());
|
|
||||||
collector.setWriter(mock(TimelineWriter.class));
|
TimelineWriter writer = mock(TimelineWriter.class);
|
||||||
|
TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth.
|
||||||
|
TimelineHealthStatus.RUNNING, "");
|
||||||
|
when(writer.getHealthStatus()).thenReturn(timelineHealth);
|
||||||
|
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5);
|
||||||
|
conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
||||||
|
500L);
|
||||||
|
|
||||||
|
collector.init(conf);
|
||||||
|
collector.setWriter(writer);
|
||||||
|
|
||||||
// Put 5 entities with different metric values.
|
// Put 5 entities with different metric values.
|
||||||
TimelineEntities entities = new TimelineEntities();
|
TimelineEntities entities = new TimelineEntities();
|
||||||
|
Loading…
Reference in New Issue
Block a user