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..93d809c
--- /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,167 @@
+/**
+ * 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;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * 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 the input table.
+ *
+ * @param conf HBase configuration from which table name will be fetched.
+ * @param tableName name of the table to be fetched
+ * @return A {@link TableName} object.
+ */
+ public static TableName getTableName(Configuration conf, String tableName) {
+ String tableSchemaPrefix = conf.get(
+ YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX);
+ return TableName.valueOf(tableSchemaPrefix + tableName);
+ }
+
+ /**
+ * Get the table name for this table.
+ *
+ * @param conf HBase configuration from which table name will be fetched.
+ * @return A {@link TableName} object.
+ */
+ public TableName getTableName(Configuration conf) {
+ String tableName = conf.get(tableNameConfName, defaultTableName);
+ return getTableName(conf, tableName);
+ }
+
+ /**
+ * Get the table name based on the input config parameters.
+ *
+ * @param conf HBase configuration from which table name will be fetched.
+ * @param tableNameInConf the table name parameter in conf.
+ * @param defaultTableName the default table name.
+ * @return A {@link TableName} object.
+ */
+ public static TableName getTableName(Configuration conf,
+ String tableNameInConf, String defaultTableName) {
+ String tableName = conf.get(tableNameInConf, defaultTableName);
+ return getTableName(conf, tableName);
+ }
+
+ /**
+ * 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 extends Mutation> 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..afe4d6a
--- /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,306 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+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 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;
+ }
+
+ /**
+ * 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;
+ }
+
+ /**
+ * @param conf Yarn configuration. Used to see if there is an explicit config
+ * pointing to the HBase config file to read. If null then a new
+ * HBase configuration will be returned.
+ * @return a configuration with the HBase configuration from the classpath,
+ * optionally overwritten by the timeline service configuration URL if
+ * specified.
+ * @throws MalformedURLException if a timeline service HBase configuration URL
+ * is specified but is a malformed URL.
+ */
+ public static Configuration getTimelineServiceHBaseConf(Configuration conf)
+ throws MalformedURLException {
+ Configuration hbaseConf;
+
+ if (conf == null) {
+ return HBaseConfiguration.create();
+ }
+
+ String timelineServiceHBaseConfFileURL =
+ conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
+ if (timelineServiceHBaseConfFileURL != null
+ && timelineServiceHBaseConfFileURL.length() > 0) {
+ // create a clone so that we don't mess with out input one
+ hbaseConf = new Configuration(conf);
+ Configuration plainHBaseConf = new Configuration(false);
+ URL hbaseSiteXML = new URL(timelineServiceHBaseConfFileURL);
+ plainHBaseConf.addResource(hbaseSiteXML);
+ HBaseConfiguration.merge(hbaseConf, plainHBaseConf);
+ } else {
+ // default to what is on the classpath
+ hbaseConf = HBaseConfiguration.create(conf);
+ }
+ return hbaseConf;
+ }
+
+ /**
+ * Given a row key prefix stored in a byte array, return a byte array for its
+ * immediate next row key.
+ *
+ * @param rowKeyPrefix The provided row key prefix, represented in an array.
+ * @return the closest next row key of the provided row key.
+ */
+ public static byte[] calculateTheClosestNextRowKeyForPrefix(
+ byte[] rowKeyPrefix) {
+ // Essentially we are treating it like an 'unsigned very very long' and
+ // doing +1 manually.
+ // Search for the place where the trailing 0xFFs start
+ int offset = rowKeyPrefix.length;
+ while (offset > 0) {
+ if (rowKeyPrefix[offset - 1] != (byte) 0xFF) {
+ break;
+ }
+ offset--;
+ }
+
+ if (offset == 0) {
+ // We got an 0xFFFF... (only FFs) stopRow value which is
+ // the last possible prefix before the end of the table.
+ // So set it to stop at the 'end of the table'
+ return HConstants.EMPTY_END_ROW;
+ }
+
+ // Copy the right length of the original
+ byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset);
+ // And increment the last one
+ newStopRow[newStopRow.length - 1]++;
+ return newStopRow;
+ }
+
+ /**
+ * Checks if passed object is of integral type(Short/Integer/Long).
+ *
+ * @param obj Object to be checked.
+ * @return true if object passed is of type Short or Integer or Long, false
+ * otherwise.
+ */
+ public static boolean isIntegralValue(Object obj) {
+ return (obj instanceof Short) || (obj instanceof Integer) ||
+ (obj instanceof Long);
+ }
+}
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..10aeec4
--- /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,239 @@
+/**
+ * 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 long entityIdPrefix;
+ private final String entityId;
+ private final KeyConverter entityRowKeyConverter =
+ new EntityRowKeyConverter();
+
+ public EntityRowKey(String clusterId, String userId, String flowName,
+ Long flowRunId, String appId, String entityType, long entityIdPrefix,
+ String entityId) {
+ this.clusterId = clusterId;
+ this.userId = userId;
+ this.flowName = flowName;
+ this.flowRunId = flowRunId;
+ this.appId = appId;
+ this.entityType = entityType;
+ this.entityIdPrefix = entityIdPrefix;
+ 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;
+ }
+
+ public long getEntityIdPrefix() {
+ return entityIdPrefix;
+ }
+
+ /**
+ * 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,
+ Bytes.SIZEOF_LONG, 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[] enitityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix());
+
+ byte[] entityId =
+ rowKey.getEntityId() == null ? Separator.EMPTY_BYTES : Separator
+ .encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
+ Separator.QUALIFIERS);
+ byte[] fourth =
+ Separator.QUALIFIERS.join(entityType, enitityIdPrefix, 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 != 8) {
+ 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);
+
+ long entityPrefixId = Bytes.toLong(rowKeyComponents[6]);
+
+ String entityId =
+ Separator.decode(Bytes.toString(rowKeyComponents[7]),
+ Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+ return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
+ entityType, entityPrefixId, 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..ef717c0
--- /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,76 @@
+/**
+ * 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) {
+ // TODO YARN-5585, change prefix id from 0L
+ super(clusterId, userId, flowName, flowRunId, appId, entityType, 0L, 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) {
+ // TODO YARN-5585, change prefix id from 0L
+ super(clusterId, userId, flowName, flowRunId, appId, null, 0L, 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..027c8d5
--- /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 | |
+ * | idPrefix! | | @timestamp2 | |
+ * | entityId | 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..122d399
--- /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 = FlowRunTable.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