YARN-5925. Extract hbase-backend-exclusive utility methods from TimelineStorageUtil. Contributed by Haibo Chen.
This commit is contained in:
parent
4aa6f0059c
commit
ef7e48072c
@ -30,7 +30,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
|
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.reader.TimelineReaderContext;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
|
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.EntityTypeReader;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.EntityTypeReader;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
|
||||||
|
@ -48,7 +48,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
|
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.Separator;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
|
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.common.TypedBufferedMutator;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
|
||||||
|
@ -39,7 +39,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
|
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.apptoflow.AppToFlowTable;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
|
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.entity.EntityTable;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
|
||||||
|
@ -206,24 +206,6 @@ public static String getAggregationCompactionDimension(List<Tag> tags) {
|
|||||||
return appId;
|
return appId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static boolean isFlowRunTable(HRegionInfo hRegionInfo,
|
|
||||||
Configuration conf) {
|
|
||||||
String regionTableName = hRegionInfo.getTable().getNameAsString();
|
|
||||||
String flowRunTableName = conf.get(FlowRunTable.TABLE_NAME_CONF_NAME,
|
|
||||||
FlowRunTable.DEFAULT_TABLE_NAME);
|
|
||||||
if (HBaseTimelineStorageUtils.LOG.isDebugEnabled()) {
|
|
||||||
HBaseTimelineStorageUtils.LOG.debug("regionTableName=" + regionTableName);
|
|
||||||
}
|
|
||||||
if (flowRunTableName.equalsIgnoreCase(regionTableName)) {
|
|
||||||
if (HBaseTimelineStorageUtils.LOG.isDebugEnabled()) {
|
|
||||||
HBaseTimelineStorageUtils.LOG.debug(
|
|
||||||
"table is the flow run table!! " + flowRunTableName);
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Converts an int into it's inverse int to be used in (row) keys
|
* Converts an int into it's inverse int to be used in (row) keys
|
||||||
* where we want to have the largest int value in the top of the table
|
* where we want to have the largest int value in the top of the table
|
||||||
@ -292,17 +274,16 @@ public static String convertApplicationIdToString(ApplicationId appId) {
|
|||||||
*/
|
*/
|
||||||
public static Configuration getTimelineServiceHBaseConf(Configuration conf)
|
public static Configuration getTimelineServiceHBaseConf(Configuration conf)
|
||||||
throws MalformedURLException {
|
throws MalformedURLException {
|
||||||
|
Configuration hbaseConf;
|
||||||
|
|
||||||
if (conf == null) {
|
if (conf == null) {
|
||||||
throw new NullPointerException();
|
return HBaseConfiguration.create();
|
||||||
}
|
}
|
||||||
|
|
||||||
Configuration hbaseConf;
|
|
||||||
String timelineServiceHBaseConfFileURL =
|
String timelineServiceHBaseConfFileURL =
|
||||||
conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
|
conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
|
||||||
if (timelineServiceHBaseConfFileURL != null
|
if (timelineServiceHBaseConfFileURL != null
|
||||||
&& timelineServiceHBaseConfFileURL.length() > 0) {
|
&& timelineServiceHBaseConfFileURL.length() > 0) {
|
||||||
LOG.info("Using hbase configuration at " +
|
|
||||||
timelineServiceHBaseConfFileURL);
|
|
||||||
// create a clone so that we don't mess with out input one
|
// create a clone so that we don't mess with out input one
|
||||||
hbaseConf = new Configuration(conf);
|
hbaseConf = new Configuration(conf);
|
||||||
Configuration plainHBaseConf = new Configuration(false);
|
Configuration plainHBaseConf = new Configuration(false);
|
||||||
@ -349,4 +330,16 @@ public static byte[] calculateTheClosestNextRowKeyForPrefix(
|
|||||||
newStopRow[newStopRow.length - 1]++;
|
newStopRow[newStopRow.length - 1]++;
|
||||||
return newStopRow;
|
return newStopRow;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if passed object is of integral type(Short/Integer/Long).
|
||||||
|
*
|
||||||
|
* @param obj Object to be checked.
|
||||||
|
* @return true if object passed is of type Short or Integer or Long, false
|
||||||
|
* otherwise.
|
||||||
|
*/
|
||||||
|
public static boolean isIntegralValue(Object obj) {
|
||||||
|
return (obj instanceof Short) || (obj instanceof Integer) ||
|
||||||
|
(obj instanceof Long);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -40,7 +40,7 @@ public LongConverter() {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] encodeValue(Object value) throws IOException {
|
public byte[] encodeValue(Object value) throws IOException {
|
||||||
if (!TimelineStorageUtils.isIntegralValue(value)) {
|
if (!HBaseTimelineStorageUtils.isIntegralValue(value)) {
|
||||||
throw new IOException("Expected integral value");
|
throw new IOException("Expected integral value");
|
||||||
}
|
}
|
||||||
return Bytes.toBytes(((Number)value).longValue());
|
return Bytes.toBytes(((Number)value).longValue());
|
||||||
|
@ -33,7 +33,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
|
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.HBaseTimelineStorageUtils;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
|
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
|
||||||
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
|
import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
|
||||||
@ -154,8 +153,8 @@ private static byte[] getNextRowKey(byte[] currRowKeyPrefix,
|
|||||||
System.arraycopy(entityTypeEncoded, 0, currRowKey, currRowKeyPrefix.length,
|
System.arraycopy(entityTypeEncoded, 0, currRowKey, currRowKeyPrefix.length,
|
||||||
entityTypeEncoded.length);
|
entityTypeEncoded.length);
|
||||||
|
|
||||||
return HBaseTimelineStorageUtils.
|
return HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix(
|
||||||
calculateTheClosestNextRowKeyForPrefix(currRowKey);
|
currRowKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
private ResultScanner getResult(Configuration hbaseConf, Connection conn,
|
private ResultScanner getResult(Configuration hbaseConf, Connection conn,
|
||||||
|
Loading…
x
Reference in New Issue
Block a user