diff --git hadoop-project/pom.xml hadoop-project/pom.xml index 9ec24ea..5a80fdd 100644 --- hadoop-project/pom.xml +++ hadoop-project/pom.xml @@ -322,6 +322,12 @@ test-jar + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase + ${project.version} + + org.apache.hadoop hadoop-yarn-applications-distributedshell diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml index 026ef75..34dce1e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml @@ -42,10 +42,10 @@ + older version of hadoop just for the hbase unit tests. We also need to + exclude hadoop-common and hadoop-hdfs on each dependency that has + transitive dependencies on them. + --> org.apache.hadoop hadoop-yarn-server-timelineservice @@ -55,6 +55,18 @@ org.apache.hadoop hadoop-common + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase + test + + + org.apache.hadoop + hadoop-common + org.apache.phoenix phoenix-core @@ -148,12 +160,6 @@ com.sun.jersey - jersey-core - test - - - - com.sun.jersey jersey-client test @@ -364,12 +370,6 @@ - - - org.mockito - mockito-all - test - org.apache.hadoop diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml new file mode 100644 index 0000000..976b66e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml @@ -0,0 +1,198 @@ + + + + hadoop-yarn-server + org.apache.hadoop + 3.0.0-alpha2-SNAPSHOT + + 4.0.0 + hadoop-yarn-server-timelineservice-hbase + Apache Hadoop YARN TimelineService HBase Backend + + + + commons-logging + commons-logging + + + + commons-lang + commons-lang + + + + commons-cli + commons-cli + + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-annotations + + + + + org.apache.hadoop + hadoop-common + + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + + org.apache.hadoop + hadoop-yarn-api + + + + + org.apache.hadoop + hadoop-yarn-common + + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + + + + org.apache.hbase + hbase-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.mortbay.jetty + jetty-util + + + + + + org.apache.hbase + hbase-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.hbase + hbase-server + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-hdfs-client + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + jetty-sslengine + + + + + + org.apache.phoenix + phoenix-core + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + net.sourceforge.findbugs + annotations + + + + + + junit + junit + test + + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + true + + + + maven-jar-plugin + + + + test-jar + + test-compile + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + junit + junit + 4.11 + + + + + + + \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/HBaseTimelineFilterUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/HBaseTimelineFilterUtils.java new file mode 100644 index 0000000..97cfcf7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/HBaseTimelineFilterUtils.java @@ -0,0 +1,298 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.reader; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; + +/** + * Set of utility methods used by timeline filter classes. + */ +public final class HBaseTimelineFilterUtils { + + private static final Log LOG = LogFactory.getLog(HBaseTimelineFilterUtils.class); + + private HBaseTimelineFilterUtils() { + } + + /** + * Returns the equivalent HBase filter list's {@link Operator}. + * + * @param op timeline filter list operator. + * @return HBase filter list's Operator. + */ + private static Operator getHBaseOperator(TimelineFilterList.Operator op) { + switch (op) { + case AND: + return Operator.MUST_PASS_ALL; + case OR: + return Operator.MUST_PASS_ONE; + default: + throw new IllegalArgumentException("Invalid operator"); + } + } + + /** + * Returns the equivalent HBase compare filter's {@link CompareOp}. + * + * @param op timeline compare op. + * @return HBase compare filter's CompareOp. + */ + private static CompareOp getHBaseCompareOp( + TimelineCompareOp op) { + switch (op) { + case LESS_THAN: + return CompareOp.LESS; + case LESS_OR_EQUAL: + return CompareOp.LESS_OR_EQUAL; + case EQUAL: + return CompareOp.EQUAL; + case NOT_EQUAL: + return CompareOp.NOT_EQUAL; + case GREATER_OR_EQUAL: + return CompareOp.GREATER_OR_EQUAL; + case GREATER_THAN: + return CompareOp.GREATER; + default: + throw new IllegalArgumentException("Invalid compare operator"); + } + } + + /** + * Converts a {@link TimelinePrefixFilter} to an equivalent HBase + * {@link QualifierFilter}. + * @param colPrefix + * @param filter + * @return a {@link QualifierFilter} object + */ + private static Filter createHBaseColQualPrefixFilter( + ColumnPrefix colPrefix, TimelinePrefixFilter filter) { + return new QualifierFilter(getHBaseCompareOp(filter.getCompareOp()), + new BinaryPrefixComparator( + colPrefix.getColumnPrefixBytes(filter.getPrefix()))); + } + + /** + * Create a HBase {@link QualifierFilter} for the passed column prefix and + * compare op. + * + * @param Describes the type of column prefix. + * @param compareOp compare op. + * @param columnPrefix column prefix. + * @return a column qualifier filter. + */ + public static Filter createHBaseQualifierFilter(CompareOp compareOp, + ColumnPrefix columnPrefix) { + return new QualifierFilter(compareOp, + new BinaryPrefixComparator( + columnPrefix.getColumnPrefixBytes(""))); + } + + /** + * Create filters for confs or metrics to retrieve. This list includes a + * configs/metrics family filter and relevant filters for confs/metrics to + * retrieve, if present. + * + * @param Describes the type of column prefix. + * @param confsOrMetricToRetrieve configs/metrics to retrieve. + * @param columnFamily config or metric column family. + * @param columnPrefix config or metric column prefix. + * @return a filter list. + * @throws IOException if any problem occurs while creating the filters. + */ + public static Filter createFilterForConfsOrMetricsToRetrieve( + TimelineFilterList confsOrMetricToRetrieve, ColumnFamily columnFamily, + ColumnPrefix columnPrefix) throws IOException { + Filter familyFilter = new FamilyFilter(CompareOp.EQUAL, + new BinaryComparator(columnFamily.getBytes())); + if (confsOrMetricToRetrieve != null && + !confsOrMetricToRetrieve.getFilterList().isEmpty()) { + // If confsOrMetricsToRetrive are specified, create a filter list based + // on it and family filter. + FilterList filter = new FilterList(familyFilter); + filter.addFilter( + createHBaseFilterList(columnPrefix, confsOrMetricToRetrieve)); + return filter; + } else { + // Only the family filter needs to be added. + return familyFilter; + } + } + + /** + * Create 2 HBase {@link SingleColumnValueFilter} filters for the specified + * value range represented by start and end value and wraps them inside a + * filter list. Start and end value should not be null. + * + * @param Describes the type of column prefix. + * @param column Column for which single column value filter is to be created. + * @param startValue Start value. + * @param endValue End value. + * @return 2 single column value filters wrapped in a filter list. + * @throws IOException if any problem is encountered while encoding value. + */ + public static FilterList createSingleColValueFiltersByRange( + Column column, Object startValue, Object endValue) throws IOException { + FilterList list = new FilterList(); + Filter singleColValFilterStart = createHBaseSingleColValueFilter( + column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), + column.getValueConverter().encodeValue(startValue), + CompareOp.GREATER_OR_EQUAL, true); + list.addFilter(singleColValFilterStart); + + Filter singleColValFilterEnd = createHBaseSingleColValueFilter( + column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), + column.getValueConverter().encodeValue(endValue), + CompareOp.LESS_OR_EQUAL, true); + list.addFilter(singleColValFilterEnd); + return list; + } + + /** + * Creates a HBase {@link SingleColumnValueFilter}. + * + * @param columnFamily Column Family represented as bytes. + * @param columnQualifier Column Qualifier represented as bytes. + * @param value Value. + * @param compareOp Compare operator. + * @param filterIfMissing This flag decides if we should filter the row if the + * specified column is missing. This is based on the filter's keyMustExist + * field. + * @return a {@link SingleColumnValueFilter} object + * @throws IOException + */ + private static SingleColumnValueFilter createHBaseSingleColValueFilter( + byte[] columnFamily, byte[] columnQualifier, byte[] value, + CompareOp compareOp, boolean filterIfMissing) throws IOException { + SingleColumnValueFilter singleColValFilter = + new SingleColumnValueFilter(columnFamily, columnQualifier, compareOp, + new BinaryComparator(value)); + singleColValFilter.setLatestVersionOnly(true); + singleColValFilter.setFilterIfMissing(filterIfMissing); + return singleColValFilter; + } + + /** + * Fetch columns from filter list containing exists and multivalue equality + * filters. This is done to fetch only required columns from back-end and + * then match event filters or relationships in reader. + * + * @param filterList filter list. + * @return set of columns. + */ + public static Set fetchColumnsFromFilterList( + TimelineFilterList filterList) { + Set strSet = new HashSet(); + for (TimelineFilter filter : filterList.getFilterList()) { + switch(filter.getFilterType()) { + case LIST: + strSet.addAll(fetchColumnsFromFilterList((TimelineFilterList)filter)); + break; + case KEY_VALUES: + strSet.add(((TimelineKeyValuesFilter)filter).getKey()); + break; + case EXISTS: + strSet.add(((TimelineExistsFilter)filter).getValue()); + break; + default: + LOG.info("Unexpected filter type " + filter.getFilterType()); + break; + } + } + return strSet; + } + + /** + * Creates equivalent HBase {@link FilterList} from {@link TimelineFilterList} + * while converting different timeline filters(of type {@link TimelineFilter}) + * into their equivalent HBase filters. + * + * @param Describes the type of column prefix. + * @param colPrefix column prefix which will be used for conversion. + * @param filterList timeline filter list which has to be converted. + * @return A {@link FilterList} object. + * @throws IOException if any problem occurs while creating the filter list. + */ + public static FilterList createHBaseFilterList(ColumnPrefix colPrefix, + TimelineFilterList filterList) throws IOException { + FilterList list = + new FilterList(getHBaseOperator(filterList.getOperator())); + for (TimelineFilter filter : filterList.getFilterList()) { + switch(filter.getFilterType()) { + case LIST: + list.addFilter(createHBaseFilterList(colPrefix, + (TimelineFilterList)filter)); + break; + case PREFIX: + list.addFilter(createHBaseColQualPrefixFilter(colPrefix, + (TimelinePrefixFilter)filter)); + break; + case COMPARE: + TimelineCompareFilter compareFilter = (TimelineCompareFilter)filter; + list.addFilter( + createHBaseSingleColValueFilter( + colPrefix.getColumnFamilyBytes(), + colPrefix.getColumnPrefixBytes(compareFilter.getKey()), + colPrefix.getValueConverter(). + encodeValue(compareFilter.getValue()), + getHBaseCompareOp(compareFilter.getCompareOp()), + compareFilter.getKeyMustExist())); + break; + case KEY_VALUE: + TimelineKeyValueFilter kvFilter = (TimelineKeyValueFilter)filter; + list.addFilter( + createHBaseSingleColValueFilter( + colPrefix.getColumnFamilyBytes(), + colPrefix.getColumnPrefixBytes(kvFilter.getKey()), + colPrefix.getValueConverter().encodeValue(kvFilter.getValue()), + getHBaseCompareOp(kvFilter.getCompareOp()), + kvFilter.getKeyMustExist())); + break; + default: + LOG.info("Unexpected filter type " + filter.getFilterType()); + break; + } + } + return list; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java new file mode 100644 index 0000000..116509a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.reader contains classes + * which can be used across reader. This package contains classes which are + * not related to storage implementations. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java new file mode 100644 index 0000000..a384a84 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage; + + +import java.io.IOException; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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; +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.reader.TimelineEntityReader; +import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory; + +/** + * HBase based implementation for {@link TimelineReader}. + */ +public class HBaseTimelineReaderImpl + extends AbstractService implements TimelineReader { + + private static final Log LOG = LogFactory + .getLog(HBaseTimelineReaderImpl.class); + + private Configuration hbaseConf = null; + private Connection conn; + + public HBaseTimelineReaderImpl() { + super(HBaseTimelineReaderImpl.class.getName()); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + hbaseConf = HBaseConfiguration.create(conf); + conn = ConnectionFactory.createConnection(hbaseConf); + } + + @Override + protected void serviceStop() throws Exception { + if (conn != null) { + LOG.info("closing the hbase Connection"); + conn.close(); + } + super.serviceStop(); + } + + @Override + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + TimelineEntityReader reader = + TimelineEntityReaderFactory.createSingleEntityReader(context, + dataToRetrieve); + return reader.readEntity(hbaseConf, conn); + } + + @Override + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + TimelineEntityReader reader = + TimelineEntityReaderFactory.createMultipleEntitiesReader(context, + filters, dataToRetrieve); + return reader.readEntities(hbaseConf, conn); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java new file mode 100644 index 0000000..b94b85f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java @@ -0,0 +1,566 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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; +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.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +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.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.TypedBufferedMutator; +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.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +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; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; + +/** + * This implements a hbase based backend for storing the timeline entity + * information. + * It writes to multiple tables at the backend + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class HBaseTimelineWriterImpl extends AbstractService implements + TimelineWriter { + + private static final Log LOG = LogFactory + .getLog(HBaseTimelineWriterImpl.class); + + private Connection conn; + private TypedBufferedMutator entityTable; + private TypedBufferedMutator appToFlowTable; + private TypedBufferedMutator applicationTable; + private TypedBufferedMutator flowActivityTable; + private TypedBufferedMutator flowRunTable; + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + /** + * Used to convert Long key components to and from storage format. + */ + private final KeyConverter longKeyConverter = new LongKeyConverter(); + + public HBaseTimelineWriterImpl() { + super(HBaseTimelineWriterImpl.class.getName()); + } + + /** + * initializes the hbase connection to write to the entity table. + */ + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + Configuration hbaseConf = HBaseConfiguration.create(conf); + conn = ConnectionFactory.createConnection(hbaseConf); + entityTable = new EntityTable().getTableMutator(hbaseConf, conn); + appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn); + applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn); + flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn); + flowActivityTable = + new FlowActivityTable().getTableMutator(hbaseConf, conn); + } + + /** + * Stores the entire information in TimelineEntities to the timeline store. + */ + @Override + public TimelineWriteResponse write(String clusterId, String userId, + String flowName, String flowVersion, long flowRunId, String appId, + 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 + if (te == null) { + continue; + } + + // if the entity is the application, the destination is the application + // table + boolean isApplication = isApplicationEntity(te); + byte[] rowKey; + if (isApplication) { + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(clusterId, userId, flowName, flowRunId, + appId); + rowKey = applicationRowKey.getRowKey(); + } else { + EntityRowKey entityRowKey = + new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + te.getType(), te.getId()); + rowKey = entityRowKey.getRowKey(); + } + + storeInfo(rowKey, te, flowVersion, isApplication); + storeEvents(rowKey, te.getEvents(), isApplication); + storeConfig(rowKey, te.getConfigs(), isApplication); + storeMetrics(rowKey, te.getMetrics(), isApplication); + storeRelations(rowKey, te, isApplication); + + if (isApplication) { + TimelineEvent event = + getApplicationEvent(te, + ApplicationMetricsConstants.CREATED_EVENT_TYPE); + FlowRunRowKey flowRunRowKey = + new FlowRunRowKey(clusterId, userId, flowName, flowRunId); + if (event != null) { + AppToFlowRowKey appToFlowRowKey = + new AppToFlowRowKey(clusterId, appId); + onApplicationCreated(flowRunRowKey, appToFlowRowKey, appId, userId, + flowVersion, te, event.getTimestamp()); + } + // if it's an application entity, store metrics + storeFlowMetricsAppRunning(flowRunRowKey, appId, te); + // if application has finished, store it's finish time and write final + // values of all metrics + event = getApplicationEvent(te, + ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + if (event != null) { + onApplicationFinished(flowRunRowKey, flowVersion, appId, te, + event.getTimestamp()); + } + } + } + return putStatus; + } + + private void onApplicationCreated(FlowRunRowKey flowRunRowKey, + AppToFlowRowKey appToFlowRowKey, String appId, String userId, + String flowVersion, TimelineEntity te, long appCreatedTimeStamp) + throws IOException { + + String flowName = flowRunRowKey.getFlowName(); + Long flowRunId = flowRunRowKey.getFlowRunId(); + + // store in App to flow table + byte[] rowKey = appToFlowRowKey.getRowKey(); + AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName); + AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId); + AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId); + + // store in flow run table + storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te); + + // store in flow activity table + byte[] flowActivityRowKeyBytes = + new FlowActivityRowKey(flowRunRowKey.getClusterId(), + appCreatedTimeStamp, flowRunRowKey.getUserId(), flowName) + .getRowKey(); + byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); + FlowActivityColumnPrefix.RUN_ID.store(flowActivityRowKeyBytes, + flowActivityTable, qualifier, null, flowVersion, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + /* + * updates the {@link FlowRunTable} with Application Created information + */ + private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te) throws IOException { + byte[] rowKey = flowRunRowKey.getRowKey(); + FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null, + te.getCreatedTime(), + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + + /* + * updates the {@link FlowRunTable} and {@link FlowActivityTable} when an + * application has finished + */ + private void onApplicationFinished(FlowRunRowKey flowRunRowKey, + String flowVersion, String appId, TimelineEntity te, + long appFinishedTimeStamp) throws IOException { + // store in flow run table + storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te, + appFinishedTimeStamp); + + // indicate in the flow activity table that the app has finished + byte[] rowKey = + new FlowActivityRowKey(flowRunRowKey.getClusterId(), + appFinishedTimeStamp, flowRunRowKey.getUserId(), + flowRunRowKey.getFlowName()).getRowKey(); + byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); + FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier, + null, flowVersion, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + /* + * Update the {@link FlowRunTable} with Application Finished information + */ + private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te, long appFinishedTimeStamp) + throws IOException { + byte[] rowKey = flowRunRowKey.getRowKey(); + Attribute attributeAppId = + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId); + FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null, + appFinishedTimeStamp, attributeAppId); + + // store the final value of metrics since application has finished + Set metrics = te.getMetrics(); + if (metrics != null) { + storeFlowMetrics(rowKey, metrics, attributeAppId, + AggregationOperation.SUM_FINAL.getAttribute()); + } + } + + /* + * Updates the {@link FlowRunTable} with Application Metrics + */ + private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te) throws IOException { + Set metrics = te.getMetrics(); + if (metrics != null) { + byte[] rowKey = flowRunRowKey.getRowKey(); + storeFlowMetrics(rowKey, metrics, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId), + AggregationOperation.SUM.getAttribute()); + } + } + + private void storeFlowMetrics(byte[] rowKey, Set metrics, + Attribute... attributes) throws IOException { + for (TimelineMetric metric : metrics) { + byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId()); + Map timeseries = metric.getValues(); + for (Map.Entry timeseriesEntry : timeseries.entrySet()) { + Long timestamp = timeseriesEntry.getKey(); + FlowRunColumnPrefix.METRIC.store(rowKey, flowRunTable, + metricColumnQualifier, timestamp, timeseriesEntry.getValue(), + attributes); + } + } + } + + private void storeRelations(byte[] rowKey, TimelineEntity te, + boolean isApplication) throws IOException { + if (isApplication) { + storeRelations(rowKey, te.getIsRelatedToEntities(), + ApplicationColumnPrefix.IS_RELATED_TO, applicationTable); + storeRelations(rowKey, te.getRelatesToEntities(), + ApplicationColumnPrefix.RELATES_TO, applicationTable); + } else { + storeRelations(rowKey, te.getIsRelatedToEntities(), + EntityColumnPrefix.IS_RELATED_TO, entityTable); + storeRelations(rowKey, te.getRelatesToEntities(), + EntityColumnPrefix.RELATES_TO, entityTable); + } + } + + /** + * Stores the Relations from the {@linkplain TimelineEntity} object. + */ + private void storeRelations(byte[] rowKey, + Map> connectedEntities, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + for (Map.Entry> connectedEntity : connectedEntities + .entrySet()) { + // id3?id4?id5 + String compoundValue = + Separator.VALUES.joinEncoded(connectedEntity.getValue()); + columnPrefix.store(rowKey, table, + stringKeyConverter.encode(connectedEntity.getKey()), null, + compoundValue); + } + } + + /** + * Stores information from the {@linkplain TimelineEntity} object. + */ + private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion, + boolean isApplication) throws IOException { + + if (isApplication) { + ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId()); + ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null, + te.getCreatedTime()); + ApplicationColumn.FLOW_VERSION.store(rowKey, applicationTable, null, + flowVersion); + Map info = te.getInfo(); + if (info != null) { + for (Map.Entry entry : info.entrySet()) { + ApplicationColumnPrefix.INFO.store(rowKey, applicationTable, + stringKeyConverter.encode(entry.getKey()), null, + entry.getValue()); + } + } + } else { + EntityColumn.ID.store(rowKey, entityTable, null, te.getId()); + EntityColumn.TYPE.store(rowKey, entityTable, null, te.getType()); + EntityColumn.CREATED_TIME.store(rowKey, entityTable, null, + te.getCreatedTime()); + EntityColumn.FLOW_VERSION.store(rowKey, entityTable, null, flowVersion); + Map info = te.getInfo(); + if (info != null) { + for (Map.Entry entry : info.entrySet()) { + EntityColumnPrefix.INFO.store(rowKey, entityTable, + stringKeyConverter.encode(entry.getKey()), null, + entry.getValue()); + } + } + } + } + + /** + * stores the config information from {@linkplain TimelineEntity}. + */ + private void storeConfig(byte[] rowKey, Map config, + boolean isApplication) throws IOException { + if (config == null) { + return; + } + for (Map.Entry entry : config.entrySet()) { + byte[] configKey = stringKeyConverter.encode(entry.getKey()); + if (isApplication) { + ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable, + configKey, null, entry.getValue()); + } else { + EntityColumnPrefix.CONFIG.store(rowKey, entityTable, configKey, + null, entry.getValue()); + } + } + } + + /** + * stores the {@linkplain TimelineMetric} information from the + * {@linkplain TimelineEvent} object. + */ + private void storeMetrics(byte[] rowKey, Set metrics, + boolean isApplication) throws IOException { + if (metrics != null) { + for (TimelineMetric metric : metrics) { + byte[] metricColumnQualifier = + stringKeyConverter.encode(metric.getId()); + Map timeseries = metric.getValues(); + for (Map.Entry timeseriesEntry : timeseries.entrySet()) { + Long timestamp = timeseriesEntry.getKey(); + if (isApplication) { + ApplicationColumnPrefix.METRIC.store(rowKey, applicationTable, + metricColumnQualifier, timestamp, timeseriesEntry.getValue()); + } else { + EntityColumnPrefix.METRIC.store(rowKey, entityTable, + metricColumnQualifier, timestamp, timeseriesEntry.getValue()); + } + } + } + } + } + + /** + * Stores the events from the {@linkplain TimelineEvent} object. + */ + private void storeEvents(byte[] rowKey, Set events, + boolean isApplication) throws IOException { + if (events != null) { + for (TimelineEvent event : events) { + if (event != null) { + String eventId = event.getId(); + if (eventId != null) { + long eventTimestamp = event.getTimestamp(); + // if the timestamp is not set, use the current timestamp + if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) { + LOG.warn("timestamp is not set for event " + eventId + + "! Using the current timestamp"); + eventTimestamp = System.currentTimeMillis(); + } + Map eventInfo = event.getInfo(); + if ((eventInfo == null) || (eventInfo.size() == 0)) { + byte[] columnQualifierBytes = + new EventColumnName(eventId, eventTimestamp, null) + .getColumnQualifier(); + if (isApplication) { + ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable, + columnQualifierBytes, null, Separator.EMPTY_BYTES); + } else { + EntityColumnPrefix.EVENT.store(rowKey, entityTable, + columnQualifierBytes, null, Separator.EMPTY_BYTES); + } + } else { + for (Map.Entry info : eventInfo.entrySet()) { + // eventId=infoKey + byte[] columnQualifierBytes = + new EventColumnName(eventId, eventTimestamp, info.getKey()) + .getColumnQualifier(); + if (isApplication) { + ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable, + columnQualifierBytes, null, info.getValue()); + } else { + EntityColumnPrefix.EVENT.store(rowKey, entityTable, + columnQualifierBytes, null, info.getValue()); + } + } // for info: eventInfo + } + } + } + } // event : events + } + } + + /** + * Checks if the input TimelineEntity object is an ApplicationEntity. + * + * @param te TimelineEntity object. + * @return true if input is an ApplicationEntity, false otherwise + */ + static boolean isApplicationEntity(TimelineEntity te) { + return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString()); + } + + /** + * @param te TimelineEntity object. + * @param eventId event with this id needs to be fetched + * @return TimelineEvent if TimelineEntity contains the desired event. + */ + private static TimelineEvent getApplicationEvent(TimelineEntity te, + String eventId) { + if (isApplicationEntity(te)) { + for (TimelineEvent event : te.getEvents()) { + if (event.getId().equals(eventId)) { + return event; + } + } + } + return null; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage + * .TimelineWriter#aggregate + * (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity, + * org.apache + * .hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack) + */ + @Override + public TimelineWriteResponse aggregate(TimelineEntity data, + TimelineAggregationTrack track) throws IOException { + return null; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter#flush + * () + */ + @Override + public void flush() throws IOException { + // flush all buffered mutators + entityTable.flush(); + appToFlowTable.flush(); + applicationTable.flush(); + flowRunTable.flush(); + flowActivityTable.flush(); + } + + /** + * close the hbase connections The close APIs perform flushing and release any + * resources held. + */ + @Override + protected void serviceStop() throws Exception { + if (entityTable != null) { + LOG.info("closing the entity table"); + // The close API performs flushing and releases any resources held + entityTable.close(); + } + if (appToFlowTable != null) { + LOG.info("closing the app_flow table"); + // The close API performs flushing and releases any resources held + appToFlowTable.close(); + } + if (applicationTable != null) { + LOG.info("closing the application table"); + applicationTable.close(); + } + if (flowRunTable != null) { + LOG.info("closing the flow run table"); + // The close API performs flushing and releases any resources held + flowRunTable.close(); + } + if (flowActivityTable != null) { + LOG.info("closing the flowActivityTable table"); + // The close API performs flushing and releases any resources held + flowActivityTable.close(); + } + if (conn != null) { + LOG.info("closing the hbase Connection"); + conn.close(); + } + super.serviceStop(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java new file mode 100644 index 0000000..130cb6c --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java @@ -0,0 +1,358 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +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.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.phoenix.util.PropertiesUtil; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** + * Offline aggregation Phoenix storage. This storage currently consists of two + * aggregation tables, one for flow level aggregation and one for user level + * aggregation. + * + * Example table record: + * + *
+ * |---------------------------|
+ * |  Primary   | Column Family|
+ * |  key       | metrics      |
+ * |---------------------------|
+ * | row_key    | metricId1:   |
+ * |            | metricValue1 |
+ * |            | @timestamp1  |
+ * |            |              |
+ * |            | metriciD1:   |
+ * |            | metricValue2 |
+ * |            | @timestamp2  |
+ * |            |              |
+ * |            | metricId2:   |
+ * |            | metricValue1 |
+ * |            | @timestamp2  |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |---------------------------|
+ * 
+ * + * For the flow aggregation table, the primary key contains user, cluster, and + * flow id. For user aggregation table,the primary key is user. + * + * Metrics column family stores all aggregated metrics for each record. + */ +@Private +@Unstable +public class PhoenixOfflineAggregationWriterImpl + extends OfflineAggregationWriter { + + private static final Log LOG + = LogFactory.getLog(PhoenixOfflineAggregationWriterImpl.class); + private static final String PHOENIX_COL_FAMILY_PLACE_HOLDER + = "timeline_cf_placeholder"; + + /** Default Phoenix JDBC driver name. */ + private static final String DRIVER_CLASS_NAME + = "org.apache.phoenix.jdbc.PhoenixDriver"; + + /** Default Phoenix timeline config column family. */ + private static final String METRIC_COLUMN_FAMILY = "m."; + /** Default Phoenix timeline info column family. */ + private static final String INFO_COLUMN_FAMILY = "i."; + /** Default separator for Phoenix storage. */ + private static final String AGGREGATION_STORAGE_SEPARATOR = ";"; + + /** Connection string to the deployed Phoenix cluster. */ + private String connString = null; + private Properties connProperties = new Properties(); + + public PhoenixOfflineAggregationWriterImpl(Properties prop) { + super(PhoenixOfflineAggregationWriterImpl.class.getName()); + connProperties = PropertiesUtil.deepCopy(prop); + } + + public PhoenixOfflineAggregationWriterImpl() { + super(PhoenixOfflineAggregationWriterImpl.class.getName()); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + Class.forName(DRIVER_CLASS_NAME); + // so check it here and only read in the config if it's not overridden. + connString = + conf.get(YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR, + YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT); + super.init(conf); + } + + @Override + public TimelineWriteResponse writeAggregatedEntity( + TimelineCollectorContext context, TimelineEntities entities, + OfflineAggregationInfo info) throws IOException { + TimelineWriteResponse response = new TimelineWriteResponse(); + String sql = "UPSERT INTO " + info.getTableName() + + " (" + StringUtils.join(info.getPrimaryKeyList(), ",") + + ", created_time, metric_names) " + + "VALUES (" + + StringUtils.repeat("?,", info.getPrimaryKeyList().length) + + "?, ?)"; + if (LOG.isDebugEnabled()) { + LOG.debug("TimelineEntity write SQL: " + sql); + } + + try (Connection conn = getConnection(); + PreparedStatement ps = conn.prepareStatement(sql)) { + for (TimelineEntity entity : entities.getEntities()) { + HashMap formattedMetrics = new HashMap<>(); + if (entity.getMetrics() != null) { + for (TimelineMetric m : entity.getMetrics()) { + formattedMetrics.put(m.getId(), m); + } + } + int idx = info.setStringsForPrimaryKey(ps, context, null, 1); + ps.setLong(idx++, entity.getCreatedTime()); + ps.setString(idx++, + StringUtils.join(formattedMetrics.keySet().toArray(), + AGGREGATION_STORAGE_SEPARATOR)); + ps.execute(); + + storeEntityVariableLengthFields(entity, formattedMetrics, context, conn, + info); + + conn.commit(); + } + } catch (SQLException se) { + LOG.error("Failed to add entity to Phoenix " + se.getMessage()); + throw new IOException(se); + } catch (Exception e) { + LOG.error("Exception on getting connection: " + e.getMessage()); + throw new IOException(e); + } + return response; + } + + /** + * Create Phoenix tables for offline aggregation storage if the tables do not + * exist. + * + * @throws IOException if any problem happens while creating Phoenix tables. + */ + public void createPhoenixTables() throws IOException { + // Create tables if necessary + try (Connection conn = getConnection(); + Statement stmt = conn.createStatement()) { + // Table schema defined as in YARN-3817. + String sql = "CREATE TABLE IF NOT EXISTS " + + OfflineAggregationInfo.FLOW_AGGREGATION_TABLE_NAME + + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, " + + "flow_name VARCHAR NOT NULL, " + + "created_time UNSIGNED_LONG, " + + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + + " VARBINARY, " + + "metric_names VARCHAR, info_keys VARCHAR " + + "CONSTRAINT pk PRIMARY KEY(" + + "user, cluster, flow_name))"; + stmt.executeUpdate(sql); + sql = "CREATE TABLE IF NOT EXISTS " + + OfflineAggregationInfo.USER_AGGREGATION_TABLE_NAME + + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, " + + "created_time UNSIGNED_LONG, " + + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + + " VARBINARY, " + + "metric_names VARCHAR, info_keys VARCHAR " + + "CONSTRAINT pk PRIMARY KEY(user, cluster))"; + stmt.executeUpdate(sql); + conn.commit(); + } catch (SQLException se) { + LOG.error("Failed in init data " + se.getLocalizedMessage()); + throw new IOException(se); + } + return; + } + + // Utility functions + @Private + @VisibleForTesting + Connection getConnection() throws IOException { + Connection conn; + try { + conn = DriverManager.getConnection(connString, connProperties); + conn.setAutoCommit(false); + } catch (SQLException se) { + LOG.error("Failed to connect to phoenix server! " + + se.getLocalizedMessage()); + throw new IOException(se); + } + return conn; + } + + // WARNING: This method will permanently drop a table! + @Private + @VisibleForTesting + void dropTable(String tableName) throws Exception { + try (Connection conn = getConnection(); + Statement stmt = conn.createStatement()) { + String sql = "DROP TABLE " + tableName; + stmt.executeUpdate(sql); + } catch (SQLException se) { + LOG.error("Failed in dropping entity table " + se.getLocalizedMessage()); + throw se; + } + } + + private static class DynamicColumns { + static final String COLUMN_FAMILY_TYPE_BYTES = " VARBINARY"; + static final String COLUMN_FAMILY_TYPE_STRING = " VARCHAR"; + private String columnFamilyPrefix; + private String type; + private Set columns; + + public DynamicColumns(String columnFamilyPrefix, String type, + Set keyValues) { + this.columnFamilyPrefix = columnFamilyPrefix; + this.columns = keyValues; + this.type = type; + } + } + + private static StringBuilder appendColumnsSQL( + StringBuilder colNames, DynamicColumns cfInfo) { + // Prepare the sql template by iterating through all keys + for (K key : cfInfo.columns) { + colNames.append(",").append(cfInfo.columnFamilyPrefix) + .append(key.toString()).append(cfInfo.type); + } + return colNames; + } + + private static int setValuesForColumnFamily( + PreparedStatement ps, Map keyValues, int startPos, + boolean converToBytes) throws SQLException { + int idx = startPos; + for (Map.Entry entry : keyValues.entrySet()) { + V value = entry.getValue(); + if (value instanceof Collection) { + ps.setString(idx++, StringUtils.join( + (Collection) value, AGGREGATION_STORAGE_SEPARATOR)); + } else { + if (converToBytes) { + try { + ps.setBytes(idx++, GenericObjectMapper.write(entry.getValue())); + } catch (IOException ie) { + LOG.error("Exception in converting values into bytes " + + ie.getMessage()); + throw new SQLException(ie); + } + } else { + ps.setString(idx++, value.toString()); + } + } + } + return idx; + } + + private static int setBytesForColumnFamily( + PreparedStatement ps, Map keyValues, int startPos) + throws SQLException { + return setValuesForColumnFamily(ps, keyValues, startPos, true); + } + + private static int setStringsForColumnFamily( + PreparedStatement ps, Map keyValues, int startPos) + throws SQLException { + return setValuesForColumnFamily(ps, keyValues, startPos, false); + } + + private static void storeEntityVariableLengthFields(TimelineEntity entity, + Map formattedMetrics, + TimelineCollectorContext context, Connection conn, + OfflineAggregationInfo aggregationInfo) throws SQLException { + int numPlaceholders = 0; + StringBuilder columnDefs = new StringBuilder( + StringUtils.join(aggregationInfo.getPrimaryKeyList(), ",")); + if (formattedMetrics != null && formattedMetrics.size() > 0) { + appendColumnsSQL(columnDefs, new DynamicColumns<>( + METRIC_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_BYTES, + formattedMetrics.keySet())); + numPlaceholders += formattedMetrics.keySet().size(); + } + if (numPlaceholders == 0) { + return; + } + StringBuilder placeholders = new StringBuilder(); + placeholders.append( + StringUtils.repeat("?,", aggregationInfo.getPrimaryKeyList().length)); + // numPlaceholders >= 1 now + placeholders.append("?") + .append(StringUtils.repeat(",?", numPlaceholders - 1)); + String sqlVariableLengthFields = new StringBuilder("UPSERT INTO ") + .append(aggregationInfo.getTableName()).append(" (").append(columnDefs) + .append(") VALUES(").append(placeholders).append(")").toString(); + if (LOG.isDebugEnabled()) { + LOG.debug("SQL statement for variable length fields: " + + sqlVariableLengthFields); + } + // Use try with resource statement for the prepared statement + try (PreparedStatement psVariableLengthFields = + conn.prepareStatement(sqlVariableLengthFields)) { + int idx = aggregationInfo.setStringsForPrimaryKey( + psVariableLengthFields, context, null, 1); + if (formattedMetrics != null && formattedMetrics.size() > 0) { + idx = setBytesForColumnFamily( + psVariableLengthFields, formattedMetrics, idx); + } + psVariableLengthFields.execute(); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java new file mode 100644 index 0000000..33f5449 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java @@ -0,0 +1,272 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.entity.EntityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; + +import com.google.common.annotations.VisibleForTesting; + +/** + * This creates the schema for a hbase based backend for storing application + * timeline information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class TimelineSchemaCreator { + private TimelineSchemaCreator() { + } + + final static String NAME = TimelineSchemaCreator.class.getSimpleName(); + private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class); + private static final String PHOENIX_OPTION_SHORT = "p"; + private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s"; + private static final String APP_TABLE_NAME_SHORT = "a"; + private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f"; + private static final String TTL_OPTION_SHORT = "m"; + private static final String ENTITY_TABLE_NAME_SHORT = "e"; + + public static void main(String[] args) throws Exception { + + Configuration hbaseConf = HBaseConfiguration.create(); + // Grab input args and allow for -Dxyz style arguments + String[] otherArgs = new GenericOptionsParser(hbaseConf, args) + .getRemainingArgs(); + + // Grab the arguments we're looking for. + CommandLine commandLine = parseArgs(otherArgs); + + // Grab the entityTableName argument + String entityTableName + = commandLine.getOptionValue(ENTITY_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(entityTableName)) { + hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName); + } + String entityTableTTLMetrics = commandLine.getOptionValue(TTL_OPTION_SHORT); + if (StringUtils.isNotBlank(entityTableTTLMetrics)) { + int metricsTTL = Integer.parseInt(entityTableTTLMetrics); + new EntityTable().setMetricsTTL(metricsTTL, hbaseConf); + } + // Grab the appToflowTableName argument + String appToflowTableName = commandLine.getOptionValue( + APP_TO_FLOW_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(appToflowTableName)) { + hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName); + } + // Grab the applicationTableName argument + String applicationTableName = commandLine.getOptionValue( + APP_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(applicationTableName)) { + hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME, + applicationTableName); + } + + List exceptions = new ArrayList<>(); + try { + boolean skipExisting + = commandLine.hasOption(SKIP_EXISTING_TABLE_OPTION_SHORT); + if (skipExisting) { + LOG.info("Will skip existing tables and continue on htable creation " + + "exceptions!"); + } + createAllTables(hbaseConf, skipExisting); + LOG.info("Successfully created HBase schema. "); + } catch (IOException e) { + LOG.error("Error in creating hbase tables: " + e.getMessage()); + exceptions.add(e); + } + + // Create Phoenix data schema if needed + if (commandLine.hasOption(PHOENIX_OPTION_SHORT)) { + Configuration phoenixConf = new Configuration(); + try { + PhoenixOfflineAggregationWriterImpl phoenixWriter = + new PhoenixOfflineAggregationWriterImpl(); + phoenixWriter.init(phoenixConf); + phoenixWriter.start(); + phoenixWriter.createPhoenixTables(); + phoenixWriter.stop(); + LOG.info("Successfully created Phoenix offline aggregation schema. "); + } catch (IOException e) { + LOG.error("Error in creating phoenix tables: " + e.getMessage()); + exceptions.add(e); + } + } + if (exceptions.size() > 0) { + LOG.warn("Schema creation finished with the following exceptions"); + for (Exception e : exceptions) { + LOG.warn(e.getMessage()); + } + System.exit(-1); + } else { + LOG.info("Schema creation finished successfully"); + } + } + + /** + * Parse command-line arguments. + * + * @param args + * command line arguments passed to program. + * @return parsed command line. + * @throws ParseException + */ + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + + // Input + Option o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true, + "entity table name"); + o.setArgName("entityTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(TTL_OPTION_SHORT, "metricsTTL", true, + "TTL for metrics column family"); + o.setArgName("metricsTTL"); + o.setRequired(false); + options.addOption(o); + + o = new Option(APP_TO_FLOW_TABLE_NAME_SHORT, "appToflowTableName", true, + "app to flow table name"); + o.setArgName("appToflowTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(APP_TABLE_NAME_SHORT, "applicationTableName", true, + "application table name"); + o.setArgName("applicationTableName"); + o.setRequired(false); + options.addOption(o); + + // Options without an argument + // No need to set arg name since we do not need an argument here + o = new Option(PHOENIX_OPTION_SHORT, "usePhoenix", false, + "create Phoenix offline aggregation tables"); + o.setRequired(false); + options.addOption(o); + + o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable", + false, "skip existing Hbase tables and continue to create new tables"); + o.setRequired(false); + options.addOption(o); + + CommandLineParser parser = new PosixParser(); + CommandLine commandLine = null; + try { + commandLine = parser.parse(options, args); + } catch (Exception e) { + LOG.error("ERROR: " + e.getMessage() + "\n"); + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(NAME + " ", options, true); + System.exit(-1); + } + + return commandLine; + } + + @VisibleForTesting + public static void createAllTables(Configuration hbaseConf, + boolean skipExisting) throws IOException { + + Connection conn = null; + try { + conn = ConnectionFactory.createConnection(hbaseConf); + Admin admin = conn.getAdmin(); + if (admin == null) { + throw new IOException("Cannot create table since admin is null"); + } + try { + new EntityTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new AppToFlowTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new ApplicationTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new FlowRunTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new FlowActivityTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + } finally { + if (conn != null) { + conn.close(); + } + } + } + + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java new file mode 100644 index 0000000..dde3911 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link ApplicationTable}. + */ +public enum ApplicationColumn implements Column { + + /** + * App id. + */ + ID(ApplicationColumnFamily.INFO, "id"), + + /** + * When the application was created. + */ + CREATED_TIME(ApplicationColumnFamily.INFO, "created_time", + new LongConverter()), + + /** + * The version of the flow that this app belongs to. + */ + FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + private ApplicationColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + private ApplicationColumn(ColumnFamily columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /** + * Retrieve an {@link ApplicationColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)}. + * + * @param columnQualifier Name of the column to retrieve + * @return the corresponding {@link ApplicationColumn} or null + */ + public static final ApplicationColumn columnFor(String columnQualifier) { + + // Match column based on value, assume column family matches. + for (ApplicationColumn ac : ApplicationColumn.values()) { + // Find a match based only on name. + if (ac.getColumnQualifier().equals(columnQualifier)) { + return ac; + } + } + + // Default to null + return null; + } + + /** + * Retrieve an {@link ApplicationColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} + * if and only if {@code a.equals(b) & x.equals(y)} or + * {@code (x == y == null)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param name Name of the column to retrieve + * @return the corresponding {@link ApplicationColumn} or null if both + * arguments don't match. + */ + public static final ApplicationColumn columnFor( + ApplicationColumnFamily columnFamily, String name) { + + for (ApplicationColumn ac : ApplicationColumn.values()) { + // Find a match based column family and on name. + if (ac.columnFamily.equals(columnFamily) + && ac.getColumnQualifier().equals(name)) { + return ac; + } + } + + // Default to null + return null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java new file mode 100644 index 0000000..97e5f7b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the application table column families. + */ +public enum ApplicationColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: a) the size + * of the config values can be very large and b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + private ApplicationColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java new file mode 100644 index 0000000..42488f4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java @@ -0,0 +1,288 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the application table. + */ +public enum ApplicationColumnPrefix implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(ApplicationColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(ApplicationColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an application. + */ + EVENT(ApplicationColumnFamily.INFO, "e"), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(ApplicationColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + private ApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, GenericConverter.getInstance()); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + private ApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, ValueConverter converter) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + private String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute...attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /** + * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there + * is no match. The following holds true: {@code columnFor(x) == columnFor(y)} + * if and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link ApplicationColumnPrefix} or null + */ + public static final ApplicationColumnPrefix columnFor(String columnPrefix) { + + // Match column based on value, assume column family matches. + for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) { + // Find a match based only on name. + if (acp.getColumnPrefix().equals(columnPrefix)) { + return acp; + } + } + + // Default to null + return null; + } + + /** + * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there + * is no match. The following holds true: + * {@code columnFor(a,x) == columnFor(b,y)} if and only if + * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link ApplicationColumnPrefix} or null if both + * arguments don't match. + */ + public static final ApplicationColumnPrefix columnFor( + ApplicationColumnFamily columnFamily, String columnPrefix) { + + // TODO: needs unit test to confirm and need to update javadoc to explain + // null prefix case. + + for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) { + // Find a match based column family and on name. + if (acp.columnFamily.equals(columnFamily) + && (((columnPrefix == null) && (acp.getColumnPrefix() == null)) || + (acp.getColumnPrefix().equals(columnPrefix)))) { + return acp; + } + } + + // Default to null + return null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java new file mode 100644 index 0000000..da62fdf --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java @@ -0,0 +1,206 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the application table. + */ +public class ApplicationRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final String appId; + private final KeyConverter appRowKeyConverter = + new ApplicationRowKeyConverter(); + + public ApplicationRowKey(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public String getAppId() { + return appId; + } + + /** + * Constructs a row key for the application table as follows: + * {@code clusterId!userName!flowName!flowRunId!AppId}. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return appRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey Byte representation of row key. + * @return An ApplicationRowKey object. + */ + public static ApplicationRowKey parseRowKey(byte[] rowKey) { + return new ApplicationRowKeyConverter().decode(rowKey); + } + + /** + * Encodes and decodes row key for application table. The row key is of the + * form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long, + * appId is encoded and decoded using {@link AppIdKeyConverter} and rest are + * strings. + *

+ */ + final private static class ApplicationRowKeyConverter implements + KeyConverter { + + private final KeyConverter appIDKeyConverter = + new AppIdKeyConverter(); + + /** + * Intended for use in ApplicationRowKey only. + */ + private ApplicationRowKeyConverter() { + } + + /** + * Application row key is of the form + * clusterId!userName!flowName!flowRunId!appId with each segment separated + * by !. The sizes below indicate sizes of each one of these segements in + * sequence. clusterId, userName and flowName are strings. flowrunId is a + * long hence 8 bytes in size. app id is represented as 12 bytes with + * cluster timestamp part of appid takes 8 bytes(long) and seq id takes 4 + * bytes(int). Strings are variable in size (i.e. end whenever separator is + * encountered). This is used while decoding and helps in determining where + * to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize() }; + + /* + * (non-Javadoc) + * + * Encodes ApplicationRowKey object into a byte array with each + * component/field in ApplicationRowKey separated by Separator#QUALIFIERS. + * This leads to an application table row key of the form + * clusterId!userName!flowName!flowRunId!appId If flowRunId in passed + * ApplicationRowKey object is null (and the fields preceding it i.e. + * clusterId, userId and flowName are not null), this returns a row key + * prefix of the form clusterId!userName!flowName! and if appId in + * ApplicationRowKey is null (other 4 components all are not null), this + * returns a row key prefix of the form + * clusterId!userName!flowName!flowRunId! flowRunId is inverted while + * encoding as it helps maintain a descending order for row keys in the + * application table. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(ApplicationRowKey rowKey) { + byte[] cluster = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] user = + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] flow = + Separator.encode(rowKey.getFlowName(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(cluster, user, flow); + // Note that flowRunId is a long, so we can't encode them all at the same + // time. + if (rowKey.getFlowRunId() == null) { + return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); + } + byte[] second = + Bytes.toBytes(LongConverter.invertLong( + rowKey.getFlowRunId())); + if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) { + return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES); + } + byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); + return Separator.QUALIFIERS.join(first, second, third); + } + + /* + * (non-Javadoc) + * + * Decodes an application row key of the form + * clusterId!userName!flowName!flowRunId!appId represented in byte format + * and converts it into an ApplicationRowKey object.flowRunId is inverted + * while decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public ApplicationRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 5) { + throw new IllegalArgumentException("the row key is not valid for " + + "an application"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + String appId = appIDKeyConverter.decode(rowKeyComponents[4]); + return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, + appId); + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java new file mode 100644 index 0000000..f61b0e9 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey (without flowName or without flowName and + * flowRunId) for the application table. + */ +public class ApplicationRowKeyPrefix extends ApplicationRowKey implements + RowKeyPrefix { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * application table: {@code clusterId!userName!flowName!}. + * + * @param clusterId the cluster on which applications ran + * @param userId the user that ran applications + * @param flowName the name of the flow that was run by the user on the + * cluster + */ + public ApplicationRowKeyPrefix(String clusterId, String userId, + String flowName) { + super(clusterId, userId, flowName, null, null); + } + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * application table: {@code clusterId!userName!flowName!flowRunId!}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the instance of this flow + */ + public ApplicationRowKeyPrefix(String clusterId, String userId, + String flowName, Long flowRunId) { + super(clusterId, userId, flowName, flowRunId, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + @Override + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java new file mode 100644 index 0000000..a02f768 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; + +/** + * The application table as column families info, config and metrics. Info + * stores information about a YARN application entity, config stores + * configuration data of a YARN application, metrics stores the metrics of a + * YARN application. This table is entirely analogous to the entity table but + * created for better performance. + * + * Example application table record: + * + *

+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
+ * | userName!  |                              | metricValue1 | configValue1 |
+ * | flowName!  | created_time:                | @timestamp1  |              |
+ * | flowRunId! | 1392993084018                |              | configKey2:  |
+ * | AppId      |                              | metriciD1:   | configValue2 |
+ * |            | i!infoKey:                   | metricValue2 |              |
+ * |            | infoValue                    | @timestamp2  |              |
+ * |            |                              |              |              |
+ * |            | r!relatesToKey:              | metricId2:   |              |
+ * |            | id3=id4=id5                  | metricValue1 |              |
+ * |            |                              | @timestamp2  |              |
+ * |            | s!isRelatedToKey:            |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public class ApplicationTable extends BaseTable { + /** application prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".application"; + + /** config param name that specifies the application table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * application table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** default value for application table name. */ + private static final String DEFAULT_TABLE_NAME = + "timelineservice.application"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000; + + private static final Log LOG = LogFactory.getLog(ApplicationTable.class); + + public ApplicationTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor applicationTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + applicationTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + applicationTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes()); + applicationTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + applicationTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(applicationTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metrics in this table. + * @param hbaseConf configuration in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java new file mode 100644 index 0000000..03f508f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.application + * contains classes related to implementation for application table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java new file mode 100644 index 0000000..ff61633 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +import java.io.IOException; + +/** + * Identifies fully qualified columns for the {@link AppToFlowTable}. + */ +public enum AppToFlowColumn implements Column { + + /** + * The flow ID. + */ + FLOW_ID(AppToFlowColumnFamily.MAPPING, "flow_id"), + + /** + * The flow run ID. + */ + FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"), + + /** + * The user. + */ + USER_ID(AppToFlowColumnFamily.MAPPING, "user_id"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + AppToFlowColumn(ColumnFamily columnFamily, + String columnQualifier) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + /** + * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnQualifier Name of the column to retrieve + * @return the corresponding {@link AppToFlowColumn} or null + */ + public static final AppToFlowColumn columnFor(String columnQualifier) { + + // Match column based on value, assume column family matches. + for (AppToFlowColumn ec : AppToFlowColumn.values()) { + // Find a match based only on name. + if (ec.getColumnQualifier().equals(columnQualifier)) { + return ec; + } + } + + // Default to null + return null; + } + + /** + * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} + * if and only if {@code a.equals(b) & x.equals(y)} or + * {@code (x == y == null)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param name Name of the column to retrieve + * @return the corresponding {@link AppToFlowColumn} or null if both arguments + * don't match. + */ + public static final AppToFlowColumn columnFor( + AppToFlowColumnFamily columnFamily, String name) { + + for (AppToFlowColumn ec : AppToFlowColumn.values()) { + // Find a match based column family and on name. + if (ec.columnFamily.equals(columnFamily) + && ec.getColumnQualifier().equals(name)) { + return ec; + } + } + + // Default to null + return null; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java new file mode 100644 index 0000000..f3f045e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the app_flow table column families. + */ +public enum AppToFlowColumnFamily implements ColumnFamily { + /** + * Mapping column family houses known columns such as flowName and flowRunId. + */ + MAPPING("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + AppToFlowColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java new file mode 100644 index 0000000..8df4407 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the app_flow table. + */ +public class AppToFlowRowKey { + private final String clusterId; + private final String appId; + private final KeyConverter appToFlowRowKeyConverter = + new AppToFlowRowKeyConverter(); + + public AppToFlowRowKey(String clusterId, String appId) { + this.clusterId = clusterId; + this.appId = appId; + } + + public String getClusterId() { + return clusterId; + } + + public String getAppId() { + return appId; + } + + /** + * Constructs a row key prefix for the app_flow table as follows: + * {@code clusterId!AppId}. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return appToFlowRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey a rowkey represented as a byte array. + * @return an AppToFlowRowKey object. + */ + public static AppToFlowRowKey parseRowKey(byte[] rowKey) { + return new AppToFlowRowKeyConverter().decode(rowKey); + } + + /** + * Encodes and decodes row key for app_flow table. The row key is of the form + * clusterId!appId. clusterId is a string and appId is encoded/decoded using + * {@link AppIdKeyConverter}. + *

+ */ + final private static class AppToFlowRowKeyConverter implements + KeyConverter { + + private final KeyConverter appIDKeyConverter = + new AppIdKeyConverter(); + + /** + * Intended for use in AppToFlowRowKey only. + */ + private AppToFlowRowKeyConverter() { + } + + + /** + * App to flow row key is of the form clusterId!appId with the 2 segments + * separated by !. The sizes below indicate sizes of both of these segments + * in sequence. clusterId is a string. appId is represented as 12 bytes w. + * cluster Timestamp part of appid taking 8 bytes(long) and seq id taking 4 + * bytes(int). Strings are variable in size (i.e. end whenever separator is + * encountered). This is used while decoding and helps in determining where + * to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT }; + + /* + * (non-Javadoc) + * + * Encodes AppToFlowRowKey object into a byte array with each + * component/field in AppToFlowRowKey separated by Separator#QUALIFIERS. + * This leads to an app to flow table row key of the form clusterId!appId + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(AppToFlowRowKey rowKey) { + byte[] first = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] second = appIDKeyConverter.encode(rowKey.getAppId()); + return Separator.QUALIFIERS.join(first, second); + } + + /* + * (non-Javadoc) + * + * Decodes an app to flow row key of the form clusterId!appId represented + * in byte format and converts it into an AppToFlowRowKey object. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public AppToFlowRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 2) { + throw new IllegalArgumentException("the row key is not valid for " + + "the app-to-flow table"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String appId = appIDKeyConverter.decode(rowKeyComponents[1]); + return new AppToFlowRowKey(clusterId, appId); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java new file mode 100644 index 0000000..301cf99 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; + +import java.io.IOException; + +/** + * The app_flow table as column families mapping. Mapping stores + * appId to flowName and flowRunId mapping information + * + * Example app_flow table record: + * + *

+ * |--------------------------------------|
+ * |  Row       | Column Family           |
+ * |  key       | info                    |
+ * |--------------------------------------|
+ * | clusterId! | flowName:               |
+ * | AppId      | foo@daily_hive_report   |
+ * |            |                         |
+ * |            | flowRunId:              |
+ * |            | 1452828720457           |
+ * |            |                         |
+ * |            | user_id:                |
+ * |            | admin                   |
+ * |            |                         |
+ * |            |                         |
+ * |            |                         |
+ * |--------------------------------------|
+ * 
+ */ +public class AppToFlowTable extends BaseTable { + /** app_flow prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow"; + + /** config param name that specifies the app_flow table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for app_flow table name. */ + private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow"; + + private static final Log LOG = LogFactory.getLog(AppToFlowTable.class); + + public AppToFlowTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor appToFlowTableDescp = new HTableDescriptor(table); + HColumnDescriptor mappCF = + new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes()); + mappCF.setBloomFilterType(BloomType.ROWCOL); + appToFlowTableDescp.addFamily(mappCF); + + appToFlowTableDescp + .setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(appToFlowTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java new file mode 100644 index 0000000..f01d982 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow + * contains classes related to implementation for app to flow table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java new file mode 100644 index 0000000..c165801 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * Encodes and decodes {@link ApplicationId} for row keys. + * App ID is stored in row key as 12 bytes, cluster timestamp section of app id + * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes). + */ +public final class AppIdKeyConverter implements KeyConverter { + + public AppIdKeyConverter() { + } + + /* + * (non-Javadoc) + * + * Converts/encodes a string app Id into a byte representation for (row) keys. + * For conversion, we extract cluster timestamp and sequence id from the + * string app id (calls ConverterUtils#toApplicationId(String) for + * conversion) and then store it in a byte array of length 12 (8 bytes (long) + * for cluster timestamp followed 4 bytes(int) for sequence id). Both cluster + * timestamp and sequence id are inverted so that the most recent cluster + * timestamp and highest sequence id appears first in the table (i.e. + * application id appears in a descending order). + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(String appIdStr) { + ApplicationId appId = ApplicationId.fromString(appIdStr); + byte[] appIdBytes = new byte[getKeySize()]; + byte[] clusterTs = Bytes.toBytes( + LongConverter.invertLong(appId.getClusterTimestamp())); + System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG); + byte[] seqId = Bytes.toBytes( + HBaseTimelineStorageUtils.invertInt(appId.getId())); + System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT); + return appIdBytes; + } + + /* + * (non-Javadoc) + * + * Converts/decodes a 12 byte representation of app id for (row) keys to an + * app id in string format which can be returned back to client. + * For decoding, 12 bytes are interpreted as 8 bytes of inverted cluster + * timestamp(long) followed by 4 bytes of inverted sequence id(int). Calls + * ApplicationId#toString to generate string representation of app id. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public String decode(byte[] appIdBytes) { + if (appIdBytes.length != getKeySize()) { + throw new IllegalArgumentException("Invalid app id in byte format"); + } + long clusterTs = LongConverter.invertLong( + Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG)); + int seqId = HBaseTimelineStorageUtils.invertInt( + Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT)); + return ApplicationId.newInstance(clusterTs, seqId).toString(); + } + + /** + * Returns the size of app id after encoding. + * + * @return size of app id after encoding. + */ + public static int getKeySize() { + return Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java new file mode 100644 index 0000000..8581aa4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; + +/** + * Implements behavior common to tables used in the timeline service storage. It + * is thread-safe, and can be used by multiple threads concurrently. + * + * @param reference to the table instance class itself for type safety. + */ +public abstract class BaseTable { + + /** + * Name of config variable that is used to point to this table. + */ + private final String tableNameConfName; + + /** + * Unless the configuration overrides, this will be the default name for the + * table when it is created. + */ + private final String defaultTableName; + + /** + * @param tableNameConfName name of config variable that is used to point to + * this table. + * @param defaultTableName Default table name if table from config is not + * found. + */ + protected BaseTable(String tableNameConfName, String defaultTableName) { + this.tableNameConfName = tableNameConfName; + this.defaultTableName = defaultTableName; + } + + /** + * Used to create a type-safe mutator for this table. + * + * @param hbaseConf used to read table name. + * @param conn used to create a table from. + * @return a type safe {@link BufferedMutator} for the entity table. + * @throws IOException if any exception occurs while creating mutator for the + * table. + */ + public TypedBufferedMutator getTableMutator(Configuration hbaseConf, + Connection conn) throws IOException { + + TableName tableName = this.getTableName(hbaseConf); + + // Plain buffered mutator + BufferedMutator bufferedMutator = conn.getBufferedMutator(tableName); + + // Now make this thing type safe. + // This is how service initialization should hang on to this variable, with + // the proper type + TypedBufferedMutator table = + new BufferedMutatorDelegator(bufferedMutator); + + return table; + } + + /** + * @param hbaseConf used to read settings that override defaults + * @param conn used to create table from + * @param scan that specifies what you want to read from this table. + * @return scanner for the table. + * @throws IOException if any exception occurs while getting the scanner. + */ + public ResultScanner getResultScanner(Configuration hbaseConf, + Connection conn, Scan scan) throws IOException { + Table table = conn.getTable(getTableName(hbaseConf)); + return table.getScanner(scan); + } + + /** + * + * @param hbaseConf used to read settings that override defaults + * @param conn used to create table from + * @param get that specifies what single row you want to get from this table + * @return result of get operation + * @throws IOException if any exception occurs while getting the result. + */ + public Result getResult(Configuration hbaseConf, Connection conn, Get get) + throws IOException { + Table table = conn.getTable(getTableName(hbaseConf)); + return table.get(get); + } + + /** + * Get the table name for this table. + * + * @param hbaseConf HBase configuration from which table name will be fetched. + * @return A {@link TableName} object. + */ + public TableName getTableName(Configuration hbaseConf) { + TableName table = + TableName.valueOf(hbaseConf.get(tableNameConfName, defaultTableName)); + return table; + + } + + /** + * Used to create the table in HBase. Should be called only once (per HBase + * instance). + * + * @param admin Used for doing HBase table operations. + * @param hbaseConf Hbase configuration. + * @throws IOException if any exception occurs while creating the table. + */ + public abstract void createTable(Admin admin, Configuration hbaseConf) + throws IOException; + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java new file mode 100644 index 0000000..cf469a5 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Mutation; + +/** + * To be used to wrap an actual {@link BufferedMutator} in a type safe manner. + * + * @param The class referring to the table to be written to. + */ +class BufferedMutatorDelegator implements TypedBufferedMutator { + + private final BufferedMutator bufferedMutator; + + /** + * @param bufferedMutator the mutator to be wrapped for delegation. Shall not + * be null. + */ + public BufferedMutatorDelegator(BufferedMutator bufferedMutator) { + this.bufferedMutator = bufferedMutator; + } + + public TableName getName() { + return bufferedMutator.getName(); + } + + public Configuration getConfiguration() { + return bufferedMutator.getConfiguration(); + } + + public void mutate(Mutation mutation) throws IOException { + bufferedMutator.mutate(mutation); + } + + public void mutate(List mutations) throws IOException { + bufferedMutator.mutate(mutations); + } + + public void close() throws IOException { + bufferedMutator.close(); + } + + public void flush() throws IOException { + bufferedMutator.flush(); + } + + public long getWriteBufferSize() { + return bufferedMutator.getWriteBufferSize(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java new file mode 100644 index 0000000..90f2de4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * A Column represents the way to store a fully qualified column in a specific + * table. + */ +public interface Column { + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes Map of attributes for this mutation. used in the + * coprocessor to set/read the cell tags. Can be null. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered during store. + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException; + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result Cannot be null + * @return result object (can be cast to whatever object was written to), or + * null when result doesn't contain this column. + * @throws IOException if there is any exception encountered while reading + * result. + */ + Object readResult(Result result) throws IOException; + + /** + * Returns column family name(as bytes) associated with this column. + * @return a byte array encoding column family for this column qualifier. + */ + byte[] getColumnFamilyBytes(); + + /** + * Get byte representation for this column qualifier. + * @return a byte array representing column qualifier. + */ + byte[] getColumnQualifierBytes(); + + /** + * Returns value converter implementation associated with this column. + * @return a {@link ValueConverter} implementation. + */ + ValueConverter getValueConverter(); +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java new file mode 100644 index 0000000..452adcd --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * Type safe column family. + * + * @param refers to the table for which this column family is used for. + */ +public interface ColumnFamily { + + /** + * Keep a local copy if you need to avoid overhead of repeated cloning. + * + * @return a clone of the byte representation of the column family. + */ + byte[] getBytes(); + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java new file mode 100644 index 0000000..be55db5 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java @@ -0,0 +1,388 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; +/** + * This class is meant to be used only by explicit Columns, and not directly to + * write by clients. + * + * @param refers to the table. + */ +public class ColumnHelper { + private static final Log LOG = LogFactory.getLog(ColumnHelper.class); + + private final ColumnFamily columnFamily; + + /** + * Local copy of bytes representation of columnFamily so that we can avoid + * cloning a new copy over and over. + */ + private final byte[] columnFamilyBytes; + + private final ValueConverter converter; + + public ColumnHelper(ColumnFamily columnFamily) { + this(columnFamily, GenericConverter.getInstance()); + } + + public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter) { + this.columnFamily = columnFamily; + columnFamilyBytes = columnFamily.getBytes(); + if (converter == null) { + this.converter = GenericConverter.getInstance(); + } else { + this.converter = converter; + } + } + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey + * identifying the row to write. Nothing gets written when null. + * @param tableMutator + * used to modify the underlying HBase table + * @param columnQualifier + * column qualifier. Nothing gets written when null. + * @param timestamp + * version timestamp. When null the current timestamp multiplied with + * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of + * app id will be used + * @param inputValue + * the value to write to the rowKey and column qualifier. Nothing + * gets written when null. + * @param attributes Attributes to be set for HBase Put. + * @throws IOException if any problem occurs during store operation(sending + * mutation to table). + */ + public void store(byte[] rowKey, TypedBufferedMutator tableMutator, + byte[] columnQualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException { + if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) { + return; + } + Put p = new Put(rowKey); + timestamp = getPutTimestamp(timestamp, attributes); + p.addColumn(columnFamilyBytes, columnQualifier, timestamp, + converter.encodeValue(inputValue)); + if ((attributes != null) && (attributes.length > 0)) { + for (Attribute attribute : attributes) { + p.setAttribute(attribute.getName(), attribute.getValue()); + } + } + tableMutator.mutate(p); + } + + /* + * Figures out the cell timestamp used in the Put For storing into flow run + * table. We would like to left shift the timestamp and supplement it with the + * AppId id so that there are no collisions in the flow run table's cells + */ + private long getPutTimestamp(Long timestamp, Attribute[] attributes) { + if (timestamp == null) { + timestamp = System.currentTimeMillis(); + } + String appId = getAppIdFromAttributes(attributes); + long supplementedTS = TimestampGenerator.getSupplementedTimestamp( + timestamp, appId); + return supplementedTS; + } + + private String getAppIdFromAttributes(Attribute[] attributes) { + if (attributes == null) { + return null; + } + String appId = null; + for (Attribute attribute : attributes) { + if (AggregationCompactionDimension.APPLICATION_ID.toString().equals( + attribute.getName())) { + appId = Bytes.toString(attribute.getValue()); + } + } + return appId; + } + + /** + * @return the column family for this column implementation. + */ + public ColumnFamily getColumnFamily() { + return columnFamily; + } + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result from which to read the value. Cannot be null + * @param columnQualifierBytes referring to the column to be read. + * @return latest version of the specified column of whichever object was + * written. + * @throws IOException if any problem occurs while reading result. + */ + public Object readResult(Result result, byte[] columnQualifierBytes) + throws IOException { + if (result == null || columnQualifierBytes == null) { + return null; + } + + // Would have preferred to be able to use getValueAsByteBuffer and get a + // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like + // that. + byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes); + return converter.decodeValue(value); + } + + /** + * @param result from which to reads data with timestamps + * @param columnPrefixBytes optional prefix to limit columns. If null all + * columns are returned. + * @param identifies the type of column name(indicated by type of key + * converter). + * @param the type of the values. The values will be cast into that type. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the cell values at each respective time in for form + * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, + * idB={timestamp3->value3}, idC={timestamp1->value4}}} + * @throws IOException if any problem occurs while reading results. + */ + @SuppressWarnings("unchecked") + public NavigableMap> + readResultsWithTimestamps(Result result, byte[] columnPrefixBytes, + KeyConverter keyConverter) throws IOException { + + NavigableMap> results = new TreeMap<>(); + + if (result != null) { + NavigableMap< + byte[], NavigableMap>> resultMap = + result.getMap(); + + NavigableMap> columnCellMap = + resultMap.get(columnFamilyBytes); + + // could be that there is no such column family. + if (columnCellMap != null) { + for (Entry> entry : columnCellMap + .entrySet()) { + K converterColumnKey = null; + if (columnPrefixBytes == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("null prefix was specified; returning all columns"); + } + try { + converterColumnKey = keyConverter.decode(entry.getKey()); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } else { + // A non-null prefix means columns are actually of the form + // prefix!columnNameRemainder + byte[][] columnNameParts = + Separator.QUALIFIERS.split(entry.getKey(), 2); + byte[] actualColumnPrefixBytes = columnNameParts[0]; + if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) + && columnNameParts.length == 2) { + try { + // This is the prefix that we want + converterColumnKey = keyConverter.decode(columnNameParts[1]); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } + } + + // If this column has the prefix we want + if (converterColumnKey != null) { + NavigableMap cellResults = + new TreeMap(); + NavigableMap cells = entry.getValue(); + if (cells != null) { + for (Entry cell : cells.entrySet()) { + V value = + (V) converter.decodeValue(cell.getValue()); + cellResults.put( + TimestampGenerator.getTruncatedTimestamp(cell.getKey()), + value); + } + } + results.put(converterColumnKey, cellResults); + } + } // for entry : columnCellMap + } // if columnCellMap != null + } // if result != null + return results; + } + + /** + * @param identifies the type of column name(indicated by type of key + * converter). + * @param result from which to read columns + * @param columnPrefixBytes optional prefix to limit columns. If null all + * columns are returned. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the latest values of columns in the column family. If the column + * prefix is null, the column qualifier is returned as Strings. For a + * non-null column prefix bytes, the column qualifier is returned as + * a list of parts, each part a byte[]. This is to facilitate + * returning byte arrays of values that were not Strings. + * @throws IOException if any problem occurs while reading results. + */ + public Map readResults(Result result, + byte[] columnPrefixBytes, KeyConverter keyConverter) + throws IOException { + Map results = new HashMap(); + + if (result != null) { + Map columns = result.getFamilyMap(columnFamilyBytes); + for (Entry entry : columns.entrySet()) { + byte[] columnKey = entry.getKey(); + if (columnKey != null && columnKey.length > 0) { + + K converterColumnKey = null; + if (columnPrefixBytes == null) { + try { + converterColumnKey = keyConverter.decode(columnKey); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } else { + // A non-null prefix means columns are actually of the form + // prefix!columnNameRemainder + byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2); + if (columnNameParts.length > 0) { + byte[] actualColumnPrefixBytes = columnNameParts[0]; + // If this is the prefix that we want + if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) + && columnNameParts.length == 2) { + try { + converterColumnKey = keyConverter.decode(columnNameParts[1]); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } + } + } // if-else + + // If the columnPrefix is null (we want all columns), or the actual + // prefix matches the given prefix we want this column + if (converterColumnKey != null) { + Object value = converter.decodeValue(entry.getValue()); + // we return the columnQualifier in parts since we don't know + // which part is of which data type. + results.put(converterColumnKey, value); + } + } + } // for entry + } + return results; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier for the remainder of the column. Any + * {@link Separator#QUALIFIERS} will be encoded in the qualifier. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + String qualifier) { + + // We don't want column names to have spaces / tabs. + byte[] encodedQualifier = + Separator.encode(qualifier, Separator.SPACE, Separator.TAB); + if (columnPrefixBytes == null) { + return encodedQualifier; + } + + // Convert qualifier to lower case, strip of separators and tag on column + // prefix. + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier); + return columnQualifier; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier for the remainder of the column. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + long qualifier) { + + if (columnPrefixBytes == null) { + return Bytes.toBytes(qualifier); + } + + // Convert qualifier to lower case, strip of separators and tag on column + // prefix. + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier)); + return columnQualifier; + } + + public ValueConverter getValueConverter() { + return converter; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier the byte representation for the remainder of the column. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + byte[] qualifier) { + + if (columnPrefixBytes == null) { + return qualifier; + } + + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, qualifier); + return columnQualifier; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java new file mode 100644 index 0000000..89aa013 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java @@ -0,0 +1,145 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Used to represent a partially qualified column, where the actual column name + * will be composed of a prefix and the remainder of the column qualifier. The + * prefix can be null, in which case the column qualifier will be completely + * determined when the values are stored. + */ +public interface ColumnPrefix { + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param qualifier column qualifier. Nothing gets written when null. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes attributes for the mutation that are used by the + * coprocessor to set/read the cell tags. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered while doing + * store operation(sending mutation to the table). + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + byte[] qualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException; + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param qualifier column qualifier. Nothing gets written when null. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes attributes for the mutation that are used by the + * coprocessor to set/read the cell tags. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered while doing + * store operation(sending mutation to the table). + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + String qualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException; + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result Cannot be null + * @param qualifier column qualifier. Nothing gets read when null. + * @return result object (can be cast to whatever object was written to) or + * null when specified column qualifier for this prefix doesn't exist + * in the result. + * @throws IOException if there is any exception encountered while reading + * result. + */ + Object readResult(Result result, String qualifier) throws IOException; + + /** + * + * @param identifies the type of key converter. + * @param result from which to read columns. + * @param keyConverter used to convert column bytes to the appropriate key + * type + * @return the latest values of columns in the column family with this prefix + * (or all of them if the prefix value is null). + * @throws IOException if there is any exception encountered while reading + * results. + */ + Map readResults(Result result, KeyConverter keyConverter) + throws IOException; + + /** + * @param result from which to reads data with timestamps. + * @param identifies the type of key converter. + * @param the type of the values. The values will be cast into that type. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the cell values at each respective time in for form + * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, + * idB={timestamp3->value3}, idC={timestamp1->value4}}} + * @throws IOException if there is any exception encountered while reading + * result. + */ + NavigableMap> readResultsWithTimestamps( + Result result, KeyConverter keyConverter) throws IOException; + + /** + * @param qualifierPrefix Column qualifier or prefix of qualifier. + * @return a byte array encoding column prefix and qualifier/prefix passed. + */ + byte[] getColumnPrefixBytes(String qualifierPrefix); + + /** + * @param qualifierPrefix Column qualifier or prefix of qualifier. + * @return a byte array encoding column prefix and qualifier/prefix passed. + */ + byte[] getColumnPrefixBytes(byte[] qualifierPrefix); + + /** + * Returns column family name(as bytes) associated with this column prefix. + * @return a byte array encoding column family for this prefix. + */ + byte[] getColumnFamilyBytes(); + + /** + * Returns value converter implementation associated with this column prefix. + * @return a {@link ValueConverter} implementation. + */ + ValueConverter getValueConverter(); +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java new file mode 100644 index 0000000..8445575 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * Encapsulates information about Event column names for application and entity + * tables. Used while encoding/decoding event column names. + */ +public class EventColumnName { + + private final String id; + private final Long timestamp; + private final String infoKey; + private final KeyConverter eventColumnNameConverter = + new EventColumnNameConverter(); + + public EventColumnName(String id, Long timestamp, String infoKey) { + this.id = id; + this.timestamp = timestamp; + this.infoKey = infoKey; + } + + public String getId() { + return id; + } + + public Long getTimestamp() { + return timestamp; + } + + public String getInfoKey() { + return infoKey; + } + + /** + * @return a byte array with each components/fields separated by + * Separator#VALUES. This leads to an event column name of the form + * eventId=timestamp=infokey. If both timestamp and infokey are null, + * then a qualifier of the form eventId=timestamp= is returned. If + * only infokey is null, then a qualifier of the form eventId= is + * returned. These prefix forms are useful for queries that intend to + * retrieve more than one specific column name. + */ + public byte[] getColumnQualifier() { + return eventColumnNameConverter.encode(this); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java new file mode 100644 index 0000000..d3ef897 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Encodes and decodes event column names for application and entity tables. + * The event column name is of the form : eventId=timestamp=infokey. + * If info is not associated with the event, event column name is of the form : + * eventId=timestamp= + * Event timestamp is long and rest are strings. + * Column prefixes are not part of the eventcolumn name passed for encoding. It + * is added later, if required in the associated ColumnPrefix implementations. + */ +public final class EventColumnNameConverter + implements KeyConverter { + + public EventColumnNameConverter() { + } + + // eventId=timestamp=infokey are of types String, Long String + // Strings are variable in size (i.e. end whenever separator is encountered). + // This is used while decoding and helps in determining where to split. + private static final int[] SEGMENT_SIZES = { + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes EventColumnName into a byte array with each component/field in + * EventColumnName separated by Separator#VALUES. This leads to an event + * column name of the form eventId=timestamp=infokey. + * If timestamp in passed EventColumnName object is null (eventId is not null) + * this returns a column prefix of the form eventId= and if infokey in + * EventColumnName is null (other 2 components are not null), this returns a + * column name of the form eventId=timestamp= + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(EventColumnName key) { + byte[] first = Separator.encode(key.getId(), Separator.SPACE, Separator.TAB, + Separator.VALUES); + if (key.getTimestamp() == null) { + return Separator.VALUES.join(first, Separator.EMPTY_BYTES); + } + byte[] second = Bytes.toBytes( + LongConverter.invertLong(key.getTimestamp())); + if (key.getInfoKey() == null) { + return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES); + } + return Separator.VALUES.join(first, second, Separator.encode( + key.getInfoKey(), Separator.SPACE, Separator.TAB, Separator.VALUES)); + } + + /* + * (non-Javadoc) + * + * Decodes an event column name of the form eventId=timestamp= or + * eventId=timestamp=infoKey represented in byte format and converts it into + * an EventColumnName object. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public EventColumnName decode(byte[] bytes) { + byte[][] components = Separator.VALUES.split(bytes, SEGMENT_SIZES); + if (components.length != 3) { + throw new IllegalArgumentException("the column name is not valid"); + } + String id = Separator.decode(Bytes.toString(components[0]), + Separator.VALUES, Separator.TAB, Separator.SPACE); + Long ts = LongConverter.invertLong(Bytes.toLong(components[1])); + String infoKey = components[2].length == 0 ? null : + Separator.decode(Bytes.toString(components[2]), + Separator.VALUES, Separator.TAB, Separator.SPACE); + return new EventColumnName(id, ts, infoKey); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java new file mode 100644 index 0000000..c34bfcb --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; + +/** + * Uses GenericObjectMapper to encode objects as bytes and decode bytes as + * objects. + */ +public final class GenericConverter implements ValueConverter { + private static final GenericConverter INSTANCE = new GenericConverter(); + + private GenericConverter() { + } + + public static GenericConverter getInstance() { + return INSTANCE; + } + + @Override + public byte[] encodeValue(Object value) throws IOException { + return GenericObjectMapper.write(value); + } + + @Override + public Object decodeValue(byte[] bytes) throws IOException { + return GenericObjectMapper.read(bytes); + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java new file mode 100644 index 0000000..b5629ae --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java @@ -0,0 +1,255 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +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.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; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * A bunch of utility functions used in HBase TimelineService backend. + */ +public final class HBaseTimelineStorageUtils { + /** milliseconds in one day. */ + public static final long MILLIS_ONE_DAY = 86400000L; + private static final Log LOG = + LogFactory.getLog(HBaseTimelineStorageUtils.class); + + private HBaseTimelineStorageUtils() { + } + + + /** + * Combines the input array of attributes and the input aggregation operation + * into a new array of attributes. + * + * @param attributes Attributes to be combined. + * @param aggOp Aggregation operation. + * @return array of combined attributes. + */ + public static Attribute[] combineAttributes(Attribute[] attributes, + AggregationOperation aggOp) { + int newLength = getNewLengthCombinedAttributes(attributes, aggOp); + Attribute[] combinedAttributes = new Attribute[newLength]; + + if (attributes != null) { + System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length); + } + + if (aggOp != null) { + Attribute a2 = aggOp.getAttribute(); + combinedAttributes[newLength - 1] = a2; + } + return combinedAttributes; + } + + /** + * Returns a number for the new array size. The new array is the combination + * of input array of attributes and the input aggregation operation. + * + * @param attributes Attributes. + * @param aggOp Aggregation operation. + * @return the size for the new array + */ + private static int getNewLengthCombinedAttributes(Attribute[] attributes, + AggregationOperation aggOp) { + int oldLength = getAttributesLength(attributes); + int aggLength = getAppOpLength(aggOp); + return oldLength + aggLength; + } + + private static int getAppOpLength(AggregationOperation aggOp) { + if (aggOp != null) { + return 1; + } + return 0; + } + + private static int getAttributesLength(Attribute[] attributes) { + if (attributes != null) { + return attributes.length; + } + return 0; + } + + /** + * Returns the first seen aggregation operation as seen in the list of input + * tags or null otherwise. + * + * @param tags list of HBase tags. + * @return AggregationOperation + */ + public static AggregationOperation getAggregationOperationFromTagsList( + List tags) { + for (AggregationOperation aggOp : AggregationOperation.values()) { + for (Tag tag : tags) { + if (tag.getType() == aggOp.getTagType()) { + return aggOp; + } + } + } + return null; + } + + /** + * Creates a {@link Tag} from the input attribute. + * + * @param attribute Attribute from which tag has to be fetched. + * @return a HBase Tag. + */ + public static Tag getTagFromAttribute(Map.Entry attribute) { + // attribute could be either an Aggregation Operation or + // an Aggregation Dimension + // Get the Tag type from either + AggregationOperation aggOp = AggregationOperation + .getAggregationOperation(attribute.getKey()); + if (aggOp != null) { + Tag t = new Tag(aggOp.getTagType(), attribute.getValue()); + return t; + } + + AggregationCompactionDimension aggCompactDim = + AggregationCompactionDimension.getAggregationCompactionDimension( + attribute.getKey()); + if (aggCompactDim != null) { + Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue()); + return t; + } + return null; + } + + /** + * creates a new cell based on the input cell but with the new value. + * + * @param origCell Original cell + * @param newValue new cell value + * @return cell + * @throws IOException while creating new cell. + */ + public static Cell createNewCell(Cell origCell, byte[] newValue) + throws IOException { + return CellUtil.createCell(CellUtil.cloneRow(origCell), + CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell), + origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue); + } + + /** + * creates a cell with the given inputs. + * + * @param row row of the cell to be created + * @param family column family name of the new cell + * @param qualifier qualifier for the new cell + * @param ts timestamp of the new cell + * @param newValue value of the new cell + * @param tags tags in the new cell + * @return cell + * @throws IOException while creating the cell. + */ + public static Cell createNewCell(byte[] row, byte[] family, byte[] qualifier, + long ts, byte[] newValue, byte[] tags) throws IOException { + return CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put, + newValue, tags); + } + + /** + * returns app id from the list of tags. + * + * @param tags cell tags to be looked into + * @return App Id as the AggregationCompactionDimension + */ + public static String getAggregationCompactionDimension(List tags) { + String appId = null; + for (Tag t : tags) { + if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t + .getType()) { + appId = Bytes.toString(t.getValue()); + 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 + * where we want to have the largest int value in the top of the table + * (scans start at the largest int first). + * + * @param key value to be inverted so that the latest version will be first in + * a scan. + * @return inverted int + */ + public static int invertInt(int key) { + return Integer.MAX_VALUE - key; + } + + /** + * returns the timestamp of that day's start (which is midnight 00:00:00 AM) + * for a given input timestamp. + * + * @param ts Timestamp. + * @return timestamp of that day's beginning (midnight) + */ + public static long getTopOfTheDayTimestamp(long ts) { + long dayTimestamp = ts - (ts % MILLIS_ONE_DAY); + return dayTimestamp; + } + + /** + * 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); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java new file mode 100644 index 0000000..4229e81 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * Interface which has to be implemented for encoding and decoding row keys and + * columns. + */ +public interface KeyConverter { + /** + * Encodes a key as a byte array. + * + * @param key key to be encoded. + * @return a byte array. + */ + byte[] encode(T key); + + /** + * Decodes a byte array and returns a key of type T. + * + * @param bytes byte representation + * @return an object(key) of type T which has been constructed after decoding + * the bytes. + */ + T decode(byte[] bytes); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java new file mode 100644 index 0000000..6ab69f7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; +import java.io.Serializable; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Encodes a value by interpreting it as a Long and converting it to bytes and + * decodes a set of bytes as a Long. + */ +public final class LongConverter implements NumericValueConverter, + Serializable { + + /** + * Added because we implement Comparator. + */ + private static final long serialVersionUID = 1L; + + public LongConverter() { + } + + @Override + public byte[] encodeValue(Object value) throws IOException { + if (!HBaseTimelineStorageUtils.isIntegralValue(value)) { + throw new IOException("Expected integral value"); + } + return Bytes.toBytes(((Number)value).longValue()); + } + + @Override + public Object decodeValue(byte[] bytes) throws IOException { + if (bytes == null) { + return null; + } + return Bytes.toLong(bytes); + } + + /** + * Compares two numbers as longs. If either number is null, it will be taken + * as 0. + * + * @param num1 the first {@code Long} to compare. + * @param num2 the second {@code Long} to compare. + * @return -1 if num1 is less than num2, 0 if num1 is equal to num2 and 1 if + * num1 is greater than num2. + */ + @Override + public int compare(Number num1, Number num2) { + return Long.compare((num1 == null) ? 0L : num1.longValue(), + (num2 == null) ? 0L : num2.longValue()); + } + + @Override + public Number add(Number num1, Number num2, Number...numbers) { + long sum = ((num1 == null) ? 0L : num1.longValue()) + + ((num2 == null) ? 0L : num2.longValue()); + for (Number num : numbers) { + sum = sum + ((num == null) ? 0L : num.longValue()); + } + return sum; + } + + /** + * Converts a timestamp into it's inverse timestamp to be used in (row) keys + * where we want to have the most recent timestamp in the top of the table + * (scans start at the most recent timestamp first). + * + * @param key value to be inverted so that the latest version will be first in + * a scan. + * @return inverted long + */ + public static long invertLong(long key) { + return Long.MAX_VALUE - key; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java new file mode 100644 index 0000000..4a724d6 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; +/** + * Encodes and decodes column names / row keys which are long. + */ +public final class LongKeyConverter implements KeyConverter { + + /** + * To delegate the actual work to. + */ + private final LongConverter longConverter = new LongConverter(); + + public LongKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(Long key) { + try { + // IOException will not be thrown here as we are explicitly passing + // Long. + return longConverter.encodeValue(key); + } catch (IOException e) { + return null; + } + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public Long decode(byte[] bytes) { + try { + return (Long) longConverter.decodeValue(bytes); + } catch (IOException e) { + return null; + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java new file mode 100644 index 0000000..8fb6536 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.util.Comparator; + +/** + * Extends ValueConverter interface for numeric converters to support numerical + * operations such as comparison, addition, etc. + */ +public interface NumericValueConverter extends ValueConverter, + Comparator { + /** + * Adds two or more numbers. If either of the numbers are null, it is taken as + * 0. + * + * @param num1 the first number to add. + * @param num2 the second number to add. + * @param numbers Rest of the numbers to be added. + * @return result after adding up the numbers. + */ + Number add(Number num1, Number num2, Number...numbers); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java new file mode 100644 index 0000000..8a2e01a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Encapsulates a range with start and end indices. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class Range { + private final int startIdx; + private final int endIdx; + + /** + * Defines a range from start index (inclusive) to end index (exclusive). + * + * @param start + * Starting index position + * @param end + * Ending index position (exclusive) + */ + public Range(int start, int end) { + if (start < 0 || end < start) { + throw new IllegalArgumentException( + "Invalid range, required that: 0 <= start <= end; start=" + start + + ", end=" + end); + } + + this.startIdx = start; + this.endIdx = end; + } + + public int start() { + return startIdx; + } + + public int end() { + return endIdx; + } + + public int length() { + return endIdx - startIdx; + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java new file mode 100644 index 0000000..6159dc7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * In queries where a single result is needed, an exact rowkey can be used + * through the corresponding rowkey#getRowKey() method. For queries that need to + * scan over a range of rowkeys, a partial (the initial part) of rowkeys are + * used. Classes implementing RowKeyPrefix indicate that they are the initial + * part of rowkeys, with different constructors with fewer number of argument to + * form a partial rowkey, a prefix. + * + * @param indicating the type of rowkey that a particular implementation is + * a prefix for. + */ +public interface RowKeyPrefix { + + /** + * Create a row key prefix, meaning a partial rowkey that can be used in range + * scans. Which fields are included in the prefix will depend on the + * constructor of the specific instance that was used. Output depends on which + * constructor was used. + * @return a prefix of the following form {@code fist!second!...!last!} + */ + byte[] getRowKeyPrefix(); + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java new file mode 100644 index 0000000..5090b4d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java @@ -0,0 +1,575 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Used to separate row qualifiers, column qualifiers and compound fields. + */ +public enum Separator { + + /** + * separator in key or column qualifier fields. + */ + QUALIFIERS("!", "%0$"), + + /** + * separator in values, and/or compound key/column qualifier fields. + */ + VALUES("=", "%1$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. Note that if we use HTML form encoding through URLEncoder, we end up + * getting a + for a space, which may already occur in strings, so we don't + * want that. + */ + SPACE(" ", "%2$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. + */ + TAB("\t", "%3$"); + + // a reserved character that starts each of the encoded values and is encoded + // first in order to escape naturally occurring instances of encoded values + // although it can be expressed as an enum instance, we define them as private + // variables to hide it from callers + private static final String PERCENT = "%"; + private static final String PERCENT_ENCODED = "%9$"; + + private static final Pattern PERCENT_PATTERN = + Pattern.compile(PERCENT, Pattern.LITERAL); + private static final String PERCENT_REPLACEMENT = + Matcher.quoteReplacement(PERCENT); + + private static final Pattern PERCENT_ENCODED_PATTERN = + Pattern.compile(PERCENT_ENCODED, Pattern.LITERAL); + private static final String PERCENT_ENCODED_REPLACEMENT = + Matcher.quoteReplacement(PERCENT_ENCODED); + + /** + * The string value of this separator. + */ + private final String value; + + /** + * The bye representation of value. + */ + private final byte[] bytes; + + // pre-compiled patterns and quoted replacements for optimization + private final Pattern valuePattern; + private final String valueReplacement; + + private final Pattern encodedValuePattern; + private final String encodedValueReplacement; + + /** + * Indicator for variable size of an individual segment in a split. The + * segment ends wherever separator is encountered. + * Typically used for string. + * Also used to indicate that there is no fixed number of splits which need to + * be returned. If split limit is specified as this, all possible splits are + * returned. + */ + public static final int VARIABLE_SIZE = 0; + + + /** empty string. */ + public static final String EMPTY_STRING = ""; + + /** empty bytes. */ + public static final byte[] EMPTY_BYTES = new byte[0]; + + /** + * @param value of the separator to use. Cannot be null or empty string. + * @param encodedValue choose something that isn't likely to occur in the data + * itself. Cannot be null or empty string. + */ + private Separator(String value, String encodedValue) { + this.value = value; + + // validation + if (value == null || value.length() == 0 || encodedValue == null + || encodedValue.length() == 0) { + throw new IllegalArgumentException( + "Cannot create separator from null or empty string."); + } + + this.bytes = Bytes.toBytes(value); + this.valuePattern = Pattern.compile(value, Pattern.LITERAL); + this.valueReplacement = Matcher.quoteReplacement(value); + + this.encodedValuePattern = Pattern.compile(encodedValue, Pattern.LITERAL); + this.encodedValueReplacement = Matcher.quoteReplacement(encodedValue); + } + + /** + * @return the original value of the separator + */ + public String getValue() { + return value; + } + + /** + * Used to make token safe to be used with this separator without collisions. + * It must be paired with {@link #decode(String)} for it to be + * decoded correctly. + *

+ * If you need to encode a given string for multiple separators, + * {@link #encode(String, Separator...)} should be used over successive + * invocations of this method. It will result in a more compact version of the + * encoded value. + * + * @param token Token to be encoded. + * @return the token with any occurrences of this separator URLEncoded. + */ + public String encode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + // first encode the percent to escape naturally occurring encoded values + String escaped = encodePercent(token); + return encodeSingle(escaped, this); + } + + private static String replace(String token, Pattern pattern, + String replacement) { + return pattern.matcher(token).replaceAll(replacement); + } + + private static String encodeSingle(String token, Separator separator) { + return replace(token, separator.valuePattern, + separator.encodedValueReplacement); + } + + private static String encodePercent(String token) { + return replace(token, PERCENT_PATTERN, PERCENT_ENCODED_REPLACEMENT); + } + + /** + * Decode the token encoded using {@link #encode(String)}. It must be + * used for the result encoded with {@link #encode(String)} to be able to + * recover the original. + * + * @param token Token to be decoded. + * @return the token with any occurrences of the encoded separator replaced by + * the separator itself. + */ + public String decode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + String escaped = decodeSingle(token, this); + // decode percent to de-escape + return decodePercent(escaped); + } + + private static String decodeSingle(String token, Separator separator) { + return replace(token, separator.encodedValuePattern, + separator.valueReplacement); + } + + private static String decodePercent(String token) { + return replace(token, PERCENT_ENCODED_PATTERN, PERCENT_REPLACEMENT); + } + + /** + * Encode the given separators in the token with their encoding equivalents. + * It must be paired with {@link #decode(byte[], Separator...)} or + * {@link #decode(String, Separator...)} with the same separators for it to be + * decoded correctly. + *

+ * If you need to encode a given string for multiple separators, this form of + * encoding should be used over successive invocations of + * {@link #encode(String)}. It will result in a more compact version of the + * encoded value. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return non-null byte representation of the token with occurrences of the + * separators encoded. + */ + public static byte[] encode(String token, Separator... separators) { + if (token == null || token.length() == 0) { + return EMPTY_BYTES; + } + String result = token; + // first encode the percent to escape naturally occurring encoded values + result = encodePercent(token); + for (Separator separator : separators) { + if (separator != null) { + result = encodeSingle(result, separator); + } + } + return Bytes.toBytes(result); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It must be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(byte[] token, Separator... separators) { + if (token == null) { + return null; + } + return decode(Bytes.toString(token), separators); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It must be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(String token, Separator... separators) { + if (token == null) { + return null; + } + String result = token; + for (Separator separator : separators) { + if (separator != null) { + result = decodeSingle(result, separator); + } + } + // decode percent to de-escape + return decodePercent(result); + } + + /** + * Returns a single byte array containing all of the individual arrays + * components separated by this separator. + * + * @param components Byte array components to be joined together. + * @return byte array after joining the components + */ + public byte[] join(byte[]... components) { + if (components == null || components.length == 0) { + return EMPTY_BYTES; + } + + int finalSize = 0; + finalSize = this.value.length() * (components.length - 1); + for (byte[] comp : components) { + if (comp != null) { + finalSize += comp.length; + } + } + + byte[] buf = new byte[finalSize]; + int offset = 0; + for (int i = 0; i < components.length; i++) { + if (components[i] != null) { + System.arraycopy(components[i], 0, buf, offset, components[i].length); + offset += components[i].length; + } + if (i < (components.length - 1)) { + System.arraycopy(this.bytes, 0, buf, offset, this.value.length()); + offset += this.value.length(); + } + } + return buf; + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(String... items) { + if (items == null || items.length == 0) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(items[0].toString())); + // Start at 1, we've already grabbed the first value at index 0 + for (int i = 1; i < items.length; i++) { + sb.append(this.value); + sb.append(encode(items[i].toString())); + } + + return sb.toString(); + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(Iterable items) { + if (items == null) { + return ""; + } + Iterator i = items.iterator(); + if (!i.hasNext()) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(i.next().toString())); + while (i.hasNext()) { + sb.append(this.value); + sb.append(encode(i.next().toString())); + } + + return sb.toString(); + } + + /** + * @param compoundValue containing individual values separated by this + * separator, which have that separator encoded. + * @return non-null set of values from the compoundValue with the separator + * decoded. + */ + public Collection splitEncoded(String compoundValue) { + List result = new ArrayList(); + if (compoundValue != null) { + for (String val : valuePattern.split(compoundValue)) { + result.add(decode(val)); + } + } + return result; + } + + /** + * Splits the source array into multiple array segments using this separator, + * up to a maximum of count items. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source to be split + * @param limit on how many segments are supposed to be returned. A + * non-positive value indicates no limit on number of segments. + * @return source split by this separator. + */ + public byte[][] split(byte[] source, int limit) { + return split(source, this.bytes, limit); + } + + /** + * Splits the source array into multiple array segments using this separator. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source byte array to be split. + * @param sizes sizes of relative components/segments. + * @return source split by this separator as per the sizes specified.. + */ + public byte[][] split(byte[] source, int[] sizes) { + return split(source, this.bytes, sizes); + } + + /** + * Splits the source array into multiple array segments using this separator, + * as many times as splits are found. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source byte array to be split + * @return source split by this separator. + */ + public byte[][] split(byte[] source) { + return split(source, this.bytes); + } + + /** + * Returns a list of ranges identifying [start, end) -- closed, open -- + * positions within the source byte array that would be split using the + * separator byte array. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source the source data + * @param separator the separator pattern to look for + * @param sizes indicate the sizes of the relative components/segments. + * @return a list of ranges. + */ + private static List splitRanges(byte[] source, byte[] separator, + int[] sizes) { + List segments = new ArrayList(); + if (source == null || separator == null) { + return segments; + } + // VARIABLE_SIZE here indicates that there is no limit to number of segments + // to return. + int limit = VARIABLE_SIZE; + if (sizes != null && sizes.length > 0) { + limit = sizes.length; + } + int start = 0; + int currentSegment = 0; + itersource: for (int i = 0; i < source.length; i++) { + for (int j = 0; j < separator.length; j++) { + if (source[i + j] != separator[j]) { + continue itersource; + } + } + // all separator elements matched + if (limit > VARIABLE_SIZE) { + if (segments.size() >= (limit - 1)) { + // everything else goes in one final segment + break; + } + if (sizes != null) { + int currentSegExpectedSize = sizes[currentSegment]; + if (currentSegExpectedSize > VARIABLE_SIZE) { + int currentSegSize = i - start; + if (currentSegSize < currentSegExpectedSize) { + // Segment not yet complete. More bytes to parse. + continue itersource; + } else if (currentSegSize > currentSegExpectedSize) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + } + } + segments.add(new Range(start, i)); + start = i + separator.length; + // i will be incremented again in outer for loop + i += separator.length - 1; + currentSegment++; + } + // add in remaining to a final range + if (start <= source.length) { + if (sizes != null) { + // Check if final segment is as per size specified. + if (sizes[currentSegment] > VARIABLE_SIZE && + source.length - start > sizes[currentSegment]) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + segments.add(new Range(start, source.length)); + } + return segments; + } + + /** + * Splits based on segments calculated based on limit/sizes specified for the + * separator. + * + * @param source byte array to be split. + * @param segments specifies the range for each segment. + * @return a byte[][] split as per the segment ranges. + */ + private static byte[][] split(byte[] source, List segments) { + byte[][] splits = new byte[segments.size()][]; + for (int i = 0; i < segments.size(); i++) { + Range r = segments.get(i); + byte[] tmp = new byte[r.length()]; + if (tmp.length > 0) { + System.arraycopy(source, r.start(), tmp, 0, r.length()); + } + splits[i] = tmp; + } + return splits; + } + + /** + * Splits the source array into multiple array segments using the given + * separator based on the sizes. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source source array. + * @param separator separator represented as a byte array. + * @param sizes sizes of relative components/segments. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator, int[] sizes) { + List segments = splitRanges(source, separator, sizes); + return split(source, segments); + } + + /** + * Splits the source array into multiple array segments using the given + * separator. This will naturally produce copied byte arrays for each of the + * split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator) { + return split(source, separator, (int[]) null); + } + + /** + * Splits the source array into multiple array segments using the given + * separator, up to a maximum of count items. This will naturally produce + * copied byte arrays for each of the split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @param limit a non-positive value indicates no limit on number of segments. + * @return byte[][] after splitting the input source. + */ + private static byte[][] split(byte[] source, byte[] separator, int limit) { + int[] sizes = null; + if (limit > VARIABLE_SIZE) { + sizes = new int[limit]; + } + List segments = splitRanges(source, separator, sizes); + return split(source, segments); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java new file mode 100644 index 0000000..282848e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * Encodes and decodes column names / row keys which are merely strings. + * Column prefixes are not part of the column name passed for encoding. It is + * added later, if required in the associated ColumnPrefix implementations. + */ +public final class StringKeyConverter implements KeyConverter { + + public StringKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(String key) { + return Separator.encode(key, Separator.SPACE, Separator.TAB); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public String decode(byte[] bytes) { + return Separator.decode(bytes, Separator.TAB, Separator.SPACE); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java new file mode 100644 index 0000000..8e6c259 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * contains the constants used in the context of schema accesses for + * {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity} + * information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class TimelineHBaseSchemaConstants { + private TimelineHBaseSchemaConstants() { + } + + /** + * Used to create a pre-split for tables starting with a username in the + * prefix. TODO: this may have to become a config variable (string with + * separators) so that different installations can presplit based on their own + * commonly occurring names. + */ + private final static byte[][] USERNAME_SPLITS = { + Bytes.toBytes("a"), Bytes.toBytes("ad"), Bytes.toBytes("an"), + Bytes.toBytes("b"), Bytes.toBytes("ca"), Bytes.toBytes("cl"), + Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), + Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), + Bytes.toBytes("j"), Bytes.toBytes("k"), Bytes.toBytes("l"), + Bytes.toBytes("m"), Bytes.toBytes("n"), Bytes.toBytes("o"), + Bytes.toBytes("q"), Bytes.toBytes("r"), Bytes.toBytes("s"), + Bytes.toBytes("se"), Bytes.toBytes("t"), Bytes.toBytes("u"), + Bytes.toBytes("v"), Bytes.toBytes("w"), Bytes.toBytes("x"), + Bytes.toBytes("y"), Bytes.toBytes("z") + }; + + /** + * The length at which keys auto-split. + */ + public static final String USERNAME_SPLIT_KEY_PREFIX_LENGTH = "4"; + + /** + * @return splits for splits where a user is a prefix. + */ + public static byte[][] getUsernameSplits() { + byte[][] kloon = USERNAME_SPLITS.clone(); + // Deep copy. + for (int row = 0; row < USERNAME_SPLITS.length; row++) { + kloon[row] = Bytes.copy(USERNAME_SPLITS[row]); + } + return kloon; + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java new file mode 100644 index 0000000..d03b37d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * Utility class that allows HBase coprocessors to interact with unique + * timestamps. + */ +public class TimestampGenerator { + + /* + * if this is changed, then reading cell timestamps written with older + * multiplier value will not work + */ + public static final long TS_MULTIPLIER = 1000000L; + + private final AtomicLong lastTimestamp = new AtomicLong(); + + /** + * Returns the current wall clock time in milliseconds, multiplied by the + * required precision. + * + * @return current timestamp. + */ + public long currentTime() { + // We want to align cell timestamps with current time. + // cell timestamps are not be less than + // System.currentTimeMillis() * TS_MULTIPLIER. + return System.currentTimeMillis() * TS_MULTIPLIER; + } + + /** + * Returns a timestamp value unique within the scope of this + * {@code TimestampGenerator} instance. For usage by HBase + * {@code RegionObserver} coprocessors, this normally means unique within a + * given region. + * + * Unlikely scenario of generating a non-unique timestamp: if there is a + * sustained rate of more than 1M hbase writes per second AND if region fails + * over within that time range of timestamps being generated then there may be + * collisions writing to a cell version of the same column. + * + * @return unique timestamp. + */ + public long getUniqueTimestamp() { + long lastTs; + long nextTs; + do { + lastTs = lastTimestamp.get(); + nextTs = Math.max(lastTs + 1, currentTime()); + } while (!lastTimestamp.compareAndSet(lastTs, nextTs)); + return nextTs; + } + + /** + * Returns a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id. + * + * Unlikely scenario of generating a timestamp that is a duplicate: If more + * than a 1M concurrent apps are running in one flow run AND write to same + * column at the same time, then say appId of 1M and 1 will overlap + * with appId of 001 and there may be collisions for that flow run's + * specific column. + * + * @param incomingTS Timestamp to be converted. + * @param appId Application Id. + * @return a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id + */ + public static long getSupplementedTimestamp(long incomingTS, String appId) { + long suffix = getAppIdSuffix(appId); + long outgoingTS = incomingTS * TS_MULTIPLIER + suffix; + return outgoingTS; + + } + + private static long getAppIdSuffix(String appIdStr) { + if (appIdStr == null) { + return 0L; + } + ApplicationId appId = ApplicationId.fromString(appIdStr); + long id = appId.getId() % TS_MULTIPLIER; + return id; + } + + /** + * truncates the last few digits of the timestamp which were supplemented by + * the TimestampGenerator#getSupplementedTimestamp function. + * + * @param incomingTS Timestamp to be truncated. + * @return a truncated timestamp value + */ + public static long getTruncatedTimestamp(long incomingTS) { + return incomingTS / TS_MULTIPLIER; + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java new file mode 100644 index 0000000..64a11f8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.hbase.client.BufferedMutator; + +/** + * Just a typed wrapper around {@link BufferedMutator} used to ensure that + * columns can write only to the table mutator for the right table. + */ +public interface TypedBufferedMutator extends BufferedMutator { + // This class is intentionally left (almost) blank +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java new file mode 100644 index 0000000..757a6d3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import java.io.IOException; + +/** + * Converter used to encode/decode value associated with a column prefix or a + * column. + */ +public interface ValueConverter { + + /** + * Encode an object as a byte array depending on the converter implementation. + * + * @param value Value to be encoded. + * @return a byte array + * @throws IOException if any problem is encountered while encoding. + */ + byte[] encodeValue(Object value) throws IOException; + + /** + * Decode a byte array and convert it into an object depending on the + * converter implementation. + * + * @param bytes Byte array to be decoded. + * @return an object + * @throws IOException if any problem is encountered while decoding. + */ + Object decodeValue(byte[] bytes) throws IOException; +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java new file mode 100644 index 0000000..0df5b8a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.common contains + * a set of utility classes used across backend storage reader and writer. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java new file mode 100644 index 0000000..93b4b36 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link EntityTable}. + */ +public enum EntityColumn implements Column { + + /** + * Identifier for the entity. + */ + ID(EntityColumnFamily.INFO, "id"), + + /** + * The type of entity. + */ + TYPE(EntityColumnFamily.INFO, "type"), + + /** + * When the entity was created. + */ + CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()), + + /** + * The version of the flow that this entity belongs to. + */ + FLOW_VERSION(EntityColumnFamily.INFO, "flow_version"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + EntityColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + EntityColumn(ColumnFamily columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + /** + * Retrieve an {@link EntityColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnQualifier Name of the column to retrieve + * @return the corresponding {@link EntityColumn} or null + */ + public static final EntityColumn columnFor(String columnQualifier) { + + // Match column based on value, assume column family matches. + for (EntityColumn ec : EntityColumn.values()) { + // Find a match based only on name. + if (ec.getColumnQualifier().equals(columnQualifier)) { + return ec; + } + } + + // Default to null + return null; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /** + * Retrieve an {@link EntityColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} + * if and only if {@code a.equals(b) & x.equals(y)} or + * {@code (x == y == null)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param name Name of the column to retrieve + * @return the corresponding {@link EntityColumn} or null if both arguments + * don't match. + */ + public static final EntityColumn columnFor(EntityColumnFamily columnFamily, + String name) { + + for (EntityColumn ec : EntityColumn.values()) { + // Find a match based column family and on name. + if (ec.columnFamily.equals(columnFamily) + && ec.getColumnQualifier().equals(name)) { + return ec; + } + } + + // Default to null + return null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java new file mode 100644 index 0000000..7c63727 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the entity table column families. + */ +public enum EntityColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: a) the size + * of the config values can be very large and b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + EntityColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java new file mode 100644 index 0000000..e410549 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java @@ -0,0 +1,300 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the entity table. + */ +public enum EntityColumnPrefix implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(EntityColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(EntityColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(EntityColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an entity. + */ + EVENT(EntityColumnFamily.INFO, "e", true), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(EntityColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(EntityColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual) { + this(columnFamily, columnPrefix, compondColQual, + GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, ValueConverter converter) { + this(columnFamily, columnPrefix, false, converter); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual, ValueConverter converter) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /** + * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no + * match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link EntityColumnPrefix} or null + */ + public static final EntityColumnPrefix columnFor(String columnPrefix) { + + // Match column based on value, assume column family matches. + for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) { + // Find a match based only on name. + if (ecp.getColumnPrefix().equals(columnPrefix)) { + return ecp; + } + } + + // Default to null + return null; + } + + /** + * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no + * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} + * if and only if {@code (x == y == null)} or + * {@code a.equals(b) & x.equals(y)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link EntityColumnPrefix} or null if both + * arguments don't match. + */ + public static final EntityColumnPrefix columnFor( + EntityColumnFamily columnFamily, String columnPrefix) { + + // TODO: needs unit test to confirm and need to update javadoc to explain + // null prefix case. + + for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) { + // Find a match based column family and on name. + if (ecp.columnFamily.equals(columnFamily) + && (((columnPrefix == null) && (ecp.getColumnPrefix() == null)) || + (ecp.getColumnPrefix().equals(columnPrefix)))) { + return ecp; + } + } + + // Default to null + return null; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java new file mode 100644 index 0000000..ff22178 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java @@ -0,0 +1,225 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the entity table. + */ +public class EntityRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final String appId; + private final String entityType; + private final String entityId; + private final KeyConverter entityRowKeyConverter = + new EntityRowKeyConverter(); + + public EntityRowKey(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, String entityId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + this.entityType = entityType; + this.entityId = entityId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public String getAppId() { + return appId; + } + + public String getEntityType() { + return entityType; + } + + public String getEntityId() { + return entityId; + } + + /** + * Constructs a row key for the entity table as follows: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * Typically used while querying a specific entity. + * + * @return byte array with the row key. + */ + public byte[] getRowKey() { + return entityRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey byte representation of row key. + * @return An EntityRowKey object. + */ + public static EntityRowKey parseRowKey(byte[] rowKey) { + return new EntityRowKeyConverter().decode(rowKey); + } + + /** + * Encodes and decodes row key for entity table. The row key is of the form : + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId + * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and + * rest are strings. + *

+ */ + final private static class EntityRowKeyConverter implements + KeyConverter { + + private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter(); + + private EntityRowKeyConverter() { + } + + /** + * Entity row key is of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each + * segment separated by !. The sizes below indicate sizes of each one of + * these segments in sequence. clusterId, userName, flowName, entityType and + * entityId are strings. flowrunId is a long hence 8 bytes in size. app id + * is represented as 12 bytes with cluster timestamp part of appid being 8 + * bytes (long) and seq id being 4 bytes(int). Strings are variable in size + * (i.e. end whenever separator is encountered). This is used while decoding + * and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes EntityRowKey object into a byte array with each component/field + * in EntityRowKey separated by Separator#QUALIFIERS. This leads to an + * entity table row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId If + * entityType in passed EntityRowKey object is null (and the fields + * preceding it i.e. clusterId, userId and flowName, flowRunId and appId + * are not null), this returns a row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId! and if entityId in + * EntityRowKey is null (other 6 components are not null), this returns a + * row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is + * inverted while encoding as it helps maintain a descending order for row + * keys in entity table. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(EntityRowKey rowKey) { + byte[] user = + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] cluster = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] flow = + Separator.encode(rowKey.getFlowName(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(user, cluster, flow); + // Note that flowRunId is a long, so we can't encode them all at the same + // time. + byte[] second = + Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); + byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); + if (rowKey.getEntityType() == null) { + return Separator.QUALIFIERS.join(first, second, third, + Separator.EMPTY_BYTES); + } + byte[] entityType = + Separator.encode(rowKey.getEntityType(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] entityId = + rowKey.getEntityId() == null ? Separator.EMPTY_BYTES : Separator + .encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId); + return Separator.QUALIFIERS.join(first, second, third, fourth); + } + + /* + * (non-Javadoc) + * + * Decodes an application row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId + * represented in byte format and converts it into an EntityRowKey object. + * flowRunId is inverted while decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public EntityRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 7) { + throw new IllegalArgumentException("the row key is not valid for " + + "an entity"); + } + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + String appId = appIDKeyConverter.decode(rowKeyComponents[4]); + String entityType = + Separator.decode(Bytes.toString(rowKeyComponents[5]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String entityId = + Separator.decode(Bytes.toString(rowKeyComponents[6]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + entityType, entityId); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java new file mode 100644 index 0000000..9146180 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey without the entityId or without entityType and + * entityId for the entity table. + * + */ +public class EntityRowKeyPrefix extends EntityRowKey implements + RowKeyPrefix { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + * @param entityType which entity type + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType) { + super(clusterId, userId, flowName, flowRunId, appId, entityType, null); + } + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + super(clusterId, userId, flowName, flowRunId, appId, null, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java new file mode 100644 index 0000000..b194f07 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; + +/** + * The entity table as column families info, config and metrics. Info stores + * information about a timeline entity object config stores configuration data + * of a timeline entity object metrics stores the metrics of a timeline entity + * object + * + * Example entity table record: + * + *

+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
+ * | clusterId! |                              | metricValue1 | configValue1 |
+ * | flowName!  | type:entityType              | @timestamp1  |              |
+ * | flowRunId! |                              |              | configKey2:  |
+ * | AppId!     | created_time:                | metricId1:   | configValue2 |
+ * | entityType!| 1392993084018                | metricValue2 |              |
+ * | entityId   |                              | @timestamp2  |              |
+ * |            | i!infoKey:                   |              |              |
+ * |            | infoValue                    | metricId1:   |              |
+ * |            |                              | metricValue1 |              |
+ * |            | r!relatesToKey:              | @timestamp2  |              |
+ * |            | id3=id4=id5                  |              |              |
+ * |            |                              |              |              |
+ * |            | s!isRelatedToKey             |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public class EntityTable extends BaseTable { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity"; + + /** config param name that specifies the entity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * entity table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** default value for entity table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.entity"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000; + + private static final Log LOG = LogFactory.getLog(EntityTable.class); + + public EntityTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor entityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(EntityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + entityTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(EntityColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + entityTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(EntityColumnFamily.METRICS.getBytes()); + entityTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + entityTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(entityTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metricss in this table. + * @param hbaseConf configururation in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java new file mode 100644 index 0000000..bb0e331 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.entity + * contains classes related to implementation for entity table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java new file mode 100644 index 0000000..4e2cf2d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Identifies the compaction dimensions for the data in the {@link FlowRunTable} + * . + */ +public enum AggregationCompactionDimension { + + /** + * the application id. + */ + APPLICATION_ID((byte) 101); + + private byte tagType; + private byte[] inBytes; + + private AggregationCompactionDimension(byte tagType) { + this.tagType = tagType; + this.inBytes = Bytes.toBytes(this.name()); + } + + public Attribute getAttribute(String attributeValue) { + return new Attribute(this.name(), Bytes.toBytes(attributeValue)); + } + + public byte getTagType() { + return tagType; + } + + public byte[] getInBytes() { + return this.inBytes.clone(); + } + + public static AggregationCompactionDimension + getAggregationCompactionDimension(String aggCompactDimStr) { + for (AggregationCompactionDimension aggDim : AggregationCompactionDimension + .values()) { + if (aggDim.name().equals(aggCompactDimStr)) { + return aggDim; + } + } + return null; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java new file mode 100644 index 0000000..40cdd2c --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Identifies the attributes to be set for puts into the {@link FlowRunTable}. + * The numbers used for tagType are prime numbers. + */ +public enum AggregationOperation { + + /** + * When the flow was started. + */ + GLOBAL_MIN((byte) 71), + + /** + * When it ended. + */ + GLOBAL_MAX((byte) 73), + + /** + * The metrics of the flow. + */ + SUM((byte) 79), + + /** + * application running. + */ + SUM_FINAL((byte) 83), + + /** + * Min value as per the latest timestamp + * seen for a given app. + */ + LATEST_MIN((byte) 89), + + /** + * Max value as per the latest timestamp + * seen for a given app. + */ + LATEST_MAX((byte) 97); + + private byte tagType; + private byte[] inBytes; + + private AggregationOperation(byte tagType) { + this.tagType = tagType; + this.inBytes = Bytes.toBytes(this.name()); + } + + public Attribute getAttribute() { + return new Attribute(this.name(), this.inBytes); + } + + public byte getTagType() { + return tagType; + } + + public byte[] getInBytes() { + return this.inBytes.clone(); + } + + /** + * returns the AggregationOperation enum that represents that string. + * @param aggOpStr Aggregation operation. + * @return the AggregationOperation enum that represents that string + */ + public static AggregationOperation getAggregationOperation(String aggOpStr) { + for (AggregationOperation aggOp : AggregationOperation.values()) { + if (aggOp.name().equals(aggOpStr)) { + return aggOp; + } + } + return null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java new file mode 100644 index 0000000..d3de518 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +/** + * Defines the attribute tuple to be set for puts into the {@link FlowRunTable}. + */ +public class Attribute { + private final String name; + private final byte[] value; + + public Attribute(String name, byte[] value) { + this.name = name; + this.value = value.clone(); + } + + public String getName() { + return name; + } + + public byte[] getValue() { + return value.clone(); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java new file mode 100644 index 0000000..f9eb5b4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the flow run table column families. + */ +public enum FlowActivityColumnFamily + implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value + * create a column family with this name. Must be lower case and + * without spaces. + */ + private FlowActivityColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java new file mode 100644 index 0000000..439e0c8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java @@ -0,0 +1,277 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +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.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies partially qualified columns for the {@link FlowActivityTable}. + */ +public enum FlowActivityColumnPrefix + implements ColumnPrefix { + + /** + * To store run ids of the flows. + */ + RUN_ID(FlowActivityColumnFamily.INFO, "r", null); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + private final AggregationOperation aggOp; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily + * that this column is stored in. + * @param columnPrefix + * for this column. + */ + private FlowActivityColumnPrefix( + ColumnFamily columnFamily, String columnPrefix, + AggregationOperation aggOp) { + this(columnFamily, columnPrefix, aggOp, false); + } + + private FlowActivityColumnPrefix( + ColumnFamily columnFamily, String columnPrefix, + AggregationOperation aggOp, boolean compoundColQual) { + column = new ColumnHelper(columnFamily); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE + .encode(columnPrefix)); + } + this.aggOp = aggOp; + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + public byte[] getColumnPrefixBytes() { + return columnPrefixBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + public AggregationOperation getAttribute() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, byte[], java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /** + * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there + * is no match. The following holds true: {@code columnFor(x) == columnFor(y)} + * if and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnPrefix + * Name of the column to retrieve + * @return the corresponding {@link FlowActivityColumnPrefix} or null + */ + public static final FlowActivityColumnPrefix columnFor(String columnPrefix) { + + // Match column based on value, assume column family matches. + for (FlowActivityColumnPrefix flowActivityColPrefix : + FlowActivityColumnPrefix.values()) { + // Find a match based only on name. + if (flowActivityColPrefix.getColumnPrefix().equals(columnPrefix)) { + return flowActivityColPrefix; + } + } + // Default to null + return null; + } + + /** + * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there + * is no match. The following holds true: + * {@code columnFor(a,x) == columnFor(b,y)} if and only if + * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} + * + * @param columnFamily + * The columnFamily for which to retrieve the column. + * @param columnPrefix + * Name of the column to retrieve + * @return the corresponding {@link FlowActivityColumnPrefix} or null if both + * arguments don't match. + */ + public static final FlowActivityColumnPrefix columnFor( + FlowActivityColumnFamily columnFamily, String columnPrefix) { + + // TODO: needs unit test to confirm and need to update javadoc to explain + // null prefix case. + + for (FlowActivityColumnPrefix flowActivityColumnPrefix : + FlowActivityColumnPrefix.values()) { + // Find a match based column family and on name. + if (flowActivityColumnPrefix.columnFamily.equals(columnFamily) + && (((columnPrefix == null) && (flowActivityColumnPrefix + .getColumnPrefix() == null)) || (flowActivityColumnPrefix + .getColumnPrefix().equals(columnPrefix)))) { + return flowActivityColumnPrefix; + } + } + // Default to null + return null; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, null, inputValue, + combinedAttributes); + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java new file mode 100644 index 0000000..bb77e36 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java @@ -0,0 +1,196 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +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.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the flow activity table. + */ +public class FlowActivityRowKey { + + private final String clusterId; + private final Long dayTs; + private final String userId; + private final String flowName; + private final KeyConverter flowActivityRowKeyConverter = + new FlowActivityRowKeyConverter(); + + /** + * @param clusterId identifying the cluster + * @param dayTs to be converted to the top of the day timestamp + * @param userId identifying user + * @param flowName identifying the flow + */ + public FlowActivityRowKey(String clusterId, Long dayTs, String userId, + String flowName) { + this(clusterId, dayTs, userId, flowName, true); + } + + /** + * @param clusterId identifying the cluster + * @param timestamp when the flow activity happened. May be converted to the + * top of the day depending on the convertDayTsToTopOfDay argument. + * @param userId identifying user + * @param flowName identifying the flow + * @param convertDayTsToTopOfDay if true and timestamp isn't null, then + * timestamp will be converted to the top-of-the day timestamp + */ + protected FlowActivityRowKey(String clusterId, Long timestamp, String userId, + String flowName, boolean convertDayTsToTopOfDay) { + this.clusterId = clusterId; + if (convertDayTsToTopOfDay && (timestamp != null)) { + this.dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(timestamp); + } else { + this.dayTs = timestamp; + } + this.userId = userId; + this.flowName = flowName; + } + + public String getClusterId() { + return clusterId; + } + + public Long getDayTimestamp() { + return dayTs; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + /** + * Constructs a row key for the flow activity table as follows: + * {@code clusterId!dayTimestamp!user!flowName}. + * + * @return byte array for the row key + */ + public byte[] getRowKey() { + return flowActivityRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey Byte representation of row key. + * @return A FlowActivityRowKey object. + */ + public static FlowActivityRowKey parseRowKey(byte[] rowKey) { + return new FlowActivityRowKeyConverter().decode(rowKey); + } + + /** + * Encodes and decodes row key for flow activity table. The row key is of the + * form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day + * timestamp) is a long and rest are strings. + *

+ */ + final private static class FlowActivityRowKeyConverter implements + KeyConverter { + + private FlowActivityRowKeyConverter() { + } + + /** + * The flow activity row key is of the form + * clusterId!dayTimestamp!user!flowName with each segment separated by !. + * The sizes below indicate sizes of each one of these segements in + * sequence. clusterId, user and flowName are strings. Top of the day + * timestamp is a long hence 8 bytes in size. Strings are variable in size + * (i.e. they end whenever separator is encountered). This is used while + * decoding and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes FlowActivityRowKey object into a byte array with each + * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS. + * This leads to an flow activity table row key of the form + * clusterId!dayTimestamp!user!flowName. If dayTimestamp in passed + * FlowActivityRowKey object is null and clusterId is not null, then this + * returns a row key prefix as clusterId! and if userId in + * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId + * and dayTimestamp are not null), this returns a row key prefix as + * clusterId!dayTimeStamp! dayTimestamp is inverted while encoding as it + * helps maintain a descending order for row keys in flow activity table. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(FlowActivityRowKey rowKey) { + if (rowKey.getDayTimestamp() == null) { + return Separator.QUALIFIERS.join(Separator.encode( + rowKey.getClusterId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.EMPTY_BYTES); + } + if (rowKey.getUserId() == null) { + return Separator.QUALIFIERS.join(Separator.encode( + rowKey.getClusterId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Bytes.toBytes(LongConverter + .invertLong(rowKey.getDayTimestamp())), Separator.EMPTY_BYTES); + } + return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Bytes + .toBytes(LongConverter.invertLong(rowKey.getDayTimestamp())), + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public FlowActivityRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 4) { + throw new IllegalArgumentException("the row key is not valid for " + + "a flow activity"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long dayTs = LongConverter.invertLong(Bytes.toLong(rowKeyComponents[1])); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[3]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + return new FlowActivityRowKey(clusterId, dayTs, userId, flowName); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java new file mode 100644 index 0000000..eb88e54 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * A prefix partial rowkey for flow activities. + */ +public class FlowActivityRowKeyPrefix extends FlowActivityRowKey implements + RowKeyPrefix { + + /** + * Constructs a row key prefix for the flow activity table as follows: + * {@code clusterId!dayTimestamp!}. + * + * @param clusterId Cluster Id. + * @param dayTs Start of the day timestamp. + */ + public FlowActivityRowKeyPrefix(String clusterId, Long dayTs) { + super(clusterId, dayTs, null, null, false); + } + + /** + * Constructs a row key prefix for the flow activity table as follows: + * {@code clusterId!}. + * + * @param clusterId identifying the cluster + */ + public FlowActivityRowKeyPrefix(String clusterId) { + super(clusterId, null, null, null, false); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java new file mode 100644 index 0000000..8a0430c --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The flow activity table has column family info + * Stores the daily activity record for flows + * Useful as a quick lookup of what flows were + * running on a given day + * + * Example flow activity table record: + * + *

+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | r!runid1:version1            |
+ * | inv Top of |                              |
+ * | Day!       | r!runid2:version7            |
+ * | userName!  |                              |
+ * | flowName   |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public class FlowActivityTable extends BaseTable { + /** flow activity table prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity"; + + /** config param name that specifies the flowactivity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowactivity table name. */ + public static final String DEFAULT_TABLE_NAME = + "timelineservice.flowactivity"; + + private static final Log LOG = LogFactory.getLog(FlowActivityTable.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowActivityTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowActivityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowActivityTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy before running in production + admin.createTable(flowActivityTableDescp); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java new file mode 100644 index 0000000..90dd345 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java @@ -0,0 +1,182 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies fully qualified columns for the {@link FlowRunTable}. + */ +public enum FlowRunColumn implements Column { + + /** + * When the flow was started. This is the minimum of currently known + * application start times. + */ + MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time", + AggregationOperation.GLOBAL_MIN, new LongConverter()), + + /** + * When the flow ended. This is the maximum of currently known application end + * times. + */ + MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time", + AggregationOperation.GLOBAL_MAX, new LongConverter()), + + /** + * The version of the flow that this flow belongs to. + */ + FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + private final AggregationOperation aggOp; + + private FlowRunColumn(ColumnFamily columnFamily, + String columnQualifier, AggregationOperation aggOp) { + this(columnFamily, columnQualifier, aggOp, + GenericConverter.getInstance()); + } + + private FlowRunColumn(ColumnFamily columnFamily, + String columnQualifier, AggregationOperation aggOp, + ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + this.aggOp = aggOp; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE + .encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + public AggregationOperation getAggregationOperation() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.Column#store + * (byte[], org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, aggOp); + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, combinedAttributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + /** + * Retrieve an {@link FlowRunColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnQualifier + * Name of the column to retrieve + * @return the corresponding {@link FlowRunColumn} or null + */ + public static final FlowRunColumn columnFor(String columnQualifier) { + + // Match column based on value, assume column family matches. + for (FlowRunColumn ec : FlowRunColumn.values()) { + // Find a match based only on name. + if (ec.getColumnQualifier().equals(columnQualifier)) { + return ec; + } + } + + // Default to null + return null; + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /** + * Retrieve an {@link FlowRunColumn} given a name, or null if there is no + * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} + * if and only if {@code a.equals(b) & x.equals(y)} or + * {@code (x == y == null)} + * + * @param columnFamily + * The columnFamily for which to retrieve the column. + * @param name + * Name of the column to retrieve + * @return the corresponding {@link FlowRunColumn} or null if both arguments + * don't match. + */ + public static final FlowRunColumn columnFor(FlowRunColumnFamily columnFamily, + String name) { + + for (FlowRunColumn ec : FlowRunColumn.values()) { + // Find a match based column family and on name. + if (ec.columnFamily.equals(columnFamily) + && ec.getColumnQualifier().equals(name)) { + return ec; + } + } + + // Default to null + return null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java new file mode 100644 index 0000000..8faf5f8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the flow run table column families. + */ +public enum FlowRunColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value + * create a column family with this name. Must be lower case and + * without spaces. + */ + private FlowRunColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java new file mode 100644 index 0000000..278d18e --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java @@ -0,0 +1,268 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +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.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies partially qualified columns for the {@link FlowRunTable}. + */ +public enum FlowRunColumnPrefix implements ColumnPrefix { + + /** + * To store flow run info values. + */ + METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + private final AggregationOperation aggOp; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + private FlowRunColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, AggregationOperation fra, ValueConverter converter) { + this(columnFamily, columnPrefix, fra, converter, false); + } + + private FlowRunColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, AggregationOperation fra, ValueConverter converter, + boolean compoundColQual) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + this.aggOp = fra; + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + public byte[] getColumnPrefixBytes() { + return columnPrefixBytes.clone(); + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, + qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, + qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + public AggregationOperation getAttribute() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /** + * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is + * no match. The following holds true: {@code columnFor(x) == columnFor(y)} if + * and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link FlowRunColumnPrefix} or null + */ + public static final FlowRunColumnPrefix columnFor(String columnPrefix) { + + // Match column based on value, assume column family matches. + for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) { + // Find a match based only on name. + if (frcp.getColumnPrefix().equals(columnPrefix)) { + return frcp; + } + } + + // Default to null + return null; + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /** + * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is + * no match. The following holds true: + * {@code columnFor(a,x) == columnFor(b,y)} if and only if + * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link FlowRunColumnPrefix} or null if both + * arguments don't match. + */ + public static final FlowRunColumnPrefix columnFor( + FlowRunColumnFamily columnFamily, String columnPrefix) { + + // TODO: needs unit test to confirm and need to update javadoc to explain + // null prefix case. + + for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) { + // Find a match based column family and on name. + if (frcp.columnFamily.equals(columnFamily) + && (((columnPrefix == null) && (frcp.getColumnPrefix() == null)) || + (frcp.getColumnPrefix().equals(columnPrefix)))) { + return frcp; + } + } + + // Default to null + return null; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java new file mode 100644 index 0000000..2be6ef8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java @@ -0,0 +1,304 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; + +/** + * Coprocessor for flow run table. + */ +public class FlowRunCoprocessor extends BaseRegionObserver { + + private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class); + private boolean isFlowRunRegion = false; + + private Region region; + /** + * generate a timestamp that is unique per row in a region this is per region. + */ + private final TimestampGenerator timestampGenerator = + new TimestampGenerator(); + + @Override + public void start(CoprocessorEnvironment e) throws IOException { + if (e instanceof RegionCoprocessorEnvironment) { + RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e; + this.region = env.getRegion(); + isFlowRunRegion = HBaseTimelineStorageUtils.isFlowRunTable( + region.getRegionInfo(), env.getConfiguration()); + } + } + + public boolean isFlowRunRegion() { + return isFlowRunRegion; + } + + /* + * (non-Javadoc) + * + * This method adds the tags onto the cells in the Put. It is presumed that + * all the cells in one Put have the same set of Tags. The existing cell + * timestamp is overwritten for non-metric cells and each such cell gets a new + * unique timestamp generated by {@link TimestampGenerator} + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#prePut(org.apache + * .hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Put, + * org.apache.hadoop.hbase.regionserver.wal.WALEdit, + * org.apache.hadoop.hbase.client.Durability) + */ + @Override + public void prePut(ObserverContext e, Put put, + WALEdit edit, Durability durability) throws IOException { + Map attributes = put.getAttributesMap(); + + if (!isFlowRunRegion) { + return; + } + // Assumption is that all the cells in a put are the same operation. + List tags = new ArrayList<>(); + if ((attributes != null) && (attributes.size() > 0)) { + for (Map.Entry attribute : attributes.entrySet()) { + Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute); + tags.add(t); + } + byte[] tagByteArray = Tag.fromList(tags); + NavigableMap> newFamilyMap = new TreeMap<>( + Bytes.BYTES_COMPARATOR); + for (Map.Entry> entry : put.getFamilyCellMap() + .entrySet()) { + List newCells = new ArrayList<>(entry.getValue().size()); + for (Cell cell : entry.getValue()) { + // for each cell in the put add the tags + // Assumption is that all the cells in + // one put are the same operation + // also, get a unique cell timestamp for non-metric cells + // this way we don't inadvertently overwrite cell versions + long cellTimestamp = getCellTimestamp(cell.getTimestamp(), tags); + newCells.add(CellUtil.createCell(CellUtil.cloneRow(cell), + CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), + cellTimestamp, KeyValue.Type.Put, CellUtil.cloneValue(cell), + tagByteArray)); + } + newFamilyMap.put(entry.getKey(), newCells); + } // for each entry + // Update the family map for the Put + put.setFamilyCellMap(newFamilyMap); + } + } + + /** + * Determines if the current cell's timestamp is to be used or a new unique + * cell timestamp is to be used. The reason this is done is to inadvertently + * overwrite cells when writes come in very fast. But for metric cells, the + * cell timestamp signifies the metric timestamp. Hence we don't want to + * overwrite it. + * + * @param timestamp + * @param tags + * @return cell timestamp + */ + private long getCellTimestamp(long timestamp, List tags) { + // if ts not set (hbase sets to HConstants.LATEST_TIMESTAMP by default) + // then use the generator + if (timestamp == HConstants.LATEST_TIMESTAMP) { + return timestampGenerator.getUniqueTimestamp(); + } else { + return timestamp; + } + } + + /* + * (non-Javadoc) + * + * Creates a {@link FlowScanner} Scan so that it can correctly process the + * contents of {@link FlowRunTable}. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preGetOp(org.apache + * .hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Get, java.util.List) + */ + @Override + public void preGetOp(ObserverContext e, + Get get, List results) throws IOException { + if (!isFlowRunRegion) { + return; + } + + Scan scan = new Scan(get); + scan.setMaxVersions(); + RegionScanner scanner = null; + try { + scanner = new FlowScanner(e.getEnvironment(), scan, + region.getScanner(scan), FlowScannerOperation.READ); + scanner.next(results); + e.bypass(); + } finally { + if (scanner != null) { + scanner.close(); + } + } + } + + /* + * (non-Javadoc) + * + * Ensures that max versions are set for the Scan so that metrics can be + * correctly aggregated and min/max can be correctly determined. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preScannerOpen(org + * .apache.hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Scan, + * org.apache.hadoop.hbase.regionserver.RegionScanner) + */ + @Override + public RegionScanner preScannerOpen( + ObserverContext e, Scan scan, + RegionScanner scanner) throws IOException { + + if (isFlowRunRegion) { + // set max versions for scan to see all + // versions to aggregate for metrics + scan.setMaxVersions(); + } + return scanner; + } + + /* + * (non-Javadoc) + * + * Creates a {@link FlowScanner} Scan so that it can correctly process the + * contents of {@link FlowRunTable}. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#postScannerOpen( + * org.apache.hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Scan, + * org.apache.hadoop.hbase.regionserver.RegionScanner) + */ + @Override + public RegionScanner postScannerOpen( + ObserverContext e, Scan scan, + RegionScanner scanner) throws IOException { + if (!isFlowRunRegion) { + return scanner; + } + return new FlowScanner(e.getEnvironment(), scan, + scanner, FlowScannerOperation.READ); + } + + @Override + public InternalScanner preFlush( + ObserverContext c, Store store, + InternalScanner scanner) throws IOException { + if (!isFlowRunRegion) { + return scanner; + } + if (LOG.isDebugEnabled()) { + if (store != null) { + LOG.debug("preFlush store = " + store.getColumnFamilyName() + + " flushableSize=" + store.getFlushableSize() + + " flushedCellsCount=" + store.getFlushedCellsCount() + + " compactedCellsCount=" + store.getCompactedCellsCount() + + " majorCompactedCellsCount=" + + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" + + store.getMemstoreFlushSize() + " memstoreSize=" + + store.getMemStoreSize() + " size=" + store.getSize() + + " storeFilesCount=" + store.getStorefilesCount()); + } + } + return new FlowScanner(c.getEnvironment(), scanner, + FlowScannerOperation.FLUSH); + } + + @Override + public void postFlush(ObserverContext c, + Store store, StoreFile resultFile) { + if (!isFlowRunRegion) { + return; + } + if (LOG.isDebugEnabled()) { + if (store != null) { + LOG.debug("postFlush store = " + store.getColumnFamilyName() + + " flushableSize=" + store.getFlushableSize() + + " flushedCellsCount=" + store.getFlushedCellsCount() + + " compactedCellsCount=" + store.getCompactedCellsCount() + + " majorCompactedCellsCount=" + + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" + + store.getMemstoreFlushSize() + " memstoreSize=" + + store.getMemStoreSize() + " size=" + store.getSize() + + " storeFilesCount=" + store.getStorefilesCount()); + } + } + } + + @Override + public InternalScanner preCompact( + ObserverContext e, Store store, + InternalScanner scanner, ScanType scanType, CompactionRequest request) + throws IOException { + + if (!isFlowRunRegion) { + return scanner; + } + FlowScannerOperation requestOp = FlowScannerOperation.MINOR_COMPACTION; + if (request != null) { + requestOp = (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION + : FlowScannerOperation.MINOR_COMPACTION); + LOG.info("Compactionrequest= " + request.toString() + " " + + requestOp.toString() + " RegionName=" + e.getEnvironment() + .getRegion().getRegionInfo().getRegionNameAsString()); + } + return new FlowScanner(e.getEnvironment(), scanner, requestOp); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java new file mode 100644 index 0000000..8fda9a8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java @@ -0,0 +1,190 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the flow run table. + */ +public class FlowRunRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final FlowRunRowKeyConverter flowRunRowKeyConverter = + new FlowRunRowKeyConverter(); + + public FlowRunRowKey(String clusterId, String userId, String flowName, + Long flowRunId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + /** + * Constructs a row key for the entity table as follows: { + * clusterId!userId!flowName!Inverted Flow Run Id}. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return flowRunRowKeyConverter.encode(this); + } + + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey Byte representation of row key. + * @return A FlowRunRowKey object. + */ + public static FlowRunRowKey parseRowKey(byte[] rowKey) { + return new FlowRunRowKeyConverter().decode(rowKey); + } + + /** + * returns the Flow Key as a verbose String output. + * @return String + */ + @Override + public String toString() { + StringBuilder flowKeyStr = new StringBuilder(); + flowKeyStr.append("{clusterId=" + clusterId); + flowKeyStr.append(" userId=" + userId); + flowKeyStr.append(" flowName=" + flowName); + flowKeyStr.append(" flowRunId="); + flowKeyStr.append(flowRunId); + flowKeyStr.append("}"); + return flowKeyStr.toString(); + } + + /** + * Encodes and decodes row key for flow run table. + * The row key is of the form : clusterId!userId!flowName!flowrunId. + * flowrunId is a long and rest are strings. + *

+ */ + final private static class FlowRunRowKeyConverter implements + KeyConverter { + + private FlowRunRowKeyConverter() { + } + + /** + * The flow run row key is of the form clusterId!userId!flowName!flowrunId + * with each segment separated by !. The sizes below indicate sizes of each + * one of these segments in sequence. clusterId, userId and flowName are + * strings. flowrunId is a long hence 8 bytes in size. Strings are variable + * in size (i.e. end whenever separator is encountered). This is used while + * decoding and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG }; + + /* + * (non-Javadoc) + * + * Encodes FlowRunRowKey object into a byte array with each component/field + * in FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an flow + * run row key of the form clusterId!userId!flowName!flowrunId If flowRunId + * in passed FlowRunRowKey object is null (and the fields preceding it i.e. + * clusterId, userId and flowName are not null), this returns a row key + * prefix of the form clusterId!userName!flowName! flowRunId is inverted + * while encoding as it helps maintain a descending order for flow keys in + * flow run table. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(FlowRunRowKey rowKey) { + byte[] first = + Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Separator + .encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); + if (rowKey.getFlowRunId() == null) { + return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); + } else { + // Note that flowRunId is a long, so we can't encode them all at the + // same + // time. + byte[] second = + Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); + return Separator.QUALIFIERS.join(first, second); + } + } + + /* + * (non-Javadoc) + * + * Decodes an flow run row key of the form + * clusterId!userId!flowName!flowrunId represented in byte format and + * converts it into an FlowRunRowKey object. flowRunId is inverted while + * decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public FlowRunRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 4) { + throw new IllegalArgumentException("the row key is not valid for " + + "a flow run"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + return new FlowRunRowKey(clusterId, userId, flowName, flowRunId); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java new file mode 100644 index 0000000..23ebc66 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey (without the flowRunId) for the flow run table. + */ +public class FlowRunRowKeyPrefix extends FlowRunRowKey implements + RowKeyPrefix { + + /** + * Constructs a row key prefix for the flow run table as follows: + * {@code clusterId!userI!flowName!}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + */ + public FlowRunRowKeyPrefix(String clusterId, String userId, + String flowName) { + super(clusterId, userId, flowName, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + // We know we're a FlowRunRowKey with null florRunId, so we can simply + // delegate + return super.getRowKey(); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java new file mode 100644 index 0000000..547bef0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The flow run table has column family info + * Stores per flow run information + * aggregated across applications. + * + * Metrics are also stored in the info column family. + * + * Example flow run table record: + * + *

+ * flow_run table
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | flow_version:version7        |
+ * | userName!  |                              |
+ * | flowName!  | running_apps:1               |
+ * | flowRunId  |                              |
+ * |            | min_start_time:1392995080000 |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | min_start_time:1392995081012 |
+ * |            | #0:appId2                    |
+ * |            |                              |
+ * |            | min_start_time:1392993083210 |
+ * |            | #0:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | max_end_time:1392993084018   |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapInputRecords:127        |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapInputRecords:31         |
+ * |            | #2:appId2                    |
+ * |            |                              |
+ * |            | m!mapInputRecords:37         |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapOutputRecords:181       |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapOutputRecords:37        |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public class FlowRunTable extends BaseTable { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun"; + + /** config param name that specifies the flowrun table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowrun table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun"; + + private static final Log LOG = LogFactory.getLog(FlowRunTable.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowRunTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowRunTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowRunTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy + flowRunTableDescp.addCoprocessor(FlowRunCoprocessor.class + .getCanonicalName()); + admin.createTable(flowRunTableDescp); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java new file mode 100644 index 0000000..0e3c8ee --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java @@ -0,0 +1,728 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.NumericValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Invoked via the coprocessor when a Get or a Scan is issued for flow run + * table. Looks through the list of cells per row, checks their tags and does + * operation on those cells as per the cell tags. Transforms reads of the stored + * metrics into calculated sums for each column Also, finds the min and max for + * start and end times in a flow run. + */ +class FlowScanner implements RegionScanner, Closeable { + + private static final Log LOG = LogFactory.getLog(FlowScanner.class); + + /** + * use a special application id to represent the flow id this is needed since + * TimestampGenerator parses the app id to generate a cell timestamp. + */ + private static final String FLOW_APP_ID = "application_00000000000_0000"; + + private final Region region; + private final InternalScanner flowRunScanner; + private final int batchSize; + private final long appFinalValueRetentionThreshold; + private RegionScanner regionScanner; + private boolean hasMore; + private byte[] currentRow; + private List availableCells = new ArrayList<>(); + private int currentIndex; + private FlowScannerOperation action = FlowScannerOperation.READ; + + FlowScanner(RegionCoprocessorEnvironment env, InternalScanner internalScanner, + FlowScannerOperation action) { + this(env, null, internalScanner, action); + } + + FlowScanner(RegionCoprocessorEnvironment env, Scan incomingScan, + InternalScanner internalScanner, FlowScannerOperation action) { + this.batchSize = incomingScan == null ? -1 : incomingScan.getBatch(); + // TODO initialize other scan attributes like Scan#maxResultSize + this.flowRunScanner = internalScanner; + if (internalScanner instanceof RegionScanner) { + this.regionScanner = (RegionScanner) internalScanner; + } + this.action = action; + if (env == null) { + this.appFinalValueRetentionThreshold = + YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD; + this.region = null; + } else { + this.region = env.getRegion(); + Configuration hbaseConf = env.getConfiguration(); + this.appFinalValueRetentionThreshold = hbaseConf.getLong( + YarnConfiguration.APP_FINAL_VALUE_RETENTION_THRESHOLD, + YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD); + } + if (LOG.isDebugEnabled()) { + LOG.debug(" batch size=" + batchSize); + } + } + + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo() + */ + @Override + public HRegionInfo getRegionInfo() { + return region.getRegionInfo(); + } + + @Override + public boolean nextRaw(List cells) throws IOException { + return nextRaw(cells, ScannerContext.newBuilder().build()); + } + + @Override + public boolean nextRaw(List cells, ScannerContext scannerContext) + throws IOException { + return nextInternal(cells, scannerContext); + } + + @Override + public boolean next(List cells) throws IOException { + return next(cells, ScannerContext.newBuilder().build()); + } + + @Override + public boolean next(List cells, ScannerContext scannerContext) + throws IOException { + return nextInternal(cells, scannerContext); + } + + /** + * Get value converter associated with a column or a column prefix. If nothing + * matches, generic converter is returned. + * @param colQualifierBytes + * @return value converter implementation. + */ + private static ValueConverter getValueConverter(byte[] colQualifierBytes) { + // Iterate over all the column prefixes for flow run table and get the + // appropriate converter for the column qualifier passed if prefix matches. + for (FlowRunColumnPrefix colPrefix : FlowRunColumnPrefix.values()) { + byte[] colPrefixBytes = colPrefix.getColumnPrefixBytes(""); + if (Bytes.compareTo(colPrefixBytes, 0, colPrefixBytes.length, + colQualifierBytes, 0, colPrefixBytes.length) == 0) { + return colPrefix.getValueConverter(); + } + } + // Iterate over all the columns for flow run table and get the + // appropriate converter for the column qualifier passed if match occurs. + for (FlowRunColumn column : FlowRunColumn.values()) { + if (Bytes.compareTo( + column.getColumnQualifierBytes(), colQualifierBytes) == 0) { + return column.getValueConverter(); + } + } + // Return generic converter if nothing matches. + return GenericConverter.getInstance(); + } + + /** + * This method loops through the cells in a given row of the + * {@link FlowRunTable}. It looks at the tags of each cell to figure out how + * to process the contents. It then calculates the sum or min or max for each + * column or returns the cell as is. + * + * @param cells + * @param scannerContext + * @return true if next row is available for the scanner, false otherwise + * @throws IOException + */ + private boolean nextInternal(List cells, ScannerContext scannerContext) + throws IOException { + Cell cell = null; + startNext(); + // Loop through all the cells in this row + // For min/max/metrics we do need to scan the entire set of cells to get the + // right one + // But with flush/compaction, the number of cells being scanned will go down + // cells are grouped per column qualifier then sorted by cell timestamp + // (latest to oldest) per column qualifier + // So all cells in one qualifier come one after the other before we see the + // next column qualifier + ByteArrayComparator comp = new ByteArrayComparator(); + byte[] previousColumnQualifier = Separator.EMPTY_BYTES; + AggregationOperation currentAggOp = null; + SortedSet currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR); + Set alreadySeenAggDim = new HashSet<>(); + int addedCnt = 0; + long currentTimestamp = System.currentTimeMillis(); + ValueConverter converter = null; + int limit = batchSize; + + while (limit <= 0 || addedCnt < limit) { + cell = peekAtNextCell(scannerContext); + if (cell == null) { + break; + } + byte[] currentColumnQualifier = CellUtil.cloneQualifier(cell); + if (previousColumnQualifier == null) { + // first time in loop + previousColumnQualifier = currentColumnQualifier; + } + + converter = getValueConverter(currentColumnQualifier); + if (comp.compare(previousColumnQualifier, currentColumnQualifier) != 0) { + addedCnt += emitCells(cells, currentColumnCells, currentAggOp, + converter, currentTimestamp); + resetState(currentColumnCells, alreadySeenAggDim); + previousColumnQualifier = currentColumnQualifier; + currentAggOp = getCurrentAggOp(cell); + converter = getValueConverter(currentColumnQualifier); + } + collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim, + converter, scannerContext); + nextCell(scannerContext); + } + if ((!currentColumnCells.isEmpty()) && ((limit <= 0 || addedCnt < limit))) { + addedCnt += emitCells(cells, currentColumnCells, currentAggOp, converter, + currentTimestamp); + if (LOG.isDebugEnabled()) { + if (addedCnt > 0) { + LOG.debug("emitted cells. " + addedCnt + " for " + this.action + + " rowKey=" + + FlowRunRowKey.parseRowKey(CellUtil.cloneRow(cells.get(0)))); + } else { + LOG.debug("emitted no cells for " + this.action); + } + } + } + return hasMore(); + } + + private AggregationOperation getCurrentAggOp(Cell cell) { + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + // We assume that all the operations for a particular column are the same + return HBaseTimelineStorageUtils.getAggregationOperationFromTagsList(tags); + } + + /** + * resets the parameters to an initialized state for next loop iteration. + * + * @param cell + * @param currentAggOp + * @param currentColumnCells + * @param alreadySeenAggDim + * @param collectedButNotEmitted + */ + private void resetState(SortedSet currentColumnCells, + Set alreadySeenAggDim) { + currentColumnCells.clear(); + alreadySeenAggDim.clear(); + } + + private void collectCells(SortedSet currentColumnCells, + AggregationOperation currentAggOp, Cell cell, + Set alreadySeenAggDim, ValueConverter converter, + ScannerContext scannerContext) throws IOException { + + if (currentAggOp == null) { + // not a min/max/metric cell, so just return it as is + currentColumnCells.add(cell); + return; + } + + switch (currentAggOp) { + case GLOBAL_MIN: + if (currentColumnCells.size() == 0) { + currentColumnCells.add(cell); + } else { + Cell currentMinCell = currentColumnCells.first(); + Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp, + (NumericValueConverter) converter); + if (!currentMinCell.equals(newMinCell)) { + currentColumnCells.remove(currentMinCell); + currentColumnCells.add(newMinCell); + } + } + break; + case GLOBAL_MAX: + if (currentColumnCells.size() == 0) { + currentColumnCells.add(cell); + } else { + Cell currentMaxCell = currentColumnCells.first(); + Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp, + (NumericValueConverter) converter); + if (!currentMaxCell.equals(newMaxCell)) { + currentColumnCells.remove(currentMaxCell); + currentColumnCells.add(newMaxCell); + } + } + break; + case SUM: + case SUM_FINAL: + if (LOG.isTraceEnabled()) { + LOG.trace("In collect cells " + + " FlowSannerOperation=" + + this.action + + " currentAggOp=" + + currentAggOp + + " cell qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(cell)) + + " cell value= " + + converter.decodeValue(CellUtil.cloneValue(cell)) + + " timestamp=" + cell.getTimestamp()); + } + + // only if this app has not been seen yet, add to current column cells + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + String aggDim = HBaseTimelineStorageUtils + .getAggregationCompactionDimension(tags); + if (!alreadySeenAggDim.contains(aggDim)) { + // if this agg dimension has already been seen, + // since they show up in sorted order + // we drop the rest which are older + // in other words, this cell is older than previously seen cells + // for that agg dim + // but when this agg dim is not seen, + // consider this cell in our working set + currentColumnCells.add(cell); + alreadySeenAggDim.add(aggDim); + } + break; + default: + break; + } // end of switch case + } + + /* + * Processes the cells in input param currentColumnCells and populates + * List cells as the output based on the input AggregationOperation + * parameter. + */ + private int emitCells(List cells, SortedSet currentColumnCells, + AggregationOperation currentAggOp, ValueConverter converter, + long currentTimestamp) throws IOException { + if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) { + return 0; + } + if (currentAggOp == null) { + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + if (LOG.isTraceEnabled()) { + LOG.trace("In emitCells " + this.action + " currentColumnCells size= " + + currentColumnCells.size() + " currentAggOp" + currentAggOp); + } + + switch (currentAggOp) { + case GLOBAL_MIN: + case GLOBAL_MAX: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + case SUM: + case SUM_FINAL: + switch (action) { + case FLUSH: + case MINOR_COMPACTION: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + case READ: + Cell sumCell = processSummation(currentColumnCells, + (NumericValueConverter) converter); + cells.add(sumCell); + return 1; + case MAJOR_COMPACTION: + List finalCells = processSummationMajorCompaction( + currentColumnCells, (NumericValueConverter) converter, + currentTimestamp); + cells.addAll(finalCells); + return finalCells.size(); + default: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + default: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + } + + /* + * Returns a cell whose value is the sum of all cell values in the input set. + * The new cell created has the timestamp of the most recent metric cell. The + * sum of a metric for a flow run is the summation at the point of the last + * metric update in that flow till that time. + */ + private Cell processSummation(SortedSet currentColumnCells, + NumericValueConverter converter) throws IOException { + Number sum = 0; + Number currentValue = 0; + long ts = 0L; + long mostCurrentTimestamp = 0L; + Cell mostRecentCell = null; + for (Cell cell : currentColumnCells) { + currentValue = (Number) converter.decodeValue(CellUtil.cloneValue(cell)); + ts = cell.getTimestamp(); + if (mostCurrentTimestamp < ts) { + mostCurrentTimestamp = ts; + mostRecentCell = cell; + } + sum = converter.add(sum, currentValue); + } + byte[] sumBytes = converter.encodeValue(sum); + Cell sumCell = + HBaseTimelineStorageUtils.createNewCell(mostRecentCell, sumBytes); + return sumCell; + } + + + /** + * Returns a list of cells that contains + * + * A) the latest cells for applications that haven't finished yet + * B) summation + * for the flow, based on applications that have completed and are older than + * a certain time + * + * The new cell created has the timestamp of the most recent metric cell. The + * sum of a metric for a flow run is the summation at the point of the last + * metric update in that flow till that time. + */ + @VisibleForTesting + List processSummationMajorCompaction( + SortedSet currentColumnCells, NumericValueConverter converter, + long currentTimestamp) + throws IOException { + Number sum = 0; + Number currentValue = 0; + long ts = 0L; + boolean summationDone = false; + List finalCells = new ArrayList(); + if (currentColumnCells == null) { + return finalCells; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("In processSummationMajorCompaction," + + " will drop cells older than " + currentTimestamp + + " CurrentColumnCells size=" + currentColumnCells.size()); + } + + for (Cell cell : currentColumnCells) { + AggregationOperation cellAggOp = getCurrentAggOp(cell); + // if this is the existing flow sum cell + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + String appId = HBaseTimelineStorageUtils + .getAggregationCompactionDimension(tags); + if (appId == FLOW_APP_ID) { + sum = converter.add(sum, currentValue); + summationDone = true; + if (LOG.isTraceEnabled()) { + LOG.trace("reading flow app id sum=" + sum); + } + } else { + currentValue = (Number) converter.decodeValue(CellUtil + .cloneValue(cell)); + // read the timestamp truncated by the generator + ts = TimestampGenerator.getTruncatedTimestamp(cell.getTimestamp()); + if ((cellAggOp == AggregationOperation.SUM_FINAL) + && ((ts + this.appFinalValueRetentionThreshold) + < currentTimestamp)) { + sum = converter.add(sum, currentValue); + summationDone = true; + if (LOG.isTraceEnabled()) { + LOG.trace("MAJOR COMPACTION loop sum= " + sum + + " discarding now: " + " qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(cell)) + " value=" + + converter.decodeValue(CellUtil.cloneValue(cell)) + + " timestamp=" + cell.getTimestamp() + " " + this.action); + } + } else { + // not a final value but it's the latest cell for this app + // so include this cell in the list of cells to write back + finalCells.add(cell); + } + } + } + if (summationDone) { + Cell anyCell = currentColumnCells.first(); + List tags = new ArrayList(); + Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + Bytes.toBytes(FLOW_APP_ID)); + tags.add(t); + t = new Tag(AggregationCompactionDimension.APPLICATION_ID.getTagType(), + Bytes.toBytes(FLOW_APP_ID)); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + Cell sumCell = HBaseTimelineStorageUtils.createNewCell( + CellUtil.cloneRow(anyCell), + CellUtil.cloneFamily(anyCell), + CellUtil.cloneQualifier(anyCell), + TimestampGenerator.getSupplementedTimestamp( + System.currentTimeMillis(), FLOW_APP_ID), + converter.encodeValue(sum), tagByteArray); + finalCells.add(sumCell); + if (LOG.isTraceEnabled()) { + LOG.trace("MAJOR COMPACTION final sum= " + sum + " for " + + Bytes.toString(CellUtil.cloneQualifier(sumCell)) + + " " + this.action); + } + LOG.info("After major compaction for qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(sumCell)) + + " with currentColumnCells.size=" + + currentColumnCells.size() + + " returning finalCells.size=" + finalCells.size() + + " with sum=" + sum.longValue() + + " with cell timestamp " + sumCell.getTimestamp()); + } else { + String qualifier = ""; + LOG.info("After major compaction for qualifier=" + qualifier + + " with currentColumnCells.size=" + + currentColumnCells.size() + + " returning finalCells.size=" + finalCells.size() + + " with zero sum=" + + sum.longValue()); + } + return finalCells; + } + + /** + * Determines which cell is to be returned based on the values in each cell + * and the comparison operation MIN or MAX. + * + * @param previouslyChosenCell + * @param currentCell + * @param currentAggOp + * @return the cell which is the min (or max) cell + * @throws IOException + */ + private Cell compareCellValues(Cell previouslyChosenCell, Cell currentCell, + AggregationOperation currentAggOp, NumericValueConverter converter) + throws IOException { + if (previouslyChosenCell == null) { + return currentCell; + } + try { + Number previouslyChosenCellValue = (Number)converter.decodeValue( + CellUtil.cloneValue(previouslyChosenCell)); + Number currentCellValue = (Number) converter.decodeValue(CellUtil + .cloneValue(currentCell)); + switch (currentAggOp) { + case GLOBAL_MIN: + if (converter.compare( + currentCellValue, previouslyChosenCellValue) < 0) { + // new value is minimum, hence return this cell + return currentCell; + } else { + // previously chosen value is miniumum, hence return previous min cell + return previouslyChosenCell; + } + case GLOBAL_MAX: + if (converter.compare( + currentCellValue, previouslyChosenCellValue) > 0) { + // new value is max, hence return this cell + return currentCell; + } else { + // previously chosen value is max, hence return previous max cell + return previouslyChosenCell; + } + default: + return currentCell; + } + } catch (IllegalArgumentException iae) { + LOG.error("caught iae during conversion to long ", iae); + return currentCell; + } + } + + @Override + public void close() throws IOException { + if (flowRunScanner != null) { + flowRunScanner.close(); + } else { + LOG.warn("scanner close called but scanner is null"); + } + } + + /** + * Called to signal the start of the next() call by the scanner. + */ + public void startNext() { + currentRow = null; + } + + /** + * Returns whether or not the underlying scanner has more rows. + */ + public boolean hasMore() { + return currentIndex < availableCells.size() ? true : hasMore; + } + + /** + * Returns the next available cell for the current row and advances the + * pointer to the next cell. This method can be called multiple times in a row + * to advance through all the available cells. + * + * @param scannerContext + * context information for the batch of cells under consideration + * @return the next available cell or null if no more cells are available for + * the current row + * @throws IOException + */ + public Cell nextCell(ScannerContext scannerContext) throws IOException { + Cell cell = peekAtNextCell(scannerContext); + if (cell != null) { + currentIndex++; + } + return cell; + } + + /** + * Returns the next available cell for the current row, without advancing the + * pointer. Calling this method multiple times in a row will continue to + * return the same cell. + * + * @param scannerContext + * context information for the batch of cells under consideration + * @return the next available cell or null if no more cells are available for + * the current row + * @throws IOException if any problem is encountered while grabbing the next + * cell. + */ + public Cell peekAtNextCell(ScannerContext scannerContext) throws IOException { + if (currentIndex >= availableCells.size()) { + // done with current batch + availableCells.clear(); + currentIndex = 0; + hasMore = flowRunScanner.next(availableCells, scannerContext); + } + Cell cell = null; + if (currentIndex < availableCells.size()) { + cell = availableCells.get(currentIndex); + if (currentRow == null) { + currentRow = CellUtil.cloneRow(cell); + } else if (!CellUtil.matchingRow(cell, currentRow)) { + // moved on to the next row + // don't use the current cell + // also signal no more cells for this row + return null; + } + } + return cell; + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize() + */ + @Override + public long getMaxResultSize() { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's scanner is not a RegionScanner"); + } + return regionScanner.getMaxResultSize(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint() + */ + @Override + public long getMvccReadPoint() { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.getMvccReadPoint(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone() + */ + @Override + public boolean isFilterDone() throws IOException { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.isFilterDone(); + + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[]) + */ + @Override + public boolean reseek(byte[] bytes) throws IOException { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.reseek() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.reseek(bytes); + } + + @Override + public int getBatch() { + return batchSize; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java new file mode 100644 index 0000000..73c666f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + + +/** + * Identifies the scanner operation on the {@link FlowRunTable}. + */ +public enum FlowScannerOperation { + + /** + * If the scanner is opened for reading + * during preGet or preScan. + */ + READ, + + /** + * If the scanner is opened during preFlush. + */ + FLUSH, + + /** + * If the scanner is opened during minor Compaction. + */ + MINOR_COMPACTION, + + /** + * If the scanner is opened during major Compaction. + */ + MAJOR_COMPACTION +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java new file mode 100644 index 0000000..04963f3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.flow + * contains classes related to implementation for flow related tables, viz. flow + * run table and flow activity table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java new file mode 100644 index 0000000..e78db2a --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage contains + * classes which define and implement reading and writing to backend storage. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java new file mode 100644 index 0000000..fbfd900 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java @@ -0,0 +1,481 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +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.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.HBaseTimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for application entities that are stored in the + * application table. + */ +class ApplicationEntityReader extends GenericEntityReader { + private static final ApplicationTable APPLICATION_TABLE = + new ApplicationTable(); + + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); + } + + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link ApplicationTable}. + */ + protected BaseTable getTable() { + return APPLICATION_TABLE; + } + + /** + * This method is called only for multiple entity reads. + */ + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + // Filters here cannot be null for multiple entity reads as they are set in + // augmentParams if null. + TimelineEntityFilters filters = getFilters(); + FilterList listBasedOnFilters = new FilterList(); + // Create filter list based on created time range and add it to + // listBasedOnFilters. + long createdTimeBegin = filters.getCreatedTimeBegin(); + long createdTimeEnd = filters.getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter( + HBaseTimelineFilterUtils.createSingleColValueFiltersByRange( + ApplicationColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd)); + } + // Create filter list based on metric filters and add it to + // listBasedOnFilters. + TimelineFilterList metricFilters = filters.getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + HBaseTimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.METRIC, metricFilters)); + } + // Create filter list based on config filters and add it to + // listBasedOnFilters. + TimelineFilterList configFilters = filters.getConfigFilters(); + if (configFilters != null && !configFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + HBaseTimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.CONFIG, configFilters)); + } + // Create filter list based on info filters and add it to listBasedOnFilters + TimelineFilterList infoFilters = filters.getInfoFilters(); + if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + HBaseTimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.INFO, infoFilters)); + } + return listBasedOnFilters; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of + * application table. + * + * @param list filter list to which qualifier filters have to be added. + */ + @Override + protected void updateFixedColumns(FilterList list) { + for (ApplicationColumn column : ApplicationColumn.values()) { + list.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryComparator(column.getColumnQualifierBytes()))); + } + } + + /** + * Creates a filter list which indicates that only some of the column + * qualifiers in the info column family will be returned in result. + * + * @return filter list. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterListForColsOfInfoFamily() + throws IOException { + FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); + // Add filters for each column in entity table. + updateFixedColumns(infoFamilyColsFilter); + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // If INFO field has to be retrieved, add a filter for fetching columns + // with INFO column prefix. + if (hasField(fieldsToRetrieve, Field.INFO)) { + infoFamilyColsFilter.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.INFO)); + } + TimelineFilterList relatesTo = getFilters().getRelatesTo(); + if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { + // If RELATES_TO field has to be retrieved, add a filter for fetching + // columns with RELATES_TO column prefix. + infoFamilyColsFilter.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.RELATES_TO)); + } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain RELATES_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // relatesTo filters are specified. relatesTo filters will then be + // matched after fetching rows from HBase. + Set relatesToCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.RELATES_TO, relatesToCols)); + } + TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + // If IS_RELATED_TO field has to be retrieved, add a filter for fetching + // columns with IS_RELATED_TO column prefix. + infoFamilyColsFilter.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); + } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain IS_RELATED_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // isRelatedTo filters are specified. isRelatedTo filters will then be + // matched after fetching rows from HBase. + Set isRelatedToCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols)); + } + TimelineFilterList eventFilters = getFilters().getEventFilters(); + if (hasField(fieldsToRetrieve, Field.EVENTS)) { + // If EVENTS field has to be retrieved, add a filter for fetching columns + // with EVENT column prefix. + infoFamilyColsFilter.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.EVENT)); + } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){ + // Even if fields to retrieve does not contain EVENTS, we still need to + // have a filter to fetch some of the column qualifiers on the basis of + // event filters specified. Event filters will then be matched after + // fetching rows from HBase. + Set eventCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.EVENT, eventCols)); + } + return infoFamilyColsFilter; + } + + /** + * Exclude column prefixes via filters which are not required(based on fields + * to retrieve) from info column family. These filters are added to filter + * list which contains a filter for getting info column family. + * + * @param infoColFamilyList filter list for info column family. + */ + private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // Events not required. + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + infoColFamilyList.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT)); + } + // info not required. + if (!hasField(fieldsToRetrieve, Field.INFO)) { + infoColFamilyList.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO)); + } + // is related to not required. + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + infoColFamilyList.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); + } + // relates to not required. + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + infoColFamilyList.addFilter( + HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO)); + } + } + + /** + * Updates filter list based on fields for confs and metrics to retrieve. + * + * @param listBasedOnFields filter list based on fields. + * @throws IOException if any problem occurs while updating filter list. + */ + private void updateFilterForConfsAndMetricsToRetrieve( + FilterList listBasedOnFields) throws IOException { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Please note that if confsToRetrieve is specified, we would have added + // CONFS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { + // Create a filter list for configs. + listBasedOnFields.addFilter(HBaseTimelineFilterUtils. + createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getConfsToRetrieve(), + ApplicationColumnFamily.CONFIGS, ApplicationColumnPrefix.CONFIG)); + } + + // Please note that if metricsToRetrieve is specified, we would have added + // METRICS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { + // Create a filter list for metrics. + listBasedOnFields.addFilter(HBaseTimelineFilterUtils. + createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getMetricsToRetrieve(), + ApplicationColumnFamily.METRICS, ApplicationColumnPrefix.METRIC)); + } + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + if (!needCreateFilterListBasedOnFields()) { + // Fetch all the columns. No need of a filter. + return null; + } + FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); + FilterList infoColFamilyList = new FilterList(); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, + new BinaryComparator(ApplicationColumnFamily.INFO.getBytes())); + infoColFamilyList.addFilter(infoColumnFamily); + if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { + // We can fetch only some of the columns from info family. + infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); + } else { + // Exclude column prefixes in info column family which are not required + // based on fields to retrieve. + excludeFieldsFromInfoColFamily(infoColFamilyList); + } + listBasedOnFields.addFilter(infoColFamilyList); + + updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); + return listBasedOnFields; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId()); + byte[] rowKey = applicationRowKey.getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + return getTable().getResult(hbaseConf, conn, get); + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull( + getDataToRetrieve(), "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + } else { + Preconditions.checkNotNull(getContext().getUserId(), + "userId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowName(), + "flowName shouldn't be null"); + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + TimelineReaderContext context = getContext(); + if (isSingleEntityRead()) { + // Get flow context information from AppToFlow table. + if (context.getFlowName() == null || context.getFlowRunId() == null + || context.getUserId() == null) { + AppToFlowRowKey appToFlowRowKey = + new AppToFlowRowKey(context.getClusterId(), context.getAppId()); + FlowContext flowContext = + lookupFlowContext(appToFlowRowKey, + hbaseConf, conn); + context.setFlowName(flowContext.getFlowName()); + context.setFlowRunId(flowContext.getFlowRunId()); + context.setUserId(flowContext.getUserId()); + } + } + // Add configs/metrics to fields to retrieve if confsToRetrieve and/or + // metricsToRetrieve are specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException { + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + // Whether or not flowRunID is null doesn't matter, the + // ApplicationRowKeyPrefix will do the right thing. + RowKeyPrefix applicationRowKeyPrefix = + new ApplicationRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName(), + context.getFlowRunId()); + scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix()); + FilterList newList = new FilterList(); + newList.addFilter(new PageFilter(getFilters().getLimit())); + if (filterList != null && !filterList.getFilters().isEmpty()) { + newList.addFilter(filterList); + } + scan.setFilter(newList); + scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + entity.setType(TimelineEntityType.YARN_APPLICATION.toString()); + String entityId = ApplicationColumn.ID.readResult(result).toString(); + entity.setId(entityId); + + TimelineEntityFilters filters = getFilters(); + // fetch created time + Long createdTime = (Long) ApplicationColumn.CREATED_TIME.readResult(result); + entity.setCreatedTime(createdTime); + + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // fetch is related to entities and match isRelatedTo filter. If isRelatedTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // isRelatedTo are not set in HBase scan. + boolean checkIsRelatedTo = + !isSingleEntityRead() && filters.getIsRelatedTo() != null && + filters.getIsRelatedTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { + readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO, + true); + if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, + Field.IS_RELATED_TO)) { + entity.getIsRelatedToEntities().clear(); + } + } + + // fetch relates to entities and match relatesTo filter. If relatesTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // relatesTo are not set in HBase scan. + boolean checkRelatesTo = + !isSingleEntityRead() && filters.getRelatesTo() != null && + filters.getRelatesTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.RELATES_TO) || + checkRelatesTo) { + readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO, + false); + if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + entity.getRelatesToEntities().clear(); + } + } + + // fetch info if fieldsToRetrieve contains INFO or ALL. + if (hasField(fieldsToRetrieve, Field.INFO)) { + readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false); + } + + // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. + if (hasField(fieldsToRetrieve, Field.CONFIGS)) { + readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true); + } + + // fetch events and match event filters if they exist. If event filters do + // not match, entity would be dropped. We have to match filters locally + // as relevant HBase filters to filter out rows on the basis of events + // are not set in HBase scan. + boolean checkEvents = + !isSingleEntityRead() && filters.getEventFilters() != null && + filters.getEventFilters().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { + readEvents(entity, result, ApplicationColumnPrefix.EVENT); + if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + entity.getEvents().clear(); + } + } + + // fetch metrics if fieldsToRetrieve contains METRICS or ALL. + if (hasField(fieldsToRetrieve, Field.METRICS)) { + readMetrics(entity, result, ApplicationColumnPrefix.METRIC); + } + return entity; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java new file mode 100644 index 0000000..9ba5e38 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +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.BaseTable; +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.flow.FlowActivityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for flow activity entities that are stored in the + * flow activity table. + */ +class FlowActivityEntityReader extends TimelineEntityReader { + private static final FlowActivityTable FLOW_ACTIVITY_TABLE = + new FlowActivityTable(); + + /** + * Used to convert Long key components to and from storage format. + */ + private final KeyConverter longKeyConverter = new LongKeyConverter(); + + + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); + } + + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link FlowActivityTable}. + */ + @Override + protected BaseTable getTable() { + return FLOW_ACTIVITY_TABLE; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + createFiltersIfNull(); + } + + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + return null; + } + + @Override + protected FilterList constructFilterListBasedOnFields() { + return null; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + throw new UnsupportedOperationException( + "we don't support a single entity query"); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException { + Scan scan = new Scan(); + String clusterId = getContext().getClusterId(); + if (getFilters().getCreatedTimeBegin() == 0L && + getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) { + // All records have to be chosen. + scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId) + .getRowKeyPrefix()); + } else { + scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters() + .getCreatedTimeEnd()).getRowKeyPrefix()); + scan.setStopRow(new FlowActivityRowKeyPrefix(clusterId, (getFilters() + .getCreatedTimeBegin() <= 0 ? 0 + : (getFilters().getCreatedTimeBegin() - 1))).getRowKeyPrefix()); + } + // use the page filter to limit the result to the page size + // the scanner may still return more than the limit; therefore we need to + // read the right number as we iterate + scan.setFilter(new PageFilter(getFilters().getLimit())); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow()); + + Long time = rowKey.getDayTimestamp(); + String user = rowKey.getUserId(); + String flowName = rowKey.getFlowName(); + + FlowActivityEntity flowActivity = new FlowActivityEntity( + getContext().getClusterId(), time, user, flowName); + // set the id + flowActivity.setId(flowActivity.getId()); + // get the list of run ids along with the version that are associated with + // this flow on this day + Map runIdsMap = + FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter); + for (Map.Entry e : runIdsMap.entrySet()) { + Long runId = e.getKey(); + String version = (String)e.getValue(); + FlowRunEntity flowRun = new FlowRunEntity(); + flowRun.setUser(user); + flowRun.setName(flowName); + flowRun.setRunId(runId); + flowRun.setVersion(version); + // set the id + flowRun.setId(flowRun.getId()); + flowActivity.addFlowRun(flowRun); + } + + return flowActivity; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java new file mode 100644 index 0000000..310486d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java @@ -0,0 +1,269 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +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.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.HBaseTimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for flow run entities that are stored in the flow run + * table. + */ +class FlowRunEntityReader extends TimelineEntityReader { + private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable(); + + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve, true); + } + + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link FlowRunTable}. + */ + @Override + protected BaseTable getTable() { + return FLOW_RUN_TABLE; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getDataToRetrieve(), + "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getUserId(), + "userId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowName(), + "flowName shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getFlowRunId(), + "flowRunId shouldn't be null"); + } + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + if (!isSingleEntityRead() && fieldsToRetrieve != null) { + for (Field field : fieldsToRetrieve) { + if (field != Field.ALL && field != Field.METRICS) { + throw new BadRequestException("Invalid field " + field + + " specified while querying flow runs."); + } + } + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) { + // Add metrics to fields to retrieve if metricsToRetrieve is specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + protected FilterList constructFilterListBasedOnFilters() throws IOException { + FilterList listBasedOnFilters = new FilterList(); + // Filter based on created time range. + Long createdTimeBegin = getFilters().getCreatedTimeBegin(); + Long createdTimeEnd = getFilters().getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils + .createSingleColValueFiltersByRange(FlowRunColumn.MIN_START_TIME, + createdTimeBegin, createdTimeEnd)); + } + // Filter based on metric filters. + TimelineFilterList metricFilters = getFilters().getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils.createHBaseFilterList( + FlowRunColumnPrefix.METRIC, metricFilters)); + } + return listBasedOnFilters; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of flow + * run table. + * + * @return filter list to which qualifier filters have been added. + */ + private FilterList updateFixedColumns() { + FilterList columnsList = new FilterList(Operator.MUST_PASS_ONE); + for (FlowRunColumn column : FlowRunColumn.values()) { + columnsList.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryComparator(column.getColumnQualifierBytes()))); + } + return columnsList; + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + FilterList list = new FilterList(Operator.MUST_PASS_ONE); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( + FlowRunColumnFamily.INFO.getBytes())); + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // If multiple entities have to be retrieved, check if metrics have to be + // retrieved and if not, add a filter so that metrics can be excluded. + // Metrics are always returned if we are reading a single entity. + if (!isSingleEntityRead() + && !hasField(dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) { + FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE); + infoColFamilyList.addFilter(infoColumnFamily); + infoColFamilyList.addFilter(new QualifierFilter(CompareOp.NOT_EQUAL, + new BinaryPrefixComparator(FlowRunColumnPrefix.METRIC + .getColumnPrefixBytes("")))); + list.addFilter(infoColFamilyList); + } else { + // Check if metricsToRetrieve are specified and if they are, create a + // filter list for info column family by adding flow run tables columns + // and a list for metrics to retrieve. Pls note that fieldsToRetrieve + // will have METRICS added to it if metricsToRetrieve are specified + // (in augmentParams()). + TimelineFilterList metricsToRetrieve = + dataToRetrieve.getMetricsToRetrieve(); + if (metricsToRetrieve != null + && !metricsToRetrieve.getFilterList().isEmpty()) { + FilterList infoColFamilyList = new FilterList(); + infoColFamilyList.addFilter(infoColumnFamily); + FilterList columnsList = updateFixedColumns(); + columnsList.addFilter(HBaseTimelineFilterUtils.createHBaseFilterList( + FlowRunColumnPrefix.METRIC, metricsToRetrieve)); + infoColFamilyList.addFilter(columnsList); + list.addFilter(infoColFamilyList); + } + } + return list; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + FlowRunRowKey flowRunRowKey = + new FlowRunRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId()); + byte[] rowKey = flowRunRowKey.getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(Integer.MAX_VALUE); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + return getTable().getResult(hbaseConf, conn, get); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + RowKeyPrefix flowRunRowKeyPrefix = + new FlowRunRowKeyPrefix(context.getClusterId(), context.getUserId(), + context.getFlowName()); + scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix()); + FilterList newList = new FilterList(); + newList.addFilter(new PageFilter(getFilters().getLimit())); + if (filterList != null && !filterList.getFilters().isEmpty()) { + newList.addFilter(filterList); + } + scan.setFilter(newList); + scan.setMaxVersions(Integer.MAX_VALUE); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + TimelineReaderContext context = getContext(); + FlowRunEntity flowRun = new FlowRunEntity(); + flowRun.setUser(context.getUserId()); + flowRun.setName(context.getFlowName()); + if (isSingleEntityRead()) { + flowRun.setRunId(context.getFlowRunId()); + } else { + FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow()); + flowRun.setRunId(rowKey.getFlowRunId()); + } + + // read the start time + Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result); + if (startTime != null) { + flowRun.setStartTime(startTime.longValue()); + } + + // read the end time if available + Long endTime = (Long) FlowRunColumn.MAX_END_TIME.readResult(result); + if (endTime != null) { + flowRun.setMaxEndTime(endTime.longValue()); + } + + // read the flow version + String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result); + if (version != null) { + flowRun.setVersion(version); + } + + // read metrics if its a single entity query or if METRICS are part of + // fieldsToRetrieve. + if (isSingleEntityRead() + || hasField(getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) { + readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC); + } + + // set the id + flowRun.setId(flowRun.getId()); + return flowRun; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java new file mode 100644 index 0000000..5c035f1 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java @@ -0,0 +1,648 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.QualifierFilter; +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.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.HBaseTimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +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.entity.EntityColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; +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.EntityTable; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for generic entities that are stored in the entity + * table. + */ +class GenericEntityReader extends TimelineEntityReader { + private static final EntityTable ENTITY_TABLE = new EntityTable(); + + /** + * Used to look up the flow context. + */ + private final AppToFlowTable appToFlowTable = new AppToFlowTable(); + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + public GenericEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, + boolean sortedKeys) { + super(ctxt, entityFilters, toRetrieve, sortedKeys); + } + + public GenericEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link EntityTable}. + */ + protected BaseTable getTable() { + return ENTITY_TABLE; + } + + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + // Filters here cannot be null for multiple entity reads as they are set in + // augmentParams if null. + FilterList listBasedOnFilters = new FilterList(); + TimelineEntityFilters filters = getFilters(); + // Create filter list based on created time range and add it to + // listBasedOnFilters. + long createdTimeBegin = filters.getCreatedTimeBegin(); + long createdTimeEnd = filters.getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils + .createSingleColValueFiltersByRange(EntityColumn.CREATED_TIME, + createdTimeBegin, createdTimeEnd)); + } + // Create filter list based on metric filters and add it to + // listBasedOnFilters. + TimelineFilterList metricFilters = filters.getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.METRIC, metricFilters)); + } + // Create filter list based on config filters and add it to + // listBasedOnFilters. + TimelineFilterList configFilters = filters.getConfigFilters(); + if (configFilters != null && !configFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.CONFIG, configFilters)); + } + // Create filter list based on info filters and add it to listBasedOnFilters + TimelineFilterList infoFilters = filters.getInfoFilters(); + if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(HBaseTimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.INFO, infoFilters)); + } + return listBasedOnFilters; + } + + /** + * Check if we need to fetch only some of the event columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + private boolean fetchPartialEventCols(TimelineFilterList eventFilters, + EnumSet fieldsToRetrieve) { + return (eventFilters != null && !eventFilters.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.EVENTS)); + } + + /** + * Check if we need to fetch only some of the relates_to columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo, + EnumSet fieldsToRetrieve) { + return (relatesTo != null && !relatesTo.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.RELATES_TO)); + } + + /** + * Check if we need to fetch only some of the is_related_to columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + private boolean fetchPartialIsRelatedToCols(TimelineFilterList isRelatedTo, + EnumSet fieldsToRetrieve) { + return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.IS_RELATED_TO)); + } + + /** + * Check if we need to fetch only some of the columns based on event filters, + * relatesto and isrelatedto from info family. + * + * @return true, if we need to fetch only some of the columns, false if we + * need to fetch all the columns under info column family. + */ + protected boolean fetchPartialColsFromInfoFamily() { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + TimelineEntityFilters filters = getFilters(); + return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) + || fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) + || fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), + fieldsToRetrieve); + } + + /** + * Check if we need to create filter list based on fields. We need to create a + * filter list iff all fields need not be retrieved or we have some specific + * fields or metrics to retrieve. We also need to create a filter list if we + * have relationships(relatesTo/isRelatedTo) and event filters specified for + * the query. + * + * @return true if we need to create the filter list, false otherwise. + */ + protected boolean needCreateFilterListBasedOnFields() { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Check if all fields are to be retrieved or not. If all fields have to + // be retrieved, also check if we have some metrics or configs to + // retrieve specified for the query because then a filter list will have + // to be created. + boolean flag = + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) + || (dataToRetrieve.getConfsToRetrieve() != null && !dataToRetrieve + .getConfsToRetrieve().getFilterList().isEmpty()) + || (dataToRetrieve.getMetricsToRetrieve() != null && !dataToRetrieve + .getMetricsToRetrieve().getFilterList().isEmpty()); + // Filters need to be checked only if we are reading multiple entities. If + // condition above is false, we check if there are relationships(relatesTo/ + // isRelatedTo) and event filters specified for the query. + if (!flag && !isSingleEntityRead()) { + TimelineEntityFilters filters = getFilters(); + flag = + (filters.getEventFilters() != null && !filters.getEventFilters() + .getFilterList().isEmpty()) + || (filters.getIsRelatedTo() != null && !filters.getIsRelatedTo() + .getFilterList().isEmpty()) + || (filters.getRelatesTo() != null && !filters.getRelatesTo() + .getFilterList().isEmpty()); + } + return flag; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of + * entity table. + * + * @param list filter list to which qualifier filters have to be added. + */ + protected void updateFixedColumns(FilterList list) { + for (EntityColumn column : EntityColumn.values()) { + list.addFilter(new QualifierFilter(CompareOp.EQUAL, new BinaryComparator( + column.getColumnQualifierBytes()))); + } + } + + /** + * Creates a filter list which indicates that only some of the column + * qualifiers in the info column family will be returned in result. + * + * @param isApplication If true, it means operations are to be performed for + * application table, otherwise for entity table. + * @return filter list. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterListForColsOfInfoFamily() throws IOException { + FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); + // Add filters for each column in entity table. + updateFixedColumns(infoFamilyColsFilter); + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // If INFO field has to be retrieved, add a filter for fetching columns + // with INFO column prefix. + if (hasField(fieldsToRetrieve, Field.INFO)) { + infoFamilyColsFilter + .addFilter(HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, EntityColumnPrefix.INFO)); + } + TimelineFilterList relatesTo = getFilters().getRelatesTo(); + if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { + // If RELATES_TO field has to be retrieved, add a filter for fetching + // columns with RELATES_TO column prefix. + infoFamilyColsFilter.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.EQUAL, + EntityColumnPrefix.RELATES_TO)); + } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain RELATES_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // relatesTo filters are specified. relatesTo filters will then be + // matched after fetching rows from HBase. + Set relatesToCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.RELATES_TO, relatesToCols)); + } + TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + // If IS_RELATED_TO field has to be retrieved, add a filter for fetching + // columns with IS_RELATED_TO column prefix. + infoFamilyColsFilter.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.EQUAL, + EntityColumnPrefix.IS_RELATED_TO)); + } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain IS_RELATED_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // isRelatedTo filters are specified. isRelatedTo filters will then be + // matched after fetching rows from HBase. + Set isRelatedToCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols)); + } + TimelineFilterList eventFilters = getFilters().getEventFilters(); + if (hasField(fieldsToRetrieve, Field.EVENTS)) { + // If EVENTS field has to be retrieved, add a filter for fetching columns + // with EVENT column prefix. + infoFamilyColsFilter + .addFilter(HBaseTimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, EntityColumnPrefix.EVENT)); + } else if (eventFilters != null && + !eventFilters.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain EVENTS, we still need to + // have a filter to fetch some of the column qualifiers on the basis of + // event filters specified. Event filters will then be matched after + // fetching rows from HBase. + Set eventCols = + HBaseTimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.EVENT, eventCols)); + } + return infoFamilyColsFilter; + } + + /** + * Exclude column prefixes via filters which are not required(based on fields + * to retrieve) from info column family. These filters are added to filter + * list which contains a filter for getting info column family. + * + * @param infoColFamilyList filter list for info column family. + */ + private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // Events not required. + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + infoColFamilyList.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.EVENT)); + } + // info not required. + if (!hasField(fieldsToRetrieve, Field.INFO)) { + infoColFamilyList.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.INFO)); + } + // is related to not required. + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + infoColFamilyList.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.IS_RELATED_TO)); + } + // relates to not required. + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + infoColFamilyList.addFilter(HBaseTimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.RELATES_TO)); + } + } + + /** + * Updates filter list based on fields for confs and metrics to retrieve. + * + * @param listBasedOnFields filter list based on fields. + * @throws IOException if any problem occurs while updating filter list. + */ + private void updateFilterForConfsAndMetricsToRetrieve( + FilterList listBasedOnFields) throws IOException { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Please note that if confsToRetrieve is specified, we would have added + // CONFS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { + // Create a filter list for configs. + listBasedOnFields.addFilter(HBaseTimelineFilterUtils + .createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getConfsToRetrieve(), EntityColumnFamily.CONFIGS, + EntityColumnPrefix.CONFIG)); + } + + // Please note that if metricsToRetrieve is specified, we would have added + // METRICS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { + // Create a filter list for metrics. + listBasedOnFields.addFilter(HBaseTimelineFilterUtils + .createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getMetricsToRetrieve(), + EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC)); + } + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + if (!needCreateFilterListBasedOnFields()) { + // Fetch all the columns. No need of a filter. + return null; + } + FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); + FilterList infoColFamilyList = new FilterList(); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( + EntityColumnFamily.INFO.getBytes())); + infoColFamilyList.addFilter(infoColumnFamily); + if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { + // We can fetch only some of the columns from info family. + infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); + } else { + // Exclude column prefixes in info column family which are not required + // based on fields to retrieve. + excludeFieldsFromInfoColFamily(infoColFamilyList); + } + listBasedOnFields.addFilter(infoColFamilyList); + updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); + return listBasedOnFields; + } + + /** + * Looks up flow context from AppToFlow table. + * + * @param appToFlowRowKey to identify Cluster and App Ids. + * @param hbaseConf HBase configuration. + * @param conn HBase Connection. + * @return flow context information. + * @throws IOException if any problem occurs while fetching flow information. + */ + protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey, + Configuration hbaseConf, Connection conn) throws IOException { + byte[] rowKey = appToFlowRowKey.getRowKey(); + Get get = new Get(rowKey); + Result result = appToFlowTable.getResult(hbaseConf, conn, get); + if (result != null && !result.isEmpty()) { + return new FlowContext(AppToFlowColumn.USER_ID.readResult(result) + .toString(), AppToFlowColumn.FLOW_ID.readResult(result).toString(), + ((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result)) + .longValue()); + } else { + throw new NotFoundException( + "Unable to find the context flow ID and flow run ID for clusterId=" + + appToFlowRowKey.getClusterId() + ", appId=" + + appToFlowRowKey.getAppId()); + } + } + + /** + * Encapsulates flow context information. + */ + protected static class FlowContext { + private final String userId; + private final String flowName; + private final Long flowRunId; + + public FlowContext(String user, String flowName, Long flowRunId) { + this.userId = user; + this.flowName = flowName; + this.flowRunId = flowRunId; + } + + protected String getUserId() { + return userId; + } + + protected String getFlowName() { + return flowName; + } + + protected Long getFlowRunId() { + return flowRunId; + } + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getDataToRetrieve(), + "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getEntityId(), + "entityId shouldn't be null"); + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + TimelineReaderContext context = getContext(); + // In reality all three should be null or neither should be null + if (context.getFlowName() == null || context.getFlowRunId() == null + || context.getUserId() == null) { + // Get flow context information from AppToFlow table. + AppToFlowRowKey appToFlowRowKey = + new AppToFlowRowKey(context.getClusterId(), context.getAppId()); + FlowContext flowContext = + lookupFlowContext(appToFlowRowKey, hbaseConf, conn); + context.setFlowName(flowContext.flowName); + context.setFlowRunId(flowContext.flowRunId); + context.setUserId(flowContext.userId); + } + // Add configs/metrics to fields to retrieve if confsToRetrieve and/or + // metricsToRetrieve are specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + byte[] rowKey = + new EntityRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId(), + context.getEntityType(), context.getEntityId()).getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + return getTable().getResult(hbaseConf, conn, get); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + // Scan through part of the table to find the entities belong to one app + // and one type + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + RowKeyPrefix entityRowKeyPrefix = + new EntityRowKeyPrefix(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId(), + context.getEntityType()); + scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix()); + scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + scan.setFilter(filterList); + } + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + String entityType = EntityColumn.TYPE.readResult(result).toString(); + entity.setType(entityType); + String entityId = EntityColumn.ID.readResult(result).toString(); + entity.setId(entityId); + + TimelineEntityFilters filters = getFilters(); + // fetch created time + Long createdTime = (Long) EntityColumn.CREATED_TIME.readResult(result); + entity.setCreatedTime(createdTime); + + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // fetch is related to entities and match isRelatedTo filter. If isRelatedTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // isRelatedTo are not set in HBase scan. + boolean checkIsRelatedTo = + !isSingleEntityRead() && filters.getIsRelatedTo() != null + && filters.getIsRelatedTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { + readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true); + if (checkIsRelatedTo + && !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + entity.getIsRelatedToEntities().clear(); + } + } + + // fetch relates to entities and match relatesTo filter. If relatesTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // relatesTo are not set in HBase scan. + boolean checkRelatesTo = + !isSingleEntityRead() && filters.getRelatesTo() != null + && filters.getRelatesTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.RELATES_TO) + || checkRelatesTo) { + readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false); + if (checkRelatesTo + && !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + entity.getRelatesToEntities().clear(); + } + } + + // fetch info if fieldsToRetrieve contains INFO or ALL. + if (hasField(fieldsToRetrieve, Field.INFO)) { + readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false); + } + + // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. + if (hasField(fieldsToRetrieve, Field.CONFIGS)) { + readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true); + } + + // fetch events and match event filters if they exist. If event filters do + // not match, entity would be dropped. We have to match filters locally + // as relevant HBase filters to filter out rows on the basis of events + // are not set in HBase scan. + boolean checkEvents = + !isSingleEntityRead() && filters.getEventFilters() != null + && filters.getEventFilters().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { + readEvents(entity, result, EntityColumnPrefix.EVENT); + if (checkEvents + && !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + entity.getEvents().clear(); + } + } + + // fetch metrics if fieldsToRetrieve contains METRICS or ALL. + if (hasField(fieldsToRetrieve, Field.METRICS)) { + readMetrics(entity, result, EntityColumnPrefix.METRIC); + } + return entity; + } + + /** + * Helper method for reading key-value pairs for either info or config. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result result from HBase. + * @param prefix column prefix. + * @param isConfig if true, means we are reading configs, otherwise info. + * @throws IOException if any problem is encountered while reading result. + */ + protected void readKeyValuePairs(TimelineEntity entity, Result result, + ColumnPrefix prefix, boolean isConfig) throws IOException { + // info and configuration are of type Map + Map columns = + prefix.readResults(result, stringKeyConverter); + if (isConfig) { + for (Map.Entry column : columns.entrySet()) { + entity.addConfig(column.getKey(), column.getValue().toString()); + } + } else { + entity.addInfo(columns); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java new file mode 100644 index 0000000..7b294a8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java @@ -0,0 +1,496 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +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.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +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.EventColumnNameConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +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.entity.EntityColumnPrefix; + +/** + * The base class for reading and deserializing timeline entities from the + * HBase storage. Different types can be defined for different types of the + * entities that are being requested. + */ +public abstract class TimelineEntityReader { + private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class); + + private final boolean singleEntityRead; + private TimelineReaderContext context; + private TimelineDataToRetrieve dataToRetrieve; + // used only for multiple entity read mode + private TimelineEntityFilters filters; + + /** + * Main table the entity reader uses. + */ + private BaseTable table; + + /** + * Specifies whether keys for this table are sorted in a manner where entities + * can be retrieved by created time. If true, it will be sufficient to collect + * the first results as specified by the limit. Otherwise all matched entities + * will be fetched and then limit applied. + */ + private boolean sortedKeys = false; + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + /** + * Instantiates a reader for multiple-entity reads. + * + * @param ctxt Reader context which defines the scope in which query has to be + * made. + * @param entityFilters Filters which limit the entities returned. + * @param toRetrieve Data to retrieve for each entity. + * @param sortedKeys Specifies whether key for this table are sorted or not. + * If sorted, entities can be retrieved by created time. + */ + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, + boolean sortedKeys) { + this.singleEntityRead = false; + this.sortedKeys = sortedKeys; + this.context = ctxt; + this.dataToRetrieve = toRetrieve; + this.filters = entityFilters; + + this.setTable(getTable()); + } + + /** + * Instantiates a reader for single-entity reads. + * + * @param ctxt Reader context which defines the scope in which query has to be + * made. + * @param toRetrieve Data to retrieve for each entity. + */ + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + this.singleEntityRead = true; + this.context = ctxt; + this.dataToRetrieve = toRetrieve; + + this.setTable(getTable()); + } + + /** + * Creates a {@link FilterList} based on fields, confs and metrics to + * retrieve. This filter list will be set in Scan/Get objects to trim down + * results fetched from HBase back-end storage. This is called only for + * multiple entity reads. + * + * @return a {@link FilterList} object. + * @throws IOException if any problem occurs while creating filter list. + */ + protected abstract FilterList constructFilterListBasedOnFields() + throws IOException; + + /** + * Creates a {@link FilterList} based on info, config and metric filters. This + * filter list will be set in HBase Get to trim down results fetched from + * HBase back-end storage. + * + * @return a {@link FilterList} object. + * @throws IOException if any problem occurs while creating filter list. + */ + protected abstract FilterList constructFilterListBasedOnFilters() + throws IOException; + + /** + * Combines filter lists created based on fields and based on filters. + * + * @return a {@link FilterList} object if it can be constructed. Returns null, + * if filter list cannot be created either on the basis of filters or on the + * basis of fields. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterList() throws IOException { + FilterList listBasedOnFilters = constructFilterListBasedOnFilters(); + boolean hasListBasedOnFilters = listBasedOnFilters != null && + !listBasedOnFilters.getFilters().isEmpty(); + FilterList listBasedOnFields = constructFilterListBasedOnFields(); + boolean hasListBasedOnFields = listBasedOnFields != null && + !listBasedOnFields.getFilters().isEmpty(); + // If filter lists based on both filters and fields can be created, + // combine them in a new filter list and return it. + // If either one of them has been created, return that filter list. + // Return null, if none of the filter lists can be created. This indicates + // that no filter list needs to be added to HBase Scan as filters are not + // specified for the query or only the default view of entity needs to be + // returned. + if (hasListBasedOnFilters && hasListBasedOnFields) { + FilterList list = new FilterList(); + list.addFilter(listBasedOnFilters); + list.addFilter(listBasedOnFields); + return list; + } else if (hasListBasedOnFilters) { + return listBasedOnFilters; + } else if (hasListBasedOnFields) { + return listBasedOnFields; + } + return null; + } + + protected TimelineReaderContext getContext() { + return context; + } + + protected TimelineDataToRetrieve getDataToRetrieve() { + return dataToRetrieve; + } + + protected TimelineEntityFilters getFilters() { + return filters; + } + + /** + * Create a {@link TimelineEntityFilters} object with default values for + * filters. + */ + protected void createFiltersIfNull() { + if (filters == null) { + filters = new TimelineEntityFilters(); + } + } + + /** + * Reads and deserializes a single timeline entity from the HBase storage. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @return A TimelineEntity object. + * @throws IOException if there is any exception encountered while reading + * entity. + */ + public TimelineEntity readEntity(Configuration hbaseConf, Connection conn) + throws IOException { + validateParams(); + augmentParams(hbaseConf, conn); + + FilterList filterList = constructFilterListBasedOnFields(); + if (LOG.isDebugEnabled() && filterList != null) { + LOG.debug("FilterList created for get is - " + filterList); + } + Result result = getResult(hbaseConf, conn, filterList); + if (result == null || result.isEmpty()) { + // Could not find a matching row. + LOG.info("Cannot find matching entity of type " + + context.getEntityType()); + return null; + } + return parseEntity(result); + } + + /** + * Reads and deserializes a set of timeline entities from the HBase storage. + * It goes through all the results available, and returns the number of + * entries as specified in the limit in the entity's natural sort order. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @return a set of TimelineEntity objects. + * @throws IOException if any exception is encountered while reading entities. + */ + public Set readEntities(Configuration hbaseConf, + Connection conn) throws IOException { + validateParams(); + augmentParams(hbaseConf, conn); + + NavigableSet entities = new TreeSet<>(); + FilterList filterList = createFilterList(); + if (LOG.isDebugEnabled() && filterList != null) { + LOG.debug("FilterList created for scan is - " + filterList); + } + ResultScanner results = getResults(hbaseConf, conn, filterList); + try { + for (Result result : results) { + TimelineEntity entity = parseEntity(result); + if (entity == null) { + continue; + } + entities.add(entity); + if (!sortedKeys) { + if (entities.size() > filters.getLimit()) { + entities.pollLast(); + } + } else { + if (entities.size() == filters.getLimit()) { + break; + } + } + } + return entities; + } finally { + results.close(); + } + } + + /** + * Returns the main table to be used by the entity reader. + * + * @return A reference to the table. + */ + protected BaseTable getTable() { + return table; + } + + /** + * Validates the required parameters to read the entities. + */ + protected abstract void validateParams(); + + /** + * Sets certain parameters to defaults if the values are not provided. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @throws IOException if any exception is encountered while setting params. + */ + protected abstract void augmentParams(Configuration hbaseConf, + Connection conn) throws IOException; + + /** + * Fetches a {@link Result} instance for a single-entity read. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @param filterList filter list which will be applied to HBase Get. + * @return the {@link Result} instance or null if no such record is found. + * @throws IOException if any exception is encountered while getting result. + */ + protected abstract Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException; + + /** + * Fetches a {@link ResultScanner} for a multi-entity read. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @param filterList filter list which will be applied to HBase Scan. + * @return the {@link ResultScanner} instance. + * @throws IOException if any exception is encountered while getting results. + */ + protected abstract ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException; + + /** + * Parses the result retrieved from HBase backend and convert it into a + * {@link TimelineEntity} object. + * + * @param result Single row result of a Get/Scan. + * @return the TimelineEntity instance or null if the entity is + * filtered. + * @throws IOException if any exception is encountered while parsing entity. + */ + protected abstract TimelineEntity parseEntity(Result result) + throws IOException; + + /** + * Helper method for reading and deserializing {@link TimelineMetric} objects + * using the specified column prefix. The timeline metrics then are added to + * the given timeline entity. + * + * @param entity {@link TimelineEntity} object. + * @param result {@link Result} object retrieved from backend. + * @param columnPrefix Metric column prefix + * @throws IOException if any exception is encountered while reading metrics. + */ + protected void readMetrics(TimelineEntity entity, Result result, + ColumnPrefix columnPrefix) throws IOException { + NavigableMap> metricsResult = + columnPrefix.readResultsWithTimestamps( + result, stringKeyConverter); + for (Map.Entry> metricResult: + metricsResult.entrySet()) { + TimelineMetric metric = new TimelineMetric(); + metric.setId(metricResult.getKey()); + // Simply assume that if the value set contains more than 1 elements, the + // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric + TimelineMetric.Type metricType = metricResult.getValue().size() > 1 ? + TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE; + metric.setType(metricType); + metric.addValues(metricResult.getValue()); + entity.addMetric(metric); + } + } + + /** + * Checks whether the reader has been created to fetch single entity or + * multiple entities. + * + * @return true, if query is for single entity, false otherwise. + */ + public boolean isSingleEntityRead() { + return singleEntityRead; + } + + protected void setTable(BaseTable baseTable) { + this.table = baseTable; + } + + /** + * Check if we have a certain field amongst fields to retrieve. This method + * checks against {@link Field#ALL} as well because that would mean field + * passed needs to be matched. + * + * @param fieldsToRetrieve fields to be retrieved. + * @param requiredField fields to be checked in fieldsToRetrieve. + * @return true if has the required field, false otherwise. + */ + protected boolean hasField(EnumSet fieldsToRetrieve, + Field requiredField) { + return fieldsToRetrieve.contains(Field.ALL) || + fieldsToRetrieve.contains(requiredField); + } + + /** + * Create a filter list of qualifier filters based on passed set of columns. + * + * @param Describes the type of column prefix. + * @param colPrefix Column Prefix. + * @param columns set of column qualifiers. + * @return filter list. + */ + protected FilterList createFiltersFromColumnQualifiers( + ColumnPrefix colPrefix, Set columns) { + FilterList list = new FilterList(Operator.MUST_PASS_ONE); + for (String column : columns) { + // For columns which have compound column qualifiers (eg. events), we need + // to include the required separator. + byte[] compoundColQual = createColQualifierPrefix(colPrefix, column); + list.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryPrefixComparator(colPrefix + .getColumnPrefixBytes(compoundColQual)))); + } + return list; + } + + protected byte[] createColQualifierPrefix(ColumnPrefix colPrefix, + String column) { + if (colPrefix == ApplicationColumnPrefix.EVENT + || colPrefix == EntityColumnPrefix.EVENT) { + return new EventColumnName(column, null, null).getColumnQualifier(); + } else { + return stringKeyConverter.encode(column); + } + } + + /** + * Helper method for reading relationship. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result result from HBase. + * @param prefix column prefix. + * @param isRelatedTo if true, means relationship is to be added to + * isRelatedTo, otherwise its added to relatesTo. + * @throws IOException if any problem is encountered while reading result. + */ + protected void readRelationship(TimelineEntity entity, Result result, + ColumnPrefix prefix, boolean isRelatedTo) throws IOException { + // isRelatedTo and relatesTo are of type Map> + Map columns = + prefix.readResults(result, stringKeyConverter); + for (Map.Entry column : columns.entrySet()) { + for (String id : Separator.VALUES.splitEncoded(column.getValue() + .toString())) { + if (isRelatedTo) { + entity.addIsRelatedToEntity(column.getKey(), id); + } else { + entity.addRelatesToEntity(column.getKey(), id); + } + } + } + } + + /** + * Read events from the entity table or the application table. The column name + * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted + * if there is no info associated with the event. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result HBase Result. + * @param prefix column prefix. + * @throws IOException if any problem is encountered while reading result. + */ + protected static void readEvents(TimelineEntity entity, Result result, + ColumnPrefix prefix) throws IOException { + Map eventsMap = new HashMap<>(); + Map eventsResult = + prefix.readResults(result, new EventColumnNameConverter()); + for (Map.Entry + eventResult : eventsResult.entrySet()) { + EventColumnName eventColumnName = eventResult.getKey(); + String key = eventColumnName.getId() + + Long.toString(eventColumnName.getTimestamp()); + // Retrieve previously seen event to add to it + TimelineEvent event = eventsMap.get(key); + if (event == null) { + // First time we're seeing this event, add it to the eventsMap + event = new TimelineEvent(); + event.setId(eventColumnName.getId()); + event.setTimestamp(eventColumnName.getTimestamp()); + eventsMap.put(key, event); + } + if (eventColumnName.getInfoKey() != null) { + event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue()); + } + } + Set eventsSet = new HashSet<>(eventsMap.values()); + entity.addEvents(eventsSet); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java new file mode 100644 index 0000000..b2a9476 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +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; + +/** + * Factory methods for instantiating a timeline entity reader. + */ +public final class TimelineEntityReaderFactory { + private TimelineEntityReaderFactory() { + } + + /** + * Creates a timeline entity reader instance for reading a single entity with + * the specified input. + * + * @param context Reader context which defines the scope in which query has to + * be made. + * @param dataToRetrieve Data to retrieve for each entity. + * @return An implementation of TimelineEntityReader object + * depending on entity type. + */ + public static TimelineEntityReader createSingleEntityReader( + TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) { + // currently the types that are handled separate from the generic entity + // table are application, flow run, and flow activity entities + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, dataToRetrieve); + } else { + // assume we're dealing with a generic entity read + return new GenericEntityReader(context, dataToRetrieve); + } + } + + /** + * Creates a timeline entity reader instance for reading set of entities with + * the specified input and predicates. + * + * @param context Reader context which defines the scope in which query has to + * be made. + * @param filters Filters which limit the entities returned. + * @param dataToRetrieve Data to retrieve for each entity. + * @return An implementation of TimelineEntityReader object + * depending on entity type. + */ + public static TimelineEntityReader createMultipleEntitiesReader( + TimelineReaderContext context, TimelineEntityFilters filters, + TimelineDataToRetrieve dataToRetrieve) { + // currently the types that are handled separate from the generic entity + // table are application, flow run, and flow activity entities + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, filters, dataToRetrieve); + } else { + // assume we're dealing with a generic entity read + return new GenericEntityReader(context, filters, dataToRetrieve, false); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java new file mode 100644 index 0000000..9814d6d --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.reader + * contains classes used to read entities from backend based on query type. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java new file mode 100644 index 0000000..58df970 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java @@ -0,0 +1,130 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.junit.Test; + +public class TestKeyConverters { + + @Test + public void testAppIdKeyConverter() { + AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); + long currentTs = System.currentTimeMillis(); + ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1); + ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2); + ApplicationId appId3 = ApplicationId.newInstance(currentTs + 300, 1); + String appIdStr1 = appId1.toString(); + String appIdStr2 = appId2.toString(); + String appIdStr3 = appId3.toString(); + byte[] appIdBytes1 = appIdKeyConverter.encode(appIdStr1); + byte[] appIdBytes2 = appIdKeyConverter.encode(appIdStr2); + byte[] appIdBytes3 = appIdKeyConverter.encode(appIdStr3); + // App ids' should be encoded in a manner wherein descending order + // is maintained. + assertTrue( + "Ordering of app ids' is incorrect", + Bytes.compareTo(appIdBytes1, appIdBytes2) > 0 + && Bytes.compareTo(appIdBytes1, appIdBytes3) > 0 + && Bytes.compareTo(appIdBytes2, appIdBytes3) > 0); + String decodedAppId1 = appIdKeyConverter.decode(appIdBytes1); + String decodedAppId2 = appIdKeyConverter.decode(appIdBytes2); + String decodedAppId3 = appIdKeyConverter.decode(appIdBytes3); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr1.equals(decodedAppId1)); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr2.equals(decodedAppId2)); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr3.equals(decodedAppId3)); + } + + @Test + public void testEventColumnNameConverter() { + String eventId = "=foo_=eve=nt="; + byte[] valSepBytes = Bytes.toBytes(Separator.VALUES.getValue()); + byte[] maxByteArr = + Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length); + byte[] ts = Bytes.add(valSepBytes, maxByteArr); + Long eventTs = Bytes.toLong(ts); + byte[] byteEventColName = + new EventColumnName(eventId, eventTs, null).getColumnQualifier(); + KeyConverter eventColumnNameConverter = + new EventColumnNameConverter(); + EventColumnName eventColName = + eventColumnNameConverter.decode(byteEventColName); + assertEquals(eventId, eventColName.getId()); + assertEquals(eventTs, eventColName.getTimestamp()); + assertNull(eventColName.getInfoKey()); + + String infoKey = "f=oo_event_in=fo=_key"; + byteEventColName = + new EventColumnName(eventId, eventTs, infoKey).getColumnQualifier(); + eventColName = eventColumnNameConverter.decode(byteEventColName); + assertEquals(eventId, eventColName.getId()); + assertEquals(eventTs, eventColName.getTimestamp()); + assertEquals(infoKey, eventColName.getInfoKey()); + } + + @Test + public void testLongKeyConverter() { + LongKeyConverter longKeyConverter = new LongKeyConverter(); + confirmLongKeyConverter(longKeyConverter, Long.MIN_VALUE); + confirmLongKeyConverter(longKeyConverter, -1234567890L); + confirmLongKeyConverter(longKeyConverter, -128L); + confirmLongKeyConverter(longKeyConverter, -127L); + confirmLongKeyConverter(longKeyConverter, -1L); + confirmLongKeyConverter(longKeyConverter, 0L); + confirmLongKeyConverter(longKeyConverter, 1L); + confirmLongKeyConverter(longKeyConverter, 127L); + confirmLongKeyConverter(longKeyConverter, 128L); + confirmLongKeyConverter(longKeyConverter, 1234567890L); + confirmLongKeyConverter(longKeyConverter, Long.MAX_VALUE); + } + + private void confirmLongKeyConverter(LongKeyConverter longKeyConverter, + Long testValue) { + Long decoded = longKeyConverter.decode(longKeyConverter.encode(testValue)); + assertEquals(testValue, decoded); + } + + @Test + public void testStringKeyConverter() { + StringKeyConverter stringKeyConverter = new StringKeyConverter(); + String phrase = "QuackAttack now!"; + + for (int i = 0; i < phrase.length(); i++) { + String sub = phrase.substring(i, phrase.length()); + confirmStrignKeyConverter(stringKeyConverter, sub); + confirmStrignKeyConverter(stringKeyConverter, sub + sub); + } + } + + private void confirmStrignKeyConverter(StringKeyConverter stringKeyConverter, + String testValue) { + String decoded = + stringKeyConverter.decode(stringKeyConverter.encode(testValue)); + assertEquals(testValue, decoded); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java new file mode 100644 index 0000000..5beb189 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java @@ -0,0 +1,246 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +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.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.junit.Test; + + +public class TestRowKeys { + + private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue(); + private final static byte[] QUALIFIER_SEP_BYTES = Bytes + .toBytes(QUALIFIER_SEP); + private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster"; + private final static String USER = QUALIFIER_SEP + "user"; + private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow" + + QUALIFIER_SEP; + private final static Long FLOW_RUN_ID; + private final static String APPLICATION_ID; + static { + long runid = Long.MAX_VALUE - 900L; + byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE); + byte[] byteArr = Bytes.toBytes(runid); + int sepByteLen = QUALIFIER_SEP_BYTES.length; + if (sepByteLen <= byteArr.length) { + for (int i = 0; i < sepByteLen; i++) { + byteArr[i] = (byte) (longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]); + } + } + FLOW_RUN_ID = Bytes.toLong(byteArr); + long clusterTs = System.currentTimeMillis(); + byteArr = Bytes.toBytes(clusterTs); + if (sepByteLen <= byteArr.length) { + for (int i = 0; i < sepByteLen; i++) { + byteArr[byteArr.length - sepByteLen + i] = + (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - + QUALIFIER_SEP_BYTES[i]); + } + } + clusterTs = Bytes.toLong(byteArr); + int seqId = 222; + APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString(); + } + + private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) { + int sepLen = QUALIFIER_SEP_BYTES.length; + for (int i = 0; i < sepLen; i++) { + assertTrue( + "Row key prefix not encoded properly.", + byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == + QUALIFIER_SEP_BYTES[i]); + } + } + + @Test + public void testApplicationRowKey() { + byte[] byteRowKey = + new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID).getRowKey(); + ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + + byte[] byteRowKeyPrefix = + new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID) + .getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE}); + assertEquals(5, splits.length); + assertEquals(0, splits[4].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + assertEquals(FLOW_RUN_ID, + (Long) LongConverter.invertLong(Bytes.toLong(splits[3]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(4, splits.length); + assertEquals(0, splits[3].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + /** + * Tests the converters indirectly through the public methods of the + * corresponding rowkey. + */ + @Test + public void testAppToFlowRowKey() { + byte[] byteRowKey = new AppToFlowRowKey(CLUSTER, + APPLICATION_ID).getRowKey(); + AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + } + + @Test + public void testEntityRowKey() { + String entityId = "!ent!ity!!id!"; + String entityType = "entity!Type"; + byte[] byteRowKey = + new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID, + entityType, entityId).getRowKey(); + EntityRowKey rowKey = EntityRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + assertEquals(entityType, rowKey.getEntityType()); + assertEquals(entityId, rowKey.getEntityId()); + + byte[] byteRowKeyPrefix = + new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID, entityType).getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split( + byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE}); + assertEquals(7, splits.length); + assertEquals(0, splits[6].length); + assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4])); + assertEquals(entityType, + Separator.QUALIFIERS.decode(Bytes.toString(splits[5]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split( + byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE}); + assertEquals(6, splits.length); + assertEquals(0, splits[5].length); + AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); + assertEquals(APPLICATION_ID, appIdKeyConverter.decode(splits[4])); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + @Test + public void testFlowActivityRowKey() { + Long ts = 1459900830000L; + Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + byte[] byteRowKey = + new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey(); + FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(dayTimestamp, rowKey.getDayTimestamp()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + + byte[] byteRowKeyPrefix = + new FlowActivityRowKeyPrefix(CLUSTER).getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(2, splits.length); + assertEquals(0, splits[1].length); + assertEquals(CLUSTER, + Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE}); + assertEquals(3, splits.length); + assertEquals(0, splits[2].length); + assertEquals(CLUSTER, + Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); + assertEquals(ts, + (Long) LongConverter.invertLong(Bytes.toLong(splits[1]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + @Test + public void testFlowRunRowKey() { + byte[] byteRowKey = + new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey(); + FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + + byte[] byteRowKeyPrefix = + new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null).getRowKey(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(4, splits.length); + assertEquals(0, splits[3].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java new file mode 100644 index 0000000..7d37206 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java @@ -0,0 +1,215 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; + +import com.google.common.collect.Iterables; + +public class TestSeparator { + + private static String villain = "Dr. Heinz Doofenshmirtz"; + private static String special = + ". * | ? + \t ( ) [ ] { } ^ $ \\ \" %"; + + /** + * + */ + @Test + public void testEncodeDecodeString() { + + for (Separator separator : Separator.values()) { + testEncodeDecode(separator, ""); + testEncodeDecode(separator, " "); + testEncodeDecode(separator, "!"); + testEncodeDecode(separator, "?"); + testEncodeDecode(separator, "&"); + testEncodeDecode(separator, "+"); + testEncodeDecode(separator, "\t"); + testEncodeDecode(separator, "Dr."); + testEncodeDecode(separator, "Heinz"); + testEncodeDecode(separator, "Doofenshmirtz"); + testEncodeDecode(separator, villain); + testEncodeDecode(separator, special); + + assertNull(separator.encode(null)); + + } + } + + private void testEncodeDecode(Separator separator, String token) { + String encoded = separator.encode(token); + String decoded = separator.decode(encoded); + String msg = "token:" + token + " separator:" + separator + "."; + assertEquals(msg, token, decoded); + } + + @Test + public void testEncodeDecode() { + testEncodeDecode("Dr.", Separator.QUALIFIERS); + testEncodeDecode("Heinz", Separator.QUALIFIERS, Separator.QUALIFIERS); + testEncodeDecode("Doofenshmirtz", Separator.QUALIFIERS, null, + Separator.QUALIFIERS); + testEncodeDecode("&Perry", Separator.QUALIFIERS, Separator.VALUES, null); + testEncodeDecode("the ", Separator.QUALIFIERS, Separator.SPACE); + testEncodeDecode("Platypus...", (Separator) null); + testEncodeDecode("The what now ?!?", Separator.QUALIFIERS, + Separator.VALUES, Separator.SPACE); + + } + @Test + public void testEncodedValues() { + testEncodeDecode("Double-escape %2$ and %9$ or %%2$ or %%3$, nor %%%2$" + + "= no problem!", + Separator.QUALIFIERS, Separator.VALUES, Separator.SPACE, Separator.TAB); + } + + @Test + public void testSplits() { + byte[] maxLongBytes = Bytes.toBytes(Long.MAX_VALUE); + byte[] maxIntBytes = Bytes.toBytes(Integer.MAX_VALUE); + for (Separator separator : Separator.values()) { + String str1 = "cl" + separator.getValue() + "us"; + String str2 = separator.getValue() + "rst"; + byte[] sepByteArr = Bytes.toBytes(separator.getValue()); + byte[] longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, + sepByteArr.length, Bytes.SIZEOF_LONG - sepByteArr.length)); + byte[] intVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxIntBytes, + sepByteArr.length, Bytes.SIZEOF_INT - sepByteArr.length)); + byte[] arr = separator.join( + Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + int[] sizes = {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT}; + byte[][] splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + longVal1Arr = Bytes.add(Bytes.copy(maxLongBytes, 0, Bytes.SIZEOF_LONG - + sepByteArr.length), sepByteArr); + intVal1Arr = Bytes.add(Bytes.copy(maxIntBytes, 0, Bytes.SIZEOF_INT - + sepByteArr.length), sepByteArr); + arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, + sepByteArr.length, 4 - sepByteArr.length), sepByteArr); + longVal1Arr = Bytes.add(longVal1Arr, Bytes.copy(maxLongBytes, 4, 3 - + sepByteArr.length), sepByteArr); + arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + arr = separator.join(Bytes.toBytes(separator.encode(str1)), + Bytes.toBytes(separator.encode(str2)), intVal1Arr, longVal1Arr); + int[] sizes1 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG}; + splits = separator.split(arr, sizes1); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(str2, separator.decode(Bytes.toString(splits[1]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[2])); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[3])); + + try { + int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Bytes.SIZEOF_INT, 7}; + splits = separator.split(arr, sizes2); + fail("Exception should have been thrown."); + } catch (IllegalArgumentException e) {} + + try { + int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2, + Bytes.SIZEOF_LONG}; + splits = separator.split(arr, sizes2); + fail("Exception should have been thrown."); + } catch (IllegalArgumentException e) {} + } + } + + /** + * Simple test to encode and decode using the same separators and confirm that + * we end up with the same as what we started with. + * + * @param token + * @param separators + */ + private static void testEncodeDecode(String token, Separator... separators) { + byte[] encoded = Separator.encode(token, separators); + String decoded = Separator.decode(encoded, separators); + assertEquals(token, decoded); + } + + @Test + public void testJoinStripped() { + List stringList = new ArrayList(0); + stringList.add("nothing"); + + String joined = Separator.VALUES.joinEncoded(stringList); + Iterable split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(stringList, split)); + + stringList = new ArrayList(3); + stringList.add("a"); + stringList.add("b?"); + stringList.add("c"); + + joined = Separator.VALUES.joinEncoded(stringList); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(stringList, split)); + + String[] stringArray1 = {"else"}; + joined = Separator.VALUES.joinEncoded(stringArray1); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray1), split)); + + String[] stringArray2 = {"d", "e?", "f"}; + joined = Separator.VALUES.joinEncoded(stringArray2); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray2), split)); + + List empty = new ArrayList(0); + split = Separator.VALUES.splitEncoded(null); + assertTrue(Iterables.elementsEqual(empty, split)); + + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml index 6b535c3..95c8a8d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml @@ -35,31 +35,37 @@ + org.apache.hadoop hadoop-common + org.apache.hadoop hadoop-annotations + org.apache.hadoop hadoop-yarn-api + org.apache.hadoop hadoop-yarn-common + org.apache.hadoop hadoop-yarn-server-common + org.apache.hadoop hadoop-yarn-server-applicationhistoryservice @@ -91,18 +97,18 @@ - com.sun.jersey - jersey-core + com.fasterxml.jackson.core + jackson-databind - com.sun.jersey - jersey-client + com.fasterxml.jackson.core + jackson-core - commons-cli - commons-cli + com.sun.jersey + jersey-client @@ -120,82 +126,6 @@ commons-csv - - org.apache.hbase - hbase-common - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.mortbay.jetty - jetty-util - - - - - - org.apache.hbase - hbase-client - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - - - - org.apache.hbase - hbase-server - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hadoop - hadoop-hdfs-client - - - org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - org.mortbay.jetty - jetty-sslengine - - - - - - org.apache.phoenix - phoenix-core - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - net.sourceforge.findbugs - annotations - - - - org.apache.hadoop diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java deleted file mode 100644 index cccae26..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java +++ /dev/null @@ -1,290 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.reader.filter; - -import java.io.IOException; -import java.util.HashSet; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; -import org.apache.hadoop.hbase.filter.FamilyFilter; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.Filter; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; - -/** - * Set of utility methods used by timeline filter classes. - */ -public final class TimelineFilterUtils { - - private static final Log LOG = LogFactory.getLog(TimelineFilterUtils.class); - - private TimelineFilterUtils() { - } - - /** - * Returns the equivalent HBase filter list's {@link Operator}. - * - * @param op timeline filter list operator. - * @return HBase filter list's Operator. - */ - private static Operator getHBaseOperator(TimelineFilterList.Operator op) { - switch (op) { - case AND: - return Operator.MUST_PASS_ALL; - case OR: - return Operator.MUST_PASS_ONE; - default: - throw new IllegalArgumentException("Invalid operator"); - } - } - - /** - * Returns the equivalent HBase compare filter's {@link CompareOp}. - * - * @param op timeline compare op. - * @return HBase compare filter's CompareOp. - */ - private static CompareOp getHBaseCompareOp( - TimelineCompareOp op) { - switch (op) { - case LESS_THAN: - return CompareOp.LESS; - case LESS_OR_EQUAL: - return CompareOp.LESS_OR_EQUAL; - case EQUAL: - return CompareOp.EQUAL; - case NOT_EQUAL: - return CompareOp.NOT_EQUAL; - case GREATER_OR_EQUAL: - return CompareOp.GREATER_OR_EQUAL; - case GREATER_THAN: - return CompareOp.GREATER; - default: - throw new IllegalArgumentException("Invalid compare operator"); - } - } - - /** - * Converts a {@link TimelinePrefixFilter} to an equivalent HBase - * {@link QualifierFilter}. - * @param colPrefix - * @param filter - * @return a {@link QualifierFilter} object - */ - private static Filter createHBaseColQualPrefixFilter( - ColumnPrefix colPrefix, TimelinePrefixFilter filter) { - return new QualifierFilter(getHBaseCompareOp(filter.getCompareOp()), - new BinaryPrefixComparator( - colPrefix.getColumnPrefixBytes(filter.getPrefix()))); - } - - /** - * Create a HBase {@link QualifierFilter} for the passed column prefix and - * compare op. - * - * @param Describes the type of column prefix. - * @param compareOp compare op. - * @param columnPrefix column prefix. - * @return a column qualifier filter. - */ - public static Filter createHBaseQualifierFilter(CompareOp compareOp, - ColumnPrefix columnPrefix) { - return new QualifierFilter(compareOp, - new BinaryPrefixComparator( - columnPrefix.getColumnPrefixBytes(""))); - } - - /** - * Create filters for confs or metrics to retrieve. This list includes a - * configs/metrics family filter and relevant filters for confs/metrics to - * retrieve, if present. - * - * @param Describes the type of column prefix. - * @param confsOrMetricToRetrieve configs/metrics to retrieve. - * @param columnFamily config or metric column family. - * @param columnPrefix config or metric column prefix. - * @return a filter list. - * @throws IOException if any problem occurs while creating the filters. - */ - public static Filter createFilterForConfsOrMetricsToRetrieve( - TimelineFilterList confsOrMetricToRetrieve, ColumnFamily columnFamily, - ColumnPrefix columnPrefix) throws IOException { - Filter familyFilter = new FamilyFilter(CompareOp.EQUAL, - new BinaryComparator(columnFamily.getBytes())); - if (confsOrMetricToRetrieve != null && - !confsOrMetricToRetrieve.getFilterList().isEmpty()) { - // If confsOrMetricsToRetrive are specified, create a filter list based - // on it and family filter. - FilterList filter = new FilterList(familyFilter); - filter.addFilter( - createHBaseFilterList(columnPrefix, confsOrMetricToRetrieve)); - return filter; - } else { - // Only the family filter needs to be added. - return familyFilter; - } - } - - /** - * Create 2 HBase {@link SingleColumnValueFilter} filters for the specified - * value range represented by start and end value and wraps them inside a - * filter list. Start and end value should not be null. - * - * @param Describes the type of column prefix. - * @param column Column for which single column value filter is to be created. - * @param startValue Start value. - * @param endValue End value. - * @return 2 single column value filters wrapped in a filter list. - * @throws IOException if any problem is encountered while encoding value. - */ - public static FilterList createSingleColValueFiltersByRange( - Column column, Object startValue, Object endValue) throws IOException { - FilterList list = new FilterList(); - Filter singleColValFilterStart = createHBaseSingleColValueFilter( - column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), - column.getValueConverter().encodeValue(startValue), - CompareOp.GREATER_OR_EQUAL, true); - list.addFilter(singleColValFilterStart); - - Filter singleColValFilterEnd = createHBaseSingleColValueFilter( - column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), - column.getValueConverter().encodeValue(endValue), - CompareOp.LESS_OR_EQUAL, true); - list.addFilter(singleColValFilterEnd); - return list; - } - - /** - * Creates a HBase {@link SingleColumnValueFilter}. - * - * @param columnFamily Column Family represented as bytes. - * @param columnQualifier Column Qualifier represented as bytes. - * @param value Value. - * @param compareOp Compare operator. - * @param filterIfMissing This flag decides if we should filter the row if the - * specified column is missing. This is based on the filter's keyMustExist - * field. - * @return a {@link SingleColumnValueFilter} object - * @throws IOException - */ - private static SingleColumnValueFilter createHBaseSingleColValueFilter( - byte[] columnFamily, byte[] columnQualifier, byte[] value, - CompareOp compareOp, boolean filterIfMissing) throws IOException { - SingleColumnValueFilter singleColValFilter = - new SingleColumnValueFilter(columnFamily, columnQualifier, compareOp, - new BinaryComparator(value)); - singleColValFilter.setLatestVersionOnly(true); - singleColValFilter.setFilterIfMissing(filterIfMissing); - return singleColValFilter; - } - - /** - * Fetch columns from filter list containing exists and multivalue equality - * filters. This is done to fetch only required columns from back-end and - * then match event filters or relationships in reader. - * - * @param filterList filter list. - * @return set of columns. - */ - public static Set fetchColumnsFromFilterList( - TimelineFilterList filterList) { - Set strSet = new HashSet(); - for (TimelineFilter filter : filterList.getFilterList()) { - switch(filter.getFilterType()) { - case LIST: - strSet.addAll(fetchColumnsFromFilterList((TimelineFilterList)filter)); - break; - case KEY_VALUES: - strSet.add(((TimelineKeyValuesFilter)filter).getKey()); - break; - case EXISTS: - strSet.add(((TimelineExistsFilter)filter).getValue()); - break; - default: - LOG.info("Unexpected filter type " + filter.getFilterType()); - break; - } - } - return strSet; - } - - /** - * Creates equivalent HBase {@link FilterList} from {@link TimelineFilterList} - * while converting different timeline filters(of type {@link TimelineFilter}) - * into their equivalent HBase filters. - * - * @param Describes the type of column prefix. - * @param colPrefix column prefix which will be used for conversion. - * @param filterList timeline filter list which has to be converted. - * @return A {@link FilterList} object. - * @throws IOException if any problem occurs while creating the filter list. - */ - public static FilterList createHBaseFilterList(ColumnPrefix colPrefix, - TimelineFilterList filterList) throws IOException { - FilterList list = - new FilterList(getHBaseOperator(filterList.getOperator())); - for (TimelineFilter filter : filterList.getFilterList()) { - switch(filter.getFilterType()) { - case LIST: - list.addFilter(createHBaseFilterList(colPrefix, - (TimelineFilterList)filter)); - break; - case PREFIX: - list.addFilter(createHBaseColQualPrefixFilter(colPrefix, - (TimelinePrefixFilter)filter)); - break; - case COMPARE: - TimelineCompareFilter compareFilter = (TimelineCompareFilter)filter; - list.addFilter( - createHBaseSingleColValueFilter( - colPrefix.getColumnFamilyBytes(), - colPrefix.getColumnPrefixBytes(compareFilter.getKey()), - colPrefix.getValueConverter(). - encodeValue(compareFilter.getValue()), - getHBaseCompareOp(compareFilter.getCompareOp()), - compareFilter.getKeyMustExist())); - break; - case KEY_VALUE: - TimelineKeyValueFilter kvFilter = (TimelineKeyValueFilter)filter; - list.addFilter( - createHBaseSingleColValueFilter( - colPrefix.getColumnFamilyBytes(), - colPrefix.getColumnPrefixBytes(kvFilter.getKey()), - colPrefix.getValueConverter().encodeValue(kvFilter.getValue()), - getHBaseCompareOp(kvFilter.getCompareOp()), - kvFilter.getKeyMustExist())); - break; - default: - LOG.info("Unexpected filter type " + filter.getFilterType()); - break; - } - } - return list; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java deleted file mode 100644 index a384a84..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java +++ /dev/null @@ -1,88 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage; - - -import java.io.IOException; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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; -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.reader.TimelineEntityReader; -import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory; - -/** - * HBase based implementation for {@link TimelineReader}. - */ -public class HBaseTimelineReaderImpl - extends AbstractService implements TimelineReader { - - private static final Log LOG = LogFactory - .getLog(HBaseTimelineReaderImpl.class); - - private Configuration hbaseConf = null; - private Connection conn; - - public HBaseTimelineReaderImpl() { - super(HBaseTimelineReaderImpl.class.getName()); - } - - @Override - public void serviceInit(Configuration conf) throws Exception { - super.serviceInit(conf); - hbaseConf = HBaseConfiguration.create(conf); - conn = ConnectionFactory.createConnection(hbaseConf); - } - - @Override - protected void serviceStop() throws Exception { - if (conn != null) { - LOG.info("closing the hbase Connection"); - conn.close(); - } - super.serviceStop(); - } - - @Override - public TimelineEntity getEntity(TimelineReaderContext context, - TimelineDataToRetrieve dataToRetrieve) throws IOException { - TimelineEntityReader reader = - TimelineEntityReaderFactory.createSingleEntityReader(context, - dataToRetrieve); - return reader.readEntity(hbaseConf, conn); - } - - @Override - public Set getEntities(TimelineReaderContext context, - TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) - throws IOException { - TimelineEntityReader reader = - TimelineEntityReaderFactory.createMultipleEntitiesReader(context, - filters, dataToRetrieve); - return reader.readEntities(hbaseConf, conn); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java deleted file mode 100644 index b94b85f..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java +++ /dev/null @@ -1,566 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage; - -import java.io.IOException; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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; -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.TimelineEntityType; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; -import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; -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.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.TypedBufferedMutator; -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.EntityRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; -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; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; - -/** - * This implements a hbase based backend for storing the timeline entity - * information. - * It writes to multiple tables at the backend - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class HBaseTimelineWriterImpl extends AbstractService implements - TimelineWriter { - - private static final Log LOG = LogFactory - .getLog(HBaseTimelineWriterImpl.class); - - private Connection conn; - private TypedBufferedMutator entityTable; - private TypedBufferedMutator appToFlowTable; - private TypedBufferedMutator applicationTable; - private TypedBufferedMutator flowActivityTable; - private TypedBufferedMutator flowRunTable; - - /** - * Used to convert strings key components to and from storage format. - */ - private final KeyConverter stringKeyConverter = - new StringKeyConverter(); - - /** - * Used to convert Long key components to and from storage format. - */ - private final KeyConverter longKeyConverter = new LongKeyConverter(); - - public HBaseTimelineWriterImpl() { - super(HBaseTimelineWriterImpl.class.getName()); - } - - /** - * initializes the hbase connection to write to the entity table. - */ - @Override - protected void serviceInit(Configuration conf) throws Exception { - super.serviceInit(conf); - Configuration hbaseConf = HBaseConfiguration.create(conf); - conn = ConnectionFactory.createConnection(hbaseConf); - entityTable = new EntityTable().getTableMutator(hbaseConf, conn); - appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn); - applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn); - flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn); - flowActivityTable = - new FlowActivityTable().getTableMutator(hbaseConf, conn); - } - - /** - * Stores the entire information in TimelineEntities to the timeline store. - */ - @Override - public TimelineWriteResponse write(String clusterId, String userId, - String flowName, String flowVersion, long flowRunId, String appId, - 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 - if (te == null) { - continue; - } - - // if the entity is the application, the destination is the application - // table - boolean isApplication = isApplicationEntity(te); - byte[] rowKey; - if (isApplication) { - ApplicationRowKey applicationRowKey = - new ApplicationRowKey(clusterId, userId, flowName, flowRunId, - appId); - rowKey = applicationRowKey.getRowKey(); - } else { - EntityRowKey entityRowKey = - new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, - te.getType(), te.getId()); - rowKey = entityRowKey.getRowKey(); - } - - storeInfo(rowKey, te, flowVersion, isApplication); - storeEvents(rowKey, te.getEvents(), isApplication); - storeConfig(rowKey, te.getConfigs(), isApplication); - storeMetrics(rowKey, te.getMetrics(), isApplication); - storeRelations(rowKey, te, isApplication); - - if (isApplication) { - TimelineEvent event = - getApplicationEvent(te, - ApplicationMetricsConstants.CREATED_EVENT_TYPE); - FlowRunRowKey flowRunRowKey = - new FlowRunRowKey(clusterId, userId, flowName, flowRunId); - if (event != null) { - AppToFlowRowKey appToFlowRowKey = - new AppToFlowRowKey(clusterId, appId); - onApplicationCreated(flowRunRowKey, appToFlowRowKey, appId, userId, - flowVersion, te, event.getTimestamp()); - } - // if it's an application entity, store metrics - storeFlowMetricsAppRunning(flowRunRowKey, appId, te); - // if application has finished, store it's finish time and write final - // values of all metrics - event = getApplicationEvent(te, - ApplicationMetricsConstants.FINISHED_EVENT_TYPE); - if (event != null) { - onApplicationFinished(flowRunRowKey, flowVersion, appId, te, - event.getTimestamp()); - } - } - } - return putStatus; - } - - private void onApplicationCreated(FlowRunRowKey flowRunRowKey, - AppToFlowRowKey appToFlowRowKey, String appId, String userId, - String flowVersion, TimelineEntity te, long appCreatedTimeStamp) - throws IOException { - - String flowName = flowRunRowKey.getFlowName(); - Long flowRunId = flowRunRowKey.getFlowRunId(); - - // store in App to flow table - byte[] rowKey = appToFlowRowKey.getRowKey(); - AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName); - AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId); - AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId); - - // store in flow run table - storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te); - - // store in flow activity table - byte[] flowActivityRowKeyBytes = - new FlowActivityRowKey(flowRunRowKey.getClusterId(), - appCreatedTimeStamp, flowRunRowKey.getUserId(), flowName) - .getRowKey(); - byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); - FlowActivityColumnPrefix.RUN_ID.store(flowActivityRowKeyBytes, - flowActivityTable, qualifier, null, flowVersion, - AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); - } - - /* - * updates the {@link FlowRunTable} with Application Created information - */ - private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey, - String appId, TimelineEntity te) throws IOException { - byte[] rowKey = flowRunRowKey.getRowKey(); - FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null, - te.getCreatedTime(), - AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); - } - - - /* - * updates the {@link FlowRunTable} and {@link FlowActivityTable} when an - * application has finished - */ - private void onApplicationFinished(FlowRunRowKey flowRunRowKey, - String flowVersion, String appId, TimelineEntity te, - long appFinishedTimeStamp) throws IOException { - // store in flow run table - storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te, - appFinishedTimeStamp); - - // indicate in the flow activity table that the app has finished - byte[] rowKey = - new FlowActivityRowKey(flowRunRowKey.getClusterId(), - appFinishedTimeStamp, flowRunRowKey.getUserId(), - flowRunRowKey.getFlowName()).getRowKey(); - byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); - FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier, - null, flowVersion, - AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); - } - - /* - * Update the {@link FlowRunTable} with Application Finished information - */ - private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey, - String appId, TimelineEntity te, long appFinishedTimeStamp) - throws IOException { - byte[] rowKey = flowRunRowKey.getRowKey(); - Attribute attributeAppId = - AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId); - FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null, - appFinishedTimeStamp, attributeAppId); - - // store the final value of metrics since application has finished - Set metrics = te.getMetrics(); - if (metrics != null) { - storeFlowMetrics(rowKey, metrics, attributeAppId, - AggregationOperation.SUM_FINAL.getAttribute()); - } - } - - /* - * Updates the {@link FlowRunTable} with Application Metrics - */ - private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey, - String appId, TimelineEntity te) throws IOException { - Set metrics = te.getMetrics(); - if (metrics != null) { - byte[] rowKey = flowRunRowKey.getRowKey(); - storeFlowMetrics(rowKey, metrics, - AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId), - AggregationOperation.SUM.getAttribute()); - } - } - - private void storeFlowMetrics(byte[] rowKey, Set metrics, - Attribute... attributes) throws IOException { - for (TimelineMetric metric : metrics) { - byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId()); - Map timeseries = metric.getValues(); - for (Map.Entry timeseriesEntry : timeseries.entrySet()) { - Long timestamp = timeseriesEntry.getKey(); - FlowRunColumnPrefix.METRIC.store(rowKey, flowRunTable, - metricColumnQualifier, timestamp, timeseriesEntry.getValue(), - attributes); - } - } - } - - private void storeRelations(byte[] rowKey, TimelineEntity te, - boolean isApplication) throws IOException { - if (isApplication) { - storeRelations(rowKey, te.getIsRelatedToEntities(), - ApplicationColumnPrefix.IS_RELATED_TO, applicationTable); - storeRelations(rowKey, te.getRelatesToEntities(), - ApplicationColumnPrefix.RELATES_TO, applicationTable); - } else { - storeRelations(rowKey, te.getIsRelatedToEntities(), - EntityColumnPrefix.IS_RELATED_TO, entityTable); - storeRelations(rowKey, te.getRelatesToEntities(), - EntityColumnPrefix.RELATES_TO, entityTable); - } - } - - /** - * Stores the Relations from the {@linkplain TimelineEntity} object. - */ - private void storeRelations(byte[] rowKey, - Map> connectedEntities, - ColumnPrefix columnPrefix, TypedBufferedMutator table) - throws IOException { - for (Map.Entry> connectedEntity : connectedEntities - .entrySet()) { - // id3?id4?id5 - String compoundValue = - Separator.VALUES.joinEncoded(connectedEntity.getValue()); - columnPrefix.store(rowKey, table, - stringKeyConverter.encode(connectedEntity.getKey()), null, - compoundValue); - } - } - - /** - * Stores information from the {@linkplain TimelineEntity} object. - */ - private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion, - boolean isApplication) throws IOException { - - if (isApplication) { - ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId()); - ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null, - te.getCreatedTime()); - ApplicationColumn.FLOW_VERSION.store(rowKey, applicationTable, null, - flowVersion); - Map info = te.getInfo(); - if (info != null) { - for (Map.Entry entry : info.entrySet()) { - ApplicationColumnPrefix.INFO.store(rowKey, applicationTable, - stringKeyConverter.encode(entry.getKey()), null, - entry.getValue()); - } - } - } else { - EntityColumn.ID.store(rowKey, entityTable, null, te.getId()); - EntityColumn.TYPE.store(rowKey, entityTable, null, te.getType()); - EntityColumn.CREATED_TIME.store(rowKey, entityTable, null, - te.getCreatedTime()); - EntityColumn.FLOW_VERSION.store(rowKey, entityTable, null, flowVersion); - Map info = te.getInfo(); - if (info != null) { - for (Map.Entry entry : info.entrySet()) { - EntityColumnPrefix.INFO.store(rowKey, entityTable, - stringKeyConverter.encode(entry.getKey()), null, - entry.getValue()); - } - } - } - } - - /** - * stores the config information from {@linkplain TimelineEntity}. - */ - private void storeConfig(byte[] rowKey, Map config, - boolean isApplication) throws IOException { - if (config == null) { - return; - } - for (Map.Entry entry : config.entrySet()) { - byte[] configKey = stringKeyConverter.encode(entry.getKey()); - if (isApplication) { - ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable, - configKey, null, entry.getValue()); - } else { - EntityColumnPrefix.CONFIG.store(rowKey, entityTable, configKey, - null, entry.getValue()); - } - } - } - - /** - * stores the {@linkplain TimelineMetric} information from the - * {@linkplain TimelineEvent} object. - */ - private void storeMetrics(byte[] rowKey, Set metrics, - boolean isApplication) throws IOException { - if (metrics != null) { - for (TimelineMetric metric : metrics) { - byte[] metricColumnQualifier = - stringKeyConverter.encode(metric.getId()); - Map timeseries = metric.getValues(); - for (Map.Entry timeseriesEntry : timeseries.entrySet()) { - Long timestamp = timeseriesEntry.getKey(); - if (isApplication) { - ApplicationColumnPrefix.METRIC.store(rowKey, applicationTable, - metricColumnQualifier, timestamp, timeseriesEntry.getValue()); - } else { - EntityColumnPrefix.METRIC.store(rowKey, entityTable, - metricColumnQualifier, timestamp, timeseriesEntry.getValue()); - } - } - } - } - } - - /** - * Stores the events from the {@linkplain TimelineEvent} object. - */ - private void storeEvents(byte[] rowKey, Set events, - boolean isApplication) throws IOException { - if (events != null) { - for (TimelineEvent event : events) { - if (event != null) { - String eventId = event.getId(); - if (eventId != null) { - long eventTimestamp = event.getTimestamp(); - // if the timestamp is not set, use the current timestamp - if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) { - LOG.warn("timestamp is not set for event " + eventId + - "! Using the current timestamp"); - eventTimestamp = System.currentTimeMillis(); - } - Map eventInfo = event.getInfo(); - if ((eventInfo == null) || (eventInfo.size() == 0)) { - byte[] columnQualifierBytes = - new EventColumnName(eventId, eventTimestamp, null) - .getColumnQualifier(); - if (isApplication) { - ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable, - columnQualifierBytes, null, Separator.EMPTY_BYTES); - } else { - EntityColumnPrefix.EVENT.store(rowKey, entityTable, - columnQualifierBytes, null, Separator.EMPTY_BYTES); - } - } else { - for (Map.Entry info : eventInfo.entrySet()) { - // eventId=infoKey - byte[] columnQualifierBytes = - new EventColumnName(eventId, eventTimestamp, info.getKey()) - .getColumnQualifier(); - if (isApplication) { - ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable, - columnQualifierBytes, null, info.getValue()); - } else { - EntityColumnPrefix.EVENT.store(rowKey, entityTable, - columnQualifierBytes, null, info.getValue()); - } - } // for info: eventInfo - } - } - } - } // event : events - } - } - - /** - * Checks if the input TimelineEntity object is an ApplicationEntity. - * - * @param te TimelineEntity object. - * @return true if input is an ApplicationEntity, false otherwise - */ - static boolean isApplicationEntity(TimelineEntity te) { - return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString()); - } - - /** - * @param te TimelineEntity object. - * @param eventId event with this id needs to be fetched - * @return TimelineEvent if TimelineEntity contains the desired event. - */ - private static TimelineEvent getApplicationEvent(TimelineEntity te, - String eventId) { - if (isApplicationEntity(te)) { - for (TimelineEvent event : te.getEvents()) { - if (event.getId().equals(eventId)) { - return event; - } - } - } - return null; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage - * .TimelineWriter#aggregate - * (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity, - * org.apache - * .hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack) - */ - @Override - public TimelineWriteResponse aggregate(TimelineEntity data, - TimelineAggregationTrack track) throws IOException { - return null; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter#flush - * () - */ - @Override - public void flush() throws IOException { - // flush all buffered mutators - entityTable.flush(); - appToFlowTable.flush(); - applicationTable.flush(); - flowRunTable.flush(); - flowActivityTable.flush(); - } - - /** - * close the hbase connections The close APIs perform flushing and release any - * resources held. - */ - @Override - protected void serviceStop() throws Exception { - if (entityTable != null) { - LOG.info("closing the entity table"); - // The close API performs flushing and releases any resources held - entityTable.close(); - } - if (appToFlowTable != null) { - LOG.info("closing the app_flow table"); - // The close API performs flushing and releases any resources held - appToFlowTable.close(); - } - if (applicationTable != null) { - LOG.info("closing the application table"); - applicationTable.close(); - } - if (flowRunTable != null) { - LOG.info("closing the flow run table"); - // The close API performs flushing and releases any resources held - flowRunTable.close(); - } - if (flowActivityTable != null) { - LOG.info("closing the flowActivityTable table"); - // The close API performs flushing and releases any resources held - flowActivityTable.close(); - } - if (conn != null) { - LOG.info("closing the hbase Connection"); - conn.close(); - } - super.serviceStop(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java deleted file mode 100644 index 130cb6c..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java +++ /dev/null @@ -1,358 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configuration; -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.TimelineMetric; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo; -import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; -import org.apache.phoenix.util.PropertiesUtil; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -/** - * Offline aggregation Phoenix storage. This storage currently consists of two - * aggregation tables, one for flow level aggregation and one for user level - * aggregation. - * - * Example table record: - * - *
- * |---------------------------|
- * |  Primary   | Column Family|
- * |  key       | metrics      |
- * |---------------------------|
- * | row_key    | metricId1:   |
- * |            | metricValue1 |
- * |            | @timestamp1  |
- * |            |              |
- * |            | metriciD1:   |
- * |            | metricValue2 |
- * |            | @timestamp2  |
- * |            |              |
- * |            | metricId2:   |
- * |            | metricValue1 |
- * |            | @timestamp2  |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |            |              |
- * |---------------------------|
- * 
- * - * For the flow aggregation table, the primary key contains user, cluster, and - * flow id. For user aggregation table,the primary key is user. - * - * Metrics column family stores all aggregated metrics for each record. - */ -@Private -@Unstable -public class PhoenixOfflineAggregationWriterImpl - extends OfflineAggregationWriter { - - private static final Log LOG - = LogFactory.getLog(PhoenixOfflineAggregationWriterImpl.class); - private static final String PHOENIX_COL_FAMILY_PLACE_HOLDER - = "timeline_cf_placeholder"; - - /** Default Phoenix JDBC driver name. */ - private static final String DRIVER_CLASS_NAME - = "org.apache.phoenix.jdbc.PhoenixDriver"; - - /** Default Phoenix timeline config column family. */ - private static final String METRIC_COLUMN_FAMILY = "m."; - /** Default Phoenix timeline info column family. */ - private static final String INFO_COLUMN_FAMILY = "i."; - /** Default separator for Phoenix storage. */ - private static final String AGGREGATION_STORAGE_SEPARATOR = ";"; - - /** Connection string to the deployed Phoenix cluster. */ - private String connString = null; - private Properties connProperties = new Properties(); - - public PhoenixOfflineAggregationWriterImpl(Properties prop) { - super(PhoenixOfflineAggregationWriterImpl.class.getName()); - connProperties = PropertiesUtil.deepCopy(prop); - } - - public PhoenixOfflineAggregationWriterImpl() { - super(PhoenixOfflineAggregationWriterImpl.class.getName()); - } - - @Override - public void serviceInit(Configuration conf) throws Exception { - Class.forName(DRIVER_CLASS_NAME); - // so check it here and only read in the config if it's not overridden. - connString = - conf.get(YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR, - YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT); - super.init(conf); - } - - @Override - public TimelineWriteResponse writeAggregatedEntity( - TimelineCollectorContext context, TimelineEntities entities, - OfflineAggregationInfo info) throws IOException { - TimelineWriteResponse response = new TimelineWriteResponse(); - String sql = "UPSERT INTO " + info.getTableName() - + " (" + StringUtils.join(info.getPrimaryKeyList(), ",") - + ", created_time, metric_names) " - + "VALUES (" - + StringUtils.repeat("?,", info.getPrimaryKeyList().length) - + "?, ?)"; - if (LOG.isDebugEnabled()) { - LOG.debug("TimelineEntity write SQL: " + sql); - } - - try (Connection conn = getConnection(); - PreparedStatement ps = conn.prepareStatement(sql)) { - for (TimelineEntity entity : entities.getEntities()) { - HashMap formattedMetrics = new HashMap<>(); - if (entity.getMetrics() != null) { - for (TimelineMetric m : entity.getMetrics()) { - formattedMetrics.put(m.getId(), m); - } - } - int idx = info.setStringsForPrimaryKey(ps, context, null, 1); - ps.setLong(idx++, entity.getCreatedTime()); - ps.setString(idx++, - StringUtils.join(formattedMetrics.keySet().toArray(), - AGGREGATION_STORAGE_SEPARATOR)); - ps.execute(); - - storeEntityVariableLengthFields(entity, formattedMetrics, context, conn, - info); - - conn.commit(); - } - } catch (SQLException se) { - LOG.error("Failed to add entity to Phoenix " + se.getMessage()); - throw new IOException(se); - } catch (Exception e) { - LOG.error("Exception on getting connection: " + e.getMessage()); - throw new IOException(e); - } - return response; - } - - /** - * Create Phoenix tables for offline aggregation storage if the tables do not - * exist. - * - * @throws IOException if any problem happens while creating Phoenix tables. - */ - public void createPhoenixTables() throws IOException { - // Create tables if necessary - try (Connection conn = getConnection(); - Statement stmt = conn.createStatement()) { - // Table schema defined as in YARN-3817. - String sql = "CREATE TABLE IF NOT EXISTS " - + OfflineAggregationInfo.FLOW_AGGREGATION_TABLE_NAME - + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, " - + "flow_name VARCHAR NOT NULL, " - + "created_time UNSIGNED_LONG, " - + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER - + " VARBINARY, " - + "metric_names VARCHAR, info_keys VARCHAR " - + "CONSTRAINT pk PRIMARY KEY(" - + "user, cluster, flow_name))"; - stmt.executeUpdate(sql); - sql = "CREATE TABLE IF NOT EXISTS " - + OfflineAggregationInfo.USER_AGGREGATION_TABLE_NAME - + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, " - + "created_time UNSIGNED_LONG, " - + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER - + " VARBINARY, " - + "metric_names VARCHAR, info_keys VARCHAR " - + "CONSTRAINT pk PRIMARY KEY(user, cluster))"; - stmt.executeUpdate(sql); - conn.commit(); - } catch (SQLException se) { - LOG.error("Failed in init data " + se.getLocalizedMessage()); - throw new IOException(se); - } - return; - } - - // Utility functions - @Private - @VisibleForTesting - Connection getConnection() throws IOException { - Connection conn; - try { - conn = DriverManager.getConnection(connString, connProperties); - conn.setAutoCommit(false); - } catch (SQLException se) { - LOG.error("Failed to connect to phoenix server! " - + se.getLocalizedMessage()); - throw new IOException(se); - } - return conn; - } - - // WARNING: This method will permanently drop a table! - @Private - @VisibleForTesting - void dropTable(String tableName) throws Exception { - try (Connection conn = getConnection(); - Statement stmt = conn.createStatement()) { - String sql = "DROP TABLE " + tableName; - stmt.executeUpdate(sql); - } catch (SQLException se) { - LOG.error("Failed in dropping entity table " + se.getLocalizedMessage()); - throw se; - } - } - - private static class DynamicColumns { - static final String COLUMN_FAMILY_TYPE_BYTES = " VARBINARY"; - static final String COLUMN_FAMILY_TYPE_STRING = " VARCHAR"; - private String columnFamilyPrefix; - private String type; - private Set columns; - - public DynamicColumns(String columnFamilyPrefix, String type, - Set keyValues) { - this.columnFamilyPrefix = columnFamilyPrefix; - this.columns = keyValues; - this.type = type; - } - } - - private static StringBuilder appendColumnsSQL( - StringBuilder colNames, DynamicColumns cfInfo) { - // Prepare the sql template by iterating through all keys - for (K key : cfInfo.columns) { - colNames.append(",").append(cfInfo.columnFamilyPrefix) - .append(key.toString()).append(cfInfo.type); - } - return colNames; - } - - private static int setValuesForColumnFamily( - PreparedStatement ps, Map keyValues, int startPos, - boolean converToBytes) throws SQLException { - int idx = startPos; - for (Map.Entry entry : keyValues.entrySet()) { - V value = entry.getValue(); - if (value instanceof Collection) { - ps.setString(idx++, StringUtils.join( - (Collection) value, AGGREGATION_STORAGE_SEPARATOR)); - } else { - if (converToBytes) { - try { - ps.setBytes(idx++, GenericObjectMapper.write(entry.getValue())); - } catch (IOException ie) { - LOG.error("Exception in converting values into bytes " - + ie.getMessage()); - throw new SQLException(ie); - } - } else { - ps.setString(idx++, value.toString()); - } - } - } - return idx; - } - - private static int setBytesForColumnFamily( - PreparedStatement ps, Map keyValues, int startPos) - throws SQLException { - return setValuesForColumnFamily(ps, keyValues, startPos, true); - } - - private static int setStringsForColumnFamily( - PreparedStatement ps, Map keyValues, int startPos) - throws SQLException { - return setValuesForColumnFamily(ps, keyValues, startPos, false); - } - - private static void storeEntityVariableLengthFields(TimelineEntity entity, - Map formattedMetrics, - TimelineCollectorContext context, Connection conn, - OfflineAggregationInfo aggregationInfo) throws SQLException { - int numPlaceholders = 0; - StringBuilder columnDefs = new StringBuilder( - StringUtils.join(aggregationInfo.getPrimaryKeyList(), ",")); - if (formattedMetrics != null && formattedMetrics.size() > 0) { - appendColumnsSQL(columnDefs, new DynamicColumns<>( - METRIC_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_BYTES, - formattedMetrics.keySet())); - numPlaceholders += formattedMetrics.keySet().size(); - } - if (numPlaceholders == 0) { - return; - } - StringBuilder placeholders = new StringBuilder(); - placeholders.append( - StringUtils.repeat("?,", aggregationInfo.getPrimaryKeyList().length)); - // numPlaceholders >= 1 now - placeholders.append("?") - .append(StringUtils.repeat(",?", numPlaceholders - 1)); - String sqlVariableLengthFields = new StringBuilder("UPSERT INTO ") - .append(aggregationInfo.getTableName()).append(" (").append(columnDefs) - .append(") VALUES(").append(placeholders).append(")").toString(); - if (LOG.isDebugEnabled()) { - LOG.debug("SQL statement for variable length fields: " - + sqlVariableLengthFields); - } - // Use try with resource statement for the prepared statement - try (PreparedStatement psVariableLengthFields = - conn.prepareStatement(sqlVariableLengthFields)) { - int idx = aggregationInfo.setStringsForPrimaryKey( - psVariableLengthFields, context, null, 1); - if (formattedMetrics != null && formattedMetrics.size() > 0) { - idx = setBytesForColumnFamily( - psVariableLengthFields, formattedMetrics, idx); - } - psVariableLengthFields.execute(); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java deleted file mode 100644 index 33f5449..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java +++ /dev/null @@ -1,272 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.cli.PosixParser; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.entity.EntityTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; - -import com.google.common.annotations.VisibleForTesting; - -/** - * This creates the schema for a hbase based backend for storing application - * timeline information. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public final class TimelineSchemaCreator { - private TimelineSchemaCreator() { - } - - final static String NAME = TimelineSchemaCreator.class.getSimpleName(); - private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class); - private static final String PHOENIX_OPTION_SHORT = "p"; - private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s"; - private static final String APP_TABLE_NAME_SHORT = "a"; - private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f"; - private static final String TTL_OPTION_SHORT = "m"; - private static final String ENTITY_TABLE_NAME_SHORT = "e"; - - public static void main(String[] args) throws Exception { - - Configuration hbaseConf = HBaseConfiguration.create(); - // Grab input args and allow for -Dxyz style arguments - String[] otherArgs = new GenericOptionsParser(hbaseConf, args) - .getRemainingArgs(); - - // Grab the arguments we're looking for. - CommandLine commandLine = parseArgs(otherArgs); - - // Grab the entityTableName argument - String entityTableName - = commandLine.getOptionValue(ENTITY_TABLE_NAME_SHORT); - if (StringUtils.isNotBlank(entityTableName)) { - hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName); - } - String entityTableTTLMetrics = commandLine.getOptionValue(TTL_OPTION_SHORT); - if (StringUtils.isNotBlank(entityTableTTLMetrics)) { - int metricsTTL = Integer.parseInt(entityTableTTLMetrics); - new EntityTable().setMetricsTTL(metricsTTL, hbaseConf); - } - // Grab the appToflowTableName argument - String appToflowTableName = commandLine.getOptionValue( - APP_TO_FLOW_TABLE_NAME_SHORT); - if (StringUtils.isNotBlank(appToflowTableName)) { - hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName); - } - // Grab the applicationTableName argument - String applicationTableName = commandLine.getOptionValue( - APP_TABLE_NAME_SHORT); - if (StringUtils.isNotBlank(applicationTableName)) { - hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME, - applicationTableName); - } - - List exceptions = new ArrayList<>(); - try { - boolean skipExisting - = commandLine.hasOption(SKIP_EXISTING_TABLE_OPTION_SHORT); - if (skipExisting) { - LOG.info("Will skip existing tables and continue on htable creation " - + "exceptions!"); - } - createAllTables(hbaseConf, skipExisting); - LOG.info("Successfully created HBase schema. "); - } catch (IOException e) { - LOG.error("Error in creating hbase tables: " + e.getMessage()); - exceptions.add(e); - } - - // Create Phoenix data schema if needed - if (commandLine.hasOption(PHOENIX_OPTION_SHORT)) { - Configuration phoenixConf = new Configuration(); - try { - PhoenixOfflineAggregationWriterImpl phoenixWriter = - new PhoenixOfflineAggregationWriterImpl(); - phoenixWriter.init(phoenixConf); - phoenixWriter.start(); - phoenixWriter.createPhoenixTables(); - phoenixWriter.stop(); - LOG.info("Successfully created Phoenix offline aggregation schema. "); - } catch (IOException e) { - LOG.error("Error in creating phoenix tables: " + e.getMessage()); - exceptions.add(e); - } - } - if (exceptions.size() > 0) { - LOG.warn("Schema creation finished with the following exceptions"); - for (Exception e : exceptions) { - LOG.warn(e.getMessage()); - } - System.exit(-1); - } else { - LOG.info("Schema creation finished successfully"); - } - } - - /** - * Parse command-line arguments. - * - * @param args - * command line arguments passed to program. - * @return parsed command line. - * @throws ParseException - */ - private static CommandLine parseArgs(String[] args) throws ParseException { - Options options = new Options(); - - // Input - Option o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true, - "entity table name"); - o.setArgName("entityTableName"); - o.setRequired(false); - options.addOption(o); - - o = new Option(TTL_OPTION_SHORT, "metricsTTL", true, - "TTL for metrics column family"); - o.setArgName("metricsTTL"); - o.setRequired(false); - options.addOption(o); - - o = new Option(APP_TO_FLOW_TABLE_NAME_SHORT, "appToflowTableName", true, - "app to flow table name"); - o.setArgName("appToflowTableName"); - o.setRequired(false); - options.addOption(o); - - o = new Option(APP_TABLE_NAME_SHORT, "applicationTableName", true, - "application table name"); - o.setArgName("applicationTableName"); - o.setRequired(false); - options.addOption(o); - - // Options without an argument - // No need to set arg name since we do not need an argument here - o = new Option(PHOENIX_OPTION_SHORT, "usePhoenix", false, - "create Phoenix offline aggregation tables"); - o.setRequired(false); - options.addOption(o); - - o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable", - false, "skip existing Hbase tables and continue to create new tables"); - o.setRequired(false); - options.addOption(o); - - CommandLineParser parser = new PosixParser(); - CommandLine commandLine = null; - try { - commandLine = parser.parse(options, args); - } catch (Exception e) { - LOG.error("ERROR: " + e.getMessage() + "\n"); - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp(NAME + " ", options, true); - System.exit(-1); - } - - return commandLine; - } - - @VisibleForTesting - public static void createAllTables(Configuration hbaseConf, - boolean skipExisting) throws IOException { - - Connection conn = null; - try { - conn = ConnectionFactory.createConnection(hbaseConf); - Admin admin = conn.getAdmin(); - if (admin == null) { - throw new IOException("Cannot create table since admin is null"); - } - try { - new EntityTable().createTable(admin, hbaseConf); - } catch (IOException e) { - if (skipExisting) { - LOG.warn("Skip and continue on: " + e.getMessage()); - } else { - throw e; - } - } - try { - new AppToFlowTable().createTable(admin, hbaseConf); - } catch (IOException e) { - if (skipExisting) { - LOG.warn("Skip and continue on: " + e.getMessage()); - } else { - throw e; - } - } - try { - new ApplicationTable().createTable(admin, hbaseConf); - } catch (IOException e) { - if (skipExisting) { - LOG.warn("Skip and continue on: " + e.getMessage()); - } else { - throw e; - } - } - try { - new FlowRunTable().createTable(admin, hbaseConf); - } catch (IOException e) { - if (skipExisting) { - LOG.warn("Skip and continue on: " + e.getMessage()); - } else { - throw e; - } - } - try { - new FlowActivityTable().createTable(admin, hbaseConf); - } catch (IOException e) { - if (skipExisting) { - LOG.warn("Skip and continue on: " + e.getMessage()); - } else { - throw e; - } - } - } finally { - if (conn != null) { - conn.close(); - } - } - } - - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java deleted file mode 100644 index dde3911..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java +++ /dev/null @@ -1,156 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import java.io.IOException; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * Identifies fully qualified columns for the {@link ApplicationTable}. - */ -public enum ApplicationColumn implements Column { - - /** - * App id. - */ - ID(ApplicationColumnFamily.INFO, "id"), - - /** - * When the application was created. - */ - CREATED_TIME(ApplicationColumnFamily.INFO, "created_time", - new LongConverter()), - - /** - * The version of the flow that this app belongs to. - */ - FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version"); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - private final String columnQualifier; - private final byte[] columnQualifierBytes; - - private ApplicationColumn(ColumnFamily columnFamily, - String columnQualifier) { - this(columnFamily, columnQualifier, GenericConverter.getInstance()); - } - - private ApplicationColumn(ColumnFamily columnFamily, - String columnQualifier, ValueConverter converter) { - this.columnFamily = columnFamily; - this.columnQualifier = columnQualifier; - // Future-proof by ensuring the right column prefix hygiene. - this.columnQualifierBytes = - Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); - this.column = new ColumnHelper(columnFamily, converter); - } - - /** - * @return the column name value - */ - private String getColumnQualifier() { - return columnQualifier; - } - - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, Long timestamp, - Object inputValue, Attribute... attributes) throws IOException { - column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, - inputValue, attributes); - } - - public Object readResult(Result result) throws IOException { - return column.readResult(result, columnQualifierBytes); - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /** - * Retrieve an {@link ApplicationColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)}. - * - * @param columnQualifier Name of the column to retrieve - * @return the corresponding {@link ApplicationColumn} or null - */ - public static final ApplicationColumn columnFor(String columnQualifier) { - - // Match column based on value, assume column family matches. - for (ApplicationColumn ac : ApplicationColumn.values()) { - // Find a match based only on name. - if (ac.getColumnQualifier().equals(columnQualifier)) { - return ac; - } - } - - // Default to null - return null; - } - - /** - * Retrieve an {@link ApplicationColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} - * if and only if {@code a.equals(b) & x.equals(y)} or - * {@code (x == y == null)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param name Name of the column to retrieve - * @return the corresponding {@link ApplicationColumn} or null if both - * arguments don't match. - */ - public static final ApplicationColumn columnFor( - ApplicationColumnFamily columnFamily, String name) { - - for (ApplicationColumn ac : ApplicationColumn.values()) { - // Find a match based column family and on name. - if (ac.columnFamily.equals(columnFamily) - && ac.getColumnQualifier().equals(name)) { - return ac; - } - } - - // Default to null - return null; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java deleted file mode 100644 index 97e5f7b..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents the application table column families. - */ -public enum ApplicationColumnFamily implements ColumnFamily { - - /** - * Info column family houses known columns, specifically ones included in - * columnfamily filters. - */ - INFO("i"), - - /** - * Configurations are in a separate column family for two reasons: a) the size - * of the config values can be very large and b) we expect that config values - * are often separately accessed from other metrics and info columns. - */ - CONFIGS("c"), - - /** - * Metrics have a separate column family, because they have a separate TTL. - */ - METRICS("m"); - - /** - * Byte representation of this column family. - */ - private final byte[] bytes; - - /** - * @param value create a column family with this name. Must be lower case and - * without spaces. - */ - private ApplicationColumnFamily(String value) { - // column families should be lower case and not contain any spaces. - this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); - } - - public byte[] getBytes() { - return Bytes.copy(bytes); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java deleted file mode 100644 index 42488f4..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java +++ /dev/null @@ -1,288 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import java.io.IOException; -import java.util.Map; -import java.util.NavigableMap; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * Identifies partially qualified columns for the application table. - */ -public enum ApplicationColumnPrefix implements ColumnPrefix { - - /** - * To store TimelineEntity getIsRelatedToEntities values. - */ - IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"), - - /** - * To store TimelineEntity getRelatesToEntities values. - */ - RELATES_TO(ApplicationColumnFamily.INFO, "r"), - - /** - * To store TimelineEntity info values. - */ - INFO(ApplicationColumnFamily.INFO, "i"), - - /** - * Lifecycle events for an application. - */ - EVENT(ApplicationColumnFamily.INFO, "e"), - - /** - * Config column stores configuration with config key as the column name. - */ - CONFIG(ApplicationColumnFamily.CONFIGS, null), - - /** - * Metrics are stored with the metric name as the column name. - */ - METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter()); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - - /** - * Can be null for those cases where the provided column qualifier is the - * entire column name. - */ - private final String columnPrefix; - private final byte[] columnPrefixBytes; - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily that this column is stored in. - * @param columnPrefix for this column. - */ - private ApplicationColumnPrefix(ColumnFamily columnFamily, - String columnPrefix) { - this(columnFamily, columnPrefix, GenericConverter.getInstance()); - } - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily that this column is stored in. - * @param columnPrefix for this column. - * @param converter used to encode/decode values to be stored in HBase for - * this column prefix. - */ - private ApplicationColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, ValueConverter converter) { - column = new ColumnHelper(columnFamily, converter); - this.columnFamily = columnFamily; - this.columnPrefix = columnPrefix; - if (columnPrefix == null) { - this.columnPrefixBytes = null; - } else { - // Future-proof by ensuring the right column prefix hygiene. - this.columnPrefixBytes = - Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); - } - } - - /** - * @return the column name value - */ - private String getColumnPrefix() { - return columnPrefix; - } - - @Override - public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - @Override - public byte[] getColumnPrefixBytes(String qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, byte[] qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - attributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, String qualifier, - Long timestamp, Object inputValue, Attribute...attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - attributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) - */ - public Object readResult(Result result, String qualifier) throws IOException { - byte[] columnQualifier = - ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); - return column.readResult(result, columnQualifier); - } - - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResults(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public Map readResults(Result result, - KeyConverter keyConverter) throws IOException { - return column.readResults(result, columnPrefixBytes, keyConverter); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public NavigableMap> - readResultsWithTimestamps(Result result, KeyConverter keyConverter) - throws IOException { - return column.readResultsWithTimestamps(result, columnPrefixBytes, - keyConverter); - } - - /** - * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there - * is no match. The following holds true: {@code columnFor(x) == columnFor(y)} - * if and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link ApplicationColumnPrefix} or null - */ - public static final ApplicationColumnPrefix columnFor(String columnPrefix) { - - // Match column based on value, assume column family matches. - for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) { - // Find a match based only on name. - if (acp.getColumnPrefix().equals(columnPrefix)) { - return acp; - } - } - - // Default to null - return null; - } - - /** - * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there - * is no match. The following holds true: - * {@code columnFor(a,x) == columnFor(b,y)} if and only if - * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link ApplicationColumnPrefix} or null if both - * arguments don't match. - */ - public static final ApplicationColumnPrefix columnFor( - ApplicationColumnFamily columnFamily, String columnPrefix) { - - // TODO: needs unit test to confirm and need to update javadoc to explain - // null prefix case. - - for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) { - // Find a match based column family and on name. - if (acp.columnFamily.equals(columnFamily) - && (((columnPrefix == null) && (acp.getColumnPrefix() == null)) || - (acp.getColumnPrefix().equals(columnPrefix)))) { - return acp; - } - } - - // Default to null - return null; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java deleted file mode 100644 index da62fdf..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java +++ /dev/null @@ -1,206 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents a rowkey for the application table. - */ -public class ApplicationRowKey { - private final String clusterId; - private final String userId; - private final String flowName; - private final Long flowRunId; - private final String appId; - private final KeyConverter appRowKeyConverter = - new ApplicationRowKeyConverter(); - - public ApplicationRowKey(String clusterId, String userId, String flowName, - Long flowRunId, String appId) { - this.clusterId = clusterId; - this.userId = userId; - this.flowName = flowName; - this.flowRunId = flowRunId; - this.appId = appId; - } - - public String getClusterId() { - return clusterId; - } - - public String getUserId() { - return userId; - } - - public String getFlowName() { - return flowName; - } - - public Long getFlowRunId() { - return flowRunId; - } - - public String getAppId() { - return appId; - } - - /** - * Constructs a row key for the application table as follows: - * {@code clusterId!userName!flowName!flowRunId!AppId}. - * - * @return byte array with the row key - */ - public byte[] getRowKey() { - return appRowKeyConverter.encode(this); - } - - /** - * Given the raw row key as bytes, returns the row key as an object. - * - * @param rowKey Byte representation of row key. - * @return An ApplicationRowKey object. - */ - public static ApplicationRowKey parseRowKey(byte[] rowKey) { - return new ApplicationRowKeyConverter().decode(rowKey); - } - - /** - * Encodes and decodes row key for application table. The row key is of the - * form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long, - * appId is encoded and decoded using {@link AppIdKeyConverter} and rest are - * strings. - *

- */ - final private static class ApplicationRowKeyConverter implements - KeyConverter { - - private final KeyConverter appIDKeyConverter = - new AppIdKeyConverter(); - - /** - * Intended for use in ApplicationRowKey only. - */ - private ApplicationRowKeyConverter() { - } - - /** - * Application row key is of the form - * clusterId!userName!flowName!flowRunId!appId with each segment separated - * by !. The sizes below indicate sizes of each one of these segements in - * sequence. clusterId, userName and flowName are strings. flowrunId is a - * long hence 8 bytes in size. app id is represented as 12 bytes with - * cluster timestamp part of appid takes 8 bytes(long) and seq id takes 4 - * bytes(int). Strings are variable in size (i.e. end whenever separator is - * encountered). This is used while decoding and helps in determining where - * to split. - */ - private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - AppIdKeyConverter.getKeySize() }; - - /* - * (non-Javadoc) - * - * Encodes ApplicationRowKey object into a byte array with each - * component/field in ApplicationRowKey separated by Separator#QUALIFIERS. - * This leads to an application table row key of the form - * clusterId!userName!flowName!flowRunId!appId If flowRunId in passed - * ApplicationRowKey object is null (and the fields preceding it i.e. - * clusterId, userId and flowName are not null), this returns a row key - * prefix of the form clusterId!userName!flowName! and if appId in - * ApplicationRowKey is null (other 4 components all are not null), this - * returns a row key prefix of the form - * clusterId!userName!flowName!flowRunId! flowRunId is inverted while - * encoding as it helps maintain a descending order for row keys in the - * application table. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#encode(java.lang.Object) - */ - @Override - public byte[] encode(ApplicationRowKey rowKey) { - byte[] cluster = - Separator.encode(rowKey.getClusterId(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] user = - Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS); - byte[] flow = - Separator.encode(rowKey.getFlowName(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] first = Separator.QUALIFIERS.join(cluster, user, flow); - // Note that flowRunId is a long, so we can't encode them all at the same - // time. - if (rowKey.getFlowRunId() == null) { - return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); - } - byte[] second = - Bytes.toBytes(LongConverter.invertLong( - rowKey.getFlowRunId())); - if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) { - return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES); - } - byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); - return Separator.QUALIFIERS.join(first, second, third); - } - - /* - * (non-Javadoc) - * - * Decodes an application row key of the form - * clusterId!userName!flowName!flowRunId!appId represented in byte format - * and converts it into an ApplicationRowKey object.flowRunId is inverted - * while decoding as it was inverted while encoding. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#decode(byte[]) - */ - @Override - public ApplicationRowKey decode(byte[] rowKey) { - byte[][] rowKeyComponents = - Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); - if (rowKeyComponents.length != 5) { - throw new IllegalArgumentException("the row key is not valid for " - + "an application"); - } - String clusterId = - Separator.decode(Bytes.toString(rowKeyComponents[0]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String userId = - Separator.decode(Bytes.toString(rowKeyComponents[1]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String flowName = - Separator.decode(Bytes.toString(rowKeyComponents[2]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - Long flowRunId = - LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); - String appId = appIDKeyConverter.decode(rowKeyComponents[4]); - return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, - appId); - } - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java deleted file mode 100644 index f61b0e9..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; - -/** - * Represents a partial rowkey (without flowName or without flowName and - * flowRunId) for the application table. - */ -public class ApplicationRowKeyPrefix extends ApplicationRowKey implements - RowKeyPrefix { - - /** - * Creates a prefix which generates the following rowKeyPrefixes for the - * application table: {@code clusterId!userName!flowName!}. - * - * @param clusterId the cluster on which applications ran - * @param userId the user that ran applications - * @param flowName the name of the flow that was run by the user on the - * cluster - */ - public ApplicationRowKeyPrefix(String clusterId, String userId, - String flowName) { - super(clusterId, userId, flowName, null, null); - } - - /** - * Creates a prefix which generates the following rowKeyPrefixes for the - * application table: {@code clusterId!userName!flowName!flowRunId!}. - * - * @param clusterId identifying the cluster - * @param userId identifying the user - * @param flowName identifying the flow - * @param flowRunId identifying the instance of this flow - */ - public ApplicationRowKeyPrefix(String clusterId, String userId, - String flowName, Long flowRunId) { - super(clusterId, userId, flowName, flowRunId, null); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.application. - * RowKeyPrefix#getRowKeyPrefix() - */ - @Override - public byte[] getRowKeyPrefix() { - return super.getRowKey(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java deleted file mode 100644 index a02f768..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java +++ /dev/null @@ -1,161 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; - -/** - * The application table as column families info, config and metrics. Info - * stores information about a YARN application entity, config stores - * configuration data of a YARN application, metrics stores the metrics of a - * YARN application. This table is entirely analogous to the entity table but - * created for better performance. - * - * Example application table record: - * - *

- * |-------------------------------------------------------------------------|
- * |  Row       | Column Family                | Column Family| Column Family|
- * |  key       | info                         | metrics      | config       |
- * |-------------------------------------------------------------------------|
- * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
- * | userName!  |                              | metricValue1 | configValue1 |
- * | flowName!  | created_time:                | @timestamp1  |              |
- * | flowRunId! | 1392993084018                |              | configKey2:  |
- * | AppId      |                              | metriciD1:   | configValue2 |
- * |            | i!infoKey:                   | metricValue2 |              |
- * |            | infoValue                    | @timestamp2  |              |
- * |            |                              |              |              |
- * |            | r!relatesToKey:              | metricId2:   |              |
- * |            | id3=id4=id5                  | metricValue1 |              |
- * |            |                              | @timestamp2  |              |
- * |            | s!isRelatedToKey:            |              |              |
- * |            | id7=id9=id6                  |              |              |
- * |            |                              |              |              |
- * |            | e!eventId=timestamp=infoKey: |              |              |
- * |            | eventInfoValue               |              |              |
- * |            |                              |              |              |
- * |            | flowVersion:                 |              |              |
- * |            | versionValue                 |              |              |
- * |-------------------------------------------------------------------------|
- * 
- */ -public class ApplicationTable extends BaseTable { - /** application prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".application"; - - /** config param name that specifies the application table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** - * config param name that specifies the TTL for metrics column family in - * application table. - */ - private static final String METRICS_TTL_CONF_NAME = PREFIX - + ".table.metrics.ttl"; - - /** default value for application table name. */ - private static final String DEFAULT_TABLE_NAME = - "timelineservice.application"; - - /** default TTL is 30 days for metrics timeseries. */ - private static final int DEFAULT_METRICS_TTL = 2592000; - - /** default max number of versions. */ - private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000; - - private static final Log LOG = LogFactory.getLog(ApplicationTable.class); - - public ApplicationTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor applicationTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - applicationTableDescp.addFamily(infoCF); - - HColumnDescriptor configCF = - new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes()); - configCF.setBloomFilterType(BloomType.ROWCOL); - configCF.setBlockCacheEnabled(true); - applicationTableDescp.addFamily(configCF); - - HColumnDescriptor metricsCF = - new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes()); - applicationTableDescp.addFamily(metricsCF); - metricsCF.setBlockCacheEnabled(true); - // always keep 1 version (the latest) - metricsCF.setMinVersions(1); - metricsCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, - DEFAULT_METRICS_TTL)); - applicationTableDescp.setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(applicationTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } - - /** - * @param metricsTTL time to live parameter for the metrics in this table. - * @param hbaseConf configuration in which to set the metrics TTL config - * variable. - */ - public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { - hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java deleted file mode 100644 index 03f508f..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Package org.apache.hadoop.yarn.server.timelineservice.storage.application - * contains classes related to implementation for application table. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -package org.apache.hadoop.yarn.server.timelineservice.storage.application; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java deleted file mode 100644 index ff61633..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java +++ /dev/null @@ -1,148 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -import java.io.IOException; - -/** - * Identifies fully qualified columns for the {@link AppToFlowTable}. - */ -public enum AppToFlowColumn implements Column { - - /** - * The flow ID. - */ - FLOW_ID(AppToFlowColumnFamily.MAPPING, "flow_id"), - - /** - * The flow run ID. - */ - FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"), - - /** - * The user. - */ - USER_ID(AppToFlowColumnFamily.MAPPING, "user_id"); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - private final String columnQualifier; - private final byte[] columnQualifierBytes; - - AppToFlowColumn(ColumnFamily columnFamily, - String columnQualifier) { - this.columnFamily = columnFamily; - this.columnQualifier = columnQualifier; - // Future-proof by ensuring the right column prefix hygiene. - this.columnQualifierBytes = - Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); - this.column = new ColumnHelper(columnFamily); - } - - /** - * @return the column name value - */ - private String getColumnQualifier() { - return columnQualifier; - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, Long timestamp, - Object inputValue, Attribute... attributes) throws IOException { - column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, - inputValue, attributes); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - public Object readResult(Result result) throws IOException { - return column.readResult(result, columnQualifierBytes); - } - - /** - * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnQualifier Name of the column to retrieve - * @return the corresponding {@link AppToFlowColumn} or null - */ - public static final AppToFlowColumn columnFor(String columnQualifier) { - - // Match column based on value, assume column family matches. - for (AppToFlowColumn ec : AppToFlowColumn.values()) { - // Find a match based only on name. - if (ec.getColumnQualifier().equals(columnQualifier)) { - return ec; - } - } - - // Default to null - return null; - } - - /** - * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} - * if and only if {@code a.equals(b) & x.equals(y)} or - * {@code (x == y == null)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param name Name of the column to retrieve - * @return the corresponding {@link AppToFlowColumn} or null if both arguments - * don't match. - */ - public static final AppToFlowColumn columnFor( - AppToFlowColumnFamily columnFamily, String name) { - - for (AppToFlowColumn ec : AppToFlowColumn.values()) { - // Find a match based column family and on name. - if (ec.columnFamily.equals(columnFamily) - && ec.getColumnQualifier().equals(name)) { - return ec; - } - } - - // Default to null - return null; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java deleted file mode 100644 index f3f045e..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents the app_flow table column families. - */ -public enum AppToFlowColumnFamily implements ColumnFamily { - /** - * Mapping column family houses known columns such as flowName and flowRunId. - */ - MAPPING("m"); - - /** - * Byte representation of this column family. - */ - private final byte[] bytes; - - /** - * @param value create a column family with this name. Must be lower case and - * without spaces. - */ - AppToFlowColumnFamily(String value) { - // column families should be lower case and not contain any spaces. - this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); - } - - public byte[] getBytes() { - return Bytes.copy(bytes); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java deleted file mode 100644 index 8df4407..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java +++ /dev/null @@ -1,143 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents a rowkey for the app_flow table. - */ -public class AppToFlowRowKey { - private final String clusterId; - private final String appId; - private final KeyConverter appToFlowRowKeyConverter = - new AppToFlowRowKeyConverter(); - - public AppToFlowRowKey(String clusterId, String appId) { - this.clusterId = clusterId; - this.appId = appId; - } - - public String getClusterId() { - return clusterId; - } - - public String getAppId() { - return appId; - } - - /** - * Constructs a row key prefix for the app_flow table as follows: - * {@code clusterId!AppId}. - * - * @return byte array with the row key - */ - public byte[] getRowKey() { - return appToFlowRowKeyConverter.encode(this); - } - - /** - * Given the raw row key as bytes, returns the row key as an object. - * - * @param rowKey a rowkey represented as a byte array. - * @return an AppToFlowRowKey object. - */ - public static AppToFlowRowKey parseRowKey(byte[] rowKey) { - return new AppToFlowRowKeyConverter().decode(rowKey); - } - - /** - * Encodes and decodes row key for app_flow table. The row key is of the form - * clusterId!appId. clusterId is a string and appId is encoded/decoded using - * {@link AppIdKeyConverter}. - *

- */ - final private static class AppToFlowRowKeyConverter implements - KeyConverter { - - private final KeyConverter appIDKeyConverter = - new AppIdKeyConverter(); - - /** - * Intended for use in AppToFlowRowKey only. - */ - private AppToFlowRowKeyConverter() { - } - - - /** - * App to flow row key is of the form clusterId!appId with the 2 segments - * separated by !. The sizes below indicate sizes of both of these segments - * in sequence. clusterId is a string. appId is represented as 12 bytes w. - * cluster Timestamp part of appid taking 8 bytes(long) and seq id taking 4 - * bytes(int). Strings are variable in size (i.e. end whenever separator is - * encountered). This is used while decoding and helps in determining where - * to split. - */ - private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, - Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT }; - - /* - * (non-Javadoc) - * - * Encodes AppToFlowRowKey object into a byte array with each - * component/field in AppToFlowRowKey separated by Separator#QUALIFIERS. - * This leads to an app to flow table row key of the form clusterId!appId - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#encode(java.lang.Object) - */ - @Override - public byte[] encode(AppToFlowRowKey rowKey) { - byte[] first = - Separator.encode(rowKey.getClusterId(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] second = appIDKeyConverter.encode(rowKey.getAppId()); - return Separator.QUALIFIERS.join(first, second); - } - - /* - * (non-Javadoc) - * - * Decodes an app to flow row key of the form clusterId!appId represented - * in byte format and converts it into an AppToFlowRowKey object. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#decode(byte[]) - */ - @Override - public AppToFlowRowKey decode(byte[] rowKey) { - byte[][] rowKeyComponents = - Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); - if (rowKeyComponents.length != 2) { - throw new IllegalArgumentException("the row key is not valid for " - + "the app-to-flow table"); - } - String clusterId = - Separator.decode(Bytes.toString(rowKeyComponents[0]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String appId = appIDKeyConverter.decode(rowKeyComponents[1]); - return new AppToFlowRowKey(clusterId, appId); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java deleted file mode 100644 index 301cf99..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; - -import java.io.IOException; - -/** - * The app_flow table as column families mapping. Mapping stores - * appId to flowName and flowRunId mapping information - * - * Example app_flow table record: - * - *

- * |--------------------------------------|
- * |  Row       | Column Family           |
- * |  key       | info                    |
- * |--------------------------------------|
- * | clusterId! | flowName:               |
- * | AppId      | foo@daily_hive_report   |
- * |            |                         |
- * |            | flowRunId:              |
- * |            | 1452828720457           |
- * |            |                         |
- * |            | user_id:                |
- * |            | admin                   |
- * |            |                         |
- * |            |                         |
- * |            |                         |
- * |--------------------------------------|
- * 
- */ -public class AppToFlowTable extends BaseTable { - /** app_flow prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow"; - - /** config param name that specifies the app_flow table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for app_flow table name. */ - private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow"; - - private static final Log LOG = LogFactory.getLog(AppToFlowTable.class); - - public AppToFlowTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor appToFlowTableDescp = new HTableDescriptor(table); - HColumnDescriptor mappCF = - new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes()); - mappCF.setBloomFilterType(BloomType.ROWCOL); - appToFlowTableDescp.addFamily(mappCF); - - appToFlowTableDescp - .setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(appToFlowTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java deleted file mode 100644 index f01d982..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow - * contains classes related to implementation for app to flow table. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java deleted file mode 100644 index c165801..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * Encodes and decodes {@link ApplicationId} for row keys. - * App ID is stored in row key as 12 bytes, cluster timestamp section of app id - * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes). - */ -public final class AppIdKeyConverter implements KeyConverter { - - public AppIdKeyConverter() { - } - - /* - * (non-Javadoc) - * - * Converts/encodes a string app Id into a byte representation for (row) keys. - * For conversion, we extract cluster timestamp and sequence id from the - * string app id (calls ConverterUtils#toApplicationId(String) for - * conversion) and then store it in a byte array of length 12 (8 bytes (long) - * for cluster timestamp followed 4 bytes(int) for sequence id). Both cluster - * timestamp and sequence id are inverted so that the most recent cluster - * timestamp and highest sequence id appears first in the table (i.e. - * application id appears in a descending order). - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #encode(java.lang.Object) - */ - @Override - public byte[] encode(String appIdStr) { - ApplicationId appId = ApplicationId.fromString(appIdStr); - byte[] appIdBytes = new byte[getKeySize()]; - byte[] clusterTs = Bytes.toBytes( - LongConverter.invertLong(appId.getClusterTimestamp())); - System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG); - byte[] seqId = Bytes.toBytes( - HBaseTimelineStorageUtils.invertInt(appId.getId())); - System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT); - return appIdBytes; - } - - /* - * (non-Javadoc) - * - * Converts/decodes a 12 byte representation of app id for (row) keys to an - * app id in string format which can be returned back to client. - * For decoding, 12 bytes are interpreted as 8 bytes of inverted cluster - * timestamp(long) followed by 4 bytes of inverted sequence id(int). Calls - * ApplicationId#toString to generate string representation of app id. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #decode(byte[]) - */ - @Override - public String decode(byte[] appIdBytes) { - if (appIdBytes.length != getKeySize()) { - throw new IllegalArgumentException("Invalid app id in byte format"); - } - long clusterTs = LongConverter.invertLong( - Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG)); - int seqId = HBaseTimelineStorageUtils.invertInt( - Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT)); - return ApplicationId.newInstance(clusterTs, seqId).toString(); - } - - /** - * Returns the size of app id after encoding. - * - * @return size of app id after encoding. - */ - public static int getKeySize() { - return Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java deleted file mode 100644 index 8581aa4..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java +++ /dev/null @@ -1,140 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; - -/** - * Implements behavior common to tables used in the timeline service storage. It - * is thread-safe, and can be used by multiple threads concurrently. - * - * @param reference to the table instance class itself for type safety. - */ -public abstract class BaseTable { - - /** - * Name of config variable that is used to point to this table. - */ - private final String tableNameConfName; - - /** - * Unless the configuration overrides, this will be the default name for the - * table when it is created. - */ - private final String defaultTableName; - - /** - * @param tableNameConfName name of config variable that is used to point to - * this table. - * @param defaultTableName Default table name if table from config is not - * found. - */ - protected BaseTable(String tableNameConfName, String defaultTableName) { - this.tableNameConfName = tableNameConfName; - this.defaultTableName = defaultTableName; - } - - /** - * Used to create a type-safe mutator for this table. - * - * @param hbaseConf used to read table name. - * @param conn used to create a table from. - * @return a type safe {@link BufferedMutator} for the entity table. - * @throws IOException if any exception occurs while creating mutator for the - * table. - */ - public TypedBufferedMutator getTableMutator(Configuration hbaseConf, - Connection conn) throws IOException { - - TableName tableName = this.getTableName(hbaseConf); - - // Plain buffered mutator - BufferedMutator bufferedMutator = conn.getBufferedMutator(tableName); - - // Now make this thing type safe. - // This is how service initialization should hang on to this variable, with - // the proper type - TypedBufferedMutator table = - new BufferedMutatorDelegator(bufferedMutator); - - return table; - } - - /** - * @param hbaseConf used to read settings that override defaults - * @param conn used to create table from - * @param scan that specifies what you want to read from this table. - * @return scanner for the table. - * @throws IOException if any exception occurs while getting the scanner. - */ - public ResultScanner getResultScanner(Configuration hbaseConf, - Connection conn, Scan scan) throws IOException { - Table table = conn.getTable(getTableName(hbaseConf)); - return table.getScanner(scan); - } - - /** - * - * @param hbaseConf used to read settings that override defaults - * @param conn used to create table from - * @param get that specifies what single row you want to get from this table - * @return result of get operation - * @throws IOException if any exception occurs while getting the result. - */ - public Result getResult(Configuration hbaseConf, Connection conn, Get get) - throws IOException { - Table table = conn.getTable(getTableName(hbaseConf)); - return table.get(get); - } - - /** - * Get the table name for this table. - * - * @param hbaseConf HBase configuration from which table name will be fetched. - * @return A {@link TableName} object. - */ - public TableName getTableName(Configuration hbaseConf) { - TableName table = - TableName.valueOf(hbaseConf.get(tableNameConfName, defaultTableName)); - return table; - - } - - /** - * Used to create the table in HBase. Should be called only once (per HBase - * instance). - * - * @param admin Used for doing HBase table operations. - * @param hbaseConf Hbase configuration. - * @throws IOException if any exception occurs while creating the table. - */ - public abstract void createTable(Admin admin, Configuration hbaseConf) - throws IOException; - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java deleted file mode 100644 index cf469a5..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.Mutation; - -/** - * To be used to wrap an actual {@link BufferedMutator} in a type safe manner. - * - * @param The class referring to the table to be written to. - */ -class BufferedMutatorDelegator implements TypedBufferedMutator { - - private final BufferedMutator bufferedMutator; - - /** - * @param bufferedMutator the mutator to be wrapped for delegation. Shall not - * be null. - */ - public BufferedMutatorDelegator(BufferedMutator bufferedMutator) { - this.bufferedMutator = bufferedMutator; - } - - public TableName getName() { - return bufferedMutator.getName(); - } - - public Configuration getConfiguration() { - return bufferedMutator.getConfiguration(); - } - - public void mutate(Mutation mutation) throws IOException { - bufferedMutator.mutate(mutation); - } - - public void mutate(List mutations) throws IOException { - bufferedMutator.mutate(mutations); - } - - public void close() throws IOException { - bufferedMutator.close(); - } - - public void flush() throws IOException { - bufferedMutator.flush(); - } - - public long getWriteBufferSize() { - return bufferedMutator.getWriteBufferSize(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java deleted file mode 100644 index 90f2de4..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * A Column represents the way to store a fully qualified column in a specific - * table. - */ -public interface Column { - - /** - * Sends a Mutation to the table. The mutations will be buffered and sent over - * the wire as part of a batch. - * - * @param rowKey identifying the row to write. Nothing gets written when null. - * @param tableMutator used to modify the underlying HBase table. Caller is - * responsible to pass a mutator for the table that actually has this - * column. - * @param timestamp version timestamp. When null the server timestamp will be - * used. - * @param attributes Map of attributes for this mutation. used in the - * coprocessor to set/read the cell tags. Can be null. - * @param inputValue the value to write to the rowKey and column qualifier. - * Nothing gets written when null. - * @throws IOException if there is any exception encountered during store. - */ - void store(byte[] rowKey, TypedBufferedMutator tableMutator, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException; - - /** - * Get the latest version of this specified column. Note: this call clones the - * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. - * - * @param result Cannot be null - * @return result object (can be cast to whatever object was written to), or - * null when result doesn't contain this column. - * @throws IOException if there is any exception encountered while reading - * result. - */ - Object readResult(Result result) throws IOException; - - /** - * Returns column family name(as bytes) associated with this column. - * @return a byte array encoding column family for this column qualifier. - */ - byte[] getColumnFamilyBytes(); - - /** - * Get byte representation for this column qualifier. - * @return a byte array representing column qualifier. - */ - byte[] getColumnQualifierBytes(); - - /** - * Returns value converter implementation associated with this column. - * @return a {@link ValueConverter} implementation. - */ - ValueConverter getValueConverter(); -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java deleted file mode 100644 index 452adcd..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -/** - * Type safe column family. - * - * @param refers to the table for which this column family is used for. - */ -public interface ColumnFamily { - - /** - * Keep a local copy if you need to avoid overhead of repeated cloning. - * - * @return a clone of the byte representation of the column family. - */ - byte[] getBytes(); - -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java deleted file mode 100644 index be55db5..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java +++ /dev/null @@ -1,388 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NavigableMap; -import java.util.TreeMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; -/** - * This class is meant to be used only by explicit Columns, and not directly to - * write by clients. - * - * @param refers to the table. - */ -public class ColumnHelper { - private static final Log LOG = LogFactory.getLog(ColumnHelper.class); - - private final ColumnFamily columnFamily; - - /** - * Local copy of bytes representation of columnFamily so that we can avoid - * cloning a new copy over and over. - */ - private final byte[] columnFamilyBytes; - - private final ValueConverter converter; - - public ColumnHelper(ColumnFamily columnFamily) { - this(columnFamily, GenericConverter.getInstance()); - } - - public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter) { - this.columnFamily = columnFamily; - columnFamilyBytes = columnFamily.getBytes(); - if (converter == null) { - this.converter = GenericConverter.getInstance(); - } else { - this.converter = converter; - } - } - - /** - * Sends a Mutation to the table. The mutations will be buffered and sent over - * the wire as part of a batch. - * - * @param rowKey - * identifying the row to write. Nothing gets written when null. - * @param tableMutator - * used to modify the underlying HBase table - * @param columnQualifier - * column qualifier. Nothing gets written when null. - * @param timestamp - * version timestamp. When null the current timestamp multiplied with - * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of - * app id will be used - * @param inputValue - * the value to write to the rowKey and column qualifier. Nothing - * gets written when null. - * @param attributes Attributes to be set for HBase Put. - * @throws IOException if any problem occurs during store operation(sending - * mutation to table). - */ - public void store(byte[] rowKey, TypedBufferedMutator tableMutator, - byte[] columnQualifier, Long timestamp, Object inputValue, - Attribute... attributes) throws IOException { - if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) { - return; - } - Put p = new Put(rowKey); - timestamp = getPutTimestamp(timestamp, attributes); - p.addColumn(columnFamilyBytes, columnQualifier, timestamp, - converter.encodeValue(inputValue)); - if ((attributes != null) && (attributes.length > 0)) { - for (Attribute attribute : attributes) { - p.setAttribute(attribute.getName(), attribute.getValue()); - } - } - tableMutator.mutate(p); - } - - /* - * Figures out the cell timestamp used in the Put For storing into flow run - * table. We would like to left shift the timestamp and supplement it with the - * AppId id so that there are no collisions in the flow run table's cells - */ - private long getPutTimestamp(Long timestamp, Attribute[] attributes) { - if (timestamp == null) { - timestamp = System.currentTimeMillis(); - } - String appId = getAppIdFromAttributes(attributes); - long supplementedTS = TimestampGenerator.getSupplementedTimestamp( - timestamp, appId); - return supplementedTS; - } - - private String getAppIdFromAttributes(Attribute[] attributes) { - if (attributes == null) { - return null; - } - String appId = null; - for (Attribute attribute : attributes) { - if (AggregationCompactionDimension.APPLICATION_ID.toString().equals( - attribute.getName())) { - appId = Bytes.toString(attribute.getValue()); - } - } - return appId; - } - - /** - * @return the column family for this column implementation. - */ - public ColumnFamily getColumnFamily() { - return columnFamily; - } - - /** - * Get the latest version of this specified column. Note: this call clones the - * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. - * - * @param result from which to read the value. Cannot be null - * @param columnQualifierBytes referring to the column to be read. - * @return latest version of the specified column of whichever object was - * written. - * @throws IOException if any problem occurs while reading result. - */ - public Object readResult(Result result, byte[] columnQualifierBytes) - throws IOException { - if (result == null || columnQualifierBytes == null) { - return null; - } - - // Would have preferred to be able to use getValueAsByteBuffer and get a - // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like - // that. - byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes); - return converter.decodeValue(value); - } - - /** - * @param result from which to reads data with timestamps - * @param columnPrefixBytes optional prefix to limit columns. If null all - * columns are returned. - * @param identifies the type of column name(indicated by type of key - * converter). - * @param the type of the values. The values will be cast into that type. - * @param keyConverter used to convert column bytes to the appropriate key - * type. - * @return the cell values at each respective time in for form - * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, - * idB={timestamp3->value3}, idC={timestamp1->value4}}} - * @throws IOException if any problem occurs while reading results. - */ - @SuppressWarnings("unchecked") - public NavigableMap> - readResultsWithTimestamps(Result result, byte[] columnPrefixBytes, - KeyConverter keyConverter) throws IOException { - - NavigableMap> results = new TreeMap<>(); - - if (result != null) { - NavigableMap< - byte[], NavigableMap>> resultMap = - result.getMap(); - - NavigableMap> columnCellMap = - resultMap.get(columnFamilyBytes); - - // could be that there is no such column family. - if (columnCellMap != null) { - for (Entry> entry : columnCellMap - .entrySet()) { - K converterColumnKey = null; - if (columnPrefixBytes == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("null prefix was specified; returning all columns"); - } - try { - converterColumnKey = keyConverter.decode(entry.getKey()); - } catch (IllegalArgumentException iae) { - LOG.error("Illegal column found, skipping this column.", iae); - continue; - } - } else { - // A non-null prefix means columns are actually of the form - // prefix!columnNameRemainder - byte[][] columnNameParts = - Separator.QUALIFIERS.split(entry.getKey(), 2); - byte[] actualColumnPrefixBytes = columnNameParts[0]; - if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) - && columnNameParts.length == 2) { - try { - // This is the prefix that we want - converterColumnKey = keyConverter.decode(columnNameParts[1]); - } catch (IllegalArgumentException iae) { - LOG.error("Illegal column found, skipping this column.", iae); - continue; - } - } - } - - // If this column has the prefix we want - if (converterColumnKey != null) { - NavigableMap cellResults = - new TreeMap(); - NavigableMap cells = entry.getValue(); - if (cells != null) { - for (Entry cell : cells.entrySet()) { - V value = - (V) converter.decodeValue(cell.getValue()); - cellResults.put( - TimestampGenerator.getTruncatedTimestamp(cell.getKey()), - value); - } - } - results.put(converterColumnKey, cellResults); - } - } // for entry : columnCellMap - } // if columnCellMap != null - } // if result != null - return results; - } - - /** - * @param identifies the type of column name(indicated by type of key - * converter). - * @param result from which to read columns - * @param columnPrefixBytes optional prefix to limit columns. If null all - * columns are returned. - * @param keyConverter used to convert column bytes to the appropriate key - * type. - * @return the latest values of columns in the column family. If the column - * prefix is null, the column qualifier is returned as Strings. For a - * non-null column prefix bytes, the column qualifier is returned as - * a list of parts, each part a byte[]. This is to facilitate - * returning byte arrays of values that were not Strings. - * @throws IOException if any problem occurs while reading results. - */ - public Map readResults(Result result, - byte[] columnPrefixBytes, KeyConverter keyConverter) - throws IOException { - Map results = new HashMap(); - - if (result != null) { - Map columns = result.getFamilyMap(columnFamilyBytes); - for (Entry entry : columns.entrySet()) { - byte[] columnKey = entry.getKey(); - if (columnKey != null && columnKey.length > 0) { - - K converterColumnKey = null; - if (columnPrefixBytes == null) { - try { - converterColumnKey = keyConverter.decode(columnKey); - } catch (IllegalArgumentException iae) { - LOG.error("Illegal column found, skipping this column.", iae); - continue; - } - } else { - // A non-null prefix means columns are actually of the form - // prefix!columnNameRemainder - byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2); - if (columnNameParts.length > 0) { - byte[] actualColumnPrefixBytes = columnNameParts[0]; - // If this is the prefix that we want - if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) - && columnNameParts.length == 2) { - try { - converterColumnKey = keyConverter.decode(columnNameParts[1]); - } catch (IllegalArgumentException iae) { - LOG.error("Illegal column found, skipping this column.", iae); - continue; - } - } - } - } // if-else - - // If the columnPrefix is null (we want all columns), or the actual - // prefix matches the given prefix we want this column - if (converterColumnKey != null) { - Object value = converter.decodeValue(entry.getValue()); - // we return the columnQualifier in parts since we don't know - // which part is of which data type. - results.put(converterColumnKey, value); - } - } - } // for entry - } - return results; - } - - /** - * @param columnPrefixBytes The byte representation for the column prefix. - * Should not contain {@link Separator#QUALIFIERS}. - * @param qualifier for the remainder of the column. Any - * {@link Separator#QUALIFIERS} will be encoded in the qualifier. - * @return fully sanitized column qualifier that is a combination of prefix - * and qualifier. If prefix is null, the result is simply the encoded - * qualifier without any separator. - */ - public static byte[] getColumnQualifier(byte[] columnPrefixBytes, - String qualifier) { - - // We don't want column names to have spaces / tabs. - byte[] encodedQualifier = - Separator.encode(qualifier, Separator.SPACE, Separator.TAB); - if (columnPrefixBytes == null) { - return encodedQualifier; - } - - // Convert qualifier to lower case, strip of separators and tag on column - // prefix. - byte[] columnQualifier = - Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier); - return columnQualifier; - } - - /** - * @param columnPrefixBytes The byte representation for the column prefix. - * Should not contain {@link Separator#QUALIFIERS}. - * @param qualifier for the remainder of the column. - * @return fully sanitized column qualifier that is a combination of prefix - * and qualifier. If prefix is null, the result is simply the encoded - * qualifier without any separator. - */ - public static byte[] getColumnQualifier(byte[] columnPrefixBytes, - long qualifier) { - - if (columnPrefixBytes == null) { - return Bytes.toBytes(qualifier); - } - - // Convert qualifier to lower case, strip of separators and tag on column - // prefix. - byte[] columnQualifier = - Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier)); - return columnQualifier; - } - - public ValueConverter getValueConverter() { - return converter; - } - - /** - * @param columnPrefixBytes The byte representation for the column prefix. - * Should not contain {@link Separator#QUALIFIERS}. - * @param qualifier the byte representation for the remainder of the column. - * @return fully sanitized column qualifier that is a combination of prefix - * and qualifier. If prefix is null, the result is simply the encoded - * qualifier without any separator. - */ - public static byte[] getColumnQualifier(byte[] columnPrefixBytes, - byte[] qualifier) { - - if (columnPrefixBytes == null) { - return qualifier; - } - - byte[] columnQualifier = - Separator.QUALIFIERS.join(columnPrefixBytes, qualifier); - return columnQualifier; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java deleted file mode 100644 index 89aa013..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java +++ /dev/null @@ -1,145 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; -import java.util.Map; -import java.util.NavigableMap; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * Used to represent a partially qualified column, where the actual column name - * will be composed of a prefix and the remainder of the column qualifier. The - * prefix can be null, in which case the column qualifier will be completely - * determined when the values are stored. - */ -public interface ColumnPrefix { - - /** - * Sends a Mutation to the table. The mutations will be buffered and sent over - * the wire as part of a batch. - * - * @param rowKey identifying the row to write. Nothing gets written when null. - * @param tableMutator used to modify the underlying HBase table. Caller is - * responsible to pass a mutator for the table that actually has this - * column. - * @param qualifier column qualifier. Nothing gets written when null. - * @param timestamp version timestamp. When null the server timestamp will be - * used. - * @param attributes attributes for the mutation that are used by the - * coprocessor to set/read the cell tags. - * @param inputValue the value to write to the rowKey and column qualifier. - * Nothing gets written when null. - * @throws IOException if there is any exception encountered while doing - * store operation(sending mutation to the table). - */ - void store(byte[] rowKey, TypedBufferedMutator tableMutator, - byte[] qualifier, Long timestamp, Object inputValue, - Attribute... attributes) throws IOException; - - /** - * Sends a Mutation to the table. The mutations will be buffered and sent over - * the wire as part of a batch. - * - * @param rowKey identifying the row to write. Nothing gets written when null. - * @param tableMutator used to modify the underlying HBase table. Caller is - * responsible to pass a mutator for the table that actually has this - * column. - * @param qualifier column qualifier. Nothing gets written when null. - * @param timestamp version timestamp. When null the server timestamp will be - * used. - * @param attributes attributes for the mutation that are used by the - * coprocessor to set/read the cell tags. - * @param inputValue the value to write to the rowKey and column qualifier. - * Nothing gets written when null. - * @throws IOException if there is any exception encountered while doing - * store operation(sending mutation to the table). - */ - void store(byte[] rowKey, TypedBufferedMutator tableMutator, - String qualifier, Long timestamp, Object inputValue, - Attribute... attributes) throws IOException; - - /** - * Get the latest version of this specified column. Note: this call clones the - * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. - * - * @param result Cannot be null - * @param qualifier column qualifier. Nothing gets read when null. - * @return result object (can be cast to whatever object was written to) or - * null when specified column qualifier for this prefix doesn't exist - * in the result. - * @throws IOException if there is any exception encountered while reading - * result. - */ - Object readResult(Result result, String qualifier) throws IOException; - - /** - * - * @param identifies the type of key converter. - * @param result from which to read columns. - * @param keyConverter used to convert column bytes to the appropriate key - * type - * @return the latest values of columns in the column family with this prefix - * (or all of them if the prefix value is null). - * @throws IOException if there is any exception encountered while reading - * results. - */ - Map readResults(Result result, KeyConverter keyConverter) - throws IOException; - - /** - * @param result from which to reads data with timestamps. - * @param identifies the type of key converter. - * @param the type of the values. The values will be cast into that type. - * @param keyConverter used to convert column bytes to the appropriate key - * type. - * @return the cell values at each respective time in for form - * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, - * idB={timestamp3->value3}, idC={timestamp1->value4}}} - * @throws IOException if there is any exception encountered while reading - * result. - */ - NavigableMap> readResultsWithTimestamps( - Result result, KeyConverter keyConverter) throws IOException; - - /** - * @param qualifierPrefix Column qualifier or prefix of qualifier. - * @return a byte array encoding column prefix and qualifier/prefix passed. - */ - byte[] getColumnPrefixBytes(String qualifierPrefix); - - /** - * @param qualifierPrefix Column qualifier or prefix of qualifier. - * @return a byte array encoding column prefix and qualifier/prefix passed. - */ - byte[] getColumnPrefixBytes(byte[] qualifierPrefix); - - /** - * Returns column family name(as bytes) associated with this column prefix. - * @return a byte array encoding column family for this prefix. - */ - byte[] getColumnFamilyBytes(); - - /** - * Returns value converter implementation associated with this column prefix. - * @return a {@link ValueConverter} implementation. - */ - ValueConverter getValueConverter(); -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java deleted file mode 100644 index 8445575..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -/** - * Encapsulates information about Event column names for application and entity - * tables. Used while encoding/decoding event column names. - */ -public class EventColumnName { - - private final String id; - private final Long timestamp; - private final String infoKey; - private final KeyConverter eventColumnNameConverter = - new EventColumnNameConverter(); - - public EventColumnName(String id, Long timestamp, String infoKey) { - this.id = id; - this.timestamp = timestamp; - this.infoKey = infoKey; - } - - public String getId() { - return id; - } - - public Long getTimestamp() { - return timestamp; - } - - public String getInfoKey() { - return infoKey; - } - - /** - * @return a byte array with each components/fields separated by - * Separator#VALUES. This leads to an event column name of the form - * eventId=timestamp=infokey. If both timestamp and infokey are null, - * then a qualifier of the form eventId=timestamp= is returned. If - * only infokey is null, then a qualifier of the form eventId= is - * returned. These prefix forms are useful for queries that intend to - * retrieve more than one specific column name. - */ - public byte[] getColumnQualifier() { - return eventColumnNameConverter.encode(this); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java deleted file mode 100644 index d3ef897..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java +++ /dev/null @@ -1,99 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Encodes and decodes event column names for application and entity tables. - * The event column name is of the form : eventId=timestamp=infokey. - * If info is not associated with the event, event column name is of the form : - * eventId=timestamp= - * Event timestamp is long and rest are strings. - * Column prefixes are not part of the eventcolumn name passed for encoding. It - * is added later, if required in the associated ColumnPrefix implementations. - */ -public final class EventColumnNameConverter - implements KeyConverter { - - public EventColumnNameConverter() { - } - - // eventId=timestamp=infokey are of types String, Long String - // Strings are variable in size (i.e. end whenever separator is encountered). - // This is used while decoding and helps in determining where to split. - private static final int[] SEGMENT_SIZES = { - Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }; - - /* - * (non-Javadoc) - * - * Encodes EventColumnName into a byte array with each component/field in - * EventColumnName separated by Separator#VALUES. This leads to an event - * column name of the form eventId=timestamp=infokey. - * If timestamp in passed EventColumnName object is null (eventId is not null) - * this returns a column prefix of the form eventId= and if infokey in - * EventColumnName is null (other 2 components are not null), this returns a - * column name of the form eventId=timestamp= - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #encode(java.lang.Object) - */ - @Override - public byte[] encode(EventColumnName key) { - byte[] first = Separator.encode(key.getId(), Separator.SPACE, Separator.TAB, - Separator.VALUES); - if (key.getTimestamp() == null) { - return Separator.VALUES.join(first, Separator.EMPTY_BYTES); - } - byte[] second = Bytes.toBytes( - LongConverter.invertLong(key.getTimestamp())); - if (key.getInfoKey() == null) { - return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES); - } - return Separator.VALUES.join(first, second, Separator.encode( - key.getInfoKey(), Separator.SPACE, Separator.TAB, Separator.VALUES)); - } - - /* - * (non-Javadoc) - * - * Decodes an event column name of the form eventId=timestamp= or - * eventId=timestamp=infoKey represented in byte format and converts it into - * an EventColumnName object. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #decode(byte[]) - */ - @Override - public EventColumnName decode(byte[] bytes) { - byte[][] components = Separator.VALUES.split(bytes, SEGMENT_SIZES); - if (components.length != 3) { - throw new IllegalArgumentException("the column name is not valid"); - } - String id = Separator.decode(Bytes.toString(components[0]), - Separator.VALUES, Separator.TAB, Separator.SPACE); - Long ts = LongConverter.invertLong(Bytes.toLong(components[1])); - String infoKey = components[2].length == 0 ? null : - Separator.decode(Bytes.toString(components[2]), - Separator.VALUES, Separator.TAB, Separator.SPACE); - return new EventColumnName(id, ts, infoKey); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java deleted file mode 100644 index c34bfcb..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; - -import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; - -/** - * Uses GenericObjectMapper to encode objects as bytes and decode bytes as - * objects. - */ -public final class GenericConverter implements ValueConverter { - private static final GenericConverter INSTANCE = new GenericConverter(); - - private GenericConverter() { - } - - public static GenericConverter getInstance() { - return INSTANCE; - } - - @Override - public byte[] encodeValue(Object value) throws IOException { - return GenericObjectMapper.write(value); - } - - @Override - public Object decodeValue(byte[] bytes) throws IOException { - return GenericObjectMapper.read(bytes); - } -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java deleted file mode 100644 index e93b470..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java +++ /dev/null @@ -1,243 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -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.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; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -/** - * A bunch of utility functions used in HBase TimelineService backend. - */ -public final class HBaseTimelineStorageUtils { - /** milliseconds in one day. */ - public static final long MILLIS_ONE_DAY = 86400000L; - private static final Log LOG = - LogFactory.getLog(HBaseTimelineStorageUtils.class); - - private HBaseTimelineStorageUtils() { - } - - - /** - * Combines the input array of attributes and the input aggregation operation - * into a new array of attributes. - * - * @param attributes Attributes to be combined. - * @param aggOp Aggregation operation. - * @return array of combined attributes. - */ - public static Attribute[] combineAttributes(Attribute[] attributes, - AggregationOperation aggOp) { - int newLength = getNewLengthCombinedAttributes(attributes, aggOp); - Attribute[] combinedAttributes = new Attribute[newLength]; - - if (attributes != null) { - System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length); - } - - if (aggOp != null) { - Attribute a2 = aggOp.getAttribute(); - combinedAttributes[newLength - 1] = a2; - } - return combinedAttributes; - } - - /** - * Returns a number for the new array size. The new array is the combination - * of input array of attributes and the input aggregation operation. - * - * @param attributes Attributes. - * @param aggOp Aggregation operation. - * @return the size for the new array - */ - private static int getNewLengthCombinedAttributes(Attribute[] attributes, - AggregationOperation aggOp) { - int oldLength = getAttributesLength(attributes); - int aggLength = getAppOpLength(aggOp); - return oldLength + aggLength; - } - - private static int getAppOpLength(AggregationOperation aggOp) { - if (aggOp != null) { - return 1; - } - return 0; - } - - private static int getAttributesLength(Attribute[] attributes) { - if (attributes != null) { - return attributes.length; - } - return 0; - } - - /** - * Returns the first seen aggregation operation as seen in the list of input - * tags or null otherwise. - * - * @param tags list of HBase tags. - * @return AggregationOperation - */ - public static AggregationOperation getAggregationOperationFromTagsList( - List tags) { - for (AggregationOperation aggOp : AggregationOperation.values()) { - for (Tag tag : tags) { - if (tag.getType() == aggOp.getTagType()) { - return aggOp; - } - } - } - return null; - } - - /** - * Creates a {@link Tag} from the input attribute. - * - * @param attribute Attribute from which tag has to be fetched. - * @return a HBase Tag. - */ - public static Tag getTagFromAttribute(Map.Entry attribute) { - // attribute could be either an Aggregation Operation or - // an Aggregation Dimension - // Get the Tag type from either - AggregationOperation aggOp = AggregationOperation - .getAggregationOperation(attribute.getKey()); - if (aggOp != null) { - Tag t = new Tag(aggOp.getTagType(), attribute.getValue()); - return t; - } - - AggregationCompactionDimension aggCompactDim = - AggregationCompactionDimension.getAggregationCompactionDimension( - attribute.getKey()); - if (aggCompactDim != null) { - Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue()); - return t; - } - return null; - } - - /** - * creates a new cell based on the input cell but with the new value. - * - * @param origCell Original cell - * @param newValue new cell value - * @return cell - * @throws IOException while creating new cell. - */ - public static Cell createNewCell(Cell origCell, byte[] newValue) - throws IOException { - return CellUtil.createCell(CellUtil.cloneRow(origCell), - CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell), - origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue); - } - - /** - * creates a cell with the given inputs. - * - * @param row row of the cell to be created - * @param family column family name of the new cell - * @param qualifier qualifier for the new cell - * @param ts timestamp of the new cell - * @param newValue value of the new cell - * @param tags tags in the new cell - * @return cell - * @throws IOException while creating the cell. - */ - public static Cell createNewCell(byte[] row, byte[] family, byte[] qualifier, - long ts, byte[] newValue, byte[] tags) throws IOException { - return CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put, - newValue, tags); - } - - /** - * returns app id from the list of tags. - * - * @param tags cell tags to be looked into - * @return App Id as the AggregationCompactionDimension - */ - public static String getAggregationCompactionDimension(List tags) { - String appId = null; - for (Tag t : tags) { - if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t - .getType()) { - appId = Bytes.toString(t.getValue()); - 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 - * where we want to have the largest int value in the top of the table - * (scans start at the largest int first). - * - * @param key value to be inverted so that the latest version will be first in - * a scan. - * @return inverted int - */ - public static int invertInt(int key) { - return Integer.MAX_VALUE - key; - } - - /** - * returns the timestamp of that day's start (which is midnight 00:00:00 AM) - * for a given input timestamp. - * - * @param ts Timestamp. - * @return timestamp of that day's beginning (midnight) - */ - public static long getTopOfTheDayTimestamp(long ts) { - long dayTimestamp = ts - (ts % MILLIS_ONE_DAY); - return dayTimestamp; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java deleted file mode 100644 index 4229e81..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -/** - * Interface which has to be implemented for encoding and decoding row keys and - * columns. - */ -public interface KeyConverter { - /** - * Encodes a key as a byte array. - * - * @param key key to be encoded. - * @return a byte array. - */ - byte[] encode(T key); - - /** - * Decodes a byte array and returns a key of type T. - * - * @param bytes byte representation - * @return an object(key) of type T which has been constructed after decoding - * the bytes. - */ - T decode(byte[] bytes); -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java deleted file mode 100644 index 600601a..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java +++ /dev/null @@ -1,94 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; -import java.io.Serializable; - -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Encodes a value by interpreting it as a Long and converting it to bytes and - * decodes a set of bytes as a Long. - */ -public final class LongConverter implements NumericValueConverter, - Serializable { - - /** - * Added because we implement Comparator. - */ - private static final long serialVersionUID = 1L; - - public LongConverter() { - } - - @Override - public byte[] encodeValue(Object value) throws IOException { - if (!TimelineStorageUtils.isIntegralValue(value)) { - throw new IOException("Expected integral value"); - } - return Bytes.toBytes(((Number)value).longValue()); - } - - @Override - public Object decodeValue(byte[] bytes) throws IOException { - if (bytes == null) { - return null; - } - return Bytes.toLong(bytes); - } - - /** - * Compares two numbers as longs. If either number is null, it will be taken - * as 0. - * - * @param num1 the first {@code Long} to compare. - * @param num2 the second {@code Long} to compare. - * @return -1 if num1 is less than num2, 0 if num1 is equal to num2 and 1 if - * num1 is greater than num2. - */ - @Override - public int compare(Number num1, Number num2) { - return Long.compare((num1 == null) ? 0L : num1.longValue(), - (num2 == null) ? 0L : num2.longValue()); - } - - @Override - public Number add(Number num1, Number num2, Number...numbers) { - long sum = ((num1 == null) ? 0L : num1.longValue()) + - ((num2 == null) ? 0L : num2.longValue()); - for (Number num : numbers) { - sum = sum + ((num == null) ? 0L : num.longValue()); - } - return sum; - } - - /** - * Converts a timestamp into it's inverse timestamp to be used in (row) keys - * where we want to have the most recent timestamp in the top of the table - * (scans start at the most recent timestamp first). - * - * @param key value to be inverted so that the latest version will be first in - * a scan. - * @return inverted long - */ - public static long invertLong(long key) { - return Long.MAX_VALUE - key; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java deleted file mode 100644 index 4a724d6..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; -/** - * Encodes and decodes column names / row keys which are long. - */ -public final class LongKeyConverter implements KeyConverter { - - /** - * To delegate the actual work to. - */ - private final LongConverter longConverter = new LongConverter(); - - public LongKeyConverter() { - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #encode(java.lang.Object) - */ - @Override - public byte[] encode(Long key) { - try { - // IOException will not be thrown here as we are explicitly passing - // Long. - return longConverter.encodeValue(key); - } catch (IOException e) { - return null; - } - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #decode(byte[]) - */ - @Override - public Long decode(byte[] bytes) { - try { - return (Long) longConverter.decodeValue(bytes); - } catch (IOException e) { - return null; - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java deleted file mode 100644 index 8fb6536..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.util.Comparator; - -/** - * Extends ValueConverter interface for numeric converters to support numerical - * operations such as comparison, addition, etc. - */ -public interface NumericValueConverter extends ValueConverter, - Comparator { - /** - * Adds two or more numbers. If either of the numbers are null, it is taken as - * 0. - * - * @param num1 the first number to add. - * @param num2 the second number to add. - * @param numbers Rest of the numbers to be added. - * @return result after adding up the numbers. - */ - Number add(Number num1, Number num2, Number...numbers); -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java deleted file mode 100644 index 8a2e01a..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Encapsulates a range with start and end indices. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class Range { - private final int startIdx; - private final int endIdx; - - /** - * Defines a range from start index (inclusive) to end index (exclusive). - * - * @param start - * Starting index position - * @param end - * Ending index position (exclusive) - */ - public Range(int start, int end) { - if (start < 0 || end < start) { - throw new IllegalArgumentException( - "Invalid range, required that: 0 <= start <= end; start=" + start - + ", end=" + end); - } - - this.startIdx = start; - this.endIdx = end; - } - - public int start() { - return startIdx; - } - - public int end() { - return endIdx; - } - - public int length() { - return endIdx - startIdx; - } -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java deleted file mode 100644 index 6159dc7..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java +++ /dev/null @@ -1,42 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -/** - * In queries where a single result is needed, an exact rowkey can be used - * through the corresponding rowkey#getRowKey() method. For queries that need to - * scan over a range of rowkeys, a partial (the initial part) of rowkeys are - * used. Classes implementing RowKeyPrefix indicate that they are the initial - * part of rowkeys, with different constructors with fewer number of argument to - * form a partial rowkey, a prefix. - * - * @param indicating the type of rowkey that a particular implementation is - * a prefix for. - */ -public interface RowKeyPrefix { - - /** - * Create a row key prefix, meaning a partial rowkey that can be used in range - * scans. Which fields are included in the prefix will depend on the - * constructor of the specific instance that was used. Output depends on which - * constructor was used. - * @return a prefix of the following form {@code fist!second!...!last!} - */ - byte[] getRowKeyPrefix(); - -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java deleted file mode 100644 index 5090b4d..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java +++ /dev/null @@ -1,575 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Used to separate row qualifiers, column qualifiers and compound fields. - */ -public enum Separator { - - /** - * separator in key or column qualifier fields. - */ - QUALIFIERS("!", "%0$"), - - /** - * separator in values, and/or compound key/column qualifier fields. - */ - VALUES("=", "%1$"), - - /** - * separator in values, often used to avoid having these in qualifiers and - * names. Note that if we use HTML form encoding through URLEncoder, we end up - * getting a + for a space, which may already occur in strings, so we don't - * want that. - */ - SPACE(" ", "%2$"), - - /** - * separator in values, often used to avoid having these in qualifiers and - * names. - */ - TAB("\t", "%3$"); - - // a reserved character that starts each of the encoded values and is encoded - // first in order to escape naturally occurring instances of encoded values - // although it can be expressed as an enum instance, we define them as private - // variables to hide it from callers - private static final String PERCENT = "%"; - private static final String PERCENT_ENCODED = "%9$"; - - private static final Pattern PERCENT_PATTERN = - Pattern.compile(PERCENT, Pattern.LITERAL); - private static final String PERCENT_REPLACEMENT = - Matcher.quoteReplacement(PERCENT); - - private static final Pattern PERCENT_ENCODED_PATTERN = - Pattern.compile(PERCENT_ENCODED, Pattern.LITERAL); - private static final String PERCENT_ENCODED_REPLACEMENT = - Matcher.quoteReplacement(PERCENT_ENCODED); - - /** - * The string value of this separator. - */ - private final String value; - - /** - * The bye representation of value. - */ - private final byte[] bytes; - - // pre-compiled patterns and quoted replacements for optimization - private final Pattern valuePattern; - private final String valueReplacement; - - private final Pattern encodedValuePattern; - private final String encodedValueReplacement; - - /** - * Indicator for variable size of an individual segment in a split. The - * segment ends wherever separator is encountered. - * Typically used for string. - * Also used to indicate that there is no fixed number of splits which need to - * be returned. If split limit is specified as this, all possible splits are - * returned. - */ - public static final int VARIABLE_SIZE = 0; - - - /** empty string. */ - public static final String EMPTY_STRING = ""; - - /** empty bytes. */ - public static final byte[] EMPTY_BYTES = new byte[0]; - - /** - * @param value of the separator to use. Cannot be null or empty string. - * @param encodedValue choose something that isn't likely to occur in the data - * itself. Cannot be null or empty string. - */ - private Separator(String value, String encodedValue) { - this.value = value; - - // validation - if (value == null || value.length() == 0 || encodedValue == null - || encodedValue.length() == 0) { - throw new IllegalArgumentException( - "Cannot create separator from null or empty string."); - } - - this.bytes = Bytes.toBytes(value); - this.valuePattern = Pattern.compile(value, Pattern.LITERAL); - this.valueReplacement = Matcher.quoteReplacement(value); - - this.encodedValuePattern = Pattern.compile(encodedValue, Pattern.LITERAL); - this.encodedValueReplacement = Matcher.quoteReplacement(encodedValue); - } - - /** - * @return the original value of the separator - */ - public String getValue() { - return value; - } - - /** - * Used to make token safe to be used with this separator without collisions. - * It must be paired with {@link #decode(String)} for it to be - * decoded correctly. - *

- * If you need to encode a given string for multiple separators, - * {@link #encode(String, Separator...)} should be used over successive - * invocations of this method. It will result in a more compact version of the - * encoded value. - * - * @param token Token to be encoded. - * @return the token with any occurrences of this separator URLEncoded. - */ - public String encode(String token) { - if (token == null || token.length() == 0) { - // Nothing to replace - return token; - } - // first encode the percent to escape naturally occurring encoded values - String escaped = encodePercent(token); - return encodeSingle(escaped, this); - } - - private static String replace(String token, Pattern pattern, - String replacement) { - return pattern.matcher(token).replaceAll(replacement); - } - - private static String encodeSingle(String token, Separator separator) { - return replace(token, separator.valuePattern, - separator.encodedValueReplacement); - } - - private static String encodePercent(String token) { - return replace(token, PERCENT_PATTERN, PERCENT_ENCODED_REPLACEMENT); - } - - /** - * Decode the token encoded using {@link #encode(String)}. It must be - * used for the result encoded with {@link #encode(String)} to be able to - * recover the original. - * - * @param token Token to be decoded. - * @return the token with any occurrences of the encoded separator replaced by - * the separator itself. - */ - public String decode(String token) { - if (token == null || token.length() == 0) { - // Nothing to replace - return token; - } - String escaped = decodeSingle(token, this); - // decode percent to de-escape - return decodePercent(escaped); - } - - private static String decodeSingle(String token, Separator separator) { - return replace(token, separator.encodedValuePattern, - separator.valueReplacement); - } - - private static String decodePercent(String token) { - return replace(token, PERCENT_ENCODED_PATTERN, PERCENT_REPLACEMENT); - } - - /** - * Encode the given separators in the token with their encoding equivalents. - * It must be paired with {@link #decode(byte[], Separator...)} or - * {@link #decode(String, Separator...)} with the same separators for it to be - * decoded correctly. - *

- * If you need to encode a given string for multiple separators, this form of - * encoding should be used over successive invocations of - * {@link #encode(String)}. It will result in a more compact version of the - * encoded value. - * - * @param token containing possible separators that need to be encoded. - * @param separators to be encoded in the token with their URLEncoding - * equivalent. - * @return non-null byte representation of the token with occurrences of the - * separators encoded. - */ - public static byte[] encode(String token, Separator... separators) { - if (token == null || token.length() == 0) { - return EMPTY_BYTES; - } - String result = token; - // first encode the percent to escape naturally occurring encoded values - result = encodePercent(token); - for (Separator separator : separators) { - if (separator != null) { - result = encodeSingle(result, separator); - } - } - return Bytes.toBytes(result); - } - - /** - * Decode the given separators in the token with their decoding equivalents. - * It must be used for the result encoded with - * {@link #encode(String, Separator...)} with the same separators to be able - * to recover the original. - * - * @param token containing possible separators that need to be encoded. - * @param separators to be encoded in the token with their URLEncoding - * equivalent. - * @return String representation of the token with occurrences of the URL - * encoded separators decoded. - */ - public static String decode(byte[] token, Separator... separators) { - if (token == null) { - return null; - } - return decode(Bytes.toString(token), separators); - } - - /** - * Decode the given separators in the token with their decoding equivalents. - * It must be used for the result encoded with - * {@link #encode(String, Separator...)} with the same separators to be able - * to recover the original. - * - * @param token containing possible separators that need to be encoded. - * @param separators to be encoded in the token with their URLEncoding - * equivalent. - * @return String representation of the token with occurrences of the URL - * encoded separators decoded. - */ - public static String decode(String token, Separator... separators) { - if (token == null) { - return null; - } - String result = token; - for (Separator separator : separators) { - if (separator != null) { - result = decodeSingle(result, separator); - } - } - // decode percent to de-escape - return decodePercent(result); - } - - /** - * Returns a single byte array containing all of the individual arrays - * components separated by this separator. - * - * @param components Byte array components to be joined together. - * @return byte array after joining the components - */ - public byte[] join(byte[]... components) { - if (components == null || components.length == 0) { - return EMPTY_BYTES; - } - - int finalSize = 0; - finalSize = this.value.length() * (components.length - 1); - for (byte[] comp : components) { - if (comp != null) { - finalSize += comp.length; - } - } - - byte[] buf = new byte[finalSize]; - int offset = 0; - for (int i = 0; i < components.length; i++) { - if (components[i] != null) { - System.arraycopy(components[i], 0, buf, offset, components[i].length); - offset += components[i].length; - } - if (i < (components.length - 1)) { - System.arraycopy(this.bytes, 0, buf, offset, this.value.length()); - offset += this.value.length(); - } - } - return buf; - } - - /** - * Concatenates items (as String), using this separator. - * - * @param items Items join, {@code toString()} will be called in each item. - * Any occurrence of the separator in the individual strings will be - * first encoded. Cannot be null. - * @return non-null joined result. Note that when separator is {@literal null} - * the result is simply all items concatenated and the process is not - * reversible through {@link #splitEncoded(String)} - */ - public String joinEncoded(String... items) { - if (items == null || items.length == 0) { - return ""; - } - - StringBuilder sb = new StringBuilder(encode(items[0].toString())); - // Start at 1, we've already grabbed the first value at index 0 - for (int i = 1; i < items.length; i++) { - sb.append(this.value); - sb.append(encode(items[i].toString())); - } - - return sb.toString(); - } - - /** - * Concatenates items (as String), using this separator. - * - * @param items Items join, {@code toString()} will be called in each item. - * Any occurrence of the separator in the individual strings will be - * first encoded. Cannot be null. - * @return non-null joined result. Note that when separator is {@literal null} - * the result is simply all items concatenated and the process is not - * reversible through {@link #splitEncoded(String)} - */ - public String joinEncoded(Iterable items) { - if (items == null) { - return ""; - } - Iterator i = items.iterator(); - if (!i.hasNext()) { - return ""; - } - - StringBuilder sb = new StringBuilder(encode(i.next().toString())); - while (i.hasNext()) { - sb.append(this.value); - sb.append(encode(i.next().toString())); - } - - return sb.toString(); - } - - /** - * @param compoundValue containing individual values separated by this - * separator, which have that separator encoded. - * @return non-null set of values from the compoundValue with the separator - * decoded. - */ - public Collection splitEncoded(String compoundValue) { - List result = new ArrayList(); - if (compoundValue != null) { - for (String val : valuePattern.split(compoundValue)) { - result.add(decode(val)); - } - } - return result; - } - - /** - * Splits the source array into multiple array segments using this separator, - * up to a maximum of count items. This will naturally produce copied byte - * arrays for each of the split segments. - * - * @param source to be split - * @param limit on how many segments are supposed to be returned. A - * non-positive value indicates no limit on number of segments. - * @return source split by this separator. - */ - public byte[][] split(byte[] source, int limit) { - return split(source, this.bytes, limit); - } - - /** - * Splits the source array into multiple array segments using this separator. - * The sizes indicate the sizes of the relative components/segments. - * In case one of the segments contains this separator before the specified - * size is reached, the separator will be considered part of that segment and - * we will continue till size is reached. - * Variable length strings cannot contain this separator and are indiced with - * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this - * separator and decoded after the results from split is returned. - * - * @param source byte array to be split. - * @param sizes sizes of relative components/segments. - * @return source split by this separator as per the sizes specified.. - */ - public byte[][] split(byte[] source, int[] sizes) { - return split(source, this.bytes, sizes); - } - - /** - * Splits the source array into multiple array segments using this separator, - * as many times as splits are found. This will naturally produce copied byte - * arrays for each of the split segments. - * - * @param source byte array to be split - * @return source split by this separator. - */ - public byte[][] split(byte[] source) { - return split(source, this.bytes); - } - - /** - * Returns a list of ranges identifying [start, end) -- closed, open -- - * positions within the source byte array that would be split using the - * separator byte array. - * The sizes indicate the sizes of the relative components/segments. - * In case one of the segments contains this separator before the specified - * size is reached, the separator will be considered part of that segment and - * we will continue till size is reached. - * Variable length strings cannot contain this separator and are indiced with - * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this - * separator and decoded after the results from split is returned. - * - * @param source the source data - * @param separator the separator pattern to look for - * @param sizes indicate the sizes of the relative components/segments. - * @return a list of ranges. - */ - private static List splitRanges(byte[] source, byte[] separator, - int[] sizes) { - List segments = new ArrayList(); - if (source == null || separator == null) { - return segments; - } - // VARIABLE_SIZE here indicates that there is no limit to number of segments - // to return. - int limit = VARIABLE_SIZE; - if (sizes != null && sizes.length > 0) { - limit = sizes.length; - } - int start = 0; - int currentSegment = 0; - itersource: for (int i = 0; i < source.length; i++) { - for (int j = 0; j < separator.length; j++) { - if (source[i + j] != separator[j]) { - continue itersource; - } - } - // all separator elements matched - if (limit > VARIABLE_SIZE) { - if (segments.size() >= (limit - 1)) { - // everything else goes in one final segment - break; - } - if (sizes != null) { - int currentSegExpectedSize = sizes[currentSegment]; - if (currentSegExpectedSize > VARIABLE_SIZE) { - int currentSegSize = i - start; - if (currentSegSize < currentSegExpectedSize) { - // Segment not yet complete. More bytes to parse. - continue itersource; - } else if (currentSegSize > currentSegExpectedSize) { - // Segment is not as per size. - throw new IllegalArgumentException( - "Segments not separated as per expected sizes"); - } - } - } - } - segments.add(new Range(start, i)); - start = i + separator.length; - // i will be incremented again in outer for loop - i += separator.length - 1; - currentSegment++; - } - // add in remaining to a final range - if (start <= source.length) { - if (sizes != null) { - // Check if final segment is as per size specified. - if (sizes[currentSegment] > VARIABLE_SIZE && - source.length - start > sizes[currentSegment]) { - // Segment is not as per size. - throw new IllegalArgumentException( - "Segments not separated as per expected sizes"); - } - } - segments.add(new Range(start, source.length)); - } - return segments; - } - - /** - * Splits based on segments calculated based on limit/sizes specified for the - * separator. - * - * @param source byte array to be split. - * @param segments specifies the range for each segment. - * @return a byte[][] split as per the segment ranges. - */ - private static byte[][] split(byte[] source, List segments) { - byte[][] splits = new byte[segments.size()][]; - for (int i = 0; i < segments.size(); i++) { - Range r = segments.get(i); - byte[] tmp = new byte[r.length()]; - if (tmp.length > 0) { - System.arraycopy(source, r.start(), tmp, 0, r.length()); - } - splits[i] = tmp; - } - return splits; - } - - /** - * Splits the source array into multiple array segments using the given - * separator based on the sizes. This will naturally produce copied byte - * arrays for each of the split segments. - * - * @param source source array. - * @param separator separator represented as a byte array. - * @param sizes sizes of relative components/segments. - * @return byte[][] after splitting the source. - */ - private static byte[][] split(byte[] source, byte[] separator, int[] sizes) { - List segments = splitRanges(source, separator, sizes); - return split(source, segments); - } - - /** - * Splits the source array into multiple array segments using the given - * separator. This will naturally produce copied byte arrays for each of the - * split segments. - * - * @param source Source array. - * @param separator Separator represented as a byte array. - * @return byte[][] after splitting the source. - */ - private static byte[][] split(byte[] source, byte[] separator) { - return split(source, separator, (int[]) null); - } - - /** - * Splits the source array into multiple array segments using the given - * separator, up to a maximum of count items. This will naturally produce - * copied byte arrays for each of the split segments. - * - * @param source Source array. - * @param separator Separator represented as a byte array. - * @param limit a non-positive value indicates no limit on number of segments. - * @return byte[][] after splitting the input source. - */ - private static byte[][] split(byte[] source, byte[] separator, int limit) { - int[] sizes = null; - if (limit > VARIABLE_SIZE) { - sizes = new int[limit]; - } - List segments = splitRanges(source, separator, sizes); - return split(source, segments); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java deleted file mode 100644 index 282848e..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -/** - * Encodes and decodes column names / row keys which are merely strings. - * Column prefixes are not part of the column name passed for encoding. It is - * added later, if required in the associated ColumnPrefix implementations. - */ -public final class StringKeyConverter implements KeyConverter { - - public StringKeyConverter() { - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #encode(java.lang.Object) - */ - @Override - public byte[] encode(String key) { - return Separator.encode(key, Separator.SPACE, Separator.TAB); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter - * #decode(byte[]) - */ - @Override - public String decode(byte[] bytes) { - return Separator.decode(bytes, Separator.TAB, Separator.SPACE); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java deleted file mode 100644 index 8e6c259..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java +++ /dev/null @@ -1,71 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * contains the constants used in the context of schema accesses for - * {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity} - * information. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public final class TimelineHBaseSchemaConstants { - private TimelineHBaseSchemaConstants() { - } - - /** - * Used to create a pre-split for tables starting with a username in the - * prefix. TODO: this may have to become a config variable (string with - * separators) so that different installations can presplit based on their own - * commonly occurring names. - */ - private final static byte[][] USERNAME_SPLITS = { - Bytes.toBytes("a"), Bytes.toBytes("ad"), Bytes.toBytes("an"), - Bytes.toBytes("b"), Bytes.toBytes("ca"), Bytes.toBytes("cl"), - Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), - Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), - Bytes.toBytes("j"), Bytes.toBytes("k"), Bytes.toBytes("l"), - Bytes.toBytes("m"), Bytes.toBytes("n"), Bytes.toBytes("o"), - Bytes.toBytes("q"), Bytes.toBytes("r"), Bytes.toBytes("s"), - Bytes.toBytes("se"), Bytes.toBytes("t"), Bytes.toBytes("u"), - Bytes.toBytes("v"), Bytes.toBytes("w"), Bytes.toBytes("x"), - Bytes.toBytes("y"), Bytes.toBytes("z") - }; - - /** - * The length at which keys auto-split. - */ - public static final String USERNAME_SPLIT_KEY_PREFIX_LENGTH = "4"; - - /** - * @return splits for splits where a user is a prefix. - */ - public static byte[][] getUsernameSplits() { - byte[][] kloon = USERNAME_SPLITS.clone(); - // Deep copy. - for (int row = 0; row < USERNAME_SPLITS.length; row++) { - kloon[row] = Bytes.copy(USERNAME_SPLITS[row]); - } - return kloon; - } - -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java deleted file mode 100644 index d03b37d..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java +++ /dev/null @@ -1,116 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * Utility class that allows HBase coprocessors to interact with unique - * timestamps. - */ -public class TimestampGenerator { - - /* - * if this is changed, then reading cell timestamps written with older - * multiplier value will not work - */ - public static final long TS_MULTIPLIER = 1000000L; - - private final AtomicLong lastTimestamp = new AtomicLong(); - - /** - * Returns the current wall clock time in milliseconds, multiplied by the - * required precision. - * - * @return current timestamp. - */ - public long currentTime() { - // We want to align cell timestamps with current time. - // cell timestamps are not be less than - // System.currentTimeMillis() * TS_MULTIPLIER. - return System.currentTimeMillis() * TS_MULTIPLIER; - } - - /** - * Returns a timestamp value unique within the scope of this - * {@code TimestampGenerator} instance. For usage by HBase - * {@code RegionObserver} coprocessors, this normally means unique within a - * given region. - * - * Unlikely scenario of generating a non-unique timestamp: if there is a - * sustained rate of more than 1M hbase writes per second AND if region fails - * over within that time range of timestamps being generated then there may be - * collisions writing to a cell version of the same column. - * - * @return unique timestamp. - */ - public long getUniqueTimestamp() { - long lastTs; - long nextTs; - do { - lastTs = lastTimestamp.get(); - nextTs = Math.max(lastTs + 1, currentTime()); - } while (!lastTimestamp.compareAndSet(lastTs, nextTs)); - return nextTs; - } - - /** - * Returns a timestamp multiplied with TS_MULTIPLIER and last few digits of - * application id. - * - * Unlikely scenario of generating a timestamp that is a duplicate: If more - * than a 1M concurrent apps are running in one flow run AND write to same - * column at the same time, then say appId of 1M and 1 will overlap - * with appId of 001 and there may be collisions for that flow run's - * specific column. - * - * @param incomingTS Timestamp to be converted. - * @param appId Application Id. - * @return a timestamp multiplied with TS_MULTIPLIER and last few digits of - * application id - */ - public static long getSupplementedTimestamp(long incomingTS, String appId) { - long suffix = getAppIdSuffix(appId); - long outgoingTS = incomingTS * TS_MULTIPLIER + suffix; - return outgoingTS; - - } - - private static long getAppIdSuffix(String appIdStr) { - if (appIdStr == null) { - return 0L; - } - ApplicationId appId = ApplicationId.fromString(appIdStr); - long id = appId.getId() % TS_MULTIPLIER; - return id; - } - - /** - * truncates the last few digits of the timestamp which were supplemented by - * the TimestampGenerator#getSupplementedTimestamp function. - * - * @param incomingTS Timestamp to be truncated. - * @return a truncated timestamp value - */ - public static long getTruncatedTimestamp(long incomingTS) { - return incomingTS / TS_MULTIPLIER; - } -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java deleted file mode 100644 index 64a11f8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import org.apache.hadoop.hbase.client.BufferedMutator; - -/** - * Just a typed wrapper around {@link BufferedMutator} used to ensure that - * columns can write only to the table mutator for the right table. - */ -public interface TypedBufferedMutator extends BufferedMutator { - // This class is intentionally left (almost) blank -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java deleted file mode 100644 index 757a6d3..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import java.io.IOException; - -/** - * Converter used to encode/decode value associated with a column prefix or a - * column. - */ -public interface ValueConverter { - - /** - * Encode an object as a byte array depending on the converter implementation. - * - * @param value Value to be encoded. - * @return a byte array - * @throws IOException if any problem is encountered while encoding. - */ - byte[] encodeValue(Object value) throws IOException; - - /** - * Decode a byte array and convert it into an object depending on the - * converter implementation. - * - * @param bytes Byte array to be decoded. - * @return an object - * @throws IOException if any problem is encountered while decoding. - */ - Object decodeValue(byte[] bytes) throws IOException; -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java deleted file mode 100644 index 93b4b36..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java +++ /dev/null @@ -1,160 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import java.io.IOException; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * Identifies fully qualified columns for the {@link EntityTable}. - */ -public enum EntityColumn implements Column { - - /** - * Identifier for the entity. - */ - ID(EntityColumnFamily.INFO, "id"), - - /** - * The type of entity. - */ - TYPE(EntityColumnFamily.INFO, "type"), - - /** - * When the entity was created. - */ - CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()), - - /** - * The version of the flow that this entity belongs to. - */ - FLOW_VERSION(EntityColumnFamily.INFO, "flow_version"); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - private final String columnQualifier; - private final byte[] columnQualifierBytes; - - EntityColumn(ColumnFamily columnFamily, - String columnQualifier) { - this(columnFamily, columnQualifier, GenericConverter.getInstance()); - } - - EntityColumn(ColumnFamily columnFamily, - String columnQualifier, ValueConverter converter) { - this.columnFamily = columnFamily; - this.columnQualifier = columnQualifier; - // Future-proof by ensuring the right column prefix hygiene. - this.columnQualifierBytes = - Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); - this.column = new ColumnHelper(columnFamily, converter); - } - - /** - * @return the column name value - */ - private String getColumnQualifier() { - return columnQualifier; - } - - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, Long timestamp, - Object inputValue, Attribute... attributes) throws IOException { - column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, - inputValue, attributes); - } - - public Object readResult(Result result) throws IOException { - return column.readResult(result, columnQualifierBytes); - } - - /** - * Retrieve an {@link EntityColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnQualifier Name of the column to retrieve - * @return the corresponding {@link EntityColumn} or null - */ - public static final EntityColumn columnFor(String columnQualifier) { - - // Match column based on value, assume column family matches. - for (EntityColumn ec : EntityColumn.values()) { - // Find a match based only on name. - if (ec.getColumnQualifier().equals(columnQualifier)) { - return ec; - } - } - - // Default to null - return null; - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /** - * Retrieve an {@link EntityColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} - * if and only if {@code a.equals(b) & x.equals(y)} or - * {@code (x == y == null)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param name Name of the column to retrieve - * @return the corresponding {@link EntityColumn} or null if both arguments - * don't match. - */ - public static final EntityColumn columnFor(EntityColumnFamily columnFamily, - String name) { - - for (EntityColumn ec : EntityColumn.values()) { - // Find a match based column family and on name. - if (ec.columnFamily.equals(columnFamily) - && ec.getColumnQualifier().equals(name)) { - return ec; - } - } - - // Default to null - return null; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java deleted file mode 100644 index 7c63727..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents the entity table column families. - */ -public enum EntityColumnFamily implements ColumnFamily { - - /** - * Info column family houses known columns, specifically ones included in - * columnfamily filters. - */ - INFO("i"), - - /** - * Configurations are in a separate column family for two reasons: a) the size - * of the config values can be very large and b) we expect that config values - * are often separately accessed from other metrics and info columns. - */ - CONFIGS("c"), - - /** - * Metrics have a separate column family, because they have a separate TTL. - */ - METRICS("m"); - - /** - * Byte representation of this column family. - */ - private final byte[] bytes; - - /** - * @param value create a column family with this name. Must be lower case and - * without spaces. - */ - EntityColumnFamily(String value) { - // column families should be lower case and not contain any spaces. - this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); - } - - public byte[] getBytes() { - return Bytes.copy(bytes); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java deleted file mode 100644 index e410549..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java +++ /dev/null @@ -1,300 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import java.io.IOException; -import java.util.Map; -import java.util.NavigableMap; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; - -/** - * Identifies partially qualified columns for the entity table. - */ -public enum EntityColumnPrefix implements ColumnPrefix { - - /** - * To store TimelineEntity getIsRelatedToEntities values. - */ - IS_RELATED_TO(EntityColumnFamily.INFO, "s"), - - /** - * To store TimelineEntity getRelatesToEntities values. - */ - RELATES_TO(EntityColumnFamily.INFO, "r"), - - /** - * To store TimelineEntity info values. - */ - INFO(EntityColumnFamily.INFO, "i"), - - /** - * Lifecycle events for an entity. - */ - EVENT(EntityColumnFamily.INFO, "e", true), - - /** - * Config column stores configuration with config key as the column name. - */ - CONFIG(EntityColumnFamily.CONFIGS, null), - - /** - * Metrics are stored with the metric name as the column name. - */ - METRIC(EntityColumnFamily.METRICS, null, new LongConverter()); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - - /** - * Can be null for those cases where the provided column qualifier is the - * entire column name. - */ - private final String columnPrefix; - private final byte[] columnPrefixBytes; - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily that this column is stored in. - * @param columnPrefix for this column. - */ - EntityColumnPrefix(ColumnFamily columnFamily, - String columnPrefix) { - this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); - } - - EntityColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, boolean compondColQual) { - this(columnFamily, columnPrefix, compondColQual, - GenericConverter.getInstance()); - } - - EntityColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, ValueConverter converter) { - this(columnFamily, columnPrefix, false, converter); - } - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily that this column is stored in. - * @param columnPrefix for this column. - * @param converter used to encode/decode values to be stored in HBase for - * this column prefix. - */ - EntityColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, boolean compondColQual, ValueConverter converter) { - column = new ColumnHelper(columnFamily, converter); - this.columnFamily = columnFamily; - this.columnPrefix = columnPrefix; - if (columnPrefix == null) { - this.columnPrefixBytes = null; - } else { - // Future-proof by ensuring the right column prefix hygiene. - this.columnPrefixBytes = - Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); - } - } - - /** - * @return the column name value - */ - public String getColumnPrefix() { - return columnPrefix; - } - - @Override - public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - @Override - public byte[] getColumnPrefixBytes(String qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, - * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, String qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - attributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, byte[] qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - attributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) - */ - public Object readResult(Result result, String qualifier) throws IOException { - byte[] columnQualifier = - ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); - return column.readResult(result, columnQualifier); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResults(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public Map readResults(Result result, - KeyConverter keyConverter) throws IOException { - return column.readResults(result, columnPrefixBytes, keyConverter); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public NavigableMap> - readResultsWithTimestamps(Result result, KeyConverter keyConverter) - throws IOException { - return column.readResultsWithTimestamps(result, columnPrefixBytes, - keyConverter); - } - - /** - * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no - * match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link EntityColumnPrefix} or null - */ - public static final EntityColumnPrefix columnFor(String columnPrefix) { - - // Match column based on value, assume column family matches. - for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) { - // Find a match based only on name. - if (ecp.getColumnPrefix().equals(columnPrefix)) { - return ecp; - } - } - - // Default to null - return null; - } - - /** - * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no - * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} - * if and only if {@code (x == y == null)} or - * {@code a.equals(b) & x.equals(y)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link EntityColumnPrefix} or null if both - * arguments don't match. - */ - public static final EntityColumnPrefix columnFor( - EntityColumnFamily columnFamily, String columnPrefix) { - - // TODO: needs unit test to confirm and need to update javadoc to explain - // null prefix case. - - for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) { - // Find a match based column family and on name. - if (ecp.columnFamily.equals(columnFamily) - && (((columnPrefix == null) && (ecp.getColumnPrefix() == null)) || - (ecp.getColumnPrefix().equals(columnPrefix)))) { - return ecp; - } - } - - // Default to null - return null; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java deleted file mode 100644 index ff22178..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java +++ /dev/null @@ -1,225 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents a rowkey for the entity table. - */ -public class EntityRowKey { - private final String clusterId; - private final String userId; - private final String flowName; - private final Long flowRunId; - private final String appId; - private final String entityType; - private final String entityId; - private final KeyConverter entityRowKeyConverter = - new EntityRowKeyConverter(); - - public EntityRowKey(String clusterId, String userId, String flowName, - Long flowRunId, String appId, String entityType, String entityId) { - this.clusterId = clusterId; - this.userId = userId; - this.flowName = flowName; - this.flowRunId = flowRunId; - this.appId = appId; - this.entityType = entityType; - this.entityId = entityId; - } - - public String getClusterId() { - return clusterId; - } - - public String getUserId() { - return userId; - } - - public String getFlowName() { - return flowName; - } - - public Long getFlowRunId() { - return flowRunId; - } - - public String getAppId() { - return appId; - } - - public String getEntityType() { - return entityType; - } - - public String getEntityId() { - return entityId; - } - - /** - * Constructs a row key for the entity table as follows: - * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. - * Typically used while querying a specific entity. - * - * @return byte array with the row key. - */ - public byte[] getRowKey() { - return entityRowKeyConverter.encode(this); - } - - /** - * Given the raw row key as bytes, returns the row key as an object. - * - * @param rowKey byte representation of row key. - * @return An EntityRowKey object. - */ - public static EntityRowKey parseRowKey(byte[] rowKey) { - return new EntityRowKeyConverter().decode(rowKey); - } - - /** - * Encodes and decodes row key for entity table. The row key is of the form : - * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId - * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and - * rest are strings. - *

- */ - final private static class EntityRowKeyConverter implements - KeyConverter { - - private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter(); - - private EntityRowKeyConverter() { - } - - /** - * Entity row key is of the form - * userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each - * segment separated by !. The sizes below indicate sizes of each one of - * these segments in sequence. clusterId, userName, flowName, entityType and - * entityId are strings. flowrunId is a long hence 8 bytes in size. app id - * is represented as 12 bytes with cluster timestamp part of appid being 8 - * bytes (long) and seq id being 4 bytes(int). Strings are variable in size - * (i.e. end whenever separator is encountered). This is used while decoding - * and helps in determining where to split. - */ - private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE }; - - /* - * (non-Javadoc) - * - * Encodes EntityRowKey object into a byte array with each component/field - * in EntityRowKey separated by Separator#QUALIFIERS. This leads to an - * entity table row key of the form - * userName!clusterId!flowName!flowRunId!appId!entityType!entityId If - * entityType in passed EntityRowKey object is null (and the fields - * preceding it i.e. clusterId, userId and flowName, flowRunId and appId - * are not null), this returns a row key prefix of the form - * userName!clusterId!flowName!flowRunId!appId! and if entityId in - * EntityRowKey is null (other 6 components are not null), this returns a - * row key prefix of the form - * userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is - * inverted while encoding as it helps maintain a descending order for row - * keys in entity table. - * - * @see org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#encode(java.lang.Object) - */ - @Override - public byte[] encode(EntityRowKey rowKey) { - byte[] user = - Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS); - byte[] cluster = - Separator.encode(rowKey.getClusterId(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] flow = - Separator.encode(rowKey.getFlowName(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] first = Separator.QUALIFIERS.join(user, cluster, flow); - // Note that flowRunId is a long, so we can't encode them all at the same - // time. - byte[] second = - Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); - byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); - if (rowKey.getEntityType() == null) { - return Separator.QUALIFIERS.join(first, second, third, - Separator.EMPTY_BYTES); - } - byte[] entityType = - Separator.encode(rowKey.getEntityType(), Separator.SPACE, - Separator.TAB, Separator.QUALIFIERS); - byte[] entityId = - rowKey.getEntityId() == null ? Separator.EMPTY_BYTES : Separator - .encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS); - byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId); - return Separator.QUALIFIERS.join(first, second, third, fourth); - } - - /* - * (non-Javadoc) - * - * Decodes an application row key of the form - * userName!clusterId!flowName!flowRunId!appId!entityType!entityId - * represented in byte format and converts it into an EntityRowKey object. - * flowRunId is inverted while decoding as it was inverted while encoding. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#decode(byte[]) - */ - @Override - public EntityRowKey decode(byte[] rowKey) { - byte[][] rowKeyComponents = - Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); - if (rowKeyComponents.length != 7) { - throw new IllegalArgumentException("the row key is not valid for " - + "an entity"); - } - String userId = - Separator.decode(Bytes.toString(rowKeyComponents[0]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String clusterId = - Separator.decode(Bytes.toString(rowKeyComponents[1]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String flowName = - Separator.decode(Bytes.toString(rowKeyComponents[2]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - Long flowRunId = - LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); - String appId = appIDKeyConverter.decode(rowKeyComponents[4]); - String entityType = - Separator.decode(Bytes.toString(rowKeyComponents[5]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String entityId = - Separator.decode(Bytes.toString(rowKeyComponents[6]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, - entityType, entityId); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java deleted file mode 100644 index 9146180..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; - -/** - * Represents a partial rowkey without the entityId or without entityType and - * entityId for the entity table. - * - */ -public class EntityRowKeyPrefix extends EntityRowKey implements - RowKeyPrefix { - - /** - * Creates a prefix which generates the following rowKeyPrefixes for the - * entity table: - * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}. - * - * @param clusterId identifying the cluster - * @param userId identifying the user - * @param flowName identifying the flow - * @param flowRunId identifying the individual run of this flow - * @param appId identifying the application - * @param entityType which entity type - */ - public EntityRowKeyPrefix(String clusterId, String userId, String flowName, - Long flowRunId, String appId, String entityType) { - super(clusterId, userId, flowName, flowRunId, appId, entityType, null); - } - - /** - * Creates a prefix which generates the following rowKeyPrefixes for the - * entity table: - * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. - * - * @param clusterId identifying the cluster - * @param userId identifying the user - * @param flowName identifying the flow - * @param flowRunId identifying the individual run of this flow - * @param appId identifying the application - */ - public EntityRowKeyPrefix(String clusterId, String userId, String flowName, - Long flowRunId, String appId) { - super(clusterId, userId, flowName, flowRunId, appId, null, null); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.application. - * RowKeyPrefix#getRowKeyPrefix() - */ - public byte[] getRowKeyPrefix() { - return super.getRowKey(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java deleted file mode 100644 index b194f07..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java +++ /dev/null @@ -1,161 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; - -/** - * The entity table as column families info, config and metrics. Info stores - * information about a timeline entity object config stores configuration data - * of a timeline entity object metrics stores the metrics of a timeline entity - * object - * - * Example entity table record: - * - *

- * |-------------------------------------------------------------------------|
- * |  Row       | Column Family                | Column Family| Column Family|
- * |  key       | info                         | metrics      | config       |
- * |-------------------------------------------------------------------------|
- * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
- * | clusterId! |                              | metricValue1 | configValue1 |
- * | flowName!  | type:entityType              | @timestamp1  |              |
- * | flowRunId! |                              |              | configKey2:  |
- * | AppId!     | created_time:                | metricId1:   | configValue2 |
- * | entityType!| 1392993084018                | metricValue2 |              |
- * | entityId   |                              | @timestamp2  |              |
- * |            | i!infoKey:                   |              |              |
- * |            | infoValue                    | metricId1:   |              |
- * |            |                              | metricValue1 |              |
- * |            | r!relatesToKey:              | @timestamp2  |              |
- * |            | id3=id4=id5                  |              |              |
- * |            |                              |              |              |
- * |            | s!isRelatedToKey             |              |              |
- * |            | id7=id9=id6                  |              |              |
- * |            |                              |              |              |
- * |            | e!eventId=timestamp=infoKey: |              |              |
- * |            | eventInfoValue               |              |              |
- * |            |                              |              |              |
- * |            | flowVersion:                 |              |              |
- * |            | versionValue                 |              |              |
- * |-------------------------------------------------------------------------|
- * 
- */ -public class EntityTable extends BaseTable { - /** entity prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity"; - - /** config param name that specifies the entity table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** - * config param name that specifies the TTL for metrics column family in - * entity table. - */ - private static final String METRICS_TTL_CONF_NAME = PREFIX - + ".table.metrics.ttl"; - - /** default value for entity table name. */ - public static final String DEFAULT_TABLE_NAME = "timelineservice.entity"; - - /** default TTL is 30 days for metrics timeseries. */ - private static final int DEFAULT_METRICS_TTL = 2592000; - - /** default max number of versions. */ - private static final int DEFAULT_METRICS_MAX_VERSIONS = 1000; - - private static final Log LOG = LogFactory.getLog(EntityTable.class); - - public EntityTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor entityTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(EntityColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - entityTableDescp.addFamily(infoCF); - - HColumnDescriptor configCF = - new HColumnDescriptor(EntityColumnFamily.CONFIGS.getBytes()); - configCF.setBloomFilterType(BloomType.ROWCOL); - configCF.setBlockCacheEnabled(true); - entityTableDescp.addFamily(configCF); - - HColumnDescriptor metricsCF = - new HColumnDescriptor(EntityColumnFamily.METRICS.getBytes()); - entityTableDescp.addFamily(metricsCF); - metricsCF.setBlockCacheEnabled(true); - // always keep 1 version (the latest) - metricsCF.setMinVersions(1); - metricsCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, - DEFAULT_METRICS_TTL)); - entityTableDescp.setRegionSplitPolicyClassName( - "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); - entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", - TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); - admin.createTable(entityTableDescp, - TimelineHBaseSchemaConstants.getUsernameSplits()); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } - - /** - * @param metricsTTL time to live parameter for the metricss in this table. - * @param hbaseConf configururation in which to set the metrics TTL config - * variable. - */ - public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { - hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java deleted file mode 100644 index bb0e331..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Package org.apache.hadoop.yarn.server.timelineservice.storage.entity - * contains classes related to implementation for entity table. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -package org.apache.hadoop.yarn.server.timelineservice.storage.entity; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java deleted file mode 100644 index 4e2cf2d..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Identifies the compaction dimensions for the data in the {@link FlowRunTable} - * . - */ -public enum AggregationCompactionDimension { - - /** - * the application id. - */ - APPLICATION_ID((byte) 101); - - private byte tagType; - private byte[] inBytes; - - private AggregationCompactionDimension(byte tagType) { - this.tagType = tagType; - this.inBytes = Bytes.toBytes(this.name()); - } - - public Attribute getAttribute(String attributeValue) { - return new Attribute(this.name(), Bytes.toBytes(attributeValue)); - } - - public byte getTagType() { - return tagType; - } - - public byte[] getInBytes() { - return this.inBytes.clone(); - } - - public static AggregationCompactionDimension - getAggregationCompactionDimension(String aggCompactDimStr) { - for (AggregationCompactionDimension aggDim : AggregationCompactionDimension - .values()) { - if (aggDim.name().equals(aggCompactDimStr)) { - return aggDim; - } - } - return null; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java deleted file mode 100644 index 40cdd2c..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java +++ /dev/null @@ -1,94 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Identifies the attributes to be set for puts into the {@link FlowRunTable}. - * The numbers used for tagType are prime numbers. - */ -public enum AggregationOperation { - - /** - * When the flow was started. - */ - GLOBAL_MIN((byte) 71), - - /** - * When it ended. - */ - GLOBAL_MAX((byte) 73), - - /** - * The metrics of the flow. - */ - SUM((byte) 79), - - /** - * application running. - */ - SUM_FINAL((byte) 83), - - /** - * Min value as per the latest timestamp - * seen for a given app. - */ - LATEST_MIN((byte) 89), - - /** - * Max value as per the latest timestamp - * seen for a given app. - */ - LATEST_MAX((byte) 97); - - private byte tagType; - private byte[] inBytes; - - private AggregationOperation(byte tagType) { - this.tagType = tagType; - this.inBytes = Bytes.toBytes(this.name()); - } - - public Attribute getAttribute() { - return new Attribute(this.name(), this.inBytes); - } - - public byte getTagType() { - return tagType; - } - - public byte[] getInBytes() { - return this.inBytes.clone(); - } - - /** - * returns the AggregationOperation enum that represents that string. - * @param aggOpStr Aggregation operation. - * @return the AggregationOperation enum that represents that string - */ - public static AggregationOperation getAggregationOperation(String aggOpStr) { - for (AggregationOperation aggOp : AggregationOperation.values()) { - if (aggOp.name().equals(aggOpStr)) { - return aggOp; - } - } - return null; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java deleted file mode 100644 index d3de518..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -/** - * Defines the attribute tuple to be set for puts into the {@link FlowRunTable}. - */ -public class Attribute { - private final String name; - private final byte[] value; - - public Attribute(String name, byte[] value) { - this.name = name; - this.value = value.clone(); - } - - public String getName() { - return name; - } - - public byte[] getValue() { - return value.clone(); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java deleted file mode 100644 index f9eb5b4..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents the flow run table column families. - */ -public enum FlowActivityColumnFamily - implements ColumnFamily { - - /** - * Info column family houses known columns, specifically ones included in - * columnfamily filters. - */ - INFO("i"); - - /** - * Byte representation of this column family. - */ - private final byte[] bytes; - - /** - * @param value - * create a column family with this name. Must be lower case and - * without spaces. - */ - private FlowActivityColumnFamily(String value) { - // column families should be lower case and not contain any spaces. - this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); - } - - public byte[] getBytes() { - return Bytes.copy(bytes); - } - -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java deleted file mode 100644 index 439e0c8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java +++ /dev/null @@ -1,277 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; -import java.util.Map; -import java.util.NavigableMap; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -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.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; - -/** - * Identifies partially qualified columns for the {@link FlowActivityTable}. - */ -public enum FlowActivityColumnPrefix - implements ColumnPrefix { - - /** - * To store run ids of the flows. - */ - RUN_ID(FlowActivityColumnFamily.INFO, "r", null); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - - /** - * Can be null for those cases where the provided column qualifier is the - * entire column name. - */ - private final String columnPrefix; - private final byte[] columnPrefixBytes; - - private final AggregationOperation aggOp; - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily - * that this column is stored in. - * @param columnPrefix - * for this column. - */ - private FlowActivityColumnPrefix( - ColumnFamily columnFamily, String columnPrefix, - AggregationOperation aggOp) { - this(columnFamily, columnPrefix, aggOp, false); - } - - private FlowActivityColumnPrefix( - ColumnFamily columnFamily, String columnPrefix, - AggregationOperation aggOp, boolean compoundColQual) { - column = new ColumnHelper(columnFamily); - this.columnFamily = columnFamily; - this.columnPrefix = columnPrefix; - if (columnPrefix == null) { - this.columnPrefixBytes = null; - } else { - // Future-proof by ensuring the right column prefix hygiene. - this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE - .encode(columnPrefix)); - } - this.aggOp = aggOp; - } - - /** - * @return the column name value - */ - public String getColumnPrefix() { - return columnPrefix; - } - - @Override - public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - @Override - public byte[] getColumnPrefixBytes(String qualifierPrefix) { - return ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifierPrefix); - } - - public byte[] getColumnPrefixBytes() { - return columnPrefixBytes.clone(); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - public AggregationOperation getAttribute() { - return aggOp; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, byte[], java.lang.Long, java.lang.Object, - * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) - */ - @Override - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, byte[] qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - Attribute[] combinedAttributes = - HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - combinedAttributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) - */ - public Object readResult(Result result, String qualifier) throws IOException { - byte[] columnQualifier = ColumnHelper.getColumnQualifier( - this.columnPrefixBytes, qualifier); - return column.readResult(result, columnQualifier); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResults(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public Map readResults(Result result, - KeyConverter keyConverter) throws IOException { - return column.readResults(result, columnPrefixBytes, keyConverter); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public NavigableMap> - readResultsWithTimestamps(Result result, KeyConverter keyConverter) - throws IOException { - return column.readResultsWithTimestamps(result, columnPrefixBytes, - keyConverter); - } - - /** - * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there - * is no match. The following holds true: {@code columnFor(x) == columnFor(y)} - * if and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnPrefix - * Name of the column to retrieve - * @return the corresponding {@link FlowActivityColumnPrefix} or null - */ - public static final FlowActivityColumnPrefix columnFor(String columnPrefix) { - - // Match column based on value, assume column family matches. - for (FlowActivityColumnPrefix flowActivityColPrefix : - FlowActivityColumnPrefix.values()) { - // Find a match based only on name. - if (flowActivityColPrefix.getColumnPrefix().equals(columnPrefix)) { - return flowActivityColPrefix; - } - } - // Default to null - return null; - } - - /** - * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there - * is no match. The following holds true: - * {@code columnFor(a,x) == columnFor(b,y)} if and only if - * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} - * - * @param columnFamily - * The columnFamily for which to retrieve the column. - * @param columnPrefix - * Name of the column to retrieve - * @return the corresponding {@link FlowActivityColumnPrefix} or null if both - * arguments don't match. - */ - public static final FlowActivityColumnPrefix columnFor( - FlowActivityColumnFamily columnFamily, String columnPrefix) { - - // TODO: needs unit test to confirm and need to update javadoc to explain - // null prefix case. - - for (FlowActivityColumnPrefix flowActivityColumnPrefix : - FlowActivityColumnPrefix.values()) { - // Find a match based column family and on name. - if (flowActivityColumnPrefix.columnFamily.equals(columnFamily) - && (((columnPrefix == null) && (flowActivityColumnPrefix - .getColumnPrefix() == null)) || (flowActivityColumnPrefix - .getColumnPrefix().equals(columnPrefix)))) { - return flowActivityColumnPrefix; - } - } - // Default to null - return null; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, - * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) - */ - @Override - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, String qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - Attribute[] combinedAttributes = - HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); - column.store(rowKey, tableMutator, columnQualifier, null, inputValue, - combinedAttributes); - } -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java deleted file mode 100644 index bb77e36..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java +++ /dev/null @@ -1,196 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; -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.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents a rowkey for the flow activity table. - */ -public class FlowActivityRowKey { - - private final String clusterId; - private final Long dayTs; - private final String userId; - private final String flowName; - private final KeyConverter flowActivityRowKeyConverter = - new FlowActivityRowKeyConverter(); - - /** - * @param clusterId identifying the cluster - * @param dayTs to be converted to the top of the day timestamp - * @param userId identifying user - * @param flowName identifying the flow - */ - public FlowActivityRowKey(String clusterId, Long dayTs, String userId, - String flowName) { - this(clusterId, dayTs, userId, flowName, true); - } - - /** - * @param clusterId identifying the cluster - * @param timestamp when the flow activity happened. May be converted to the - * top of the day depending on the convertDayTsToTopOfDay argument. - * @param userId identifying user - * @param flowName identifying the flow - * @param convertDayTsToTopOfDay if true and timestamp isn't null, then - * timestamp will be converted to the top-of-the day timestamp - */ - protected FlowActivityRowKey(String clusterId, Long timestamp, String userId, - String flowName, boolean convertDayTsToTopOfDay) { - this.clusterId = clusterId; - if (convertDayTsToTopOfDay && (timestamp != null)) { - this.dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(timestamp); - } else { - this.dayTs = timestamp; - } - this.userId = userId; - this.flowName = flowName; - } - - public String getClusterId() { - return clusterId; - } - - public Long getDayTimestamp() { - return dayTs; - } - - public String getUserId() { - return userId; - } - - public String getFlowName() { - return flowName; - } - - /** - * Constructs a row key for the flow activity table as follows: - * {@code clusterId!dayTimestamp!user!flowName}. - * - * @return byte array for the row key - */ - public byte[] getRowKey() { - return flowActivityRowKeyConverter.encode(this); - } - - /** - * Given the raw row key as bytes, returns the row key as an object. - * - * @param rowKey Byte representation of row key. - * @return A FlowActivityRowKey object. - */ - public static FlowActivityRowKey parseRowKey(byte[] rowKey) { - return new FlowActivityRowKeyConverter().decode(rowKey); - } - - /** - * Encodes and decodes row key for flow activity table. The row key is of the - * form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day - * timestamp) is a long and rest are strings. - *

- */ - final private static class FlowActivityRowKeyConverter implements - KeyConverter { - - private FlowActivityRowKeyConverter() { - } - - /** - * The flow activity row key is of the form - * clusterId!dayTimestamp!user!flowName with each segment separated by !. - * The sizes below indicate sizes of each one of these segements in - * sequence. clusterId, user and flowName are strings. Top of the day - * timestamp is a long hence 8 bytes in size. Strings are variable in size - * (i.e. they end whenever separator is encountered). This is used while - * decoding and helps in determining where to split. - */ - private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, - Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }; - - /* - * (non-Javadoc) - * - * Encodes FlowActivityRowKey object into a byte array with each - * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS. - * This leads to an flow activity table row key of the form - * clusterId!dayTimestamp!user!flowName. If dayTimestamp in passed - * FlowActivityRowKey object is null and clusterId is not null, then this - * returns a row key prefix as clusterId! and if userId in - * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId - * and dayTimestamp are not null), this returns a row key prefix as - * clusterId!dayTimeStamp! dayTimestamp is inverted while encoding as it - * helps maintain a descending order for row keys in flow activity table. - * - * @see org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#encode(java.lang.Object) - */ - @Override - public byte[] encode(FlowActivityRowKey rowKey) { - if (rowKey.getDayTimestamp() == null) { - return Separator.QUALIFIERS.join(Separator.encode( - rowKey.getClusterId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS), Separator.EMPTY_BYTES); - } - if (rowKey.getUserId() == null) { - return Separator.QUALIFIERS.join(Separator.encode( - rowKey.getClusterId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS), Bytes.toBytes(LongConverter - .invertLong(rowKey.getDayTimestamp())), Separator.EMPTY_BYTES); - } - return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), - Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Bytes - .toBytes(LongConverter.invertLong(rowKey.getDayTimestamp())), - Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), - Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#decode(byte[]) - */ - @Override - public FlowActivityRowKey decode(byte[] rowKey) { - byte[][] rowKeyComponents = - Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); - if (rowKeyComponents.length != 4) { - throw new IllegalArgumentException("the row key is not valid for " - + "a flow activity"); - } - String clusterId = - Separator.decode(Bytes.toString(rowKeyComponents[0]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - Long dayTs = LongConverter.invertLong(Bytes.toLong(rowKeyComponents[1])); - String userId = - Separator.decode(Bytes.toString(rowKeyComponents[2]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String flowName = - Separator.decode(Bytes.toString(rowKeyComponents[3]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - return new FlowActivityRowKey(clusterId, dayTs, userId, flowName); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java deleted file mode 100644 index eb88e54..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; - -/** - * A prefix partial rowkey for flow activities. - */ -public class FlowActivityRowKeyPrefix extends FlowActivityRowKey implements - RowKeyPrefix { - - /** - * Constructs a row key prefix for the flow activity table as follows: - * {@code clusterId!dayTimestamp!}. - * - * @param clusterId Cluster Id. - * @param dayTs Start of the day timestamp. - */ - public FlowActivityRowKeyPrefix(String clusterId, Long dayTs) { - super(clusterId, dayTs, null, null, false); - } - - /** - * Constructs a row key prefix for the flow activity table as follows: - * {@code clusterId!}. - * - * @param clusterId identifying the cluster - */ - public FlowActivityRowKeyPrefix(String clusterId) { - super(clusterId, null, null, null, false); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.application. - * RowKeyPrefix#getRowKeyPrefix() - */ - public byte[] getRowKeyPrefix() { - return super.getRowKey(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java deleted file mode 100644 index 8a0430c..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; - -/** - * The flow activity table has column family info - * Stores the daily activity record for flows - * Useful as a quick lookup of what flows were - * running on a given day - * - * Example flow activity table record: - * - *

- * |-------------------------------------------|
- * |  Row key   | Column Family                |
- * |            | info                         |
- * |-------------------------------------------|
- * | clusterId! | r!runid1:version1            |
- * | inv Top of |                              |
- * | Day!       | r!runid2:version7            |
- * | userName!  |                              |
- * | flowName   |                              |
- * |-------------------------------------------|
- * 
- */ -public class FlowActivityTable extends BaseTable { - /** flow activity table prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity"; - - /** config param name that specifies the flowactivity table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for flowactivity table name. */ - public static final String DEFAULT_TABLE_NAME = - "timelineservice.flowactivity"; - - private static final Log LOG = LogFactory.getLog(FlowActivityTable.class); - - /** default max number of versions. */ - public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; - - public FlowActivityTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor flowActivityTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - flowActivityTableDescp.addFamily(infoCF); - infoCF.setMinVersions(1); - infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - - // TODO: figure the split policy before running in production - admin.createTable(flowActivityTableDescp); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } -} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java deleted file mode 100644 index 90dd345..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; - -/** - * Identifies fully qualified columns for the {@link FlowRunTable}. - */ -public enum FlowRunColumn implements Column { - - /** - * When the flow was started. This is the minimum of currently known - * application start times. - */ - MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time", - AggregationOperation.GLOBAL_MIN, new LongConverter()), - - /** - * When the flow ended. This is the maximum of currently known application end - * times. - */ - MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time", - AggregationOperation.GLOBAL_MAX, new LongConverter()), - - /** - * The version of the flow that this flow belongs to. - */ - FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - private final String columnQualifier; - private final byte[] columnQualifierBytes; - private final AggregationOperation aggOp; - - private FlowRunColumn(ColumnFamily columnFamily, - String columnQualifier, AggregationOperation aggOp) { - this(columnFamily, columnQualifier, aggOp, - GenericConverter.getInstance()); - } - - private FlowRunColumn(ColumnFamily columnFamily, - String columnQualifier, AggregationOperation aggOp, - ValueConverter converter) { - this.columnFamily = columnFamily; - this.columnQualifier = columnQualifier; - this.aggOp = aggOp; - // Future-proof by ensuring the right column prefix hygiene. - this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE - .encode(columnQualifier)); - this.column = new ColumnHelper(columnFamily, converter); - } - - /** - * @return the column name value - */ - private String getColumnQualifier() { - return columnQualifier; - } - - @Override - public byte[] getColumnQualifierBytes() { - return columnQualifierBytes.clone(); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - public AggregationOperation getAggregationOperation() { - return aggOp; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.Column#store - * (byte[], org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.Long, java.lang.Object, - * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, Long timestamp, - Object inputValue, Attribute... attributes) throws IOException { - - Attribute[] combinedAttributes = - HBaseTimelineStorageUtils.combineAttributes(attributes, aggOp); - column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, - inputValue, combinedAttributes); - } - - public Object readResult(Result result) throws IOException { - return column.readResult(result, columnQualifierBytes); - } - - /** - * Retrieve an {@link FlowRunColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnQualifier - * Name of the column to retrieve - * @return the corresponding {@link FlowRunColumn} or null - */ - public static final FlowRunColumn columnFor(String columnQualifier) { - - // Match column based on value, assume column family matches. - for (FlowRunColumn ec : FlowRunColumn.values()) { - // Find a match based only on name. - if (ec.getColumnQualifier().equals(columnQualifier)) { - return ec; - } - } - - // Default to null - return null; - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /** - * Retrieve an {@link FlowRunColumn} given a name, or null if there is no - * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)} - * if and only if {@code a.equals(b) & x.equals(y)} or - * {@code (x == y == null)} - * - * @param columnFamily - * The columnFamily for which to retrieve the column. - * @param name - * Name of the column to retrieve - * @return the corresponding {@link FlowRunColumn} or null if both arguments - * don't match. - */ - public static final FlowRunColumn columnFor(FlowRunColumnFamily columnFamily, - String name) { - - for (FlowRunColumn ec : FlowRunColumn.values()) { - // Find a match based column family and on name. - if (ec.columnFamily.equals(columnFamily) - && ec.getColumnQualifier().equals(name)) { - return ec; - } - } - - // Default to null - return null; - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java deleted file mode 100644 index 8faf5f8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents the flow run table column families. - */ -public enum FlowRunColumnFamily implements ColumnFamily { - - /** - * Info column family houses known columns, specifically ones included in - * columnfamily filters. - */ - INFO("i"); - - /** - * Byte representation of this column family. - */ - private final byte[] bytes; - - /** - * @param value - * create a column family with this name. Must be lower case and - * without spaces. - */ - private FlowRunColumnFamily(String value) { - // column families should be lower case and not contain any spaces. - this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); - } - - public byte[] getBytes() { - return Bytes.copy(bytes); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java deleted file mode 100644 index 278d18e..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java +++ /dev/null @@ -1,268 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; -import java.util.Map; -import java.util.NavigableMap; - -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -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.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; - -/** - * Identifies partially qualified columns for the {@link FlowRunTable}. - */ -public enum FlowRunColumnPrefix implements ColumnPrefix { - - /** - * To store flow run info values. - */ - METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter()); - - private final ColumnHelper column; - private final ColumnFamily columnFamily; - - /** - * Can be null for those cases where the provided column qualifier is the - * entire column name. - */ - private final String columnPrefix; - private final byte[] columnPrefixBytes; - - private final AggregationOperation aggOp; - - /** - * Private constructor, meant to be used by the enum definition. - * - * @param columnFamily that this column is stored in. - * @param columnPrefix for this column. - */ - private FlowRunColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, AggregationOperation fra, ValueConverter converter) { - this(columnFamily, columnPrefix, fra, converter, false); - } - - private FlowRunColumnPrefix(ColumnFamily columnFamily, - String columnPrefix, AggregationOperation fra, ValueConverter converter, - boolean compoundColQual) { - column = new ColumnHelper(columnFamily, converter); - this.columnFamily = columnFamily; - this.columnPrefix = columnPrefix; - if (columnPrefix == null) { - this.columnPrefixBytes = null; - } else { - // Future-proof by ensuring the right column prefix hygiene. - this.columnPrefixBytes = - Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); - } - this.aggOp = fra; - } - - /** - * @return the column name value - */ - public String getColumnPrefix() { - return columnPrefix; - } - - public byte[] getColumnPrefixBytes() { - return columnPrefixBytes.clone(); - } - - @Override - public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { - return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, - qualifierPrefix); - } - - @Override - public byte[] getColumnPrefixBytes(String qualifierPrefix) { - return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, - qualifierPrefix); - } - - @Override - public byte[] getColumnFamilyBytes() { - return columnFamily.getBytes(); - } - - public AggregationOperation getAttribute() { - return aggOp; - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, String qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - Attribute[] combinedAttributes = - HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - combinedAttributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #store(byte[], - * org.apache.hadoop.yarn.server.timelineservice.storage.common. - * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) - */ - public void store(byte[] rowKey, - TypedBufferedMutator tableMutator, byte[] qualifier, - Long timestamp, Object inputValue, Attribute... attributes) - throws IOException { - - // Null check - if (qualifier == null) { - throw new IOException("Cannot store column with null qualifier in " - + tableMutator.getName().getNameAsString()); - } - - byte[] columnQualifier = getColumnPrefixBytes(qualifier); - Attribute[] combinedAttributes = - HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); - column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, - combinedAttributes); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) - */ - public Object readResult(Result result, String qualifier) throws IOException { - byte[] columnQualifier = - ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); - return column.readResult(result, columnQualifier); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResults(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public Map readResults(Result result, - KeyConverter keyConverter) throws IOException { - return column.readResults(result, columnPrefixBytes, keyConverter); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix - * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, - * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) - */ - public NavigableMap> - readResultsWithTimestamps(Result result, KeyConverter keyConverter) - throws IOException { - return column.readResultsWithTimestamps(result, columnPrefixBytes, - keyConverter); - } - - /** - * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is - * no match. The following holds true: {@code columnFor(x) == columnFor(y)} if - * and only if {@code x.equals(y)} or {@code (x == y == null)} - * - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link FlowRunColumnPrefix} or null - */ - public static final FlowRunColumnPrefix columnFor(String columnPrefix) { - - // Match column based on value, assume column family matches. - for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) { - // Find a match based only on name. - if (frcp.getColumnPrefix().equals(columnPrefix)) { - return frcp; - } - } - - // Default to null - return null; - } - - @Override - public ValueConverter getValueConverter() { - return column.getValueConverter(); - } - - /** - * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is - * no match. The following holds true: - * {@code columnFor(a,x) == columnFor(b,y)} if and only if - * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} - * - * @param columnFamily The columnFamily for which to retrieve the column. - * @param columnPrefix Name of the column to retrieve - * @return the corresponding {@link FlowRunColumnPrefix} or null if both - * arguments don't match. - */ - public static final FlowRunColumnPrefix columnFor( - FlowRunColumnFamily columnFamily, String columnPrefix) { - - // TODO: needs unit test to confirm and need to update javadoc to explain - // null prefix case. - - for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) { - // Find a match based column family and on name. - if (frcp.columnFamily.equals(columnFamily) - && (((columnPrefix == null) && (frcp.getColumnPrefix() == null)) || - (frcp.getColumnPrefix().equals(columnPrefix)))) { - return frcp; - } - } - - // Default to null - return null; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java deleted file mode 100644 index 2be6ef8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java +++ /dev/null @@ -1,304 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; - -/** - * Coprocessor for flow run table. - */ -public class FlowRunCoprocessor extends BaseRegionObserver { - - private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class); - private boolean isFlowRunRegion = false; - - private Region region; - /** - * generate a timestamp that is unique per row in a region this is per region. - */ - private final TimestampGenerator timestampGenerator = - new TimestampGenerator(); - - @Override - public void start(CoprocessorEnvironment e) throws IOException { - if (e instanceof RegionCoprocessorEnvironment) { - RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e; - this.region = env.getRegion(); - isFlowRunRegion = HBaseTimelineStorageUtils.isFlowRunTable( - region.getRegionInfo(), env.getConfiguration()); - } - } - - public boolean isFlowRunRegion() { - return isFlowRunRegion; - } - - /* - * (non-Javadoc) - * - * This method adds the tags onto the cells in the Put. It is presumed that - * all the cells in one Put have the same set of Tags. The existing cell - * timestamp is overwritten for non-metric cells and each such cell gets a new - * unique timestamp generated by {@link TimestampGenerator} - * - * @see - * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#prePut(org.apache - * .hadoop.hbase.coprocessor.ObserverContext, - * org.apache.hadoop.hbase.client.Put, - * org.apache.hadoop.hbase.regionserver.wal.WALEdit, - * org.apache.hadoop.hbase.client.Durability) - */ - @Override - public void prePut(ObserverContext e, Put put, - WALEdit edit, Durability durability) throws IOException { - Map attributes = put.getAttributesMap(); - - if (!isFlowRunRegion) { - return; - } - // Assumption is that all the cells in a put are the same operation. - List tags = new ArrayList<>(); - if ((attributes != null) && (attributes.size() > 0)) { - for (Map.Entry attribute : attributes.entrySet()) { - Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute); - tags.add(t); - } - byte[] tagByteArray = Tag.fromList(tags); - NavigableMap> newFamilyMap = new TreeMap<>( - Bytes.BYTES_COMPARATOR); - for (Map.Entry> entry : put.getFamilyCellMap() - .entrySet()) { - List newCells = new ArrayList<>(entry.getValue().size()); - for (Cell cell : entry.getValue()) { - // for each cell in the put add the tags - // Assumption is that all the cells in - // one put are the same operation - // also, get a unique cell timestamp for non-metric cells - // this way we don't inadvertently overwrite cell versions - long cellTimestamp = getCellTimestamp(cell.getTimestamp(), tags); - newCells.add(CellUtil.createCell(CellUtil.cloneRow(cell), - CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), - cellTimestamp, KeyValue.Type.Put, CellUtil.cloneValue(cell), - tagByteArray)); - } - newFamilyMap.put(entry.getKey(), newCells); - } // for each entry - // Update the family map for the Put - put.setFamilyCellMap(newFamilyMap); - } - } - - /** - * Determines if the current cell's timestamp is to be used or a new unique - * cell timestamp is to be used. The reason this is done is to inadvertently - * overwrite cells when writes come in very fast. But for metric cells, the - * cell timestamp signifies the metric timestamp. Hence we don't want to - * overwrite it. - * - * @param timestamp - * @param tags - * @return cell timestamp - */ - private long getCellTimestamp(long timestamp, List tags) { - // if ts not set (hbase sets to HConstants.LATEST_TIMESTAMP by default) - // then use the generator - if (timestamp == HConstants.LATEST_TIMESTAMP) { - return timestampGenerator.getUniqueTimestamp(); - } else { - return timestamp; - } - } - - /* - * (non-Javadoc) - * - * Creates a {@link FlowScanner} Scan so that it can correctly process the - * contents of {@link FlowRunTable}. - * - * @see - * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preGetOp(org.apache - * .hadoop.hbase.coprocessor.ObserverContext, - * org.apache.hadoop.hbase.client.Get, java.util.List) - */ - @Override - public void preGetOp(ObserverContext e, - Get get, List results) throws IOException { - if (!isFlowRunRegion) { - return; - } - - Scan scan = new Scan(get); - scan.setMaxVersions(); - RegionScanner scanner = null; - try { - scanner = new FlowScanner(e.getEnvironment(), scan, - region.getScanner(scan), FlowScannerOperation.READ); - scanner.next(results); - e.bypass(); - } finally { - if (scanner != null) { - scanner.close(); - } - } - } - - /* - * (non-Javadoc) - * - * Ensures that max versions are set for the Scan so that metrics can be - * correctly aggregated and min/max can be correctly determined. - * - * @see - * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preScannerOpen(org - * .apache.hadoop.hbase.coprocessor.ObserverContext, - * org.apache.hadoop.hbase.client.Scan, - * org.apache.hadoop.hbase.regionserver.RegionScanner) - */ - @Override - public RegionScanner preScannerOpen( - ObserverContext e, Scan scan, - RegionScanner scanner) throws IOException { - - if (isFlowRunRegion) { - // set max versions for scan to see all - // versions to aggregate for metrics - scan.setMaxVersions(); - } - return scanner; - } - - /* - * (non-Javadoc) - * - * Creates a {@link FlowScanner} Scan so that it can correctly process the - * contents of {@link FlowRunTable}. - * - * @see - * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#postScannerOpen( - * org.apache.hadoop.hbase.coprocessor.ObserverContext, - * org.apache.hadoop.hbase.client.Scan, - * org.apache.hadoop.hbase.regionserver.RegionScanner) - */ - @Override - public RegionScanner postScannerOpen( - ObserverContext e, Scan scan, - RegionScanner scanner) throws IOException { - if (!isFlowRunRegion) { - return scanner; - } - return new FlowScanner(e.getEnvironment(), scan, - scanner, FlowScannerOperation.READ); - } - - @Override - public InternalScanner preFlush( - ObserverContext c, Store store, - InternalScanner scanner) throws IOException { - if (!isFlowRunRegion) { - return scanner; - } - if (LOG.isDebugEnabled()) { - if (store != null) { - LOG.debug("preFlush store = " + store.getColumnFamilyName() - + " flushableSize=" + store.getFlushableSize() - + " flushedCellsCount=" + store.getFlushedCellsCount() - + " compactedCellsCount=" + store.getCompactedCellsCount() - + " majorCompactedCellsCount=" - + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" - + store.getMemstoreFlushSize() + " memstoreSize=" - + store.getMemStoreSize() + " size=" + store.getSize() - + " storeFilesCount=" + store.getStorefilesCount()); - } - } - return new FlowScanner(c.getEnvironment(), scanner, - FlowScannerOperation.FLUSH); - } - - @Override - public void postFlush(ObserverContext c, - Store store, StoreFile resultFile) { - if (!isFlowRunRegion) { - return; - } - if (LOG.isDebugEnabled()) { - if (store != null) { - LOG.debug("postFlush store = " + store.getColumnFamilyName() - + " flushableSize=" + store.getFlushableSize() - + " flushedCellsCount=" + store.getFlushedCellsCount() - + " compactedCellsCount=" + store.getCompactedCellsCount() - + " majorCompactedCellsCount=" - + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" - + store.getMemstoreFlushSize() + " memstoreSize=" - + store.getMemStoreSize() + " size=" + store.getSize() - + " storeFilesCount=" + store.getStorefilesCount()); - } - } - } - - @Override - public InternalScanner preCompact( - ObserverContext e, Store store, - InternalScanner scanner, ScanType scanType, CompactionRequest request) - throws IOException { - - if (!isFlowRunRegion) { - return scanner; - } - FlowScannerOperation requestOp = FlowScannerOperation.MINOR_COMPACTION; - if (request != null) { - requestOp = (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION - : FlowScannerOperation.MINOR_COMPACTION); - LOG.info("Compactionrequest= " + request.toString() + " " - + requestOp.toString() + " RegionName=" + e.getEnvironment() - .getRegion().getRegionInfo().getRegionNameAsString()); - } - return new FlowScanner(e.getEnvironment(), scanner, requestOp); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java deleted file mode 100644 index 8fda9a8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java +++ /dev/null @@ -1,190 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; - -/** - * Represents a rowkey for the flow run table. - */ -public class FlowRunRowKey { - private final String clusterId; - private final String userId; - private final String flowName; - private final Long flowRunId; - private final FlowRunRowKeyConverter flowRunRowKeyConverter = - new FlowRunRowKeyConverter(); - - public FlowRunRowKey(String clusterId, String userId, String flowName, - Long flowRunId) { - this.clusterId = clusterId; - this.userId = userId; - this.flowName = flowName; - this.flowRunId = flowRunId; - } - - public String getClusterId() { - return clusterId; - } - - public String getUserId() { - return userId; - } - - public String getFlowName() { - return flowName; - } - - public Long getFlowRunId() { - return flowRunId; - } - - /** - * Constructs a row key for the entity table as follows: { - * clusterId!userId!flowName!Inverted Flow Run Id}. - * - * @return byte array with the row key - */ - public byte[] getRowKey() { - return flowRunRowKeyConverter.encode(this); - } - - - /** - * Given the raw row key as bytes, returns the row key as an object. - * - * @param rowKey Byte representation of row key. - * @return A FlowRunRowKey object. - */ - public static FlowRunRowKey parseRowKey(byte[] rowKey) { - return new FlowRunRowKeyConverter().decode(rowKey); - } - - /** - * returns the Flow Key as a verbose String output. - * @return String - */ - @Override - public String toString() { - StringBuilder flowKeyStr = new StringBuilder(); - flowKeyStr.append("{clusterId=" + clusterId); - flowKeyStr.append(" userId=" + userId); - flowKeyStr.append(" flowName=" + flowName); - flowKeyStr.append(" flowRunId="); - flowKeyStr.append(flowRunId); - flowKeyStr.append("}"); - return flowKeyStr.toString(); - } - - /** - * Encodes and decodes row key for flow run table. - * The row key is of the form : clusterId!userId!flowName!flowrunId. - * flowrunId is a long and rest are strings. - *

- */ - final private static class FlowRunRowKeyConverter implements - KeyConverter { - - private FlowRunRowKeyConverter() { - } - - /** - * The flow run row key is of the form clusterId!userId!flowName!flowrunId - * with each segment separated by !. The sizes below indicate sizes of each - * one of these segments in sequence. clusterId, userId and flowName are - * strings. flowrunId is a long hence 8 bytes in size. Strings are variable - * in size (i.e. end whenever separator is encountered). This is used while - * decoding and helps in determining where to split. - */ - private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG }; - - /* - * (non-Javadoc) - * - * Encodes FlowRunRowKey object into a byte array with each component/field - * in FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an flow - * run row key of the form clusterId!userId!flowName!flowrunId If flowRunId - * in passed FlowRunRowKey object is null (and the fields preceding it i.e. - * clusterId, userId and flowName are not null), this returns a row key - * prefix of the form clusterId!userName!flowName! flowRunId is inverted - * while encoding as it helps maintain a descending order for flow keys in - * flow run table. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#encode(java.lang.Object) - */ - @Override - public byte[] encode(FlowRunRowKey rowKey) { - byte[] first = - Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), - Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Separator - .encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, - Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), - Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); - if (rowKey.getFlowRunId() == null) { - return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); - } else { - // Note that flowRunId is a long, so we can't encode them all at the - // same - // time. - byte[] second = - Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); - return Separator.QUALIFIERS.join(first, second); - } - } - - /* - * (non-Javadoc) - * - * Decodes an flow run row key of the form - * clusterId!userId!flowName!flowrunId represented in byte format and - * converts it into an FlowRunRowKey object. flowRunId is inverted while - * decoding as it was inverted while encoding. - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.common - * .KeyConverter#decode(byte[]) - */ - @Override - public FlowRunRowKey decode(byte[] rowKey) { - byte[][] rowKeyComponents = - Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); - if (rowKeyComponents.length != 4) { - throw new IllegalArgumentException("the row key is not valid for " - + "a flow run"); - } - String clusterId = - Separator.decode(Bytes.toString(rowKeyComponents[0]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String userId = - Separator.decode(Bytes.toString(rowKeyComponents[1]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - String flowName = - Separator.decode(Bytes.toString(rowKeyComponents[2]), - Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); - Long flowRunId = - LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); - return new FlowRunRowKey(clusterId, userId, flowName, flowRunId); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java deleted file mode 100644 index 23ebc66..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; - -/** - * Represents a partial rowkey (without the flowRunId) for the flow run table. - */ -public class FlowRunRowKeyPrefix extends FlowRunRowKey implements - RowKeyPrefix { - - /** - * Constructs a row key prefix for the flow run table as follows: - * {@code clusterId!userI!flowName!}. - * - * @param clusterId identifying the cluster - * @param userId identifying the user - * @param flowName identifying the flow - */ - public FlowRunRowKeyPrefix(String clusterId, String userId, - String flowName) { - super(clusterId, userId, flowName, null); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.application. - * RowKeyPrefix#getRowKeyPrefix() - */ - public byte[] getRowKeyPrefix() { - // We know we're a FlowRunRowKey with null florRunId, so we can simply - // delegate - return super.getRowKey(); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java deleted file mode 100644 index 547bef0..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; - -/** - * The flow run table has column family info - * Stores per flow run information - * aggregated across applications. - * - * Metrics are also stored in the info column family. - * - * Example flow run table record: - * - *

- * flow_run table
- * |-------------------------------------------|
- * |  Row key   | Column Family                |
- * |            | info                         |
- * |-------------------------------------------|
- * | clusterId! | flow_version:version7        |
- * | userName!  |                              |
- * | flowName!  | running_apps:1               |
- * | flowRunId  |                              |
- * |            | min_start_time:1392995080000 |
- * |            | #0:""                        |
- * |            |                              |
- * |            | min_start_time:1392995081012 |
- * |            | #0:appId2                    |
- * |            |                              |
- * |            | min_start_time:1392993083210 |
- * |            | #0:appId3                    |
- * |            |                              |
- * |            |                              |
- * |            | max_end_time:1392993084018   |
- * |            | #0:""                        |
- * |            |                              |
- * |            |                              |
- * |            | m!mapInputRecords:127        |
- * |            | #0:""                        |
- * |            |                              |
- * |            | m!mapInputRecords:31         |
- * |            | #2:appId2                    |
- * |            |                              |
- * |            | m!mapInputRecords:37         |
- * |            | #1:appId3                    |
- * |            |                              |
- * |            |                              |
- * |            | m!mapOutputRecords:181       |
- * |            | #0:""                        |
- * |            |                              |
- * |            | m!mapOutputRecords:37        |
- * |            | #1:appId3                    |
- * |            |                              |
- * |            |                              |
- * |-------------------------------------------|
- * 
- */ -public class FlowRunTable extends BaseTable { - /** entity prefix. */ - private static final String PREFIX = - YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun"; - - /** config param name that specifies the flowrun table name. */ - public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; - - /** default value for flowrun table name. */ - public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun"; - - private static final Log LOG = LogFactory.getLog(FlowRunTable.class); - - /** default max number of versions. */ - public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; - - public FlowRunTable() { - super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); - } - - /* - * (non-Javadoc) - * - * @see - * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable - * (org.apache.hadoop.hbase.client.Admin, - * org.apache.hadoop.conf.Configuration) - */ - public void createTable(Admin admin, Configuration hbaseConf) - throws IOException { - - TableName table = getTableName(hbaseConf); - if (admin.tableExists(table)) { - // do not disable / delete existing table - // similar to the approach taken by map-reduce jobs when - // output directory exists - throw new IOException("Table " + table.getNameAsString() - + " already exists."); - } - - HTableDescriptor flowRunTableDescp = new HTableDescriptor(table); - HColumnDescriptor infoCF = - new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes()); - infoCF.setBloomFilterType(BloomType.ROWCOL); - flowRunTableDescp.addFamily(infoCF); - infoCF.setMinVersions(1); - infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); - - // TODO: figure the split policy - flowRunTableDescp.addCoprocessor(FlowRunCoprocessor.class - .getCanonicalName()); - admin.createTable(flowRunTableDescp); - LOG.info("Status of table creation for " + table.getNameAsString() + "=" - + admin.tableExists(table)); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java deleted file mode 100644 index 0e3c8ee..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java +++ /dev/null @@ -1,728 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.NumericValueConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; - -import com.google.common.annotations.VisibleForTesting; - -/** - * Invoked via the coprocessor when a Get or a Scan is issued for flow run - * table. Looks through the list of cells per row, checks their tags and does - * operation on those cells as per the cell tags. Transforms reads of the stored - * metrics into calculated sums for each column Also, finds the min and max for - * start and end times in a flow run. - */ -class FlowScanner implements RegionScanner, Closeable { - - private static final Log LOG = LogFactory.getLog(FlowScanner.class); - - /** - * use a special application id to represent the flow id this is needed since - * TimestampGenerator parses the app id to generate a cell timestamp. - */ - private static final String FLOW_APP_ID = "application_00000000000_0000"; - - private final Region region; - private final InternalScanner flowRunScanner; - private final int batchSize; - private final long appFinalValueRetentionThreshold; - private RegionScanner regionScanner; - private boolean hasMore; - private byte[] currentRow; - private List availableCells = new ArrayList<>(); - private int currentIndex; - private FlowScannerOperation action = FlowScannerOperation.READ; - - FlowScanner(RegionCoprocessorEnvironment env, InternalScanner internalScanner, - FlowScannerOperation action) { - this(env, null, internalScanner, action); - } - - FlowScanner(RegionCoprocessorEnvironment env, Scan incomingScan, - InternalScanner internalScanner, FlowScannerOperation action) { - this.batchSize = incomingScan == null ? -1 : incomingScan.getBatch(); - // TODO initialize other scan attributes like Scan#maxResultSize - this.flowRunScanner = internalScanner; - if (internalScanner instanceof RegionScanner) { - this.regionScanner = (RegionScanner) internalScanner; - } - this.action = action; - if (env == null) { - this.appFinalValueRetentionThreshold = - YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD; - this.region = null; - } else { - this.region = env.getRegion(); - Configuration hbaseConf = env.getConfiguration(); - this.appFinalValueRetentionThreshold = hbaseConf.getLong( - YarnConfiguration.APP_FINAL_VALUE_RETENTION_THRESHOLD, - YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD); - } - if (LOG.isDebugEnabled()) { - LOG.debug(" batch size=" + batchSize); - } - } - - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo() - */ - @Override - public HRegionInfo getRegionInfo() { - return region.getRegionInfo(); - } - - @Override - public boolean nextRaw(List cells) throws IOException { - return nextRaw(cells, ScannerContext.newBuilder().build()); - } - - @Override - public boolean nextRaw(List cells, ScannerContext scannerContext) - throws IOException { - return nextInternal(cells, scannerContext); - } - - @Override - public boolean next(List cells) throws IOException { - return next(cells, ScannerContext.newBuilder().build()); - } - - @Override - public boolean next(List cells, ScannerContext scannerContext) - throws IOException { - return nextInternal(cells, scannerContext); - } - - /** - * Get value converter associated with a column or a column prefix. If nothing - * matches, generic converter is returned. - * @param colQualifierBytes - * @return value converter implementation. - */ - private static ValueConverter getValueConverter(byte[] colQualifierBytes) { - // Iterate over all the column prefixes for flow run table and get the - // appropriate converter for the column qualifier passed if prefix matches. - for (FlowRunColumnPrefix colPrefix : FlowRunColumnPrefix.values()) { - byte[] colPrefixBytes = colPrefix.getColumnPrefixBytes(""); - if (Bytes.compareTo(colPrefixBytes, 0, colPrefixBytes.length, - colQualifierBytes, 0, colPrefixBytes.length) == 0) { - return colPrefix.getValueConverter(); - } - } - // Iterate over all the columns for flow run table and get the - // appropriate converter for the column qualifier passed if match occurs. - for (FlowRunColumn column : FlowRunColumn.values()) { - if (Bytes.compareTo( - column.getColumnQualifierBytes(), colQualifierBytes) == 0) { - return column.getValueConverter(); - } - } - // Return generic converter if nothing matches. - return GenericConverter.getInstance(); - } - - /** - * This method loops through the cells in a given row of the - * {@link FlowRunTable}. It looks at the tags of each cell to figure out how - * to process the contents. It then calculates the sum or min or max for each - * column or returns the cell as is. - * - * @param cells - * @param scannerContext - * @return true if next row is available for the scanner, false otherwise - * @throws IOException - */ - private boolean nextInternal(List cells, ScannerContext scannerContext) - throws IOException { - Cell cell = null; - startNext(); - // Loop through all the cells in this row - // For min/max/metrics we do need to scan the entire set of cells to get the - // right one - // But with flush/compaction, the number of cells being scanned will go down - // cells are grouped per column qualifier then sorted by cell timestamp - // (latest to oldest) per column qualifier - // So all cells in one qualifier come one after the other before we see the - // next column qualifier - ByteArrayComparator comp = new ByteArrayComparator(); - byte[] previousColumnQualifier = Separator.EMPTY_BYTES; - AggregationOperation currentAggOp = null; - SortedSet currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR); - Set alreadySeenAggDim = new HashSet<>(); - int addedCnt = 0; - long currentTimestamp = System.currentTimeMillis(); - ValueConverter converter = null; - int limit = batchSize; - - while (limit <= 0 || addedCnt < limit) { - cell = peekAtNextCell(scannerContext); - if (cell == null) { - break; - } - byte[] currentColumnQualifier = CellUtil.cloneQualifier(cell); - if (previousColumnQualifier == null) { - // first time in loop - previousColumnQualifier = currentColumnQualifier; - } - - converter = getValueConverter(currentColumnQualifier); - if (comp.compare(previousColumnQualifier, currentColumnQualifier) != 0) { - addedCnt += emitCells(cells, currentColumnCells, currentAggOp, - converter, currentTimestamp); - resetState(currentColumnCells, alreadySeenAggDim); - previousColumnQualifier = currentColumnQualifier; - currentAggOp = getCurrentAggOp(cell); - converter = getValueConverter(currentColumnQualifier); - } - collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim, - converter, scannerContext); - nextCell(scannerContext); - } - if ((!currentColumnCells.isEmpty()) && ((limit <= 0 || addedCnt < limit))) { - addedCnt += emitCells(cells, currentColumnCells, currentAggOp, converter, - currentTimestamp); - if (LOG.isDebugEnabled()) { - if (addedCnt > 0) { - LOG.debug("emitted cells. " + addedCnt + " for " + this.action - + " rowKey=" - + FlowRunRowKey.parseRowKey(CellUtil.cloneRow(cells.get(0)))); - } else { - LOG.debug("emitted no cells for " + this.action); - } - } - } - return hasMore(); - } - - private AggregationOperation getCurrentAggOp(Cell cell) { - List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); - // We assume that all the operations for a particular column are the same - return HBaseTimelineStorageUtils.getAggregationOperationFromTagsList(tags); - } - - /** - * resets the parameters to an initialized state for next loop iteration. - * - * @param cell - * @param currentAggOp - * @param currentColumnCells - * @param alreadySeenAggDim - * @param collectedButNotEmitted - */ - private void resetState(SortedSet currentColumnCells, - Set alreadySeenAggDim) { - currentColumnCells.clear(); - alreadySeenAggDim.clear(); - } - - private void collectCells(SortedSet currentColumnCells, - AggregationOperation currentAggOp, Cell cell, - Set alreadySeenAggDim, ValueConverter converter, - ScannerContext scannerContext) throws IOException { - - if (currentAggOp == null) { - // not a min/max/metric cell, so just return it as is - currentColumnCells.add(cell); - return; - } - - switch (currentAggOp) { - case GLOBAL_MIN: - if (currentColumnCells.size() == 0) { - currentColumnCells.add(cell); - } else { - Cell currentMinCell = currentColumnCells.first(); - Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp, - (NumericValueConverter) converter); - if (!currentMinCell.equals(newMinCell)) { - currentColumnCells.remove(currentMinCell); - currentColumnCells.add(newMinCell); - } - } - break; - case GLOBAL_MAX: - if (currentColumnCells.size() == 0) { - currentColumnCells.add(cell); - } else { - Cell currentMaxCell = currentColumnCells.first(); - Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp, - (NumericValueConverter) converter); - if (!currentMaxCell.equals(newMaxCell)) { - currentColumnCells.remove(currentMaxCell); - currentColumnCells.add(newMaxCell); - } - } - break; - case SUM: - case SUM_FINAL: - if (LOG.isTraceEnabled()) { - LOG.trace("In collect cells " - + " FlowSannerOperation=" - + this.action - + " currentAggOp=" - + currentAggOp - + " cell qualifier=" - + Bytes.toString(CellUtil.cloneQualifier(cell)) - + " cell value= " - + converter.decodeValue(CellUtil.cloneValue(cell)) - + " timestamp=" + cell.getTimestamp()); - } - - // only if this app has not been seen yet, add to current column cells - List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); - String aggDim = HBaseTimelineStorageUtils - .getAggregationCompactionDimension(tags); - if (!alreadySeenAggDim.contains(aggDim)) { - // if this agg dimension has already been seen, - // since they show up in sorted order - // we drop the rest which are older - // in other words, this cell is older than previously seen cells - // for that agg dim - // but when this agg dim is not seen, - // consider this cell in our working set - currentColumnCells.add(cell); - alreadySeenAggDim.add(aggDim); - } - break; - default: - break; - } // end of switch case - } - - /* - * Processes the cells in input param currentColumnCells and populates - * List cells as the output based on the input AggregationOperation - * parameter. - */ - private int emitCells(List cells, SortedSet currentColumnCells, - AggregationOperation currentAggOp, ValueConverter converter, - long currentTimestamp) throws IOException { - if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) { - return 0; - } - if (currentAggOp == null) { - cells.addAll(currentColumnCells); - return currentColumnCells.size(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("In emitCells " + this.action + " currentColumnCells size= " - + currentColumnCells.size() + " currentAggOp" + currentAggOp); - } - - switch (currentAggOp) { - case GLOBAL_MIN: - case GLOBAL_MAX: - cells.addAll(currentColumnCells); - return currentColumnCells.size(); - case SUM: - case SUM_FINAL: - switch (action) { - case FLUSH: - case MINOR_COMPACTION: - cells.addAll(currentColumnCells); - return currentColumnCells.size(); - case READ: - Cell sumCell = processSummation(currentColumnCells, - (NumericValueConverter) converter); - cells.add(sumCell); - return 1; - case MAJOR_COMPACTION: - List finalCells = processSummationMajorCompaction( - currentColumnCells, (NumericValueConverter) converter, - currentTimestamp); - cells.addAll(finalCells); - return finalCells.size(); - default: - cells.addAll(currentColumnCells); - return currentColumnCells.size(); - } - default: - cells.addAll(currentColumnCells); - return currentColumnCells.size(); - } - } - - /* - * Returns a cell whose value is the sum of all cell values in the input set. - * The new cell created has the timestamp of the most recent metric cell. The - * sum of a metric for a flow run is the summation at the point of the last - * metric update in that flow till that time. - */ - private Cell processSummation(SortedSet currentColumnCells, - NumericValueConverter converter) throws IOException { - Number sum = 0; - Number currentValue = 0; - long ts = 0L; - long mostCurrentTimestamp = 0L; - Cell mostRecentCell = null; - for (Cell cell : currentColumnCells) { - currentValue = (Number) converter.decodeValue(CellUtil.cloneValue(cell)); - ts = cell.getTimestamp(); - if (mostCurrentTimestamp < ts) { - mostCurrentTimestamp = ts; - mostRecentCell = cell; - } - sum = converter.add(sum, currentValue); - } - byte[] sumBytes = converter.encodeValue(sum); - Cell sumCell = - HBaseTimelineStorageUtils.createNewCell(mostRecentCell, sumBytes); - return sumCell; - } - - - /** - * Returns a list of cells that contains - * - * A) the latest cells for applications that haven't finished yet - * B) summation - * for the flow, based on applications that have completed and are older than - * a certain time - * - * The new cell created has the timestamp of the most recent metric cell. The - * sum of a metric for a flow run is the summation at the point of the last - * metric update in that flow till that time. - */ - @VisibleForTesting - List processSummationMajorCompaction( - SortedSet currentColumnCells, NumericValueConverter converter, - long currentTimestamp) - throws IOException { - Number sum = 0; - Number currentValue = 0; - long ts = 0L; - boolean summationDone = false; - List finalCells = new ArrayList(); - if (currentColumnCells == null) { - return finalCells; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("In processSummationMajorCompaction," - + " will drop cells older than " + currentTimestamp - + " CurrentColumnCells size=" + currentColumnCells.size()); - } - - for (Cell cell : currentColumnCells) { - AggregationOperation cellAggOp = getCurrentAggOp(cell); - // if this is the existing flow sum cell - List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); - String appId = HBaseTimelineStorageUtils - .getAggregationCompactionDimension(tags); - if (appId == FLOW_APP_ID) { - sum = converter.add(sum, currentValue); - summationDone = true; - if (LOG.isTraceEnabled()) { - LOG.trace("reading flow app id sum=" + sum); - } - } else { - currentValue = (Number) converter.decodeValue(CellUtil - .cloneValue(cell)); - // read the timestamp truncated by the generator - ts = TimestampGenerator.getTruncatedTimestamp(cell.getTimestamp()); - if ((cellAggOp == AggregationOperation.SUM_FINAL) - && ((ts + this.appFinalValueRetentionThreshold) - < currentTimestamp)) { - sum = converter.add(sum, currentValue); - summationDone = true; - if (LOG.isTraceEnabled()) { - LOG.trace("MAJOR COMPACTION loop sum= " + sum - + " discarding now: " + " qualifier=" - + Bytes.toString(CellUtil.cloneQualifier(cell)) + " value=" - + converter.decodeValue(CellUtil.cloneValue(cell)) - + " timestamp=" + cell.getTimestamp() + " " + this.action); - } - } else { - // not a final value but it's the latest cell for this app - // so include this cell in the list of cells to write back - finalCells.add(cell); - } - } - } - if (summationDone) { - Cell anyCell = currentColumnCells.first(); - List tags = new ArrayList(); - Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), - Bytes.toBytes(FLOW_APP_ID)); - tags.add(t); - t = new Tag(AggregationCompactionDimension.APPLICATION_ID.getTagType(), - Bytes.toBytes(FLOW_APP_ID)); - tags.add(t); - byte[] tagByteArray = Tag.fromList(tags); - Cell sumCell = HBaseTimelineStorageUtils.createNewCell( - CellUtil.cloneRow(anyCell), - CellUtil.cloneFamily(anyCell), - CellUtil.cloneQualifier(anyCell), - TimestampGenerator.getSupplementedTimestamp( - System.currentTimeMillis(), FLOW_APP_ID), - converter.encodeValue(sum), tagByteArray); - finalCells.add(sumCell); - if (LOG.isTraceEnabled()) { - LOG.trace("MAJOR COMPACTION final sum= " + sum + " for " - + Bytes.toString(CellUtil.cloneQualifier(sumCell)) - + " " + this.action); - } - LOG.info("After major compaction for qualifier=" - + Bytes.toString(CellUtil.cloneQualifier(sumCell)) - + " with currentColumnCells.size=" - + currentColumnCells.size() - + " returning finalCells.size=" + finalCells.size() - + " with sum=" + sum.longValue() - + " with cell timestamp " + sumCell.getTimestamp()); - } else { - String qualifier = ""; - LOG.info("After major compaction for qualifier=" + qualifier - + " with currentColumnCells.size=" - + currentColumnCells.size() - + " returning finalCells.size=" + finalCells.size() - + " with zero sum=" - + sum.longValue()); - } - return finalCells; - } - - /** - * Determines which cell is to be returned based on the values in each cell - * and the comparison operation MIN or MAX. - * - * @param previouslyChosenCell - * @param currentCell - * @param currentAggOp - * @return the cell which is the min (or max) cell - * @throws IOException - */ - private Cell compareCellValues(Cell previouslyChosenCell, Cell currentCell, - AggregationOperation currentAggOp, NumericValueConverter converter) - throws IOException { - if (previouslyChosenCell == null) { - return currentCell; - } - try { - Number previouslyChosenCellValue = (Number)converter.decodeValue( - CellUtil.cloneValue(previouslyChosenCell)); - Number currentCellValue = (Number) converter.decodeValue(CellUtil - .cloneValue(currentCell)); - switch (currentAggOp) { - case GLOBAL_MIN: - if (converter.compare( - currentCellValue, previouslyChosenCellValue) < 0) { - // new value is minimum, hence return this cell - return currentCell; - } else { - // previously chosen value is miniumum, hence return previous min cell - return previouslyChosenCell; - } - case GLOBAL_MAX: - if (converter.compare( - currentCellValue, previouslyChosenCellValue) > 0) { - // new value is max, hence return this cell - return currentCell; - } else { - // previously chosen value is max, hence return previous max cell - return previouslyChosenCell; - } - default: - return currentCell; - } - } catch (IllegalArgumentException iae) { - LOG.error("caught iae during conversion to long ", iae); - return currentCell; - } - } - - @Override - public void close() throws IOException { - if (flowRunScanner != null) { - flowRunScanner.close(); - } else { - LOG.warn("scanner close called but scanner is null"); - } - } - - /** - * Called to signal the start of the next() call by the scanner. - */ - public void startNext() { - currentRow = null; - } - - /** - * Returns whether or not the underlying scanner has more rows. - */ - public boolean hasMore() { - return currentIndex < availableCells.size() ? true : hasMore; - } - - /** - * Returns the next available cell for the current row and advances the - * pointer to the next cell. This method can be called multiple times in a row - * to advance through all the available cells. - * - * @param scannerContext - * context information for the batch of cells under consideration - * @return the next available cell or null if no more cells are available for - * the current row - * @throws IOException - */ - public Cell nextCell(ScannerContext scannerContext) throws IOException { - Cell cell = peekAtNextCell(scannerContext); - if (cell != null) { - currentIndex++; - } - return cell; - } - - /** - * Returns the next available cell for the current row, without advancing the - * pointer. Calling this method multiple times in a row will continue to - * return the same cell. - * - * @param scannerContext - * context information for the batch of cells under consideration - * @return the next available cell or null if no more cells are available for - * the current row - * @throws IOException if any problem is encountered while grabbing the next - * cell. - */ - public Cell peekAtNextCell(ScannerContext scannerContext) throws IOException { - if (currentIndex >= availableCells.size()) { - // done with current batch - availableCells.clear(); - currentIndex = 0; - hasMore = flowRunScanner.next(availableCells, scannerContext); - } - Cell cell = null; - if (currentIndex < availableCells.size()) { - cell = availableCells.get(currentIndex); - if (currentRow == null) { - currentRow = CellUtil.cloneRow(cell); - } else if (!CellUtil.matchingRow(cell, currentRow)) { - // moved on to the next row - // don't use the current cell - // also signal no more cells for this row - return null; - } - } - return cell; - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize() - */ - @Override - public long getMaxResultSize() { - if (regionScanner == null) { - throw new IllegalStateException( - "RegionScanner.isFilterDone() called when the flow " - + "scanner's scanner is not a RegionScanner"); - } - return regionScanner.getMaxResultSize(); - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint() - */ - @Override - public long getMvccReadPoint() { - if (regionScanner == null) { - throw new IllegalStateException( - "RegionScanner.isFilterDone() called when the flow " - + "scanner's internal scanner is not a RegionScanner"); - } - return regionScanner.getMvccReadPoint(); - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone() - */ - @Override - public boolean isFilterDone() throws IOException { - if (regionScanner == null) { - throw new IllegalStateException( - "RegionScanner.isFilterDone() called when the flow " - + "scanner's internal scanner is not a RegionScanner"); - } - return regionScanner.isFilterDone(); - - } - - /* - * (non-Javadoc) - * - * @see org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[]) - */ - @Override - public boolean reseek(byte[] bytes) throws IOException { - if (regionScanner == null) { - throw new IllegalStateException( - "RegionScanner.reseek() called when the flow " - + "scanner's internal scanner is not a RegionScanner"); - } - return regionScanner.reseek(bytes); - } - - @Override - public int getBatch() { - return batchSize; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java deleted file mode 100644 index 73c666f..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java +++ /dev/null @@ -1,46 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - - -/** - * Identifies the scanner operation on the {@link FlowRunTable}. - */ -public enum FlowScannerOperation { - - /** - * If the scanner is opened for reading - * during preGet or preScan. - */ - READ, - - /** - * If the scanner is opened during preFlush. - */ - FLUSH, - - /** - * If the scanner is opened during minor Compaction. - */ - MINOR_COMPACTION, - - /** - * If the scanner is opened during major Compaction. - */ - MAJOR_COMPACTION -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java deleted file mode 100644 index 04963f3..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Package org.apache.hadoop.yarn.server.timelineservice.storage.flow - * contains classes related to implementation for flow related tables, viz. flow - * run table and flow activity table. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -package org.apache.hadoop.yarn.server.timelineservice.storage.flow; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java deleted file mode 100644 index aa2bfda..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java +++ /dev/null @@ -1,481 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import java.io.IOException; -import java.util.EnumSet; -import java.util.Set; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.FamilyFilter; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.hbase.filter.PageFilter; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; -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.reader.filter.TimelineFilterList; -import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; - -import com.google.common.base.Preconditions; - -/** - * Timeline entity reader for application entities that are stored in the - * application table. - */ -class ApplicationEntityReader extends GenericEntityReader { - private static final ApplicationTable APPLICATION_TABLE = - new ApplicationTable(); - - public ApplicationEntityReader(TimelineReaderContext ctxt, - TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { - super(ctxt, entityFilters, toRetrieve, true); - } - - public ApplicationEntityReader(TimelineReaderContext ctxt, - TimelineDataToRetrieve toRetrieve) { - super(ctxt, toRetrieve); - } - - /** - * Uses the {@link ApplicationTable}. - */ - protected BaseTable getTable() { - return APPLICATION_TABLE; - } - - /** - * This method is called only for multiple entity reads. - */ - @Override - protected FilterList constructFilterListBasedOnFilters() throws IOException { - // Filters here cannot be null for multiple entity reads as they are set in - // augmentParams if null. - TimelineEntityFilters filters = getFilters(); - FilterList listBasedOnFilters = new FilterList(); - // Create filter list based on created time range and add it to - // listBasedOnFilters. - long createdTimeBegin = filters.getCreatedTimeBegin(); - long createdTimeEnd = filters.getCreatedTimeEnd(); - if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { - listBasedOnFilters.addFilter( - TimelineFilterUtils.createSingleColValueFiltersByRange( - ApplicationColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd)); - } - // Create filter list based on metric filters and add it to - // listBasedOnFilters. - TimelineFilterList metricFilters = filters.getMetricFilters(); - if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter( - TimelineFilterUtils.createHBaseFilterList( - ApplicationColumnPrefix.METRIC, metricFilters)); - } - // Create filter list based on config filters and add it to - // listBasedOnFilters. - TimelineFilterList configFilters = filters.getConfigFilters(); - if (configFilters != null && !configFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter( - TimelineFilterUtils.createHBaseFilterList( - ApplicationColumnPrefix.CONFIG, configFilters)); - } - // Create filter list based on info filters and add it to listBasedOnFilters - TimelineFilterList infoFilters = filters.getInfoFilters(); - if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter( - TimelineFilterUtils.createHBaseFilterList( - ApplicationColumnPrefix.INFO, infoFilters)); - } - return listBasedOnFilters; - } - - /** - * Add {@link QualifierFilter} filters to filter list for each column of - * application table. - * - * @param list filter list to which qualifier filters have to be added. - */ - @Override - protected void updateFixedColumns(FilterList list) { - for (ApplicationColumn column : ApplicationColumn.values()) { - list.addFilter(new QualifierFilter(CompareOp.EQUAL, - new BinaryComparator(column.getColumnQualifierBytes()))); - } - } - - /** - * Creates a filter list which indicates that only some of the column - * qualifiers in the info column family will be returned in result. - * - * @return filter list. - * @throws IOException if any problem occurs while creating filter list. - */ - private FilterList createFilterListForColsOfInfoFamily() - throws IOException { - FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); - // Add filters for each column in entity table. - updateFixedColumns(infoFamilyColsFilter); - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // If INFO field has to be retrieved, add a filter for fetching columns - // with INFO column prefix. - if (hasField(fieldsToRetrieve, Field.INFO)) { - infoFamilyColsFilter.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, ApplicationColumnPrefix.INFO)); - } - TimelineFilterList relatesTo = getFilters().getRelatesTo(); - if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { - // If RELATES_TO field has to be retrieved, add a filter for fetching - // columns with RELATES_TO column prefix. - infoFamilyColsFilter.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, ApplicationColumnPrefix.RELATES_TO)); - } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { - // Even if fields to retrieve does not contain RELATES_TO, we still - // need to have a filter to fetch some of the column qualifiers if - // relatesTo filters are specified. relatesTo filters will then be - // matched after fetching rows from HBase. - Set relatesToCols = - TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - ApplicationColumnPrefix.RELATES_TO, relatesToCols)); - } - TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); - if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { - // If IS_RELATED_TO field has to be retrieved, add a filter for fetching - // columns with IS_RELATED_TO column prefix. - infoFamilyColsFilter.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); - } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { - // Even if fields to retrieve does not contain IS_RELATED_TO, we still - // need to have a filter to fetch some of the column qualifiers if - // isRelatedTo filters are specified. isRelatedTo filters will then be - // matched after fetching rows from HBase. - Set isRelatedToCols = - TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols)); - } - TimelineFilterList eventFilters = getFilters().getEventFilters(); - if (hasField(fieldsToRetrieve, Field.EVENTS)) { - // If EVENTS field has to be retrieved, add a filter for fetching columns - // with EVENT column prefix. - infoFamilyColsFilter.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, ApplicationColumnPrefix.EVENT)); - } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){ - // Even if fields to retrieve does not contain EVENTS, we still need to - // have a filter to fetch some of the column qualifiers on the basis of - // event filters specified. Event filters will then be matched after - // fetching rows from HBase. - Set eventCols = - TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - ApplicationColumnPrefix.EVENT, eventCols)); - } - return infoFamilyColsFilter; - } - - /** - * Exclude column prefixes via filters which are not required(based on fields - * to retrieve) from info column family. These filters are added to filter - * list which contains a filter for getting info column family. - * - * @param infoColFamilyList filter list for info column family. - */ - private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // Events not required. - if (!hasField(fieldsToRetrieve, Field.EVENTS)) { - infoColFamilyList.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT)); - } - // info not required. - if (!hasField(fieldsToRetrieve, Field.INFO)) { - infoColFamilyList.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO)); - } - // is related to not required. - if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { - infoColFamilyList.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); - } - // relates to not required. - if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { - infoColFamilyList.addFilter( - TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO)); - } - } - - /** - * Updates filter list based on fields for confs and metrics to retrieve. - * - * @param listBasedOnFields filter list based on fields. - * @throws IOException if any problem occurs while updating filter list. - */ - private void updateFilterForConfsAndMetricsToRetrieve( - FilterList listBasedOnFields) throws IOException { - TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); - // Please note that if confsToRetrieve is specified, we would have added - // CONFS to fields to retrieve in augmentParams() even if not specified. - if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { - // Create a filter list for configs. - listBasedOnFields.addFilter(TimelineFilterUtils. - createFilterForConfsOrMetricsToRetrieve( - dataToRetrieve.getConfsToRetrieve(), - ApplicationColumnFamily.CONFIGS, ApplicationColumnPrefix.CONFIG)); - } - - // Please note that if metricsToRetrieve is specified, we would have added - // METRICS to fields to retrieve in augmentParams() even if not specified. - if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { - // Create a filter list for metrics. - listBasedOnFields.addFilter(TimelineFilterUtils. - createFilterForConfsOrMetricsToRetrieve( - dataToRetrieve.getMetricsToRetrieve(), - ApplicationColumnFamily.METRICS, ApplicationColumnPrefix.METRIC)); - } - } - - @Override - protected FilterList constructFilterListBasedOnFields() throws IOException { - if (!needCreateFilterListBasedOnFields()) { - // Fetch all the columns. No need of a filter. - return null; - } - FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); - FilterList infoColFamilyList = new FilterList(); - // By default fetch everything in INFO column family. - FamilyFilter infoColumnFamily = - new FamilyFilter(CompareOp.EQUAL, - new BinaryComparator(ApplicationColumnFamily.INFO.getBytes())); - infoColFamilyList.addFilter(infoColumnFamily); - if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { - // We can fetch only some of the columns from info family. - infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); - } else { - // Exclude column prefixes in info column family which are not required - // based on fields to retrieve. - excludeFieldsFromInfoColFamily(infoColFamilyList); - } - listBasedOnFields.addFilter(infoColFamilyList); - - updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); - return listBasedOnFields; - } - - @Override - protected Result getResult(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - TimelineReaderContext context = getContext(); - ApplicationRowKey applicationRowKey = - new ApplicationRowKey(context.getClusterId(), context.getUserId(), - context.getFlowName(), context.getFlowRunId(), context.getAppId()); - byte[] rowKey = applicationRowKey.getRowKey(); - Get get = new Get(rowKey); - get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); - if (filterList != null && !filterList.getFilters().isEmpty()) { - get.setFilter(filterList); - } - return getTable().getResult(hbaseConf, conn, get); - } - - @Override - protected void validateParams() { - Preconditions.checkNotNull(getContext(), "context shouldn't be null"); - Preconditions.checkNotNull( - getDataToRetrieve(), "data to retrieve shouldn't be null"); - Preconditions.checkNotNull(getContext().getClusterId(), - "clusterId shouldn't be null"); - Preconditions.checkNotNull(getContext().getEntityType(), - "entityType shouldn't be null"); - if (isSingleEntityRead()) { - Preconditions.checkNotNull(getContext().getAppId(), - "appId shouldn't be null"); - } else { - Preconditions.checkNotNull(getContext().getUserId(), - "userId shouldn't be null"); - Preconditions.checkNotNull(getContext().getFlowName(), - "flowName shouldn't be null"); - } - } - - @Override - protected void augmentParams(Configuration hbaseConf, Connection conn) - throws IOException { - TimelineReaderContext context = getContext(); - if (isSingleEntityRead()) { - // Get flow context information from AppToFlow table. - if (context.getFlowName() == null || context.getFlowRunId() == null - || context.getUserId() == null) { - AppToFlowRowKey appToFlowRowKey = - new AppToFlowRowKey(context.getClusterId(), context.getAppId()); - FlowContext flowContext = - lookupFlowContext(appToFlowRowKey, - hbaseConf, conn); - context.setFlowName(flowContext.getFlowName()); - context.setFlowRunId(flowContext.getFlowRunId()); - context.setUserId(flowContext.getUserId()); - } - } - // Add configs/metrics to fields to retrieve if confsToRetrieve and/or - // metricsToRetrieve are specified. - getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); - if (!isSingleEntityRead()) { - createFiltersIfNull(); - } - } - - @Override - protected ResultScanner getResults(Configuration hbaseConf, - Connection conn, FilterList filterList) throws IOException { - Scan scan = new Scan(); - TimelineReaderContext context = getContext(); - // Whether or not flowRunID is null doesn't matter, the - // ApplicationRowKeyPrefix will do the right thing. - RowKeyPrefix applicationRowKeyPrefix = - new ApplicationRowKeyPrefix(context.getClusterId(), - context.getUserId(), context.getFlowName(), - context.getFlowRunId()); - scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix()); - FilterList newList = new FilterList(); - newList.addFilter(new PageFilter(getFilters().getLimit())); - if (filterList != null && !filterList.getFilters().isEmpty()) { - newList.addFilter(filterList); - } - scan.setFilter(newList); - scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); - return getTable().getResultScanner(hbaseConf, conn, scan); - } - - @Override - protected TimelineEntity parseEntity(Result result) throws IOException { - if (result == null || result.isEmpty()) { - return null; - } - TimelineEntity entity = new TimelineEntity(); - entity.setType(TimelineEntityType.YARN_APPLICATION.toString()); - String entityId = ApplicationColumn.ID.readResult(result).toString(); - entity.setId(entityId); - - TimelineEntityFilters filters = getFilters(); - // fetch created time - Long createdTime = (Long) ApplicationColumn.CREATED_TIME.readResult(result); - entity.setCreatedTime(createdTime); - - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // fetch is related to entities and match isRelatedTo filter. If isRelatedTo - // filters do not match, entity would be dropped. We have to match filters - // locally as relevant HBase filters to filter out rows on the basis of - // isRelatedTo are not set in HBase scan. - boolean checkIsRelatedTo = - !isSingleEntityRead() && filters.getIsRelatedTo() != null && - filters.getIsRelatedTo().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { - readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO, - true); - if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity, - filters.getIsRelatedTo())) { - return null; - } - if (!hasField(fieldsToRetrieve, - Field.IS_RELATED_TO)) { - entity.getIsRelatedToEntities().clear(); - } - } - - // fetch relates to entities and match relatesTo filter. If relatesTo - // filters do not match, entity would be dropped. We have to match filters - // locally as relevant HBase filters to filter out rows on the basis of - // relatesTo are not set in HBase scan. - boolean checkRelatesTo = - !isSingleEntityRead() && filters.getRelatesTo() != null && - filters.getRelatesTo().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.RELATES_TO) || - checkRelatesTo) { - readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO, - false); - if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity, - filters.getRelatesTo())) { - return null; - } - if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { - entity.getRelatesToEntities().clear(); - } - } - - // fetch info if fieldsToRetrieve contains INFO or ALL. - if (hasField(fieldsToRetrieve, Field.INFO)) { - readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false); - } - - // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. - if (hasField(fieldsToRetrieve, Field.CONFIGS)) { - readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true); - } - - // fetch events and match event filters if they exist. If event filters do - // not match, entity would be dropped. We have to match filters locally - // as relevant HBase filters to filter out rows on the basis of events - // are not set in HBase scan. - boolean checkEvents = - !isSingleEntityRead() && filters.getEventFilters() != null && - filters.getEventFilters().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { - readEvents(entity, result, ApplicationColumnPrefix.EVENT); - if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity, - filters.getEventFilters())) { - return null; - } - if (!hasField(fieldsToRetrieve, Field.EVENTS)) { - entity.getEvents().clear(); - } - } - - // fetch metrics if fieldsToRetrieve contains METRICS or ALL. - if (hasField(fieldsToRetrieve, Field.METRICS)) { - readMetrics(entity, result, ApplicationColumnPrefix.METRIC); - } - return entity; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java deleted file mode 100644 index 9ba5e38..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java +++ /dev/null @@ -1,163 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import java.io.IOException; -import java.util.Map; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.PageFilter; -import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; -import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; -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.BaseTable; -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.flow.FlowActivityColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; - -import com.google.common.base.Preconditions; - -/** - * Timeline entity reader for flow activity entities that are stored in the - * flow activity table. - */ -class FlowActivityEntityReader extends TimelineEntityReader { - private static final FlowActivityTable FLOW_ACTIVITY_TABLE = - new FlowActivityTable(); - - /** - * Used to convert Long key components to and from storage format. - */ - private final KeyConverter longKeyConverter = new LongKeyConverter(); - - - public FlowActivityEntityReader(TimelineReaderContext ctxt, - TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { - super(ctxt, entityFilters, toRetrieve, true); - } - - public FlowActivityEntityReader(TimelineReaderContext ctxt, - TimelineDataToRetrieve toRetrieve) { - super(ctxt, toRetrieve); - } - - /** - * Uses the {@link FlowActivityTable}. - */ - @Override - protected BaseTable getTable() { - return FLOW_ACTIVITY_TABLE; - } - - @Override - protected void validateParams() { - Preconditions.checkNotNull(getContext().getClusterId(), - "clusterId shouldn't be null"); - } - - @Override - protected void augmentParams(Configuration hbaseConf, Connection conn) - throws IOException { - createFiltersIfNull(); - } - - @Override - protected FilterList constructFilterListBasedOnFilters() throws IOException { - return null; - } - - @Override - protected FilterList constructFilterListBasedOnFields() { - return null; - } - - @Override - protected Result getResult(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - throw new UnsupportedOperationException( - "we don't support a single entity query"); - } - - @Override - protected ResultScanner getResults(Configuration hbaseConf, - Connection conn, FilterList filterList) throws IOException { - Scan scan = new Scan(); - String clusterId = getContext().getClusterId(); - if (getFilters().getCreatedTimeBegin() == 0L && - getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) { - // All records have to be chosen. - scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId) - .getRowKeyPrefix()); - } else { - scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters() - .getCreatedTimeEnd()).getRowKeyPrefix()); - scan.setStopRow(new FlowActivityRowKeyPrefix(clusterId, (getFilters() - .getCreatedTimeBegin() <= 0 ? 0 - : (getFilters().getCreatedTimeBegin() - 1))).getRowKeyPrefix()); - } - // use the page filter to limit the result to the page size - // the scanner may still return more than the limit; therefore we need to - // read the right number as we iterate - scan.setFilter(new PageFilter(getFilters().getLimit())); - return getTable().getResultScanner(hbaseConf, conn, scan); - } - - @Override - protected TimelineEntity parseEntity(Result result) throws IOException { - FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow()); - - Long time = rowKey.getDayTimestamp(); - String user = rowKey.getUserId(); - String flowName = rowKey.getFlowName(); - - FlowActivityEntity flowActivity = new FlowActivityEntity( - getContext().getClusterId(), time, user, flowName); - // set the id - flowActivity.setId(flowActivity.getId()); - // get the list of run ids along with the version that are associated with - // this flow on this day - Map runIdsMap = - FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter); - for (Map.Entry e : runIdsMap.entrySet()) { - Long runId = e.getKey(); - String version = (String)e.getValue(); - FlowRunEntity flowRun = new FlowRunEntity(); - flowRun.setUser(user); - flowRun.setName(flowName); - flowRun.setRunId(runId); - flowRun.setVersion(version); - // set the id - flowRun.setId(flowRun.getId()); - flowActivity.addFlowRun(flowRun); - } - - return flowActivity; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java deleted file mode 100644 index 986a28f..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java +++ /dev/null @@ -1,269 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import java.io.IOException; -import java.util.EnumSet; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.FamilyFilter; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.hbase.filter.PageFilter; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; -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.reader.filter.TimelineFilterList; -import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; -import org.apache.hadoop.yarn.webapp.BadRequestException; - -import com.google.common.base.Preconditions; - -/** - * Timeline entity reader for flow run entities that are stored in the flow run - * table. - */ -class FlowRunEntityReader extends TimelineEntityReader { - private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable(); - - public FlowRunEntityReader(TimelineReaderContext ctxt, - TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { - super(ctxt, entityFilters, toRetrieve, true); - } - - public FlowRunEntityReader(TimelineReaderContext ctxt, - TimelineDataToRetrieve toRetrieve) { - super(ctxt, toRetrieve); - } - - /** - * Uses the {@link FlowRunTable}. - */ - @Override - protected BaseTable getTable() { - return FLOW_RUN_TABLE; - } - - @Override - protected void validateParams() { - Preconditions.checkNotNull(getContext(), "context shouldn't be null"); - Preconditions.checkNotNull(getDataToRetrieve(), - "data to retrieve shouldn't be null"); - Preconditions.checkNotNull(getContext().getClusterId(), - "clusterId shouldn't be null"); - Preconditions.checkNotNull(getContext().getUserId(), - "userId shouldn't be null"); - Preconditions.checkNotNull(getContext().getFlowName(), - "flowName shouldn't be null"); - if (isSingleEntityRead()) { - Preconditions.checkNotNull(getContext().getFlowRunId(), - "flowRunId shouldn't be null"); - } - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - if (!isSingleEntityRead() && fieldsToRetrieve != null) { - for (Field field : fieldsToRetrieve) { - if (field != Field.ALL && field != Field.METRICS) { - throw new BadRequestException("Invalid field " + field - + " specified while querying flow runs."); - } - } - } - } - - @Override - protected void augmentParams(Configuration hbaseConf, Connection conn) { - // Add metrics to fields to retrieve if metricsToRetrieve is specified. - getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); - if (!isSingleEntityRead()) { - createFiltersIfNull(); - } - } - - protected FilterList constructFilterListBasedOnFilters() throws IOException { - FilterList listBasedOnFilters = new FilterList(); - // Filter based on created time range. - Long createdTimeBegin = getFilters().getCreatedTimeBegin(); - Long createdTimeEnd = getFilters().getCreatedTimeEnd(); - if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { - listBasedOnFilters.addFilter(TimelineFilterUtils - .createSingleColValueFiltersByRange(FlowRunColumn.MIN_START_TIME, - createdTimeBegin, createdTimeEnd)); - } - // Filter based on metric filters. - TimelineFilterList metricFilters = getFilters().getMetricFilters(); - if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( - FlowRunColumnPrefix.METRIC, metricFilters)); - } - return listBasedOnFilters; - } - - /** - * Add {@link QualifierFilter} filters to filter list for each column of flow - * run table. - * - * @return filter list to which qualifier filters have been added. - */ - private FilterList updateFixedColumns() { - FilterList columnsList = new FilterList(Operator.MUST_PASS_ONE); - for (FlowRunColumn column : FlowRunColumn.values()) { - columnsList.addFilter(new QualifierFilter(CompareOp.EQUAL, - new BinaryComparator(column.getColumnQualifierBytes()))); - } - return columnsList; - } - - @Override - protected FilterList constructFilterListBasedOnFields() throws IOException { - FilterList list = new FilterList(Operator.MUST_PASS_ONE); - // By default fetch everything in INFO column family. - FamilyFilter infoColumnFamily = - new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( - FlowRunColumnFamily.INFO.getBytes())); - TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); - // If multiple entities have to be retrieved, check if metrics have to be - // retrieved and if not, add a filter so that metrics can be excluded. - // Metrics are always returned if we are reading a single entity. - if (!isSingleEntityRead() - && !hasField(dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) { - FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE); - infoColFamilyList.addFilter(infoColumnFamily); - infoColFamilyList.addFilter(new QualifierFilter(CompareOp.NOT_EQUAL, - new BinaryPrefixComparator(FlowRunColumnPrefix.METRIC - .getColumnPrefixBytes("")))); - list.addFilter(infoColFamilyList); - } else { - // Check if metricsToRetrieve are specified and if they are, create a - // filter list for info column family by adding flow run tables columns - // and a list for metrics to retrieve. Pls note that fieldsToRetrieve - // will have METRICS added to it if metricsToRetrieve are specified - // (in augmentParams()). - TimelineFilterList metricsToRetrieve = - dataToRetrieve.getMetricsToRetrieve(); - if (metricsToRetrieve != null - && !metricsToRetrieve.getFilterList().isEmpty()) { - FilterList infoColFamilyList = new FilterList(); - infoColFamilyList.addFilter(infoColumnFamily); - FilterList columnsList = updateFixedColumns(); - columnsList.addFilter(TimelineFilterUtils.createHBaseFilterList( - FlowRunColumnPrefix.METRIC, metricsToRetrieve)); - infoColFamilyList.addFilter(columnsList); - list.addFilter(infoColFamilyList); - } - } - return list; - } - - @Override - protected Result getResult(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - TimelineReaderContext context = getContext(); - FlowRunRowKey flowRunRowKey = - new FlowRunRowKey(context.getClusterId(), context.getUserId(), - context.getFlowName(), context.getFlowRunId()); - byte[] rowKey = flowRunRowKey.getRowKey(); - Get get = new Get(rowKey); - get.setMaxVersions(Integer.MAX_VALUE); - if (filterList != null && !filterList.getFilters().isEmpty()) { - get.setFilter(filterList); - } - return getTable().getResult(hbaseConf, conn, get); - } - - @Override - protected ResultScanner getResults(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - Scan scan = new Scan(); - TimelineReaderContext context = getContext(); - RowKeyPrefix flowRunRowKeyPrefix = - new FlowRunRowKeyPrefix(context.getClusterId(), context.getUserId(), - context.getFlowName()); - scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix()); - FilterList newList = new FilterList(); - newList.addFilter(new PageFilter(getFilters().getLimit())); - if (filterList != null && !filterList.getFilters().isEmpty()) { - newList.addFilter(filterList); - } - scan.setFilter(newList); - scan.setMaxVersions(Integer.MAX_VALUE); - return getTable().getResultScanner(hbaseConf, conn, scan); - } - - @Override - protected TimelineEntity parseEntity(Result result) throws IOException { - TimelineReaderContext context = getContext(); - FlowRunEntity flowRun = new FlowRunEntity(); - flowRun.setUser(context.getUserId()); - flowRun.setName(context.getFlowName()); - if (isSingleEntityRead()) { - flowRun.setRunId(context.getFlowRunId()); - } else { - FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow()); - flowRun.setRunId(rowKey.getFlowRunId()); - } - - // read the start time - Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result); - if (startTime != null) { - flowRun.setStartTime(startTime.longValue()); - } - - // read the end time if available - Long endTime = (Long) FlowRunColumn.MAX_END_TIME.readResult(result); - if (endTime != null) { - flowRun.setMaxEndTime(endTime.longValue()); - } - - // read the flow version - String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result); - if (version != null) { - flowRun.setVersion(version); - } - - // read metrics if its a single entity query or if METRICS are part of - // fieldsToRetrieve. - if (isSingleEntityRead() - || hasField(getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) { - readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC); - } - - // set the id - flowRun.setId(flowRun.getId()); - return flowRun; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java deleted file mode 100644 index 4e1ab8a..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java +++ /dev/null @@ -1,648 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import java.io.IOException; -import java.util.EnumSet; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.FamilyFilter; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.hbase.filter.QualifierFilter; -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.reader.filter.TimelineFilterList; -import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; -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.entity.EntityColumn; -import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily; -import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; -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.EntityTable; -import org.apache.hadoop.yarn.webapp.NotFoundException; - -import com.google.common.base.Preconditions; - -/** - * Timeline entity reader for generic entities that are stored in the entity - * table. - */ -class GenericEntityReader extends TimelineEntityReader { - private static final EntityTable ENTITY_TABLE = new EntityTable(); - - /** - * Used to look up the flow context. - */ - private final AppToFlowTable appToFlowTable = new AppToFlowTable(); - - /** - * Used to convert strings key components to and from storage format. - */ - private final KeyConverter stringKeyConverter = - new StringKeyConverter(); - - public GenericEntityReader(TimelineReaderContext ctxt, - TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, - boolean sortedKeys) { - super(ctxt, entityFilters, toRetrieve, sortedKeys); - } - - public GenericEntityReader(TimelineReaderContext ctxt, - TimelineDataToRetrieve toRetrieve) { - super(ctxt, toRetrieve); - } - - /** - * Uses the {@link EntityTable}. - */ - protected BaseTable getTable() { - return ENTITY_TABLE; - } - - @Override - protected FilterList constructFilterListBasedOnFilters() throws IOException { - // Filters here cannot be null for multiple entity reads as they are set in - // augmentParams if null. - FilterList listBasedOnFilters = new FilterList(); - TimelineEntityFilters filters = getFilters(); - // Create filter list based on created time range and add it to - // listBasedOnFilters. - long createdTimeBegin = filters.getCreatedTimeBegin(); - long createdTimeEnd = filters.getCreatedTimeEnd(); - if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { - listBasedOnFilters.addFilter(TimelineFilterUtils - .createSingleColValueFiltersByRange(EntityColumn.CREATED_TIME, - createdTimeBegin, createdTimeEnd)); - } - // Create filter list based on metric filters and add it to - // listBasedOnFilters. - TimelineFilterList metricFilters = filters.getMetricFilters(); - if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( - EntityColumnPrefix.METRIC, metricFilters)); - } - // Create filter list based on config filters and add it to - // listBasedOnFilters. - TimelineFilterList configFilters = filters.getConfigFilters(); - if (configFilters != null && !configFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( - EntityColumnPrefix.CONFIG, configFilters)); - } - // Create filter list based on info filters and add it to listBasedOnFilters - TimelineFilterList infoFilters = filters.getInfoFilters(); - if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { - listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( - EntityColumnPrefix.INFO, infoFilters)); - } - return listBasedOnFilters; - } - - /** - * Check if we need to fetch only some of the event columns. - * - * @return true if we need to fetch some of the columns, false otherwise. - */ - private boolean fetchPartialEventCols(TimelineFilterList eventFilters, - EnumSet fieldsToRetrieve) { - return (eventFilters != null && !eventFilters.getFilterList().isEmpty() && - !hasField(fieldsToRetrieve, Field.EVENTS)); - } - - /** - * Check if we need to fetch only some of the relates_to columns. - * - * @return true if we need to fetch some of the columns, false otherwise. - */ - private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo, - EnumSet fieldsToRetrieve) { - return (relatesTo != null && !relatesTo.getFilterList().isEmpty() && - !hasField(fieldsToRetrieve, Field.RELATES_TO)); - } - - /** - * Check if we need to fetch only some of the is_related_to columns. - * - * @return true if we need to fetch some of the columns, false otherwise. - */ - private boolean fetchPartialIsRelatedToCols(TimelineFilterList isRelatedTo, - EnumSet fieldsToRetrieve) { - return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() && - !hasField(fieldsToRetrieve, Field.IS_RELATED_TO)); - } - - /** - * Check if we need to fetch only some of the columns based on event filters, - * relatesto and isrelatedto from info family. - * - * @return true, if we need to fetch only some of the columns, false if we - * need to fetch all the columns under info column family. - */ - protected boolean fetchPartialColsFromInfoFamily() { - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - TimelineEntityFilters filters = getFilters(); - return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) - || fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) - || fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), - fieldsToRetrieve); - } - - /** - * Check if we need to create filter list based on fields. We need to create a - * filter list iff all fields need not be retrieved or we have some specific - * fields or metrics to retrieve. We also need to create a filter list if we - * have relationships(relatesTo/isRelatedTo) and event filters specified for - * the query. - * - * @return true if we need to create the filter list, false otherwise. - */ - protected boolean needCreateFilterListBasedOnFields() { - TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); - // Check if all fields are to be retrieved or not. If all fields have to - // be retrieved, also check if we have some metrics or configs to - // retrieve specified for the query because then a filter list will have - // to be created. - boolean flag = - !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) - || (dataToRetrieve.getConfsToRetrieve() != null && !dataToRetrieve - .getConfsToRetrieve().getFilterList().isEmpty()) - || (dataToRetrieve.getMetricsToRetrieve() != null && !dataToRetrieve - .getMetricsToRetrieve().getFilterList().isEmpty()); - // Filters need to be checked only if we are reading multiple entities. If - // condition above is false, we check if there are relationships(relatesTo/ - // isRelatedTo) and event filters specified for the query. - if (!flag && !isSingleEntityRead()) { - TimelineEntityFilters filters = getFilters(); - flag = - (filters.getEventFilters() != null && !filters.getEventFilters() - .getFilterList().isEmpty()) - || (filters.getIsRelatedTo() != null && !filters.getIsRelatedTo() - .getFilterList().isEmpty()) - || (filters.getRelatesTo() != null && !filters.getRelatesTo() - .getFilterList().isEmpty()); - } - return flag; - } - - /** - * Add {@link QualifierFilter} filters to filter list for each column of - * entity table. - * - * @param list filter list to which qualifier filters have to be added. - */ - protected void updateFixedColumns(FilterList list) { - for (EntityColumn column : EntityColumn.values()) { - list.addFilter(new QualifierFilter(CompareOp.EQUAL, new BinaryComparator( - column.getColumnQualifierBytes()))); - } - } - - /** - * Creates a filter list which indicates that only some of the column - * qualifiers in the info column family will be returned in result. - * - * @param isApplication If true, it means operations are to be performed for - * application table, otherwise for entity table. - * @return filter list. - * @throws IOException if any problem occurs while creating filter list. - */ - private FilterList createFilterListForColsOfInfoFamily() throws IOException { - FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); - // Add filters for each column in entity table. - updateFixedColumns(infoFamilyColsFilter); - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // If INFO field has to be retrieved, add a filter for fetching columns - // with INFO column prefix. - if (hasField(fieldsToRetrieve, Field.INFO)) { - infoFamilyColsFilter - .addFilter(TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, EntityColumnPrefix.INFO)); - } - TimelineFilterList relatesTo = getFilters().getRelatesTo(); - if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { - // If RELATES_TO field has to be retrieved, add a filter for fetching - // columns with RELATES_TO column prefix. - infoFamilyColsFilter.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.EQUAL, - EntityColumnPrefix.RELATES_TO)); - } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { - // Even if fields to retrieve does not contain RELATES_TO, we still - // need to have a filter to fetch some of the column qualifiers if - // relatesTo filters are specified. relatesTo filters will then be - // matched after fetching rows from HBase. - Set relatesToCols = - TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - EntityColumnPrefix.RELATES_TO, relatesToCols)); - } - TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); - if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { - // If IS_RELATED_TO field has to be retrieved, add a filter for fetching - // columns with IS_RELATED_TO column prefix. - infoFamilyColsFilter.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.EQUAL, - EntityColumnPrefix.IS_RELATED_TO)); - } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { - // Even if fields to retrieve does not contain IS_RELATED_TO, we still - // need to have a filter to fetch some of the column qualifiers if - // isRelatedTo filters are specified. isRelatedTo filters will then be - // matched after fetching rows from HBase. - Set isRelatedToCols = - TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols)); - } - TimelineFilterList eventFilters = getFilters().getEventFilters(); - if (hasField(fieldsToRetrieve, Field.EVENTS)) { - // If EVENTS field has to be retrieved, add a filter for fetching columns - // with EVENT column prefix. - infoFamilyColsFilter - .addFilter(TimelineFilterUtils.createHBaseQualifierFilter( - CompareOp.EQUAL, EntityColumnPrefix.EVENT)); - } else if (eventFilters != null && - !eventFilters.getFilterList().isEmpty()) { - // Even if fields to retrieve does not contain EVENTS, we still need to - // have a filter to fetch some of the column qualifiers on the basis of - // event filters specified. Event filters will then be matched after - // fetching rows from HBase. - Set eventCols = - TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); - infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( - EntityColumnPrefix.EVENT, eventCols)); - } - return infoFamilyColsFilter; - } - - /** - * Exclude column prefixes via filters which are not required(based on fields - * to retrieve) from info column family. These filters are added to filter - * list which contains a filter for getting info column family. - * - * @param infoColFamilyList filter list for info column family. - */ - private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // Events not required. - if (!hasField(fieldsToRetrieve, Field.EVENTS)) { - infoColFamilyList.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, - EntityColumnPrefix.EVENT)); - } - // info not required. - if (!hasField(fieldsToRetrieve, Field.INFO)) { - infoColFamilyList.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, - EntityColumnPrefix.INFO)); - } - // is related to not required. - if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { - infoColFamilyList.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, - EntityColumnPrefix.IS_RELATED_TO)); - } - // relates to not required. - if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { - infoColFamilyList.addFilter(TimelineFilterUtils - .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, - EntityColumnPrefix.RELATES_TO)); - } - } - - /** - * Updates filter list based on fields for confs and metrics to retrieve. - * - * @param listBasedOnFields filter list based on fields. - * @throws IOException if any problem occurs while updating filter list. - */ - private void updateFilterForConfsAndMetricsToRetrieve( - FilterList listBasedOnFields) throws IOException { - TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); - // Please note that if confsToRetrieve is specified, we would have added - // CONFS to fields to retrieve in augmentParams() even if not specified. - if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { - // Create a filter list for configs. - listBasedOnFields.addFilter(TimelineFilterUtils - .createFilterForConfsOrMetricsToRetrieve( - dataToRetrieve.getConfsToRetrieve(), EntityColumnFamily.CONFIGS, - EntityColumnPrefix.CONFIG)); - } - - // Please note that if metricsToRetrieve is specified, we would have added - // METRICS to fields to retrieve in augmentParams() even if not specified. - if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { - // Create a filter list for metrics. - listBasedOnFields.addFilter(TimelineFilterUtils - .createFilterForConfsOrMetricsToRetrieve( - dataToRetrieve.getMetricsToRetrieve(), - EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC)); - } - } - - @Override - protected FilterList constructFilterListBasedOnFields() throws IOException { - if (!needCreateFilterListBasedOnFields()) { - // Fetch all the columns. No need of a filter. - return null; - } - FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); - FilterList infoColFamilyList = new FilterList(); - // By default fetch everything in INFO column family. - FamilyFilter infoColumnFamily = - new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( - EntityColumnFamily.INFO.getBytes())); - infoColFamilyList.addFilter(infoColumnFamily); - if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { - // We can fetch only some of the columns from info family. - infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); - } else { - // Exclude column prefixes in info column family which are not required - // based on fields to retrieve. - excludeFieldsFromInfoColFamily(infoColFamilyList); - } - listBasedOnFields.addFilter(infoColFamilyList); - updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); - return listBasedOnFields; - } - - /** - * Looks up flow context from AppToFlow table. - * - * @param appToFlowRowKey to identify Cluster and App Ids. - * @param hbaseConf HBase configuration. - * @param conn HBase Connection. - * @return flow context information. - * @throws IOException if any problem occurs while fetching flow information. - */ - protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey, - Configuration hbaseConf, Connection conn) throws IOException { - byte[] rowKey = appToFlowRowKey.getRowKey(); - Get get = new Get(rowKey); - Result result = appToFlowTable.getResult(hbaseConf, conn, get); - if (result != null && !result.isEmpty()) { - return new FlowContext(AppToFlowColumn.USER_ID.readResult(result) - .toString(), AppToFlowColumn.FLOW_ID.readResult(result).toString(), - ((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result)) - .longValue()); - } else { - throw new NotFoundException( - "Unable to find the context flow ID and flow run ID for clusterId=" - + appToFlowRowKey.getClusterId() + ", appId=" - + appToFlowRowKey.getAppId()); - } - } - - /** - * Encapsulates flow context information. - */ - protected static class FlowContext { - private final String userId; - private final String flowName; - private final Long flowRunId; - - public FlowContext(String user, String flowName, Long flowRunId) { - this.userId = user; - this.flowName = flowName; - this.flowRunId = flowRunId; - } - - protected String getUserId() { - return userId; - } - - protected String getFlowName() { - return flowName; - } - - protected Long getFlowRunId() { - return flowRunId; - } - } - - @Override - protected void validateParams() { - Preconditions.checkNotNull(getContext(), "context shouldn't be null"); - Preconditions.checkNotNull(getDataToRetrieve(), - "data to retrieve shouldn't be null"); - Preconditions.checkNotNull(getContext().getClusterId(), - "clusterId shouldn't be null"); - Preconditions.checkNotNull(getContext().getAppId(), - "appId shouldn't be null"); - Preconditions.checkNotNull(getContext().getEntityType(), - "entityType shouldn't be null"); - if (isSingleEntityRead()) { - Preconditions.checkNotNull(getContext().getEntityId(), - "entityId shouldn't be null"); - } - } - - @Override - protected void augmentParams(Configuration hbaseConf, Connection conn) - throws IOException { - TimelineReaderContext context = getContext(); - // In reality all three should be null or neither should be null - if (context.getFlowName() == null || context.getFlowRunId() == null - || context.getUserId() == null) { - // Get flow context information from AppToFlow table. - AppToFlowRowKey appToFlowRowKey = - new AppToFlowRowKey(context.getClusterId(), context.getAppId()); - FlowContext flowContext = - lookupFlowContext(appToFlowRowKey, hbaseConf, conn); - context.setFlowName(flowContext.flowName); - context.setFlowRunId(flowContext.flowRunId); - context.setUserId(flowContext.userId); - } - // Add configs/metrics to fields to retrieve if confsToRetrieve and/or - // metricsToRetrieve are specified. - getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); - if (!isSingleEntityRead()) { - createFiltersIfNull(); - } - } - - @Override - protected Result getResult(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - TimelineReaderContext context = getContext(); - byte[] rowKey = - new EntityRowKey(context.getClusterId(), context.getUserId(), - context.getFlowName(), context.getFlowRunId(), context.getAppId(), - context.getEntityType(), context.getEntityId()).getRowKey(); - Get get = new Get(rowKey); - get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); - if (filterList != null && !filterList.getFilters().isEmpty()) { - get.setFilter(filterList); - } - return getTable().getResult(hbaseConf, conn, get); - } - - @Override - protected ResultScanner getResults(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException { - // Scan through part of the table to find the entities belong to one app - // and one type - Scan scan = new Scan(); - TimelineReaderContext context = getContext(); - RowKeyPrefix entityRowKeyPrefix = - new EntityRowKeyPrefix(context.getClusterId(), context.getUserId(), - context.getFlowName(), context.getFlowRunId(), context.getAppId(), - context.getEntityType()); - scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix()); - scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); - if (filterList != null && !filterList.getFilters().isEmpty()) { - scan.setFilter(filterList); - } - return getTable().getResultScanner(hbaseConf, conn, scan); - } - - @Override - protected TimelineEntity parseEntity(Result result) throws IOException { - if (result == null || result.isEmpty()) { - return null; - } - TimelineEntity entity = new TimelineEntity(); - String entityType = EntityColumn.TYPE.readResult(result).toString(); - entity.setType(entityType); - String entityId = EntityColumn.ID.readResult(result).toString(); - entity.setId(entityId); - - TimelineEntityFilters filters = getFilters(); - // fetch created time - Long createdTime = (Long) EntityColumn.CREATED_TIME.readResult(result); - entity.setCreatedTime(createdTime); - - EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); - // fetch is related to entities and match isRelatedTo filter. If isRelatedTo - // filters do not match, entity would be dropped. We have to match filters - // locally as relevant HBase filters to filter out rows on the basis of - // isRelatedTo are not set in HBase scan. - boolean checkIsRelatedTo = - !isSingleEntityRead() && filters.getIsRelatedTo() != null - && filters.getIsRelatedTo().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { - readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true); - if (checkIsRelatedTo - && !TimelineStorageUtils.matchIsRelatedTo(entity, - filters.getIsRelatedTo())) { - return null; - } - if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { - entity.getIsRelatedToEntities().clear(); - } - } - - // fetch relates to entities and match relatesTo filter. If relatesTo - // filters do not match, entity would be dropped. We have to match filters - // locally as relevant HBase filters to filter out rows on the basis of - // relatesTo are not set in HBase scan. - boolean checkRelatesTo = - !isSingleEntityRead() && filters.getRelatesTo() != null - && filters.getRelatesTo().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.RELATES_TO) - || checkRelatesTo) { - readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false); - if (checkRelatesTo - && !TimelineStorageUtils.matchRelatesTo(entity, - filters.getRelatesTo())) { - return null; - } - if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { - entity.getRelatesToEntities().clear(); - } - } - - // fetch info if fieldsToRetrieve contains INFO or ALL. - if (hasField(fieldsToRetrieve, Field.INFO)) { - readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false); - } - - // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. - if (hasField(fieldsToRetrieve, Field.CONFIGS)) { - readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true); - } - - // fetch events and match event filters if they exist. If event filters do - // not match, entity would be dropped. We have to match filters locally - // as relevant HBase filters to filter out rows on the basis of events - // are not set in HBase scan. - boolean checkEvents = - !isSingleEntityRead() && filters.getEventFilters() != null - && filters.getEventFilters().getFilterList().size() > 0; - if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { - readEvents(entity, result, EntityColumnPrefix.EVENT); - if (checkEvents - && !TimelineStorageUtils.matchEventFilters(entity, - filters.getEventFilters())) { - return null; - } - if (!hasField(fieldsToRetrieve, Field.EVENTS)) { - entity.getEvents().clear(); - } - } - - // fetch metrics if fieldsToRetrieve contains METRICS or ALL. - if (hasField(fieldsToRetrieve, Field.METRICS)) { - readMetrics(entity, result, EntityColumnPrefix.METRIC); - } - return entity; - } - - /** - * Helper method for reading key-value pairs for either info or config. - * - * @param Describes the type of column prefix. - * @param entity entity to fill. - * @param result result from HBase. - * @param prefix column prefix. - * @param isConfig if true, means we are reading configs, otherwise info. - * @throws IOException if any problem is encountered while reading result. - */ - protected void readKeyValuePairs(TimelineEntity entity, Result result, - ColumnPrefix prefix, boolean isConfig) throws IOException { - // info and configuration are of type Map - Map columns = - prefix.readResults(result, stringKeyConverter); - if (isConfig) { - for (Map.Entry column : columns.entrySet()) { - entity.addConfig(column.getKey(), column.getValue().toString()); - } - } else { - entity.addInfo(columns); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java deleted file mode 100644 index 7b294a8..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java +++ /dev/null @@ -1,496 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import java.io.IOException; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -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.TimelineReader.Field; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; -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.EventColumnNameConverter; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; -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.entity.EntityColumnPrefix; - -/** - * The base class for reading and deserializing timeline entities from the - * HBase storage. Different types can be defined for different types of the - * entities that are being requested. - */ -public abstract class TimelineEntityReader { - private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class); - - private final boolean singleEntityRead; - private TimelineReaderContext context; - private TimelineDataToRetrieve dataToRetrieve; - // used only for multiple entity read mode - private TimelineEntityFilters filters; - - /** - * Main table the entity reader uses. - */ - private BaseTable table; - - /** - * Specifies whether keys for this table are sorted in a manner where entities - * can be retrieved by created time. If true, it will be sufficient to collect - * the first results as specified by the limit. Otherwise all matched entities - * will be fetched and then limit applied. - */ - private boolean sortedKeys = false; - - /** - * Used to convert strings key components to and from storage format. - */ - private final KeyConverter stringKeyConverter = - new StringKeyConverter(); - - /** - * Instantiates a reader for multiple-entity reads. - * - * @param ctxt Reader context which defines the scope in which query has to be - * made. - * @param entityFilters Filters which limit the entities returned. - * @param toRetrieve Data to retrieve for each entity. - * @param sortedKeys Specifies whether key for this table are sorted or not. - * If sorted, entities can be retrieved by created time. - */ - protected TimelineEntityReader(TimelineReaderContext ctxt, - TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve, - boolean sortedKeys) { - this.singleEntityRead = false; - this.sortedKeys = sortedKeys; - this.context = ctxt; - this.dataToRetrieve = toRetrieve; - this.filters = entityFilters; - - this.setTable(getTable()); - } - - /** - * Instantiates a reader for single-entity reads. - * - * @param ctxt Reader context which defines the scope in which query has to be - * made. - * @param toRetrieve Data to retrieve for each entity. - */ - protected TimelineEntityReader(TimelineReaderContext ctxt, - TimelineDataToRetrieve toRetrieve) { - this.singleEntityRead = true; - this.context = ctxt; - this.dataToRetrieve = toRetrieve; - - this.setTable(getTable()); - } - - /** - * Creates a {@link FilterList} based on fields, confs and metrics to - * retrieve. This filter list will be set in Scan/Get objects to trim down - * results fetched from HBase back-end storage. This is called only for - * multiple entity reads. - * - * @return a {@link FilterList} object. - * @throws IOException if any problem occurs while creating filter list. - */ - protected abstract FilterList constructFilterListBasedOnFields() - throws IOException; - - /** - * Creates a {@link FilterList} based on info, config and metric filters. This - * filter list will be set in HBase Get to trim down results fetched from - * HBase back-end storage. - * - * @return a {@link FilterList} object. - * @throws IOException if any problem occurs while creating filter list. - */ - protected abstract FilterList constructFilterListBasedOnFilters() - throws IOException; - - /** - * Combines filter lists created based on fields and based on filters. - * - * @return a {@link FilterList} object if it can be constructed. Returns null, - * if filter list cannot be created either on the basis of filters or on the - * basis of fields. - * @throws IOException if any problem occurs while creating filter list. - */ - private FilterList createFilterList() throws IOException { - FilterList listBasedOnFilters = constructFilterListBasedOnFilters(); - boolean hasListBasedOnFilters = listBasedOnFilters != null && - !listBasedOnFilters.getFilters().isEmpty(); - FilterList listBasedOnFields = constructFilterListBasedOnFields(); - boolean hasListBasedOnFields = listBasedOnFields != null && - !listBasedOnFields.getFilters().isEmpty(); - // If filter lists based on both filters and fields can be created, - // combine them in a new filter list and return it. - // If either one of them has been created, return that filter list. - // Return null, if none of the filter lists can be created. This indicates - // that no filter list needs to be added to HBase Scan as filters are not - // specified for the query or only the default view of entity needs to be - // returned. - if (hasListBasedOnFilters && hasListBasedOnFields) { - FilterList list = new FilterList(); - list.addFilter(listBasedOnFilters); - list.addFilter(listBasedOnFields); - return list; - } else if (hasListBasedOnFilters) { - return listBasedOnFilters; - } else if (hasListBasedOnFields) { - return listBasedOnFields; - } - return null; - } - - protected TimelineReaderContext getContext() { - return context; - } - - protected TimelineDataToRetrieve getDataToRetrieve() { - return dataToRetrieve; - } - - protected TimelineEntityFilters getFilters() { - return filters; - } - - /** - * Create a {@link TimelineEntityFilters} object with default values for - * filters. - */ - protected void createFiltersIfNull() { - if (filters == null) { - filters = new TimelineEntityFilters(); - } - } - - /** - * Reads and deserializes a single timeline entity from the HBase storage. - * - * @param hbaseConf HBase Configuration. - * @param conn HBase Connection. - * @return A TimelineEntity object. - * @throws IOException if there is any exception encountered while reading - * entity. - */ - public TimelineEntity readEntity(Configuration hbaseConf, Connection conn) - throws IOException { - validateParams(); - augmentParams(hbaseConf, conn); - - FilterList filterList = constructFilterListBasedOnFields(); - if (LOG.isDebugEnabled() && filterList != null) { - LOG.debug("FilterList created for get is - " + filterList); - } - Result result = getResult(hbaseConf, conn, filterList); - if (result == null || result.isEmpty()) { - // Could not find a matching row. - LOG.info("Cannot find matching entity of type " + - context.getEntityType()); - return null; - } - return parseEntity(result); - } - - /** - * Reads and deserializes a set of timeline entities from the HBase storage. - * It goes through all the results available, and returns the number of - * entries as specified in the limit in the entity's natural sort order. - * - * @param hbaseConf HBase Configuration. - * @param conn HBase Connection. - * @return a set of TimelineEntity objects. - * @throws IOException if any exception is encountered while reading entities. - */ - public Set readEntities(Configuration hbaseConf, - Connection conn) throws IOException { - validateParams(); - augmentParams(hbaseConf, conn); - - NavigableSet entities = new TreeSet<>(); - FilterList filterList = createFilterList(); - if (LOG.isDebugEnabled() && filterList != null) { - LOG.debug("FilterList created for scan is - " + filterList); - } - ResultScanner results = getResults(hbaseConf, conn, filterList); - try { - for (Result result : results) { - TimelineEntity entity = parseEntity(result); - if (entity == null) { - continue; - } - entities.add(entity); - if (!sortedKeys) { - if (entities.size() > filters.getLimit()) { - entities.pollLast(); - } - } else { - if (entities.size() == filters.getLimit()) { - break; - } - } - } - return entities; - } finally { - results.close(); - } - } - - /** - * Returns the main table to be used by the entity reader. - * - * @return A reference to the table. - */ - protected BaseTable getTable() { - return table; - } - - /** - * Validates the required parameters to read the entities. - */ - protected abstract void validateParams(); - - /** - * Sets certain parameters to defaults if the values are not provided. - * - * @param hbaseConf HBase Configuration. - * @param conn HBase Connection. - * @throws IOException if any exception is encountered while setting params. - */ - protected abstract void augmentParams(Configuration hbaseConf, - Connection conn) throws IOException; - - /** - * Fetches a {@link Result} instance for a single-entity read. - * - * @param hbaseConf HBase Configuration. - * @param conn HBase Connection. - * @param filterList filter list which will be applied to HBase Get. - * @return the {@link Result} instance or null if no such record is found. - * @throws IOException if any exception is encountered while getting result. - */ - protected abstract Result getResult(Configuration hbaseConf, Connection conn, - FilterList filterList) throws IOException; - - /** - * Fetches a {@link ResultScanner} for a multi-entity read. - * - * @param hbaseConf HBase Configuration. - * @param conn HBase Connection. - * @param filterList filter list which will be applied to HBase Scan. - * @return the {@link ResultScanner} instance. - * @throws IOException if any exception is encountered while getting results. - */ - protected abstract ResultScanner getResults(Configuration hbaseConf, - Connection conn, FilterList filterList) throws IOException; - - /** - * Parses the result retrieved from HBase backend and convert it into a - * {@link TimelineEntity} object. - * - * @param result Single row result of a Get/Scan. - * @return the TimelineEntity instance or null if the entity is - * filtered. - * @throws IOException if any exception is encountered while parsing entity. - */ - protected abstract TimelineEntity parseEntity(Result result) - throws IOException; - - /** - * Helper method for reading and deserializing {@link TimelineMetric} objects - * using the specified column prefix. The timeline metrics then are added to - * the given timeline entity. - * - * @param entity {@link TimelineEntity} object. - * @param result {@link Result} object retrieved from backend. - * @param columnPrefix Metric column prefix - * @throws IOException if any exception is encountered while reading metrics. - */ - protected void readMetrics(TimelineEntity entity, Result result, - ColumnPrefix columnPrefix) throws IOException { - NavigableMap> metricsResult = - columnPrefix.readResultsWithTimestamps( - result, stringKeyConverter); - for (Map.Entry> metricResult: - metricsResult.entrySet()) { - TimelineMetric metric = new TimelineMetric(); - metric.setId(metricResult.getKey()); - // Simply assume that if the value set contains more than 1 elements, the - // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric - TimelineMetric.Type metricType = metricResult.getValue().size() > 1 ? - TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE; - metric.setType(metricType); - metric.addValues(metricResult.getValue()); - entity.addMetric(metric); - } - } - - /** - * Checks whether the reader has been created to fetch single entity or - * multiple entities. - * - * @return true, if query is for single entity, false otherwise. - */ - public boolean isSingleEntityRead() { - return singleEntityRead; - } - - protected void setTable(BaseTable baseTable) { - this.table = baseTable; - } - - /** - * Check if we have a certain field amongst fields to retrieve. This method - * checks against {@link Field#ALL} as well because that would mean field - * passed needs to be matched. - * - * @param fieldsToRetrieve fields to be retrieved. - * @param requiredField fields to be checked in fieldsToRetrieve. - * @return true if has the required field, false otherwise. - */ - protected boolean hasField(EnumSet fieldsToRetrieve, - Field requiredField) { - return fieldsToRetrieve.contains(Field.ALL) || - fieldsToRetrieve.contains(requiredField); - } - - /** - * Create a filter list of qualifier filters based on passed set of columns. - * - * @param Describes the type of column prefix. - * @param colPrefix Column Prefix. - * @param columns set of column qualifiers. - * @return filter list. - */ - protected FilterList createFiltersFromColumnQualifiers( - ColumnPrefix colPrefix, Set columns) { - FilterList list = new FilterList(Operator.MUST_PASS_ONE); - for (String column : columns) { - // For columns which have compound column qualifiers (eg. events), we need - // to include the required separator. - byte[] compoundColQual = createColQualifierPrefix(colPrefix, column); - list.addFilter(new QualifierFilter(CompareOp.EQUAL, - new BinaryPrefixComparator(colPrefix - .getColumnPrefixBytes(compoundColQual)))); - } - return list; - } - - protected byte[] createColQualifierPrefix(ColumnPrefix colPrefix, - String column) { - if (colPrefix == ApplicationColumnPrefix.EVENT - || colPrefix == EntityColumnPrefix.EVENT) { - return new EventColumnName(column, null, null).getColumnQualifier(); - } else { - return stringKeyConverter.encode(column); - } - } - - /** - * Helper method for reading relationship. - * - * @param Describes the type of column prefix. - * @param entity entity to fill. - * @param result result from HBase. - * @param prefix column prefix. - * @param isRelatedTo if true, means relationship is to be added to - * isRelatedTo, otherwise its added to relatesTo. - * @throws IOException if any problem is encountered while reading result. - */ - protected void readRelationship(TimelineEntity entity, Result result, - ColumnPrefix prefix, boolean isRelatedTo) throws IOException { - // isRelatedTo and relatesTo are of type Map> - Map columns = - prefix.readResults(result, stringKeyConverter); - for (Map.Entry column : columns.entrySet()) { - for (String id : Separator.VALUES.splitEncoded(column.getValue() - .toString())) { - if (isRelatedTo) { - entity.addIsRelatedToEntity(column.getKey(), id); - } else { - entity.addRelatesToEntity(column.getKey(), id); - } - } - } - } - - /** - * Read events from the entity table or the application table. The column name - * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted - * if there is no info associated with the event. - * - * @param Describes the type of column prefix. - * @param entity entity to fill. - * @param result HBase Result. - * @param prefix column prefix. - * @throws IOException if any problem is encountered while reading result. - */ - protected static void readEvents(TimelineEntity entity, Result result, - ColumnPrefix prefix) throws IOException { - Map eventsMap = new HashMap<>(); - Map eventsResult = - prefix.readResults(result, new EventColumnNameConverter()); - for (Map.Entry - eventResult : eventsResult.entrySet()) { - EventColumnName eventColumnName = eventResult.getKey(); - String key = eventColumnName.getId() + - Long.toString(eventColumnName.getTimestamp()); - // Retrieve previously seen event to add to it - TimelineEvent event = eventsMap.get(key); - if (event == null) { - // First time we're seeing this event, add it to the eventsMap - event = new TimelineEvent(); - event.setId(eventColumnName.getId()); - event.setTimestamp(eventColumnName.getTimestamp()); - eventsMap.put(key, event); - } - if (eventColumnName.getInfoKey() != null) { - event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue()); - } - } - Set eventsSet = new HashSet<>(eventsMap.values()); - entity.addEvents(eventsSet); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java deleted file mode 100644 index b2a9476..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java +++ /dev/null @@ -1,89 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; -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; - -/** - * Factory methods for instantiating a timeline entity reader. - */ -public final class TimelineEntityReaderFactory { - private TimelineEntityReaderFactory() { - } - - /** - * Creates a timeline entity reader instance for reading a single entity with - * the specified input. - * - * @param context Reader context which defines the scope in which query has to - * be made. - * @param dataToRetrieve Data to retrieve for each entity. - * @return An implementation of TimelineEntityReader object - * depending on entity type. - */ - public static TimelineEntityReader createSingleEntityReader( - TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) { - // currently the types that are handled separate from the generic entity - // table are application, flow run, and flow activity entities - if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { - return new ApplicationEntityReader(context, dataToRetrieve); - } else if (TimelineEntityType. - YARN_FLOW_RUN.matches(context.getEntityType())) { - return new FlowRunEntityReader(context, dataToRetrieve); - } else if (TimelineEntityType. - YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { - return new FlowActivityEntityReader(context, dataToRetrieve); - } else { - // assume we're dealing with a generic entity read - return new GenericEntityReader(context, dataToRetrieve); - } - } - - /** - * Creates a timeline entity reader instance for reading set of entities with - * the specified input and predicates. - * - * @param context Reader context which defines the scope in which query has to - * be made. - * @param filters Filters which limit the entities returned. - * @param dataToRetrieve Data to retrieve for each entity. - * @return An implementation of TimelineEntityReader object - * depending on entity type. - */ - public static TimelineEntityReader createMultipleEntitiesReader( - TimelineReaderContext context, TimelineEntityFilters filters, - TimelineDataToRetrieve dataToRetrieve) { - // currently the types that are handled separate from the generic entity - // table are application, flow run, and flow activity entities - if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { - return new ApplicationEntityReader(context, filters, dataToRetrieve); - } else if (TimelineEntityType. - YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { - return new FlowActivityEntityReader(context, filters, dataToRetrieve); - } else if (TimelineEntityType. - YARN_FLOW_RUN.matches(context.getEntityType())) { - return new FlowRunEntityReader(context, filters, dataToRetrieve); - } else { - // assume we're dealing with a generic entity read - return new GenericEntityReader(context, filters, dataToRetrieve, false); - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java deleted file mode 100644 index 9814d6d..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Package org.apache.hadoop.yarn.server.timelineservice.storage.reader - * contains classes used to read entities from backend based on query type. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -package org.apache.hadoop.yarn.server.timelineservice.storage.reader; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java deleted file mode 100644 index 58df970..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java +++ /dev/null @@ -1,130 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.junit.Test; - -public class TestKeyConverters { - - @Test - public void testAppIdKeyConverter() { - AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); - long currentTs = System.currentTimeMillis(); - ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1); - ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2); - ApplicationId appId3 = ApplicationId.newInstance(currentTs + 300, 1); - String appIdStr1 = appId1.toString(); - String appIdStr2 = appId2.toString(); - String appIdStr3 = appId3.toString(); - byte[] appIdBytes1 = appIdKeyConverter.encode(appIdStr1); - byte[] appIdBytes2 = appIdKeyConverter.encode(appIdStr2); - byte[] appIdBytes3 = appIdKeyConverter.encode(appIdStr3); - // App ids' should be encoded in a manner wherein descending order - // is maintained. - assertTrue( - "Ordering of app ids' is incorrect", - Bytes.compareTo(appIdBytes1, appIdBytes2) > 0 - && Bytes.compareTo(appIdBytes1, appIdBytes3) > 0 - && Bytes.compareTo(appIdBytes2, appIdBytes3) > 0); - String decodedAppId1 = appIdKeyConverter.decode(appIdBytes1); - String decodedAppId2 = appIdKeyConverter.decode(appIdBytes2); - String decodedAppId3 = appIdKeyConverter.decode(appIdBytes3); - assertTrue("Decoded app id is not same as the app id encoded", - appIdStr1.equals(decodedAppId1)); - assertTrue("Decoded app id is not same as the app id encoded", - appIdStr2.equals(decodedAppId2)); - assertTrue("Decoded app id is not same as the app id encoded", - appIdStr3.equals(decodedAppId3)); - } - - @Test - public void testEventColumnNameConverter() { - String eventId = "=foo_=eve=nt="; - byte[] valSepBytes = Bytes.toBytes(Separator.VALUES.getValue()); - byte[] maxByteArr = - Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length); - byte[] ts = Bytes.add(valSepBytes, maxByteArr); - Long eventTs = Bytes.toLong(ts); - byte[] byteEventColName = - new EventColumnName(eventId, eventTs, null).getColumnQualifier(); - KeyConverter eventColumnNameConverter = - new EventColumnNameConverter(); - EventColumnName eventColName = - eventColumnNameConverter.decode(byteEventColName); - assertEquals(eventId, eventColName.getId()); - assertEquals(eventTs, eventColName.getTimestamp()); - assertNull(eventColName.getInfoKey()); - - String infoKey = "f=oo_event_in=fo=_key"; - byteEventColName = - new EventColumnName(eventId, eventTs, infoKey).getColumnQualifier(); - eventColName = eventColumnNameConverter.decode(byteEventColName); - assertEquals(eventId, eventColName.getId()); - assertEquals(eventTs, eventColName.getTimestamp()); - assertEquals(infoKey, eventColName.getInfoKey()); - } - - @Test - public void testLongKeyConverter() { - LongKeyConverter longKeyConverter = new LongKeyConverter(); - confirmLongKeyConverter(longKeyConverter, Long.MIN_VALUE); - confirmLongKeyConverter(longKeyConverter, -1234567890L); - confirmLongKeyConverter(longKeyConverter, -128L); - confirmLongKeyConverter(longKeyConverter, -127L); - confirmLongKeyConverter(longKeyConverter, -1L); - confirmLongKeyConverter(longKeyConverter, 0L); - confirmLongKeyConverter(longKeyConverter, 1L); - confirmLongKeyConverter(longKeyConverter, 127L); - confirmLongKeyConverter(longKeyConverter, 128L); - confirmLongKeyConverter(longKeyConverter, 1234567890L); - confirmLongKeyConverter(longKeyConverter, Long.MAX_VALUE); - } - - private void confirmLongKeyConverter(LongKeyConverter longKeyConverter, - Long testValue) { - Long decoded = longKeyConverter.decode(longKeyConverter.encode(testValue)); - assertEquals(testValue, decoded); - } - - @Test - public void testStringKeyConverter() { - StringKeyConverter stringKeyConverter = new StringKeyConverter(); - String phrase = "QuackAttack now!"; - - for (int i = 0; i < phrase.length(); i++) { - String sub = phrase.substring(i, phrase.length()); - confirmStrignKeyConverter(stringKeyConverter, sub); - confirmStrignKeyConverter(stringKeyConverter, sub + sub); - } - } - - private void confirmStrignKeyConverter(StringKeyConverter stringKeyConverter, - String testValue) { - String decoded = - stringKeyConverter.decode(stringKeyConverter.encode(testValue)); - assertEquals(testValue, decoded); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java deleted file mode 100644 index 5beb189..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java +++ /dev/null @@ -1,246 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; -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.flow.FlowActivityRowKey; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; -import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; -import org.junit.Test; - - -public class TestRowKeys { - - private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue(); - private final static byte[] QUALIFIER_SEP_BYTES = Bytes - .toBytes(QUALIFIER_SEP); - private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster"; - private final static String USER = QUALIFIER_SEP + "user"; - private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow" - + QUALIFIER_SEP; - private final static Long FLOW_RUN_ID; - private final static String APPLICATION_ID; - static { - long runid = Long.MAX_VALUE - 900L; - byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE); - byte[] byteArr = Bytes.toBytes(runid); - int sepByteLen = QUALIFIER_SEP_BYTES.length; - if (sepByteLen <= byteArr.length) { - for (int i = 0; i < sepByteLen; i++) { - byteArr[i] = (byte) (longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]); - } - } - FLOW_RUN_ID = Bytes.toLong(byteArr); - long clusterTs = System.currentTimeMillis(); - byteArr = Bytes.toBytes(clusterTs); - if (sepByteLen <= byteArr.length) { - for (int i = 0; i < sepByteLen; i++) { - byteArr[byteArr.length - sepByteLen + i] = - (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - - QUALIFIER_SEP_BYTES[i]); - } - } - clusterTs = Bytes.toLong(byteArr); - int seqId = 222; - APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString(); - } - - private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) { - int sepLen = QUALIFIER_SEP_BYTES.length; - for (int i = 0; i < sepLen; i++) { - assertTrue( - "Row key prefix not encoded properly.", - byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == - QUALIFIER_SEP_BYTES[i]); - } - } - - @Test - public void testApplicationRowKey() { - byte[] byteRowKey = - new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, - APPLICATION_ID).getRowKey(); - ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(byteRowKey); - assertEquals(CLUSTER, rowKey.getClusterId()); - assertEquals(USER, rowKey.getUserId()); - assertEquals(FLOW_NAME, rowKey.getFlowName()); - assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); - assertEquals(APPLICATION_ID, rowKey.getAppId()); - - byte[] byteRowKeyPrefix = - new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID) - .getRowKeyPrefix(); - byte[][] splits = - Separator.QUALIFIERS.split(byteRowKeyPrefix, - new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - Separator.VARIABLE_SIZE}); - assertEquals(5, splits.length); - assertEquals(0, splits[4].length); - assertEquals(FLOW_NAME, - Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); - assertEquals(FLOW_RUN_ID, - (Long) LongConverter.invertLong(Bytes.toLong(splits[3]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - - byteRowKeyPrefix = - new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME).getRowKeyPrefix(); - splits = - Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); - assertEquals(4, splits.length); - assertEquals(0, splits[3].length); - assertEquals(FLOW_NAME, - Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - } - - /** - * Tests the converters indirectly through the public methods of the - * corresponding rowkey. - */ - @Test - public void testAppToFlowRowKey() { - byte[] byteRowKey = new AppToFlowRowKey(CLUSTER, - APPLICATION_ID).getRowKey(); - AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey); - assertEquals(CLUSTER, rowKey.getClusterId()); - assertEquals(APPLICATION_ID, rowKey.getAppId()); - } - - @Test - public void testEntityRowKey() { - String entityId = "!ent!ity!!id!"; - String entityType = "entity!Type"; - byte[] byteRowKey = - new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID, - entityType, entityId).getRowKey(); - EntityRowKey rowKey = EntityRowKey.parseRowKey(byteRowKey); - assertEquals(CLUSTER, rowKey.getClusterId()); - assertEquals(USER, rowKey.getUserId()); - assertEquals(FLOW_NAME, rowKey.getFlowName()); - assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); - assertEquals(APPLICATION_ID, rowKey.getAppId()); - assertEquals(entityType, rowKey.getEntityType()); - assertEquals(entityId, rowKey.getEntityId()); - - byte[] byteRowKeyPrefix = - new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, - APPLICATION_ID, entityType).getRowKeyPrefix(); - byte[][] splits = - Separator.QUALIFIERS.split( - byteRowKeyPrefix, - new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE}); - assertEquals(7, splits.length); - assertEquals(0, splits[6].length); - assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4])); - assertEquals(entityType, - Separator.QUALIFIERS.decode(Bytes.toString(splits[5]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - - byteRowKeyPrefix = - new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, - APPLICATION_ID).getRowKeyPrefix(); - splits = - Separator.QUALIFIERS.split( - byteRowKeyPrefix, - new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE}); - assertEquals(6, splits.length); - assertEquals(0, splits[5].length); - AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); - assertEquals(APPLICATION_ID, appIdKeyConverter.decode(splits[4])); - verifyRowPrefixBytes(byteRowKeyPrefix); - } - - @Test - public void testFlowActivityRowKey() { - Long ts = 1459900830000L; - Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); - byte[] byteRowKey = - new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey(); - FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey); - assertEquals(CLUSTER, rowKey.getClusterId()); - assertEquals(dayTimestamp, rowKey.getDayTimestamp()); - assertEquals(USER, rowKey.getUserId()); - assertEquals(FLOW_NAME, rowKey.getFlowName()); - - byte[] byteRowKeyPrefix = - new FlowActivityRowKeyPrefix(CLUSTER).getRowKeyPrefix(); - byte[][] splits = - Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); - assertEquals(2, splits.length); - assertEquals(0, splits[1].length); - assertEquals(CLUSTER, - Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - - byteRowKeyPrefix = - new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix(); - splits = - Separator.QUALIFIERS.split(byteRowKeyPrefix, - new int[] {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - Separator.VARIABLE_SIZE}); - assertEquals(3, splits.length); - assertEquals(0, splits[2].length); - assertEquals(CLUSTER, - Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); - assertEquals(ts, - (Long) LongConverter.invertLong(Bytes.toLong(splits[1]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - } - - @Test - public void testFlowRunRowKey() { - byte[] byteRowKey = - new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey(); - FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(byteRowKey); - assertEquals(CLUSTER, rowKey.getClusterId()); - assertEquals(USER, rowKey.getUserId()); - assertEquals(FLOW_NAME, rowKey.getFlowName()); - assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); - - byte[] byteRowKeyPrefix = - new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null).getRowKey(); - byte[][] splits = - Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); - assertEquals(4, splits.length); - assertEquals(0, splits[3].length); - assertEquals(FLOW_NAME, - Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); - verifyRowPrefixBytes(byteRowKeyPrefix); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java deleted file mode 100644 index 7d37206..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java +++ /dev/null @@ -1,215 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.yarn.server.timelineservice.storage.common; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; - -import com.google.common.collect.Iterables; - -public class TestSeparator { - - private static String villain = "Dr. Heinz Doofenshmirtz"; - private static String special = - ". * | ? + \t ( ) [ ] { } ^ $ \\ \" %"; - - /** - * - */ - @Test - public void testEncodeDecodeString() { - - for (Separator separator : Separator.values()) { - testEncodeDecode(separator, ""); - testEncodeDecode(separator, " "); - testEncodeDecode(separator, "!"); - testEncodeDecode(separator, "?"); - testEncodeDecode(separator, "&"); - testEncodeDecode(separator, "+"); - testEncodeDecode(separator, "\t"); - testEncodeDecode(separator, "Dr."); - testEncodeDecode(separator, "Heinz"); - testEncodeDecode(separator, "Doofenshmirtz"); - testEncodeDecode(separator, villain); - testEncodeDecode(separator, special); - - assertNull(separator.encode(null)); - - } - } - - private void testEncodeDecode(Separator separator, String token) { - String encoded = separator.encode(token); - String decoded = separator.decode(encoded); - String msg = "token:" + token + " separator:" + separator + "."; - assertEquals(msg, token, decoded); - } - - @Test - public void testEncodeDecode() { - testEncodeDecode("Dr.", Separator.QUALIFIERS); - testEncodeDecode("Heinz", Separator.QUALIFIERS, Separator.QUALIFIERS); - testEncodeDecode("Doofenshmirtz", Separator.QUALIFIERS, null, - Separator.QUALIFIERS); - testEncodeDecode("&Perry", Separator.QUALIFIERS, Separator.VALUES, null); - testEncodeDecode("the ", Separator.QUALIFIERS, Separator.SPACE); - testEncodeDecode("Platypus...", (Separator) null); - testEncodeDecode("The what now ?!?", Separator.QUALIFIERS, - Separator.VALUES, Separator.SPACE); - - } - @Test - public void testEncodedValues() { - testEncodeDecode("Double-escape %2$ and %9$ or %%2$ or %%3$, nor %%%2$" + - "= no problem!", - Separator.QUALIFIERS, Separator.VALUES, Separator.SPACE, Separator.TAB); - } - - @Test - public void testSplits() { - byte[] maxLongBytes = Bytes.toBytes(Long.MAX_VALUE); - byte[] maxIntBytes = Bytes.toBytes(Integer.MAX_VALUE); - for (Separator separator : Separator.values()) { - String str1 = "cl" + separator.getValue() + "us"; - String str2 = separator.getValue() + "rst"; - byte[] sepByteArr = Bytes.toBytes(separator.getValue()); - byte[] longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, - sepByteArr.length, Bytes.SIZEOF_LONG - sepByteArr.length)); - byte[] intVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxIntBytes, - sepByteArr.length, Bytes.SIZEOF_INT - sepByteArr.length)); - byte[] arr = separator.join( - Bytes.toBytes(separator.encode(str1)), longVal1Arr, - Bytes.toBytes(separator.encode(str2)), intVal1Arr); - int[] sizes = {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, - Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT}; - byte[][] splits = separator.split(arr, sizes); - assertEquals(4, splits.length); - assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); - assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); - assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); - assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); - - longVal1Arr = Bytes.add(Bytes.copy(maxLongBytes, 0, Bytes.SIZEOF_LONG - - sepByteArr.length), sepByteArr); - intVal1Arr = Bytes.add(Bytes.copy(maxIntBytes, 0, Bytes.SIZEOF_INT - - sepByteArr.length), sepByteArr); - arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, - Bytes.toBytes(separator.encode(str2)), intVal1Arr); - splits = separator.split(arr, sizes); - assertEquals(4, splits.length); - assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); - assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); - assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); - assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); - - longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, - sepByteArr.length, 4 - sepByteArr.length), sepByteArr); - longVal1Arr = Bytes.add(longVal1Arr, Bytes.copy(maxLongBytes, 4, 3 - - sepByteArr.length), sepByteArr); - arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, - Bytes.toBytes(separator.encode(str2)), intVal1Arr); - splits = separator.split(arr, sizes); - assertEquals(4, splits.length); - assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); - assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); - assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); - assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); - - arr = separator.join(Bytes.toBytes(separator.encode(str1)), - Bytes.toBytes(separator.encode(str2)), intVal1Arr, longVal1Arr); - int[] sizes1 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG}; - splits = separator.split(arr, sizes1); - assertEquals(4, splits.length); - assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); - assertEquals(str2, separator.decode(Bytes.toString(splits[1]))); - assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[2])); - assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[3])); - - try { - int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, - Bytes.SIZEOF_INT, 7}; - splits = separator.split(arr, sizes2); - fail("Exception should have been thrown."); - } catch (IllegalArgumentException e) {} - - try { - int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2, - Bytes.SIZEOF_LONG}; - splits = separator.split(arr, sizes2); - fail("Exception should have been thrown."); - } catch (IllegalArgumentException e) {} - } - } - - /** - * Simple test to encode and decode using the same separators and confirm that - * we end up with the same as what we started with. - * - * @param token - * @param separators - */ - private static void testEncodeDecode(String token, Separator... separators) { - byte[] encoded = Separator.encode(token, separators); - String decoded = Separator.decode(encoded, separators); - assertEquals(token, decoded); - } - - @Test - public void testJoinStripped() { - List stringList = new ArrayList(0); - stringList.add("nothing"); - - String joined = Separator.VALUES.joinEncoded(stringList); - Iterable split = Separator.VALUES.splitEncoded(joined); - assertTrue(Iterables.elementsEqual(stringList, split)); - - stringList = new ArrayList(3); - stringList.add("a"); - stringList.add("b?"); - stringList.add("c"); - - joined = Separator.VALUES.joinEncoded(stringList); - split = Separator.VALUES.splitEncoded(joined); - assertTrue(Iterables.elementsEqual(stringList, split)); - - String[] stringArray1 = {"else"}; - joined = Separator.VALUES.joinEncoded(stringArray1); - split = Separator.VALUES.splitEncoded(joined); - assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray1), split)); - - String[] stringArray2 = {"d", "e?", "f"}; - joined = Separator.VALUES.joinEncoded(stringArray2); - split = Separator.VALUES.splitEncoded(joined); - assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray2), split)); - - List empty = new ArrayList(0); - split = Separator.VALUES.splitEncoded(null); - assertTrue(Iterables.elementsEqual(empty, split)); - - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml index a35e7c5..c9df9e5 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml @@ -43,6 +43,7 @@ hadoop-yarn-server-applicationhistoryservice hadoop-yarn-server-timeline-pluginstorage hadoop-yarn-server-timelineservice + hadoop-yarn-server-timelineservice-hbase hadoop-yarn-server-timelineservice-hbase-tests diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md index 6b7bd08..bcdb0d0 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md @@ -172,7 +172,7 @@ Once you have an Apache HBase cluster ready to use for this purpose, perform the First, add the timeline service jar to the HBase classpath in all HBase machines in the cluster. It is needed for the coprocessor as well as the schema creator. For example, - cp hadoop-yarn-server-timelineservice-3.0.0-alpha1-SNAPSHOT.jar /usr/hbase/lib/ + cp hadoop-yarn-server-timelineservice-hbase-3.0.0-alpha1-SNAPSHOT.jar /usr/hbase/lib/ Then, enable the coprocessor that handles the aggregation. To enable it, add the following entry in region servers' `hbase-site.xml` file (generally located in the `conf` directory) as follows: diff --git hadoop-yarn-project/pom.xml hadoop-yarn-project/pom.xml index a20ef55..abade34 100644 --- hadoop-yarn-project/pom.xml +++ hadoop-yarn-project/pom.xml @@ -74,6 +74,11 @@ org.apache.hadoop hadoop-yarn-server-web-proxy
+ + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase + +