YARN-7931. [atsv2 read acls] Include domain table creation as part of schema 
creator. (Vrushali C via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b8597858
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b8597858
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b8597858

Branch: refs/heads/HDFS-7240
Commit: b8597858b17e40a99611e3a384cdd241293af83f
Parents: 113af12
Author: Haibo Chen <haiboc...@apache.org>
Authored: Thu Apr 12 06:38:30 2018 -0700
Committer: Haibo Chen <haiboc...@apache.org>
Committed: Thu Apr 12 06:38:30 2018 -0700

----------------------------------------------------------------------
 .../storage/TimelineSchemaCreator.java          |  10 ++
 .../storage/domain/DomainTableRW.java           |  92 ++++++++++
 .../storage/domain/package-info.java            |  28 +++
 .../storage/domain/DomainColumn.java            | 111 ++++++++++++
 .../storage/domain/DomainColumnFamily.java      |  52 ++++++
 .../storage/domain/DomainRowKey.java            | 179 +++++++++++++++++++
 .../storage/domain/DomainTable.java             |  45 +++++
 .../storage/domain/package-info.java            |  28 +++
 .../storage/common/TestRowKeys.java             |  32 ++++
 9 files changed, 577 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index e9e4770..37ed50c 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -44,6 +44,7 @@ import 
org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableR
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTableRW;
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTableRW;
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW;
+import 
org.apache.hadoop.yarn.server.timelineservice.storage.domain.DomainTableRW;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -357,6 +358,15 @@ public final class TimelineSchemaCreator {
           throw e;
         }
       }
+      try {
+        new DomainTableRW().createTable(admin, hbaseConf);
+      } catch (IOException e) {
+        if (skipExisting) {
+          LOG.warn("Skip and continue on: " + e.getMessage());
+        } else {
+          throw e;
+        }
+      }
     } finally {
       if (conn != null) {
         conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTableRW.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTableRW.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTableRW.java
new file mode 100644
index 0000000..1d58e40
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTableRW.java
@@ -0,0 +1,92 @@
+/**
+ * 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.domain;
+
+
+import java.io.IOException;
+
+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.BaseTableRW;
+import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Create, read and write to the domain Table.
+ */
+public class DomainTableRW extends BaseTableRW<DomainTable> {
+  /** domain prefix. */
+  private static final String PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "domain";
+
+  /** config param name that specifies the domain table name. */
+  public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
+
+  /** default value for domain table name. */
+  private static final String DEFAULT_TABLE_NAME = "timelineservice.domain";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DomainTableRW.class);
+
+  public DomainTableRW() {
+    super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTableRW#
+   * 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 domainTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor mappCF =
+        new HColumnDescriptor(DomainColumnFamily.INFO.getBytes());
+    mappCF.setBloomFilterType(BloomType.ROWCOL);
+    domainTableDescp.addFamily(mappCF);
+
+    domainTableDescp
+        .setRegionSplitPolicyClassName(
+            "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy");
+    domainTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length",
+        TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
+    admin.createTable(domainTableDescp,
+        TimelineHBaseSchemaConstants.getUsernameSplits());
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
new file mode 100644
index 0000000..49882e6
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/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.domain 
contains
+ * a set of classes used for the domain table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.domain;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumn.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumn.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumn.java
new file mode 100644
index 0000000..73233af
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumn.java
@@ -0,0 +1,111 @@
+/**
+ * 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.domain;
+
+
+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.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+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 DomainTable}.
+ */
+public enum DomainColumn implements Column<DomainTable> {
+
+  /**
+   * The created time.
+   */
+  CREATED_TIME(DomainColumnFamily.INFO, "created_time"),
+
+  /**
+   * The description of the domain.
+   */
+  DESCRIPTION(DomainColumnFamily.INFO, "description"),
+
+  /**
+   * The modification time.
+   */
+  MODIFICATION_TIME(DomainColumnFamily.INFO, "modification_time"),
+
+  /**
+   * The owner.
+   */
+  OWNER(DomainColumnFamily.INFO, "owner"),
+
+  /**
+   * The readers.
+   */
+  READERS(DomainColumnFamily.INFO, "readers"),
+
+  /**
+   * The Writers.
+   */
+  WRITERS(DomainColumnFamily.INFO, "writers");
+
+
+  private final ColumnFamily<DomainTable> columnFamily;
+  private final String columnQualifier;
+  private final byte[] columnQualifierBytes;
+  private final ValueConverter valueConverter;
+
+  DomainColumn(ColumnFamily<DomainTable> 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.valueConverter = GenericConverter.getInstance();
+  }
+
+  /**
+   * @return the column name value
+   */
+  private String getColumnQualifier() {
+    return columnQualifier;
+  }
+
+  @Override
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return valueConverter;
+  }
+
+  @Override
+  public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) {
+    return attributes;
+  }
+
+  @Override
+  public boolean supplementCellTimestamp() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumnFamily.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumnFamily.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumnFamily.java
new file mode 100644
index 0000000..276e1ad
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainColumnFamily.java
@@ -0,0 +1,52 @@
+/**
+ * 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.domain;
+
+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 domain table column families.
+ */
+public enum DomainColumnFamily implements ColumnFamily<DomainTable> {
+  /**
+   * Info column family houses known columns such as created_time, owners,
+   * readers.
+   */
+  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.
+   */
+  DomainColumnFamily(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/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainRowKey.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainRowKey.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainRowKey.java
new file mode 100644
index 0000000..6ffaef7
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainRowKey.java
@@ -0,0 +1,179 @@
+/**
+ * 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.domain;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import 
org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import 
org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
+
+/**
+ * Represents a row key for the domain table, which is the
+ * cluster ! domain id.
+ */
+public class DomainRowKey {
+  private final String clusterId;
+  private final String domainId;
+  private final DomainRowKeyConverter domainIdKeyConverter =
+      new DomainRowKeyConverter();
+
+  public DomainRowKey(String clusterId, String domainId) {
+    this.clusterId = clusterId;
+    this.domainId = domainId;
+  }
+
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public String getDomainId() {
+    return domainId;
+  }
+
+  /**
+   * Constructs a row key prefix for the domain table.
+   *
+   * @return byte array with the row key
+   */
+  public  byte[] getRowKey() {
+
+    return domainIdKeyConverter.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 <cite>DomainRowKey</cite> object.
+   */
+  public static DomainRowKey parseRowKey(byte[] rowKey) {
+    return new DomainRowKeyConverter().decode(rowKey);
+  }
+
+  /**
+   * Constructs a row key for the domain table as follows:
+   * <p>
+   * {@code clusterId!domainId}.
+   * </p>
+   * @return String representation of row key.
+   */
+  public String getRowKeyAsString() {
+    return domainIdKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the encoded row key as string, returns the row key as an object.
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>DomainRowKey</cite> object.
+   */
+  public static DomainRowKey parseRowKeyFromString(String encodedRowKey) {
+    return new DomainRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
+   * Encodes and decodes row key for the domain table.
+   * The row key is of the
+   * form : domainId
+   * <p>
+   */
+  final private static class DomainRowKeyConverter
+      implements KeyConverter<DomainRowKey>,
+      KeyConverterToString<DomainRowKey> {
+
+    private DomainRowKeyConverter() {
+    }
+
+    /**
+     * The domain row key is of the form
+     * clusterId!domainId with each segment separated by !.
+     * The sizes below indicate sizes of each one of
+     * these segements in sequence.
+     * clusterId and domainId are strings.
+     * 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,
+        Separator.VARIABLE_SIZE};
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes DomainRowKey object into a byte array
+     *
+     * @see org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(DomainRowKey rowKey) {
+      if (rowKey == null) {
+        return Separator.EMPTY_BYTES;
+      }
+
+      byte[] cluster =
+          Separator.encode(rowKey.getClusterId(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] domainIdBytes =
+          Separator.encode(rowKey.getDomainId(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+
+      return Separator.QUALIFIERS.join(cluster, domainIdBytes);
+    }
+
+    @Override
+    public DomainRowKey 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 "
+            + "a domain id");
+      }
+      String clusterId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+
+      String domainId =
+          Separator.decode(Bytes.toString(rowKeyComponents[1]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+
+      return new DomainRowKey(clusterId, domainId);
+    }
+
+    @Override
+    public String encodeAsString(DomainRowKey key) {
+      return TimelineReaderUtils.joinAndEscapeStrings(
+          new String[] {key.clusterId, key.domainId});
+    }
+
+    @Override
+    public DomainRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 2) {
+        throw new IllegalArgumentException(
+            "Invalid row key for domain id.");
+      }
+      return new DomainRowKey(split.get(0), split.get(1));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTable.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTable.java
new file mode 100644
index 0000000..c02fed3
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/DomainTable.java
@@ -0,0 +1,45 @@
+/**
+ * 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.domain;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+
+/**
+ * The domain table has column family info. Info stores
+ * information about a timeline domain object
+ *
+ * Example domain table record:
+ *
+ * <pre>
+ * |-------------------------------------------|
+ * |  Row       | Column Family                |
+ * |  key       | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | created_time:1521676928000   |
+ * | domainI    | description: "domain         |
+ * |            | information for XYZ job"     |
+ * |            | owners: "user1, yarn"        |
+ * |            | readers:                     |
+ * |            | "user2,user33 yarn,group2"   |
+ * |            |                              |
+ * |-------------------------------------------|
+ * </pre>
+ */
+public final class DomainTable extends BaseTable<DomainTable> {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/package-info.java
new file mode 100644
index 0000000..39e53fa
--- /dev/null
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/domain/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.domain
+ * contains classes related to implementation for domain table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.domain;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8597858/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
----------------------------------------------------------------------
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index d05cbad..5b7c7dc 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -32,6 +32,7 @@ import 
org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRo
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
 import 
org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey;
+import 
org.apache.hadoop.yarn.server.timelineservice.storage.domain.DomainRowKey;
 import org.junit.Test;
 
 
@@ -273,4 +274,35 @@ public class TestRowKeys {
     assertEquals(USER, rowKey.getUserId());
   }
 
+  @Test
+  public void testDomainRowKey() {
+    String clusterId = "cluster1@dc1";
+    String domainId = "helloworld";
+    byte[] byteRowKey =
+        new DomainRowKey(clusterId, domainId).getRowKey();
+    DomainRowKey rowKey = DomainRowKey.parseRowKey(byteRowKey);
+    assertEquals(clusterId, rowKey.getClusterId());
+    assertEquals(domainId, rowKey.getDomainId());
+
+    String rowKeyStr = rowKey.getRowKeyAsString();
+    DomainRowKey drk = DomainRowKey.parseRowKeyFromString(rowKeyStr);
+    assertEquals(drk.getClusterId(), rowKey.getClusterId());
+    assertEquals(drk.getDomainId(), rowKey.getDomainId());
+  }
+
+  @Test
+  public void testDomainRowKeySpecialChars() {
+    String clusterId = "cluster1!temp!dc1";
+    String domainId = "hello=world";
+    byte[] byteRowKey =
+        new DomainRowKey(clusterId, domainId).getRowKey();
+    DomainRowKey rowKey = DomainRowKey.parseRowKey(byteRowKey);
+    assertEquals(clusterId, rowKey.getClusterId());
+    assertEquals(domainId, rowKey.getDomainId());
+
+    String rowKeyStr = rowKey.getRowKeyAsString();
+    DomainRowKey drk = DomainRowKey.parseRowKeyFromString(rowKeyStr);
+    assertEquals(drk.getClusterId(), rowKey.getClusterId());
+    assertEquals(drk.getDomainId(), rowKey.getDomainId());
+  }
 }


---------------------------------------------------------------------
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