YARN-9822.TimelineCollectorWebService#putEntities blocked when ATSV2 HBase is down (#4492)

Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
This commit is contained in:
Ashutosh Gupta 2022-06-28 05:02:07 +01:00 committed by GitHub
parent 43112bd472
commit a177232ebc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 196 additions and 21 deletions

View File

@ -39,12 +39,12 @@ public class TimelineHealth {
* Timline health status.
*
* RUNNING - Service is up and running
* READER_CONNECTION_FAULURE - isConnectionAlive() of reader implementation
* CONNECTION_FAULURE - isConnectionAlive() of reader / writer implementation
* reported an error
*/
public enum TimelineHealthStatus {
RUNNING,
READER_CONNECTION_FAILURE
CONNECTION_FAILURE
}
private TimelineHealthStatus healthStatus;

View File

@ -108,7 +108,7 @@ public class DocumentStoreTimelineReaderImpl
"");
} else {
return new TimelineHealth(
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
"Timeline store reader not initialized.");
}
}
@ -131,4 +131,4 @@ public class DocumentStoreTimelineReaderImpl
}
return timelineEntities;
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.*;
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
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.documentstore.lib.DocumentStoreVendor;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack;
@ -151,6 +152,11 @@ public class DocumentStoreTimelineWriterImpl extends AbstractService
return null;
}
@Override
public TimelineHealth getHealthStatus() {
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, "");
}
private void appendSubAppUserIfExists(TimelineCollectorContext context,
String subApplicationUser) {
String userId = context.getUserId();
@ -282,4 +288,4 @@ public class DocumentStoreTimelineWriterImpl extends AbstractService
@Override
public void flush() {
}
}
}

View File

@ -115,7 +115,7 @@ public class HBaseTimelineReaderImpl
"");
} catch (IOException e){
return new TimelineHealth(
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
"HBase connection is down");
}
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.security.UserGroupInformation;
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.SubApplicationEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain;
@ -604,6 +605,19 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
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)
*

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
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.TimelineMetricOperation;
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 int maxWriteRetries;
private long writeRetryInterval;
public TimelineCollector(String name) {
super(name);
}
@ -86,6 +90,13 @@ public abstract class TimelineCollector extends CompositeService {
new ArrayBlockingQueue<>(capacity));
pool.setRejectedExecutionHandler(
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
@ -153,18 +164,54 @@ public abstract class TimelineCollector extends CompositeService {
UserGroupInformation callerUgi) throws IOException {
LOG.debug("putEntities(entities={}, callerUgi={})", entities, callerUgi);
TimelineWriteResponse response;
// synchronize on the writer object so that no other threads can
// flush the writer buffer concurrently and swallow any exception
// caused by the timeline enitites that are being put here.
synchronized (writer) {
response = writeTimelineEntities(entities, callerUgi);
flushBufferedTimelineEntities();
TimelineWriteResponse response = null;
try {
boolean isStorageUp = checkRetryWithSleep();
if (isStorageUp) {
// synchronize on the writer object so that no other threads can
// flush the writer buffer concurrently and swallow any exception
// caused by the timeline enitites that are being put here.
synchronized (writer) {
response = writeTimelineEntities(entities, callerUgi);
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;
}
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
* and the admin can update it.
@ -179,11 +226,25 @@ public abstract class TimelineCollector extends CompositeService {
UserGroupInformation callerUgi) throws IOException {
LOG.debug("putDomain(domain={}, callerUgi={})", domain, callerUgi);
TimelineWriteResponse response;
synchronized (writer) {
final TimelineCollectorContext context = getTimelineEntityContext();
response = writer.write(context, domain);
flushBufferedTimelineEntities();
TimelineWriteResponse response = null;
try {
boolean isStorageUp = checkRetryWithSleep();
if (isStorageUp) {
synchronized (writer) {
final TimelineCollectorContext context = getTimelineEntityContext();
response = writer.write(context, domain);
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;

View File

@ -454,7 +454,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
fs.exists(rootPath);
} catch (IOException e) {
return new TimelineHealth(
TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE,
TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE,
e.getMessage()
);
}

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
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.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@ -194,6 +195,20 @@ public class FileSystemTimelineWriterImpl extends AbstractService
// 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 {
for (Path path: paths) {
if (!existsWithRetries(path)) {

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
import org.apache.hadoop.security.UserGroupInformation;
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.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@ -77,4 +78,10 @@ public class NoOpTimelineWriterImpl extends AbstractService implements
public void flush() throws IOException {
LOG.debug("NoOpTimelineWriter is configured. Ignoring flush call");
}
@Override
public TimelineHealth getHealthStatus() {
return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING,
"NoOpTimelineWriter is configured. ");
}
}

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.security.UserGroupInformation;
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.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@ -95,4 +96,13 @@ public interface TimelineWriter extends Service {
* entities to the backend storage.
*/
void flush() throws IOException;
/**
* Check if writer connection is working properly.
*
* @return True if writer connection works as expected, false otherwise.
*/
TimelineHealth getHealthStatus();
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.timelineservice.collector;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
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.TimelineMetricOperation;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -155,7 +156,17 @@ public class TestTimelineCollector {
@Test
public void testPutEntity() throws IOException {
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);
collector.init(conf);
TimelineEntities entities = generateTestEntities(1, 1);
collector.putEntities(
@ -166,6 +177,36 @@ public class TestTimelineCollector {
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
* putEntityAsync() calls.
@ -222,7 +263,17 @@ public class TestTimelineCollector {
*/
@Test public void testPutDomain() throws IOException {
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);
collector.init(conf);
TimelineDomain domain =
generateDomain("id", "desc", "owner", "reader1,reader2", "writer", 0L,
@ -287,8 +338,19 @@ public class TestTimelineCollector {
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.
TimelineEntities entities = new TimelineEntities();