http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index f3f045e..0000000
--- 
a/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
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-
-/**
- * Represents the app_flow table column families.
- */
-public enum AppToFlowColumnFamily implements ColumnFamily<AppToFlowTable> {
-  /**
-   * 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);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
deleted file mode 100644
index 752a380..0000000
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-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.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;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Identifies partially qualified columns for the app-to-flow table.
- */
-public enum AppToFlowColumnPrefix implements ColumnPrefix<AppToFlowTable> {
-
-  /**
-   * The flow name.
-   */
-  FLOW_NAME(AppToFlowColumnFamily.MAPPING, "flow_name"),
-
-  /**
-   * The flow run ID.
-   */
-  FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"),
-
-  /**
-   * The user.
-   */
-  USER_ID(AppToFlowColumnFamily.MAPPING, "user_id");
-
-  private final ColumnHelper<AppToFlowTable> column;
-  private final ColumnFamily<AppToFlowTable> columnFamily;
-  private final String columnPrefix;
-  private final byte[] columnPrefixBytes;
-
-  AppToFlowColumnPrefix(ColumnFamily<AppToFlowTable> columnFamily,
-      String columnPrefix) {
-    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.column = new ColumnHelper<AppToFlowTable>(columnFamily);
-  }
-
-  @Override
-  public byte[] getColumnPrefixBytes(String qualifierPrefix) {
-    return ColumnHelper.getColumnQualifier(
-        columnPrefixBytes, qualifierPrefix);
-  }
-
-  @Override
-  public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
-    return ColumnHelper.getColumnQualifier(
-        columnPrefixBytes, qualifierPrefix);
-  }
-
-  @Override
-  public byte[] getColumnFamilyBytes() {
-    return columnFamily.getBytes();
-  }
-
-  @Override
-  public void store(byte[] rowKey,
-      TypedBufferedMutator<AppToFlowTable> 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);
-  }
-
-  @Override
-  public void store(byte[] rowKey,
-      TypedBufferedMutator<AppToFlowTable> 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);
-  }
-
-  @Override
-  public ValueConverter getValueConverter() {
-    return column.getValueConverter();
-  }
-
-  @Override
-  public Object readResult(Result result, String qualifier) throws IOException 
{
-    byte[] columnQualifier =
-        ColumnHelper.getColumnQualifier(columnPrefixBytes, qualifier);
-    return column.readResult(result, columnQualifier);
-  }
-
-  @Override
-  public <K> Map<K, Object> readResults(Result result,
-      KeyConverter<K> keyConverter)
-      throws IOException {
-    return column.readResults(result, columnPrefixBytes, keyConverter);
-  }
-
-  @Override
-  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result,
-      KeyConverter<K> keyConverter) throws IOException {
-    return column.readResultsWithTimestamps(result, columnPrefixBytes,
-        keyConverter);
-  }
-
-  /**
-   * Retrieve an {@link AppToFlowColumnPrefix} 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 AppToFlowColumnPrefix} or null
-   */
-  public static final AppToFlowColumnPrefix columnFor(String columnPrefix) {
-
-    // Match column based on value, assume column family matches.
-    for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
-      // Find a match based only on name.
-      if (afcp.columnPrefix.equals(columnPrefix)) {
-        return afcp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link AppToFlowColumnPrefix} 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 AppToFlowColumnPrefix} or null if both
-   *         arguments don't match.
-   */
-  public static final AppToFlowColumnPrefix columnFor(
-      AppToFlowColumnFamily columnFamily, String columnPrefix) {
-
-    // TODO: needs unit test to confirm and need to update javadoc to explain
-    // null prefix case.
-
-    for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
-      // Find a match based column family and on name.
-      if (afcp.columnFamily.equals(columnFamily)
-          && (((columnPrefix == null) && (afcp.columnPrefix == null)) ||
-          (afcp.columnPrefix.equals(columnPrefix)))) {
-        return afcp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 146c475..0000000
--- 
a/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
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
-import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-
-/**
- * Represents a row key for the app_flow table, which is the app id.
- */
-public class AppToFlowRowKey {
-  private final String appId;
-  private final KeyConverter<String> appIdKeyConverter =
-      new AppIdKeyConverter();
-
-  public AppToFlowRowKey(String appId) {
-    this.appId = appId;
-  }
-
-  public String getAppId() {
-    return appId;
-  }
-
-  /**
-   * Constructs a row key prefix for the app_flow table.
-   *
-   * @return byte array with the row key
-   */
-  public  byte[] getRowKey() {
-    return appIdKeyConverter.encode(appId);
-  }
-
-  /**
-   * 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 <cite>AppToFlowRowKey</cite> object.
-   */
-  public static AppToFlowRowKey parseRowKey(byte[] rowKey) {
-    String appId = new AppIdKeyConverter().decode(rowKey);
-    return new AppToFlowRowKey(appId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 04da5c7..0000000
--- 
a/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
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-
-import org.apache.hadoop.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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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:
- *
- * <pre>
- * |--------------------------------------|
- * |  Row       | Column Family           |
- * |  key       | mapping                 |
- * |--------------------------------------|
- * | appId      | flow_name!cluster1:     |
- * |            | foo@daily_hive_report   |
- * |            |                         |
- * |            | flow_run_id!cluster1:   |
- * |            | 1452828720457           |
- * |            |                         |
- * |            | user_id!cluster1:       |
- * |            | admin                   |
- * |            |                         |
- * |            | flow_name!cluster2:     |
- * |            | bar@ad_hoc_query        |
- * |            |                         |
- * |            | flow_run_id!cluster2:   |
- * |            | 1452828498752           |
- * |            |                         |
- * |            | user_id!cluster2:       |
- * |            | joe                     |
- * |            |                         |
- * |--------------------------------------|
- * </pre>
- *
- * It is possible (although unlikely) in a multi-cluster environment that there
- * may be more than one applications for a given app id. Different clusters are
- * recorded as different sets of columns.
- */
-public class AppToFlowTable extends BaseTable<AppToFlowTable> {
-  /** 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 Logger LOG =
-      LoggerFactory.getLogger(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));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index f01d982..0000000
--- 
a/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
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow
- * contains classes related to implementation for app to flow table.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 51604f0..0000000
--- 
a/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
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-/**
- * Encodes and decodes {@link ApplicationId} for row keys.
- * App ID is stored in row key as 12 bytes, cluster timestamp section of app id
- * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes).
- */
-public final class AppIdKeyConverter implements KeyConverter<String> {
-
-  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 HBaseTimelineStorageUtils.convertApplicationIdToString(
-        ApplicationId.newInstance(clusterTs, seqId));
-  }
-
-  /**
-   * 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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 93d809c..0000000
--- 
a/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
+++ /dev/null
@@ -1,167 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-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 <T> reference to the table instance class itself for type safety.
- */
-public abstract class BaseTable<T> {
-
-  /**
-   * 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<T> 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<T> table =
-        new BufferedMutatorDelegator<T>(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;
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index cf469a5..0000000
--- 
a/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
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.Mutation;
-
-/**
- * To be used to wrap an actual {@link BufferedMutator} in a type safe manner.
- *
- * @param <T> The class referring to the table to be written to.
- */
-class BufferedMutatorDelegator<T> implements TypedBufferedMutator<T> {
-
-  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();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 90f2de4..0000000
--- 
a/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
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * A Column represents the way to store a fully qualified column in a specific
- * table.
- */
-public interface Column<T> {
-
-  /**
-   * 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<T> 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 452adcd..0000000
--- 
a/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
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-/**
- * Type safe column family.
- *
- * @param <T> refers to the table for which this column family is used for.
- */
-public interface ColumnFamily<T> {
-
-  /**
-   * 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 9f95d44..0000000
--- 
a/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
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import org.apache.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;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is meant to be used only by explicit Columns, and not directly to
- * write by clients.
- *
- * @param <T> refers to the table.
- */
-public class ColumnHelper<T> {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ColumnHelper.class);
-
-  private final ColumnFamily<T> 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;
-
-  private final boolean supplementTs;
-
-  public ColumnHelper(ColumnFamily<T> columnFamily) {
-    this(columnFamily, GenericConverter.getInstance());
-  }
-
-  public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter) {
-    this(columnFamily, converter, false);
-  }
-
-  /**
-   * @param columnFamily column family implementation.
-   * @param converter converter use to encode/decode values stored in the 
column
-   *     or column prefix.
-   * @param needSupplementTs flag to indicate if cell timestamp needs to be
-   *     modified for this column by calling
-   *     {@link TimestampGenerator#getSupplementedTimestamp(long, String)}. 
This
-   *     would be required for columns(such as metrics in flow run table) where
-   *     potential collisions can occur due to same timestamp.
-   */
-  public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter,
-      boolean needSupplementTs) {
-    this.columnFamily = columnFamily;
-    columnFamilyBytes = columnFamily.getBytes();
-    if (converter == null) {
-      this.converter = GenericConverter.getInstance();
-    } else {
-      this.converter = converter;
-    }
-    this.supplementTs = needSupplementTs;
-  }
-
-  /**
-   * 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.
-   * Will supplement the timestamp if required. Typically done for flow run
-   * table.If we supplement the timestamp, we 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();
-    }
-    if (!this.supplementTs) {
-      return timestamp;
-    } else {
-      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<T> 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 <K> identifies the type of column name(indicated by type of key
-   *     converter).
-   * @param <V> 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 <K, V> NavigableMap<K, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result, byte[] columnPrefixBytes,
-          KeyConverter<K> keyConverter) throws IOException {
-
-    NavigableMap<K, NavigableMap<Long, V>> results = new TreeMap<>();
-
-    if (result != null) {
-      NavigableMap<
-          byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> resultMap =
-              result.getMap();
-
-      NavigableMap<byte[], NavigableMap<Long, byte[]>> columnCellMap =
-          resultMap.get(columnFamilyBytes);
-      // could be that there is no such column family.
-      if (columnCellMap != null) {
-        for (Entry<byte[], NavigableMap<Long, byte[]>> 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<Long, V> cellResults =
-                new TreeMap<Long, V>();
-            NavigableMap<Long, byte[]> cells = entry.getValue();
-            if (cells != null) {
-              for (Entry<Long, byte[]> cell : cells.entrySet()) {
-                V value =
-                    (V) converter.decodeValue(cell.getValue());
-                Long ts = supplementTs ? TimestampGenerator.
-                    getTruncatedTimestamp(cell.getKey()) : cell.getKey();
-                cellResults.put(ts, value);
-              }
-            }
-            results.put(converterColumnKey, cellResults);
-          }
-        } // for entry : columnCellMap
-      } // if columnCellMap != null
-    } // if result != null
-    return results;
-  }
-
-  /**
-   * @param <K> 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 <K> Map<K, Object> readResults(Result result,
-      byte[] columnPrefixBytes, KeyConverter<K> keyConverter)
-      throws IOException {
-    Map<K, Object> results = new HashMap<K, Object>();
-
-    if (result != null) {
-      Map<byte[], byte[]> columns = result.getFamilyMap(columnFamilyBytes);
-      for (Entry<byte[], byte[]> 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.
-   *          {@link Separator#QUALIFIERS} is permissible in the qualifier
-   *          as it is joined only with the column prefix bytes.
-   * @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;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 89aa013..0000000
--- 
a/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
+++ /dev/null
@@ -1,145 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * Used to represent a partially qualified column, where the actual column name
- * will be composed of a prefix and the remainder of the column qualifier. The
- * prefix can be null, in which case the column qualifier will be completely
- * determined when the values are stored.
- */
-public interface ColumnPrefix<T> {
-
-  /**
-   * 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<T> 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<T> 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 <K> 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.
-   */
-  <K> Map<K, Object> readResults(Result result, KeyConverter<K> keyConverter)
-      throws IOException;
-
-  /**
-   * @param result from which to reads data with timestamps.
-   * @param <K> identifies the type of key converter.
-   * @param <V> 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.
-   */
-  <K, V> NavigableMap<K, NavigableMap<Long, V>> readResultsWithTimestamps(
-      Result result, KeyConverter<K> 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index 8445575..0000000
--- 
a/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
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-/**
- * Encapsulates information about Event column names for application and entity
- * tables. Used while encoding/decoding event column names.
- */
-public class EventColumnName {
-
-  private final String id;
-  private final Long timestamp;
-  private final String infoKey;
-  private final KeyConverter<EventColumnName> 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);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index d3ef897..0000000
--- 
a/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
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Encodes and decodes event column names for application and entity tables.
- * The event column name is of the form : eventId=timestamp=infokey.
- * If info is not associated with the event, event column name is of the form :
- * eventId=timestamp=
- * Event timestamp is long and rest are strings.
- * Column prefixes are not part of the eventcolumn name passed for encoding. It
- * is added later, if required in the associated ColumnPrefix implementations.
- */
-public final class EventColumnNameConverter
-    implements KeyConverter<EventColumnName> {
-
-  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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/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
----------------------------------------------------------------------
diff --git 
a/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
 
b/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
deleted file mode 100644
index c34bfcb..0000000
--- 
a/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
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-
-/**
- * Uses GenericObjectMapper to encode objects as bytes and decode bytes as
- * objects.
- */
-public final class GenericConverter implements ValueConverter {
-  private static final GenericConverter INSTANCE = new GenericConverter();
-
-  private GenericConverter() {
-  }
-
-  public static GenericConverter getInstance() {
-    return INSTANCE;
-  }
-
-  @Override
-  public byte[] encodeValue(Object value) throws IOException {
-    return GenericObjectMapper.write(value);
-  }
-
-  @Override
-  public Object decodeValue(byte[] bytes) throws IOException {
-    return GenericObjectMapper.read(bytes);
-  }
-}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to