You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2023/01/10 01:23:42 UTC

[GitHub] [ozone] fapifta commented on a diff in pull request #4138: HDDS-5934. Add OM NodeID to VERSION file.

fapifta commented on code in PR #4138:
URL: https://github.com/apache/ozone/pull/4138#discussion_r1065251181


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java:
##########
@@ -32,52 +32,178 @@
 import static org.apache.hadoop.ozone.om.OmUpgradeConfig.ConfigStrings.OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION;
 
 /**
- * OMStorage is responsible for management of the StorageDirectories used by
- * the Ozone Manager.
+ * Ozone Manager VERSION file representation.
+ * On top of what is defined in the base Storage class, this class adds
+ * functionality to hold Ozone Manager related data in its VERSION file.
+ * The additional values stored:
+ * - Ozone Manager ID - a UUID that identifies this Ozone Manager.
+ *                      The value can not be changed once initialized.
+ * - Ozone Manager Node Id - the node id defined for this Ozone manager in the
+ *                           configuration. The value can not be changed after
+ *                           it was set.

Review Comment:
   Yes you are right, I am updating this doc, with a note that this one is not used anymore. It is a required part of a protocol message OzoneManagerDetailsProto in HDDSProtos, so I am hesitant to remove it fully. We can do it but that should be part of a separate JIRA if we want to, however I think as this is a UUID even if we do not use it for anything at the moment, it is still an identifier of the OM, and does not require too much maintenance so I would not remove it yet.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -459,6 +459,7 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption)
     this.omNodeDetails = omhaNodeDetails.getLocalNodeDetails();
 
     omStorage = new OMStorage(conf);
+    omStorage.validateOrPersistOmNodeId(omNodeDetails.getNodeId());

Review Comment:
   Comment on the sidenote:
   As a possible solution we can bump the layout version, and add an UpgradeAction to handle the change in the VERSION file, however I am not sure whether this is an overkill for just adding this value. Probably it is not, as we are changing the metadata, but I am still a bit hesitant to go and do it without hearing your opinion on this :)
   
   I thought adding a comment here is just redundant, as the doc of the method is pretty much trying to tell what it is doing, but anyway... why not :) I have added a code comment before the method call to explain, however if we decide to do this within the upgrade framework, this one will be meaningless anyway ;)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java:
##########
@@ -32,52 +32,178 @@
 import static org.apache.hadoop.ozone.om.OmUpgradeConfig.ConfigStrings.OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION;
 
 /**
- * OMStorage is responsible for management of the StorageDirectories used by
- * the Ozone Manager.
+ * Ozone Manager VERSION file representation.
+ * On top of what is defined in the base Storage class, this class adds
+ * functionality to hold Ozone Manager related data in its VERSION file.
+ * The additional values stored:
+ * - Ozone Manager ID - a UUID that identifies this Ozone Manager.
+ *                      The value can not be changed once initialized.
+ * - Ozone Manager Node Id - the node id defined for this Ozone manager in the
+ *                           configuration. The value can not be changed after
+ *                           it was set.
+ * - Ozone Manager Certificate Serial Id - the serial id of the Ozone Manager's
+ *                                         SSL certificate if one exists.
  */
 public class OMStorage extends Storage {
 
-  public static final String STORAGE_DIR = "om";
-  public static final String OM_ID = "omUuid";
-  public static final String OM_CERT_SERIAL_ID = "omCertSerialId";
+  static final String ERROR_OM_IS_ALREADY_INITIALIZED =
+      "OM is already initialized.";
+  static final String ERROR_UNEXPECTED_OM_NODE_ID_TEMPLATE =
+      "OM NodeId: {} does not match existing nodeId from VERSION file: {}";

Review Comment:
   Oh... my bad, I don't know what I was doing here :D Fixing it.



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMStorage.java:
##########
@@ -17,76 +17,312 @@
 package org.apache.hadoop.ozone.om;
 
 import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
 
 import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdds.conf.MutableConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.ozone.test.GenericTestUtils;
-
-import org.apache.commons.io.FileUtils;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.hadoop.ozone.common.Storage.StorageState.INITIALIZED;
+import static org.apache.hadoop.ozone.om.OMStorage.ERROR_OM_IS_ALREADY_INITIALIZED;
+import static org.apache.hadoop.ozone.om.OMStorage.ERROR_UNEXPECTED_OM_NODE_ID_TEMPLATE;
+import static org.apache.hadoop.ozone.om.OMStorage.OM_CERT_SERIAL_ID;
+import static org.apache.hadoop.ozone.om.OMStorage.OM_ID;
+import static org.apache.hadoop.ozone.om.OMStorage.OM_NODE_ID;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.fail;
 
 /**
  * Testing OMStorage class.
+ * Assumptions tested:
+ *   1. certificate serial ID can be set and unset anytime.
+ *   2. OmId the UUID of the Ozone Manager can be set only when the OMStorage
+ *       is not initialized already. Once initialized, setting OmId throws
+ *       IOException
+ *   3. OmNodeId:
+ *     3.1. can be set when the storage is not initialized, once initialize,
+ *         setting OmNodeId throws IOException
+ *     3.2. verifying the OmNodeId is possible once the storage is initialized,
+ *         until it is not initialized, verification throws IOException
+ *     3.3. verifying the OmNodeId does not do anything if the provided value is
+ *         equal to the stored value, throws an IOException otherwise
+ *   4. Configuration parsing:
+ *     4.1. getOmDbDir returns the configured
+ *         {@link OMConfigKeys#OZONE_OM_DB_DIRS} value
+ *     4.2. getOmDbDir falls back to {@link HddsConfigKeys#OZONE_METADATA_DIRS}
+ *         when {@link OMConfigKeys#OZONE_OM_DB_DIRS} is not set
+ *     4.3. getOmDbDir throws exception if none of the above properties are set
+ *   5. the protected getNodeProperties method properly returns all the keys
+ *       that are set properly in the OMStorage object.
  */
 public class TestOMStorage {
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
-  /**
-   * Test {@link OMStorage#getOmDbDir}.
-   */
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  private static final String OM_ID_STR = new UUID(1L, 1L).toString();
+
   @Test
-  public void testGetOmDbDir() {
-    final File testDir = createTestDir();
+  public void testGetOmDbDir() throws Exception {
+    final File testDir = tmpFolder.newFolder();
     final File dbDir = new File(testDir, "omDbDir");
-    final File metaDir = new File(testDir, "metaDir");   // should be ignored.
-    final MutableConfigurationSource conf = new OzoneConfiguration();
-    conf.set(OMConfigKeys.OZONE_OM_DB_DIRS, dbDir.getPath());
-    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+    final File metaDir = new File(testDir, "metaDir");
+    OzoneConfiguration conf = confWithHDDSMetaAndOMDBDir(metaDir, dbDir);
 
+    assertThat(dbDir, equalTo(OMStorage.getOmDbDir(conf)));
+    assertThat(dbDir.exists(), is(true));
+    assertThat(metaDir.exists(), is(false));
+  }
+
+  @Test
+  public void testGetOmDbDirWithFallback() throws Exception {
+    File metaDir = tmpFolder.newFolder();
+    OzoneConfiguration conf = confWithHDDSMetadataDir(metaDir);
+
+    assertThat(metaDir, equalTo(OMStorage.getOmDbDir(conf)));
+    assertThat(metaDir.exists(), is(true));
+  }
+
+  @Test
+  public void testNoOmDbDirConfigured() {
+    thrown.expect(IllegalArgumentException.class);
+    OMStorage.getOmDbDir(new OzoneConfiguration());
+  }
+
+  @Test
+  public void testSetOmIdOnNotInitializedStorage() throws Exception {
+    OMStorage storage = new OMStorage(configWithOMDBDir());
+    assertThat(storage.getState(), is(not(INITIALIZED)));
+
+    String omId = "omId";
     try {
-      assertEquals(dbDir, OMStorage.getOmDbDir(conf));
-      assertTrue(dbDir.exists());          // should have been created.
-    } finally {
-      FileUtils.deleteQuietly(dbDir);
+      storage.setOmId(omId);
+    } catch (IOException e) {
+      fail("Can not set OmId on a Storage that is not initialized.");
     }
+    assertThat(storage.getOmId(), is(omId));
+    assertGetNodeProperties(storage, omId);
   }
 
-  /**
-   * Test {@link OMStorage#getOmDbDir} with fallback to OZONE_METADATA_DIRS
-   * when OZONE_OM_DB_DIRS is undefined.
-   */
   @Test
-  public void testGetOmDbDirWithFallback() {
-    final File testDir = createTestDir();
-    final File metaDir = new File(testDir, "metaDir");
-    final MutableConfigurationSource conf = new OzoneConfiguration();
-    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+  public void testSetOmIdOnInitializedStorage() throws Exception {
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFile(conf);
+    thrown.expect(IOException.class);
+    thrown.expectMessage(ERROR_OM_IS_ALREADY_INITIALIZED);
 
+    OMStorage storage = new OMStorage(conf);
+    storage.setOmId("omId");
+  }
+
+  @Test
+  public void testCertSerialIdOperations() throws Exception {
+    OzoneConfiguration conf = configWithOMDBDir();
+    OMStorage storage = new OMStorage(conf);
+
+    assertThat(storage.getState(), is(not(INITIALIZED)));
+    assertCertOps(storage);
+    storage.initialize();
+    storage.persistCurrentState();
+
+    storage = new OMStorage(conf);
+    assertThat(storage.getState(), is(INITIALIZED));
+    assertCertOps(storage);
+  }
+
+  @Test
+  public void testSetOmNodeIdOnNotInitializedStorage() throws Exception {
+    OMStorage storage = new OMStorage(configWithOMDBDir());
+    assertThat(storage.getState(), is(not(INITIALIZED)));
+
+    String nodeId = "nodeId";
     try {
-      assertEquals(metaDir, OMStorage.getOmDbDir(conf));
-      assertTrue(metaDir.exists());        // should have been created.
-    } finally {
-      FileUtils.deleteQuietly(metaDir);
+      storage.setOmNodeId(nodeId);
+    } catch (IOException e) {
+      fail("Can not set OmNodeId on a Storage that is not initialized.");
     }
+    assertThat(storage.getOmNodeId(), is(nodeId));
+    assertGetNodeProperties(storage, null, nodeId);
   }
 
   @Test
-  public void testNoOmDbDirConfigured() {
-    thrown.expect(IllegalArgumentException.class);
-    OMStorage.getOmDbDir(new OzoneConfiguration());
+  public void testSetOMNodeIdOnInitializedStorageWithoutNodeID()
+      throws Exception {
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFile(conf);
+    thrown.expect(IOException.class);
+    thrown.expectMessage(ERROR_OM_IS_ALREADY_INITIALIZED);
+
+    OMStorage storage = new OMStorage(conf);
+    storage.setOmNodeId("nodeId");
   }
 
-  public File createTestDir() {
-    File dir = new File(GenericTestUtils.getRandomizedTestDir(),
-        TestOMStorage.class.getSimpleName());
-    dir.mkdirs();
-    return dir;
+  @Test
+  public void testSetOMNodeIdOnInitializedStorageWithNodeID() throws Exception {
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFileWithNodeId(conf, "nodeId");
+    thrown.expect(IOException.class);
+    thrown.expectMessage(ERROR_OM_IS_ALREADY_INITIALIZED);
+
+    OMStorage storage = new OMStorage(conf);
+    storage.setOmNodeId("nodeId");
   }
+
+  @Test
+  public void testValidateOrPersistOmNodeIdPersistsNewlySetValue()
+      throws Exception {
+    String nodeId = "nodeId";
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFile(conf);
+
+    OMStorage storage = new OMStorage(conf);
+    assertThat(storage.getState(), is(INITIALIZED));
+    assertThat(storage.getOmNodeId(), is(nullValue()));
+
+    storage.validateOrPersistOmNodeId(nodeId);
+    assertThat(storage.getOmNodeId(), is(nodeId));
+    assertGetNodeProperties(storage, OM_ID_STR, nodeId);
+
+    storage = new OMStorage(conf);
+    assertThat(storage.getOmNodeId(), is(nodeId));
+    assertGetNodeProperties(storage, OM_ID_STR, nodeId);
+  }
+
+  @Test
+  public void testValidateOrPersistOmNodeIdDoesRunWithSameNodeIdAsInFile()
+      throws Exception {
+    String nodeId = "nodeId";
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFileWithNodeId(conf, nodeId);
+
+    OMStorage storage = new OMStorage(conf);
+    assertThat(storage.getState(), is(INITIALIZED));
+    assertThat(storage.getOmNodeId(), is(nodeId));
+    assertGetNodeProperties(storage, OM_ID_STR, nodeId);
+
+    storage.validateOrPersistOmNodeId(nodeId);
+
+    assertThat(storage.getOmNodeId(), is(nodeId));
+    assertGetNodeProperties(storage, OM_ID_STR, nodeId);
+  }
+
+  @Test
+  public void testValidateOrPersistOmNodeIdThrowsWithDifferentNodeIdAsInFile()
+      throws Exception {
+    String nodeId = "nodeId";
+    String newId = "newId";
+    OzoneConfiguration conf = configWithOMDBDir();
+    setupAPersistedVersionFileWithNodeId(conf, nodeId);
+
+    OMStorage storage = new OMStorage(conf);
+    assertThat(storage.getState(), is(INITIALIZED));
+    assertThat(storage.getOmNodeId(), is(nodeId));
+
+    thrown.expect(IOException.class);
+    String expectedMsg =
+        String.format(ERROR_UNEXPECTED_OM_NODE_ID_TEMPLATE, nodeId, newId);
+    thrown.expectMessage(expectedMsg);
+
+    storage.validateOrPersistOmNodeId(newId);
+  }
+
+  private void assertCertOps(OMStorage storage) throws IOException {
+    String certSerialId = "12345";
+    String certSerialId2 = "54321";
+    storage.setOmCertSerialId(certSerialId);
+    assertThat(storage.getOmCertSerialId(), is(certSerialId));
+    assertGetNodeProperties(storage, null, null, certSerialId);
+
+    storage.setOmCertSerialId(certSerialId2);
+    assertThat(storage.getOmCertSerialId(), is(certSerialId2));
+    assertGetNodeProperties(storage, null, null, certSerialId2);
+
+    storage.unsetOmCertSerialId();
+    assertThat(storage.getOmCertSerialId(), is(nullValue()));
+    assertGetNodeProperties(storage, null, null, null);
+  }
+
+  private void assertGetNodeProperties(OMStorage storage, String... values) {
+    Properties p = storage.getNodeProperties();
+    Map<String, String> e = toExpectedPropertyMapping(values);
+
+    if (e.get(OM_ID) != null) {
+      assertThat(p.getProperty(OM_ID), is(e.get(OM_ID)));
+    }
+    if (e.get(OM_NODE_ID) != null) {
+      assertThat(p.get(OM_NODE_ID), is(e.get(OM_NODE_ID)));
+    }
+    if (e.get(OM_CERT_SERIAL_ID) != null) {
+      assertThat(p.get(OM_CERT_SERIAL_ID), is(e.get(OM_CERT_SERIAL_ID)));
+    }
+  }
+
+  private Map<String, String> toExpectedPropertyMapping(String... values) {
+    Map<String, String> ret = new HashMap<>();
+    String[] propNames = new String[]{OM_ID, OM_NODE_ID, OM_CERT_SERIAL_ID};
+    for (int i = 0; i < values.length; i++) {
+      ret.put(propNames[i], values[i]);
+    }
+    return ret;
+  }
+
+  private OMStorage setupAPersistedVersionFile(OzoneConfiguration conf)

Review Comment:
   First it was used, then I realized that an OMStorage object never gets initialized just if it is being created with a config that points to a directory that contains a proper VERSION file, after that the return value was not necessary anymore, but I did not realized this. Thank you for recognizing this, I am changing the return value to void.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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