YARN-5265. Make HBase configuration for the timeline service configurable. Contributed by Joep Rottinghuis.

This commit is contained in:
Sangjin Lee 2016-11-03 10:18:04 -07:00 committed by Varun Saxena
parent 05ff04439e
commit 643a20a358
7 changed files with 89 additions and 7 deletions

View File

@ -2145,6 +2145,15 @@ public class YarnConfiguration extends Configuration {
TIMELINE_SERVICE_PREFIX
+ "hbase.coprocessor.app-final-value-retention-milliseconds";
/**
* The name for setting that points to an optional HBase configuration
* (hbase-site.xml file) with settings that will override the ones found on
* the classpath.
*/
public static final String TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE =
TIMELINE_SERVICE_PREFIX
+ "hbase.configuration.file";
/**
* The setting that controls how long the final value of a metric of a
* completed app is retained before merging into the flow sum. Up to this time

View File

@ -2353,6 +2353,18 @@
<value>prod.</value>
</property>
<property>
<description> Optional URL to an hbase-site.xml configuration file to be
used to connect to the timeline-service hbase cluster. If empty or not
specified, then the HBase configuration will be loaded from the classpath.
When specified the values in the specified configuration file will override
those from the ones that are present on the classpath.
</description>
<name>yarn.timeline-service.hbase.configuration.file
</name>
<value></value>
</property>
<!-- Shared Cache Configuration -->
<property>

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.service.AbstractService;
@ -30,6 +29,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
import org.slf4j.Logger;
@ -54,7 +55,7 @@ public class HBaseTimelineReaderImpl
@Override
public void serviceInit(Configuration conf) throws Exception {
super.serviceInit(conf);
hbaseConf = HBaseConfiguration.create(conf);
hbaseConf = HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf);
conn = ConnectionFactory.createConnection(hbaseConf);
}

View File

@ -24,7 +24,6 @@ import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.service.AbstractService;
@ -44,10 +43,12 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
@ -107,7 +108,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
@Override
protected void serviceInit(Configuration conf) throws Exception {
super.serviceInit(conf);
Configuration hbaseConf = HBaseConfiguration.create(conf);
Configuration hbaseConf =
HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf);
conn = ConnectionFactory.createConnection(hbaseConf);
entityTable = new EntityTable().getTableMutator(hbaseConf, conn);
appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn);

View File

@ -32,13 +32,14 @@ import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
@ -71,7 +72,8 @@ public final class TimelineSchemaCreator {
public static void main(String[] args) throws Exception {
Configuration hbaseConf = HBaseConfiguration.create();
Configuration hbaseConf =
HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null);
// Grab input args and allow for -Dxyz style arguments
String[] otherArgs = new GenericOptionsParser(hbaseConf, args)
.getRemainingArgs();

View File

@ -17,14 +17,18 @@
package org.apache.hadoop.yarn.server.timelineservice.storage.common;
import java.net.MalformedURLException;
import java.net.URL;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
@ -273,4 +277,40 @@ public final class HBaseTimelineStorageUtils {
sb.append(APP_ID_FORMAT.get().format(appId.getId()));
return sb.toString();
}
/**
* @param conf Yarn configuration. Used to see if there is an explicit config
* pointing to the HBase config file to read. It should not be null
* or a NullPointerException will be thrown.
* @return a configuration with the HBase configuration from the classpath,
* optionally overwritten by the timeline service configuration URL if
* specified.
* @throws MalformedURLException if a timeline service HBase configuration URL
* is specified but is a malformed URL.
*/
public static Configuration getTimelineServiceHBaseConf(Configuration conf)
throws MalformedURLException {
if (conf == null) {
throw new NullPointerException();
}
Configuration hbaseConf;
String timelineServiceHBaseConfFileURL =
conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
if (timelineServiceHBaseConfFileURL != null
&& timelineServiceHBaseConfFileURL.length() > 0) {
LOG.info("Using hbase configuration at " +
timelineServiceHBaseConfFileURL);
// create a clone so that we don't mess with out input one
hbaseConf = new Configuration(conf);
Configuration plainHBaseConf = new Configuration(false);
URL hbaseSiteXML = new URL(timelineServiceHBaseConfFileURL);
plainHBaseConf.addResource(hbaseSiteXML);
HBaseConfiguration.merge(hbaseConf, plainHBaseConf);
} else {
// default to what is on the classpath
hbaseConf = HBaseConfiguration.create(conf);
}
return hbaseConf;
}
}

View File

@ -137,6 +137,7 @@ New configuration parameters that are introduced with v.2 are marked bold.
| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. |
| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
| **`yarn.timeline-service.hbase.configuration.file`** | Optional URL to an hbase-site.xml configuration file to be used to connect to the timeline-service hbase cluster. If empty or not specified, then the HBase configuration will be loaded from the classpath. When specified the values in the specified configuration file will override those from the ones that are present on the classpath. Defaults to `null`. |
| **`yarn.timeline-service.writer.flush-interval-seconds`** | The setting that controls how often the timeline collector flushes the timeline writer. Defaults to `60`. |
| **`yarn.timeline-service.app-collector.linger-period.ms`** | Time period till which the application collector will be alive in NM, after the application master container finishes. Defaults to `1000` (1 second). |
| **`yarn.timeline-service.timeline-client.number-of-async-entities-to-merge`** | Time line V2 client tries to merge these many number of async entities (if available) and then call the REST ATS V2 API to submit. Defaults to `10`. |
@ -243,7 +244,22 @@ are using multiple clusters to store data in the same Apache HBase storage:
```
Also, add the `hbase-site.xml` configuration file to the client Hadoop cluster configuration so
that it can write data to the Apache HBase cluster you are using.
that it can write data to the Apache HBase cluster you are using, or set
`yarn.timeline-service.hbase.configuration.file` to the file URL pointing to
`hbase-site.xml` for the same. For example:
```
<property>
<description> Optional URL to an hbase-site.xml configuration file to be
used to connect to the timeline-service hbase cluster. If empty or not
specified, then the HBase configuration will be loaded from the classpath.
When specified the values in the specified configuration file will override
those from the ones that are present on the classpath.
</description>
<name>yarn.timeline-service.hbase.configuration.file</name>
<value>file:/etc/hbase/hbase-ats-dc1/hbase-site.xml</value>
</property>
```
#### Running Timeline Service v.2
Restart the resource manager as well as the node managers to pick up the new configuration. The