You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by si...@apache.org on 2021/07/28 21:24:34 UTC

[ozone] branch HDDS-4944 updated: HDDS-5485. Add new OM DB tables for AssignUserToTenant (#2453)

This is an automated email from the ASF dual-hosted git repository.

siyao pushed a commit to branch HDDS-4944
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-4944 by this push:
     new ab84d9e  HDDS-5485. Add new OM DB tables for AssignUserToTenant (#2453)
ab84d9e is described below

commit ab84d9ea4756a0d5098f4642a0001f054ab02dee
Author: Siyao Meng <50...@users.noreply.github.com>
AuthorDate: Wed Jul 28 14:24:15 2021 -0700

    HDDS-5485. Add new OM DB tables for AssignUserToTenant (#2453)
---
 .../hadoop/ozone/om/helpers/OmDBAccessIdInfo.java  | 127 +++++++++++++++++++++
 .../om/helpers/OmDBKerberosPrincipalInfo.java      | 102 +++++++++++++++++
 .../apache/hadoop/ozone/om/OMMetadataManager.java  |   6 +
 .../ozone/om/codec/OmDBAccessIdInfoCodec.java      |  55 +++++++++
 .../om/codec/OmDBKerberosPrincipalInfoCodec.java   |  58 ++++++++++
 .../hadoop/ozone/om/OmMetadataManagerImpl.java     |  48 +++++++-
 .../hadoop/ozone/om/codec/OMDBDefinition.java      |  25 +++-
 7 files changed, 416 insertions(+), 5 deletions(-)

diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBAccessIdInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBAccessIdInfo.java
new file mode 100644
index 0000000..0676c05
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBAccessIdInfo.java
@@ -0,0 +1,127 @@
+/*
+ * 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.ozone.om.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.StringUtils;
+
+/**
+ * This class is used for storing Ozone tenant accessId info.
+ */
+public final class OmDBAccessIdInfo {
+  /**
+   * Name of the tenant.
+   */
+  private final String tenantId;
+  /**
+   * Kerberos principal this accessId belongs to.
+   */
+  private final String kerberosPrincipal;
+  /**
+   * Shared secret of the accessId. TODO: Encryption?
+   */
+  private final String sharedSecret;
+
+  // This implies above String fields should NOT contain the split key.
+  public static final String SERIALIZATION_SPLIT_KEY = ";";
+
+  public OmDBAccessIdInfo(String tenantId,
+      String kerberosPrincipal, String sharedSecret) {
+    this.tenantId = tenantId;
+    this.kerberosPrincipal = kerberosPrincipal;
+    this.sharedSecret = sharedSecret;
+  }
+
+  private OmDBAccessIdInfo(String accessIdInfoString) {
+    String[] tInfo = accessIdInfoString.split(SERIALIZATION_SPLIT_KEY);
+    Preconditions.checkState(tInfo.length == 3,
+        "Incorrect accessIdInfoString");
+
+    tenantId = tInfo[0];
+    kerberosPrincipal = tInfo[1];
+    sharedSecret = tInfo[2];
+  }
+
+  public String getTenantId() {
+    return tenantId;
+  }
+
+  private String serialize() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(tenantId).append(SERIALIZATION_SPLIT_KEY);
+    sb.append(kerberosPrincipal).append(SERIALIZATION_SPLIT_KEY);
+    sb.append(sharedSecret);
+    return sb.toString();
+  }
+
+  /**
+   * Convert OmDBAccessIdInfo to byteArray to be persisted to DB.
+   * @return byte[]
+   */
+  public byte[] convertToByteArray() {
+    return StringUtils.string2Bytes(serialize());
+  }
+
+  /**
+   * Convert byte array to OmDBAccessIdInfo.
+   */
+  public static OmDBAccessIdInfo getFromByteArray(byte[] bytes) {
+    String tInfo = StringUtils.bytes2String(bytes);
+    return new OmDBAccessIdInfo(tInfo);
+  }
+
+  public String getKerberosPrincipal() {
+    return kerberosPrincipal;
+  }
+
+  public String getSharedSecret() {
+    return sharedSecret;
+  }
+
+  /**
+   * Builder for OmDBAccessIdInfo.
+   */
+  @SuppressWarnings("checkstyle:hiddenfield")
+  public static final class Builder {
+    private String tenantId;
+    private String kerberosPrincipal;
+    private String sharedSecret;
+
+    private Builder() {
+    }
+
+    public Builder setTenantName(String tenantId) {
+      this.tenantId = tenantId;
+      return this;
+    }
+
+    public Builder setKerberosPrincipal(String kerberosPrincipal) {
+      this.kerberosPrincipal = kerberosPrincipal;
+      return this;
+    }
+
+    public Builder setSharedSecret(String sharedSecret) {
+      this.sharedSecret = sharedSecret;
+      return this;
+    }
+
+    public OmDBAccessIdInfo build() {
+      return new OmDBAccessIdInfo(tenantId, kerberosPrincipal, sharedSecret);
+    }
+  }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBKerberosPrincipalInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBKerberosPrincipalInfo.java
new file mode 100644
index 0000000..8721706
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBKerberosPrincipalInfo.java
@@ -0,0 +1,102 @@
+/*
+ * 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.ozone.om.helpers;
+
+import org.apache.hadoop.hdds.StringUtils;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * This class is used for storing info related to the Kerberos principal.
+ *
+ * For now this is merely used to store a list of accessIds associates with the
+ * principal, but can be extended to store more fields later.
+ */
+public final class OmDBKerberosPrincipalInfo {
+  /**
+   * A set of accessIds.
+   */
+  private final Set<String> accessIds;
+
+  // This implies above String fields should NOT contain the split key.
+  // TODO: Reject user input accessId if it contains the split key.
+  public static final String SERIALIZATION_SPLIT_KEY = ";";
+
+  public OmDBKerberosPrincipalInfo(Set<String> accessIds) {
+    this.accessIds = new HashSet<>(accessIds);
+  }
+
+  private OmDBKerberosPrincipalInfo(String serialized) {
+    accessIds = new HashSet<>(
+        Arrays.asList(serialized.split(SERIALIZATION_SPLIT_KEY)));
+  }
+
+  public boolean addAccessId(String accessId) {
+    return accessIds.add(accessId);
+  }
+
+  public boolean removeAccessId(String accessId) {
+    return accessIds.remove(accessId);
+  }
+
+  public boolean hasAccessId(String accessId) {
+    return accessIds.contains(accessId);
+  }
+
+  private String serialize() {
+    return String.join(SERIALIZATION_SPLIT_KEY, accessIds);
+  }
+
+  /**
+   * Convert OmDBKerberosPrincipalInfo to byteArray to be persisted to DB.
+   * @return byte[]
+   */
+  public byte[] convertToByteArray() {
+    return StringUtils.string2Bytes(serialize());
+  }
+
+  /**
+   * Convert byte array to OmDBKerberosPrincipalInfo.
+   */
+  public static OmDBKerberosPrincipalInfo getFromByteArray(byte[] bytes) {
+    String tInfo = StringUtils.bytes2String(bytes);
+    return new OmDBKerberosPrincipalInfo(tInfo);
+  }
+
+  /**
+   * Builder for OmDBKerberosPrincipalInfo.
+   */
+  @SuppressWarnings("checkstyle:hiddenfield")
+  public static final class Builder {
+    private Set<String> accessIds;
+
+    private Builder() {
+    }
+
+    public Builder setAccessIds(Set<String> accessIds) {
+      this.accessIds = accessIds;
+      return this;
+    }
+
+    public OmDBKerberosPrincipalInfo build() {
+      return new OmDBKerberosPrincipalInfo(accessIds);
+    }
+  }
+}
diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
index 8b68a19..519bb08 100644
--- a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
+++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBKerberosPrincipalInfo;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
@@ -350,6 +352,10 @@ public interface OMMetadataManager extends DBStoreHAManager {
 
   Table<String, String> getTenantUserTable();
 
+  Table<String, OmDBAccessIdInfo> getTenantAccessIdTable();
+
+  Table<String, OmDBKerberosPrincipalInfo> getPrincipalToAccessIdsTable();
+
   Table<String, OmDBTenantInfo> getTenantStateTable();
 
   Table<String, String> getTenantGroupTable();
diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBAccessIdInfoCodec.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBAccessIdInfoCodec.java
new file mode 100644
index 0000000..fa8ad73
--- /dev/null
+++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBAccessIdInfoCodec.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode OmDBAccessIdInfo as byte array.
+ */
+public class OmDBAccessIdInfoCodec implements Codec<OmDBAccessIdInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBAccessIdInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(OmDBAccessIdInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.convertToByteArray();
+  }
+
+  @Override
+  public OmDBAccessIdInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return OmDBAccessIdInfo.getFromByteArray(rawData);
+  }
+
+  @Override
+  public OmDBAccessIdInfo copyObject(OmDBAccessIdInfo object) {
+    // TODO: Not really a "copy". See OMTransactionInfoCodec
+    return object;
+  }
+}
diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBKerberosPrincipalInfoCodec.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBKerberosPrincipalInfoCodec.java
new file mode 100644
index 0000000..9ddfe1a
--- /dev/null
+++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBKerberosPrincipalInfoCodec.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ozone.om.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.om.helpers.OmDBKerberosPrincipalInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode OmDBKerberosPrincipalInfo as byte array.
+ */
+public class OmDBKerberosPrincipalInfoCodec
+    implements Codec<OmDBKerberosPrincipalInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBKerberosPrincipalInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(OmDBKerberosPrincipalInfo object)
+      throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.convertToByteArray();
+  }
+
+  @Override
+  public OmDBKerberosPrincipalInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return OmDBKerberosPrincipalInfo.getFromByteArray(rawData);
+  }
+
+  @Override
+  public OmDBKerberosPrincipalInfo copyObject(
+      OmDBKerberosPrincipalInfo object) {
+    // TODO: Not really a "copy". See OMTransactionInfoCodec
+    return object;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index c703b40..8d8257b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -62,6 +62,8 @@ import org.apache.hadoop.ozone.om.codec.UserVolumeInfoCodec;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBKerberosPrincipalInfo;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
@@ -130,10 +132,19 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    * |----------------------------------------------------------------------|
    * | transactionInfoTable| #TRANSACTIONINFO -> OMTransactionInfo          |
    * |----------------------------------------------------------------------|
-   * | tenantUserTable    |  tenant user name -> a tenant (organization)    |
-   * | tenantStateTable   |  tenant user name -> OmDBTenantInfo             |
-   * | tenantGroupTable   |  tenant user name -> [tenant group A, B, ...]   |
-   * | tenantRoleTable    |  tenant user name -> roles [admin, roleB, ...]  |
+   *
+   * Multi-Tenant Tables:
+   * |----------------------------------------------------------------------|
+   * | tenantAccessIdTable|  accessId -> OmTenantAccessIdInfo               |
+   * |----------------------------------------------------------------------|
+   * | principalToAccessIdsTable | Principal -> OmDBKerberosPrincipalInfo   |
+   * |----------------------------------------------------------------------|
+   * | tenantStateTable   |  accessId -> OmDBTenantInfo                     |
+   * |----------------------------------------------------------------------|
+   * | tenantGroupTable   |  accessId -> [tenant group A, B, ...]           |
+   * |----------------------------------------------------------------------|
+   * | tenantRoleTable    |  accessId -> roles [admin, roleB, ...]          |
+   * |----------------------------------------------------------------------|
    * | tenantPolicyTable  |  policyGroup -> [policyId1, policyId2]          |
    * |----------------------------------------------------------------------|
    *
@@ -181,6 +192,9 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
 
   // Tables for S3 multi-tenancy
   public static final String TENANT_USER_TABLE = "tenantUserTable";
+  public static final String TENANT_ACCESS_ID_TABLE = "tenantAccessIdTable";
+  public static final String PRINCIPAL_TO_ACCESS_IDS_TABLE =
+      "principalToAccessIdsTable";
   public static final String TENANT_STATE_TABLE = "tenantStateTable";
   public static final String TENANT_GROUP_TABLE = "tenantGroupTable";
   public static final String TENANT_ROLE_TABLE = "tenantRoleTable";
@@ -207,6 +221,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private Table transactionInfoTable;
   // Tables for S3 multi-tenancy
   private Table tenantUserTable;
+  private Table tenantAccessIdTable;
+  private Table principalToAccessIdsTable;
   private Table tenantStateTable;
   private Table tenantGroupTable;
   private Table tenantRoleTable;
@@ -411,6 +427,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addTable(DELETED_DIR_TABLE)
         .addTable(TRANSACTION_INFO_TABLE)
         .addTable(TENANT_USER_TABLE)
+        .addTable(TENANT_ACCESS_ID_TABLE)
+        .addTable(PRINCIPAL_TO_ACCESS_IDS_TABLE)
         .addTable(TENANT_STATE_TABLE)
         .addTable(TENANT_GROUP_TABLE)
         .addTable(TENANT_ROLE_TABLE)
@@ -506,6 +524,17 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         String.class, String.class);
     checkTableStatus(tenantUserTable, TENANT_USER_TABLE);
 
+    // tenantId -> OmDBAccessIdInfo (tenantId, secret, Kerberos principal)
+    tenantAccessIdTable = this.store.getTable(TENANT_ACCESS_ID_TABLE,
+        String.class, OmDBAccessIdInfo.class);
+    checkTableStatus(tenantAccessIdTable, TENANT_ACCESS_ID_TABLE);
+
+    // Kerberos principal -> OmDBKerberosPrincipalInfo (A list of accessIds)
+    principalToAccessIdsTable = this.store.getTable(
+        PRINCIPAL_TO_ACCESS_IDS_TABLE,
+        String.class, OmDBKerberosPrincipalInfo.class);
+    checkTableStatus(principalToAccessIdsTable, PRINCIPAL_TO_ACCESS_IDS_TABLE);
+
     // tenant name -> tenant (tenant states)
     tenantStateTable = this.store.getTable(TENANT_STATE_TABLE,
         String.class, OmDBTenantInfo.class);
@@ -1266,6 +1295,17 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   }
 
   @Override
+  public Table<String, OmDBAccessIdInfo> getTenantAccessIdTable() {
+    return tenantAccessIdTable;
+  }
+
+  @Override
+  public Table<String, OmDBKerberosPrincipalInfo>
+      getPrincipalToAccessIdsTable() {
+    return principalToAccessIdsTable;
+  }
+
+  @Override
   public Table<String, OmDBTenantInfo> getTenantStateTable() {
     return tenantStateTable;
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java
index fb83e5e..c1d53e6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OMDBDefinition.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDBKerberosPrincipalInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
@@ -180,6 +182,7 @@ public class OMDBDefinition implements DBDefinition {
 
   // Tables for S3 multi-tenancy
 
+  // TODO: this table will be removed with the disappearance of CreateUser API.
   public static final DBColumnFamilyDefinition<String, String>
             TENANT_USER_TABLE =
             new DBColumnFamilyDefinition<>(
@@ -189,6 +192,25 @@ public class OMDBDefinition implements DBDefinition {
                     String.class,
                     new StringCodec());
 
+  public static final DBColumnFamilyDefinition<String, OmDBAccessIdInfo>
+            TENANT_ACCESS_ID_TABLE =
+            new DBColumnFamilyDefinition<>(
+                    OmMetadataManagerImpl.TENANT_ACCESS_ID_TABLE,
+                    String.class,  // accessId
+                    new StringCodec(),
+                    OmDBAccessIdInfo.class,  // tenantId, secret, principal
+                    new OmDBAccessIdInfoCodec());
+
+  public static final DBColumnFamilyDefinition<String,
+            OmDBKerberosPrincipalInfo>
+            PRINCIPAL_TO_ACCESS_IDS_TABLE =
+            new DBColumnFamilyDefinition<>(
+                    OmMetadataManagerImpl.PRINCIPAL_TO_ACCESS_IDS_TABLE,
+                    String.class,  // Kerberos principal
+                    new StringCodec(),
+                    OmDBKerberosPrincipalInfo.class,  // List of accessIds
+                    new OmDBKerberosPrincipalInfoCodec());
+
   public static final DBColumnFamilyDefinition<String, OmDBTenantInfo>
             TENANT_STATE_TABLE =
             new DBColumnFamilyDefinition<>(
@@ -244,7 +266,8 @@ public class OMDBDefinition implements DBDefinition {
         BUCKET_TABLE, MULTIPART_INFO_TABLE, PREFIX_TABLE, DTOKEN_TABLE,
         S3_SECRET_TABLE, TRANSACTION_INFO_TABLE, DIRECTORY_TABLE,
         FILE_TABLE, OPEN_FILE_TABLE, DELETED_DIR_TABLE,
-        TENANT_USER_TABLE, TENANT_STATE_TABLE, TENANT_GROUP_TABLE,
+        TENANT_USER_TABLE, TENANT_ACCESS_ID_TABLE,
+        PRINCIPAL_TO_ACCESS_IDS_TABLE, TENANT_STATE_TABLE, TENANT_GROUP_TABLE,
         TENANT_ROLE_TABLE, TENANT_POLICY_TABLE };
   }
 }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org