From 29b5a2894223bec817cd7da9e315a612cac77a84 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 7 Jun 2019 15:51:08 +0100 Subject: [PATCH 1/7] Change the Ozone Manager CLI startup / init code to use Picocli --- .../hadoop/ozone/MiniOzoneClusterImpl.java | 2 +- .../hadoop/ozone/MiniOzoneHAClusterImpl.java | 2 +- .../hadoop/ozone/TestSecureOzoneCluster.java | 10 +- .../apache/hadoop/ozone/om/TestOmInit.java | 6 +- .../hadoop/ozone/om/TestOzoneManager.java | 4 +- .../hadoop/ozone/om/OMStarterInterface.java | 33 ++++ .../apache/hadoop/ozone/om/OzoneManager.java | 144 ++-------------- .../hadoop/ozone/om/OzoneManagerStarter.java | 130 +++++++++++++++ .../ozone/om/TestOzoneManagerStarter.java | 154 ++++++++++++++++++ 9 files changed, 340 insertions(+), 145 deletions(-) create mode 100644 hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java create mode 100644 hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java create mode 100644 hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java index ee1e34ad7bbce..3c4407ce84a0b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java @@ -517,7 +517,7 @@ OzoneManager createOM() configureOM(); OMStorage omStore = new OMStorage(conf); initializeOmStorage(omStore); - return OzoneManager.createOm(null, conf); + return OzoneManager.createOm(conf); } /** diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java index 7818d9ea7e544..816b99ce7f947 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java @@ -211,7 +211,7 @@ private Map createOMService() throws IOException, OMStorage omStore = new OMStorage(conf); initializeOmStorage(omStore); - OzoneManager om = OzoneManager.createOm(null, conf); + OzoneManager om = OzoneManager.createOm(conf); om.setCertClient(certClient); omMap.put(nodeId, om); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java index 498261999783d..6a3ae68e77fef 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java @@ -383,7 +383,7 @@ public void testSecureOMInitializationFailure() throws Exception { setupOm(conf); conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY, "non-existent-user@EXAMPLE.com"); - testCommonKerberosFailures(() -> OzoneManager.createOm(null, conf)); + testCommonKerberosFailures(() -> OzoneManager.createOm(conf)); } /** @@ -662,7 +662,7 @@ private void setupOm(OzoneConfiguration config) throws Exception { // writes the version file properties omStore.initialize(); OzoneManager.setTestSecureOmFlag(true); - om = OzoneManager.createOm(null, config); + om = OzoneManager.createOm(config); } @Test @@ -725,7 +725,7 @@ public void testSecureOmReInit() throws Exception { OMStorage omStore = new OMStorage(conf); initializeOmStorage(omStore); OzoneManager.setTestSecureOmFlag(true); - om = OzoneManager.createOm(null, conf); + om = OzoneManager.createOm(conf); assertNull(om.getCertificateClient()); assertFalse(omLogs.getOutput().contains("Init response: GETCERT")); @@ -735,7 +735,7 @@ public void testSecureOmReInit() throws Exception { conf.setBoolean(OZONE_SECURITY_ENABLED_KEY, true); OzoneManager.omInit(conf); om.stop(); - om = OzoneManager.createOm(null, conf); + om = OzoneManager.createOm(conf); Assert.assertNotNull(om.getCertificateClient()); Assert.assertNotNull(om.getCertificateClient().getPublicKey()); @@ -771,7 +771,7 @@ public void testSecureOmInitSuccess() throws Exception { OMStorage omStore = new OMStorage(conf); initializeOmStorage(omStore); OzoneManager.setTestSecureOmFlag(true); - om = OzoneManager.createOm(null, conf); + om = OzoneManager.createOm(conf); Assert.assertNotNull(om.getCertificateClient()); Assert.assertNotNull(om.getCertificateClient().getPublicKey()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmInit.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmInit.java index b8ea2350ff942..4b736c9436d64 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmInit.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmInit.java @@ -23,6 +23,7 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs; import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.utils.OzoneUtils; +import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -90,10 +91,11 @@ public static void shutdown() { /** * Tests the OM Initialization. - * @throws IOException + * @throws IOException, AuthenticationException */ @Test - public void testOmInitAgain() throws IOException { + public void testOmInitAgain() throws IOException, + AuthenticationException { // Stop the Ozone Manager cluster.getOzoneManager().stop(); // Now try to init the OM again. It should succeed diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java index 1057e7a23ba28..62464ba2ef394 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java @@ -1326,7 +1326,7 @@ public void testOmInitializationFailure() throws Exception { conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY)); OzoneTestUtils.expectOmException(ResultCodes.OM_NOT_INITIALIZED, () -> { - OzoneManager.createOm(null, config); + OzoneManager.createOm(config); }); OzoneTestUtils @@ -1336,7 +1336,7 @@ public void testOmInitializationFailure() throws Exception { omStore.setScmId("testScmId"); // writes the version file properties omStore.initialize(); - OzoneManager.createOm(null, config); + OzoneManager.createOm(config); }); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java new file mode 100644 index 0000000000000..f632ad143c461 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java @@ -0,0 +1,33 @@ +/** + * 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; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import java.io.IOException; + +/** + * This interface is used by the OzoneManagerStarter class to allow the + * dependencies to be injected to the CLI class. + */ +public interface OMStarterInterface { + void start(OzoneConfiguration conf) throws IOException, + AuthenticationException; + boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException; +} \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 4041670eaa9ce..287c2ded96055 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -142,12 +142,10 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.JvmPauseMonitor; import org.apache.hadoop.util.KMSUtil; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ShutdownHookManager; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.utils.RetriableTask; import org.apache.ratis.util.LifeCycle; import org.bouncycastle.pkcs.PKCS10CertificationRequest; @@ -160,7 +158,6 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStreamWriter; -import java.io.PrintStream; import java.net.InetSocketAddress; import java.nio.file.Files; import java.nio.file.StandardCopyOption; @@ -206,7 +203,6 @@ import static org.apache.hadoop.ozone.protocol.proto .OzoneManagerProtocolProtos.OzoneManagerService .newReflectiveBlockingService; -import static org.apache.hadoop.util.ExitUtil.terminate; /** * Ozone Manager is the metadata manager of ozone. @@ -220,10 +216,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl private static final AuditLogger AUDIT = new AuditLogger( AuditLoggerType.OMLOGGER); - private static final String USAGE = - "Usage: \n ozone om [genericOptions] " + "[ " - + StartupOption.INIT.getName() + " ]\n " + "ozone om [ " - + StartupOption.HELP.getName() + " ]\n"; private static final String OM_DAEMON = "om"; private static boolean securityEnabled = false; private OzoneDelegationTokenSecretManager delegationTokenMgr; @@ -924,109 +916,36 @@ private RPC.Server startRpcServer(OzoneConfiguration conf, return rpcServer; } - /** - * Main entry point for starting OzoneManager. - * - * @param argv arguments - * @throws IOException if startup fails due to I/O error - */ - public static void main(String[] argv) throws IOException { - if (DFSUtil.parseHelpArgument(argv, USAGE, System.out, true)) { - System.exit(0); - } - try { - TracingUtil.initTracing("OzoneManager"); - OzoneConfiguration conf = new OzoneConfiguration(); - GenericOptionsParser hParser = new GenericOptionsParser(conf, argv); - if (!hParser.isParseSuccessful()) { - System.err.println("USAGE: " + USAGE + " \n"); - hParser.printGenericCommandUsage(System.err); - System.exit(1); - } - OzoneManager om = createOm(hParser.getRemainingArgs(), conf, true); - if (om != null) { - om.start(); - om.join(); - } - } catch (Throwable t) { - LOG.error("Failed to start the OzoneManager.", t); - terminate(1, t); - } - } - - private static void printUsage(PrintStream out) { - out.println(USAGE + "\n"); - } - private static boolean isOzoneSecurityEnabled() { return securityEnabled; } /** - * Constructs OM instance based on command line arguments. - * - * This method is intended for unit tests only. It suppresses the - * startup/shutdown message and skips registering Unix signal - * handlers. + * Constructs OM instance based on the configuration. * - * @param argv Command line arguments * @param conf OzoneConfiguration * @return OM instance * @throws IOException, AuthenticationException in case OM instance * creation fails. */ - @VisibleForTesting - public static OzoneManager createOm( - String[] argv, OzoneConfiguration conf) + public static OzoneManager createOm(OzoneConfiguration conf) throws IOException, AuthenticationException { - return createOm(argv, conf, false); + loginOMUserIfSecurityEnabled(conf); + return new OzoneManager(conf); } /** - * Constructs OM instance based on command line arguments. + * Logs in the OM use if security is enabled in the configuration. * - * @param argv Command line arguments * @param conf OzoneConfiguration - * @param printBanner if true then log a verbose startup message. - * @return OM instance - * @throws IOException, AuthenticationException in case OM instance - * creation fails. + * @throws IOException, AuthenticationException in case login failes. */ - private static OzoneManager createOm(String[] argv, - OzoneConfiguration conf, boolean printBanner) + private static void loginOMUserIfSecurityEnabled(OzoneConfiguration conf) throws IOException, AuthenticationException { - StartupOption startOpt = parseArguments(argv); - if (startOpt == null) { - printUsage(System.err); - terminate(1); - return null; - } - securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf); if (securityEnabled) { loginOMUser(conf); } - - switch (startOpt) { - case INIT: - if (printBanner) { - StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG); - } - terminate(omInit(conf) ? 0 : 1); - return null; - case HELP: - printUsage(System.err); - terminate(0); - return null; - default: - if (argv == null) { - argv = new String[]{}; - } - if (printBanner) { - StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG); - } - return new OzoneManager(conf); - } } /** @@ -1038,7 +957,9 @@ private static OzoneManager createOm(String[] argv, * accessible */ @VisibleForTesting - public static boolean omInit(OzoneConfiguration conf) throws IOException { + public static boolean omInit(OzoneConfiguration conf) throws IOException, + AuthenticationException { + loginOMUserIfSecurityEnabled(conf); OMStorage omStorage = new OMStorage(conf); StorageState state = omStorage.getState(); if (state != StorageState.INITIALIZED) { @@ -1135,23 +1056,6 @@ private static ScmInfo getScmInfo(OzoneConfiguration conf) } } - /** - * Parses the command line options for OM initialization. - * - * @param args command line arguments - * @return StartupOption if options are valid, null otherwise - */ - private static StartupOption parseArguments(String[] args) { - if (args == null || args.length == 0) { - return StartupOption.REGULAR; - } else { - if (args.length == 1) { - return StartupOption.parse(args[0]); - } - } - return null; - } - /** * Builds a message for logging startup information about an RPC server. * @@ -3144,34 +3048,6 @@ public List getAcl(OzoneObj obj) throws IOException { } } - /** - * Startup options. - */ - public enum StartupOption { - INIT("--init"), - HELP("--help"), - REGULAR("--regular"); - - private final String name; - - StartupOption(String arg) { - this.name = arg; - } - - public static StartupOption parse(String value) { - for (StartupOption option : StartupOption.values()) { - if (option.name.equalsIgnoreCase(value)) { - return option; - } - } - return null; - } - - public String getName() { - return name; - } - } - public static Logger getLogger() { return LOG; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java new file mode 100644 index 0000000000000..8a0c317858756 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java @@ -0,0 +1,130 @@ +/** + * 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; + +import org.apache.hadoop.hdds.cli.GenericCli; +import org.apache.hadoop.hdds.cli.HddsVersionProvider; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import picocli.CommandLine; +import picocli.CommandLine.Command; + +import java.io.IOException; + +/** + * This class provides a command line interface to start the OM + * using Picocli. + */ +@Command(name = "ozone om", + hidden = true, description = "Start or initialize the Ozone Manager.", + versionProvider = HddsVersionProvider.class, + mixinStandardHelpOptions = true) +public class OzoneManagerStarter extends GenericCli { + + private OzoneConfiguration conf; + private OMStarterInterface receiver; + private static final Logger LOG = + LoggerFactory.getLogger(OzoneManagerStarter.class); + + public static void main(String[] args) throws Exception { + TracingUtil.initTracing("OzoneManager"); + new OzoneManagerStarter( + new OzoneManagerStarter.OMStarterHelper()).run(args); + } + + public OzoneManagerStarter(OMStarterInterface receiverObj) { + super(); + receiver = receiverObj; + } + + @Override + public Void call() throws Exception { + /** + * This method is invoked only when a sub-command is not called. Therefore + * if someone runs "ozone om" with no parameters, this is the methood + * which runs and starts the OM. + */ + commonInit(); + startOm(); + return null; + } + + /** + * This function is used by the command line to start the OM. + */ + private void startOm() throws Exception { + receiver.start(conf); + } + + /** + * This function implements a sub-command to allow the OM to be + * initialized from the command line. + */ + @CommandLine.Command(name = "--init", + customSynopsis = "ozone om [global options] --init", + hidden = false, + description = "Initialize the Ozone Manager if not already initialized", + mixinStandardHelpOptions = true, + versionProvider = HddsVersionProvider.class) + public void initOm() + throws Exception { + commonInit(); + boolean result = receiver.init(conf); + if (!result) { + throw new IOException("OM Init failed."); + } + } + + /** + * This function should be called by each command to ensure the configuration + * is set and print the startup banner message. + */ + private void commonInit() { + conf = createOzoneConfiguration(); + + String[] originalArgs = getCmd().getParseResult().originalArgs() + .toArray(new String[0]); + StringUtils.startupShutdownMessage(OzoneManager.class, + originalArgs, LOG); + } + + /** + * This static class wraps the external dependencies needed for this command + * to execute its tasks. This allows the dependency to be injected for unit + * testing. + */ + static class OMStarterHelper implements OMStarterInterface{ + + public void start(OzoneConfiguration conf) throws IOException, + AuthenticationException { + OzoneManager om = OzoneManager.createOm(conf); + om.start(); + om.join(); + } + + public boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException { + return OzoneManager.omInit(conf); + } + } + +} \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java new file mode 100644 index 0000000000000..80281693c1b62 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java @@ -0,0 +1,154 @@ +/** + * 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; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import static org.junit.Assert.*; + +/** + * This class is used to test the CLI provided by OzoneManagerStarter, which is + * used to start and init the OzoneManager. The calls to the Ozone Manager are + * mocked so the tests only validate the CLI calls the correct methods are + * invoked. + */ +public class TestOzoneManagerStarter { + + private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); + private final ByteArrayOutputStream errContent = new ByteArrayOutputStream(); + private final PrintStream originalOut = System.out; + private final PrintStream originalErr = System.err; + + private MockOMStarter mock; + + @Before + public void setUpStreams() { + System.setOut(new PrintStream(outContent)); + System.setErr(new PrintStream(errContent)); + mock = new MockOMStarter(); + } + + @After + public void restoreStreams() { + System.setOut(originalOut); + System.setErr(originalErr); + } + + @Test + public void testCallsStartWhenServerStarted() throws Exception { + executeCommand(); + assertTrue(mock.startCalled); + } + + @Test + public void testExceptionThrownWhenStartFails() throws Exception { + mock.throwOnStart = true; + try { + executeCommand(); + fail("Exception should have been thrown"); + } catch (Exception e) { + assertTrue(true); + } + } + + @Test + public void testStartNotCalledWithInvalidParam() throws Exception { + executeCommand("--invalid"); + assertFalse(mock.startCalled); + } + + @Test + public void testPassingInitSwitchCallsInit() { + executeCommand("--init"); + assertTrue(mock.initCalled); + } + + @Test + public void testInitSwitchWithInvalidParamDoesNotRun() { + executeCommand("--init", "--invalid"); + assertFalse(mock.initCalled); + } + + @Test + public void testUnSuccessfulInitThrowsException() { + mock.throwOnInit = true; + try { + executeCommand("--init"); + fail("Exception show have been thrown"); + } catch (Exception e) { + assertTrue(true); + } + } + + @Test + public void testInitThatReturnsFalseThrowsException() { + mock.initStatus = false; + try { + executeCommand("--init"); + fail("Exception show have been thrown"); + } catch (Exception e) { + assertTrue(true); + } + } + + @Test + public void testUsagePrintedOnInvalidInput() { + executeCommand("--invalid"); + Pattern p = Pattern.compile("^Unknown option:.*--invalid.*\nUsage"); + Matcher m = p.matcher(errContent.toString()); + assertTrue(m.find()); + } + + private void executeCommand(String... args) { + new OzoneManagerStarter(mock).execute(args); + } + + static class MockOMStarter implements OMStarterInterface { + + private boolean startCalled = false; + private boolean initCalled = false; + private boolean initStatus = true; + private boolean throwOnStart = false; + private boolean throwOnInit = false; + + public void start(OzoneConfiguration conf) throws IOException, + AuthenticationException { + startCalled = true; + if (throwOnStart) { + throw new IOException("Simulated Exception"); + } + } + + public boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException { + initCalled = true; + if (throwOnInit) { + throw new IOException("Simulated Exception"); + } + return initStatus; + } + } +} \ No newline at end of file From daff57969a827b2c99049c078ab81a32eafd2944 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 7 Jun 2019 15:54:36 +0100 Subject: [PATCH 2/7] Change the ozone executable script to run the new OzoneManagerStarter class when running OM commands --- hadoop-ozone/common/src/main/bin/ozone | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone index de8f47f1ae8f5..f6fe147e5076c 100755 --- a/hadoop-ozone/common/src/main/bin/ozone +++ b/hadoop-ozone/common/src/main/bin/ozone @@ -129,7 +129,7 @@ function ozonecmd_case ;; om) HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" - HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager + HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManagerStarter HDFS_OM_OPTS="${HDFS_OM_OPTS} -Dlog4j.configurationFile=${HADOOP_CONF_DIR}/om-audit-log4j2.properties" HADOOP_OPTS="${HADOOP_OPTS} ${HDFS_OM_OPTS}" OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-ozone-manager" From ca9909fc1c53ebeb6299db4889992a1ffcc6559f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 7 Jun 2019 16:30:15 +0100 Subject: [PATCH 3/7] Changed the dev-support Intellij configurations to use the new OzoneManagerStarter script --- .../dev-support/intellij/runConfigurations/OzoneManager.xml | 4 ++-- .../intellij/runConfigurations/OzoneManagerInit.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml b/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml index 2d4a308798a58..c2aaf1c829b27 100644 --- a/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml +++ b/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml @@ -16,7 +16,7 @@ --> - - \ No newline at end of file + diff --git a/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManagerInit.xml b/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManagerInit.xml index 7988ff17f0810..70fab5df640f2 100644 --- a/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManagerInit.xml +++ b/hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManagerInit.xml @@ -16,7 +16,7 @@ --> - - \ No newline at end of file + From 2c3b134b36650997c0231ab2e9ff7ad2b4179a4e Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 7 Jun 2019 16:31:45 +0100 Subject: [PATCH 4/7] Changed reference to the old OzoneManager.createOm in GenesisUtil to use the new method spec --- .../main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java index 6bff82b5755dc..d08d26ad28938 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java @@ -186,7 +186,7 @@ static OzoneManager getOm(OzoneConfiguration conf) omStorage.setOmId(UUID.randomUUID().toString()); omStorage.initialize(); } - return OzoneManager.createOm(null, conf); + return OzoneManager.createOm(conf); } static void configureOM(Configuration conf, int numHandlers) { From 8ad4c71ae6184945bd786f1ccbaf79bba1d2e08f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 7 Jun 2019 20:18:19 +0100 Subject: [PATCH 5/7] Removed the loginIfSecurityEnabled method from omInit as it was causing secure test failures. --- .../hadoop/ozone/om/OMStarterInterface.java | 3 +-- .../apache/hadoop/ozone/om/OzoneManager.java | 17 ++--------------- .../hadoop/ozone/om/OzoneManagerStarter.java | 3 +-- .../ozone/om/TestOzoneManagerStarter.java | 3 +-- 4 files changed, 5 insertions(+), 21 deletions(-) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java index f632ad143c461..b90b67e568ce7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java @@ -28,6 +28,5 @@ public interface OMStarterInterface { void start(OzoneConfiguration conf) throws IOException, AuthenticationException; - boolean init(OzoneConfiguration conf) throws IOException, - AuthenticationException; + boolean init(OzoneConfiguration conf) throws IOException; } \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 287c2ded96055..279550f8167bf 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -930,22 +930,11 @@ private static boolean isOzoneSecurityEnabled() { */ public static OzoneManager createOm(OzoneConfiguration conf) throws IOException, AuthenticationException { - loginOMUserIfSecurityEnabled(conf); - return new OzoneManager(conf); - } - - /** - * Logs in the OM use if security is enabled in the configuration. - * - * @param conf OzoneConfiguration - * @throws IOException, AuthenticationException in case login failes. - */ - private static void loginOMUserIfSecurityEnabled(OzoneConfiguration conf) - throws IOException, AuthenticationException { securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf); if (securityEnabled) { loginOMUser(conf); } + return new OzoneManager(conf); } /** @@ -957,9 +946,7 @@ private static void loginOMUserIfSecurityEnabled(OzoneConfiguration conf) * accessible */ @VisibleForTesting - public static boolean omInit(OzoneConfiguration conf) throws IOException, - AuthenticationException { - loginOMUserIfSecurityEnabled(conf); + public static boolean omInit(OzoneConfiguration conf) throws IOException { OMStorage omStorage = new OMStorage(conf); StorageState state = omStorage.getState(); if (state != StorageState.INITIALIZED) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java index 8a0c317858756..c01bd2fe63903 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java @@ -121,8 +121,7 @@ public void start(OzoneConfiguration conf) throws IOException, om.join(); } - public boolean init(OzoneConfiguration conf) throws IOException, - AuthenticationException { + public boolean init(OzoneConfiguration conf) throws IOException { return OzoneManager.omInit(conf); } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java index 80281693c1b62..d9586e8d814e2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java @@ -142,8 +142,7 @@ public void start(OzoneConfiguration conf) throws IOException, } } - public boolean init(OzoneConfiguration conf) throws IOException, - AuthenticationException { + public boolean init(OzoneConfiguration conf) throws IOException { initCalled = true; if (throwOnInit) { throw new IOException("Simulated Exception"); From 538d12ff6f12a65af3b1cd7df4a53d99c5ebd089 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 13 Jun 2019 10:41:32 +0100 Subject: [PATCH 6/7] Revert "Removed the loginIfSecurityEnabled method from omInit as it was causing secure test failures." This is because it appears to be the test which is broken rather than the refactored code. This reverts commit 021796b964ba05edd3c7458dad96b4f2ed2c919f. --- .../hadoop/ozone/om/OMStarterInterface.java | 3 ++- .../apache/hadoop/ozone/om/OzoneManager.java | 17 +++++++++++++++-- .../hadoop/ozone/om/OzoneManagerStarter.java | 3 ++- .../ozone/om/TestOzoneManagerStarter.java | 3 ++- 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java index b90b67e568ce7..f632ad143c461 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java @@ -28,5 +28,6 @@ public interface OMStarterInterface { void start(OzoneConfiguration conf) throws IOException, AuthenticationException; - boolean init(OzoneConfiguration conf) throws IOException; + boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException; } \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 279550f8167bf..287c2ded96055 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -930,11 +930,22 @@ private static boolean isOzoneSecurityEnabled() { */ public static OzoneManager createOm(OzoneConfiguration conf) throws IOException, AuthenticationException { + loginOMUserIfSecurityEnabled(conf); + return new OzoneManager(conf); + } + + /** + * Logs in the OM use if security is enabled in the configuration. + * + * @param conf OzoneConfiguration + * @throws IOException, AuthenticationException in case login failes. + */ + private static void loginOMUserIfSecurityEnabled(OzoneConfiguration conf) + throws IOException, AuthenticationException { securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf); if (securityEnabled) { loginOMUser(conf); } - return new OzoneManager(conf); } /** @@ -946,7 +957,9 @@ public static OzoneManager createOm(OzoneConfiguration conf) * accessible */ @VisibleForTesting - public static boolean omInit(OzoneConfiguration conf) throws IOException { + public static boolean omInit(OzoneConfiguration conf) throws IOException, + AuthenticationException { + loginOMUserIfSecurityEnabled(conf); OMStorage omStorage = new OMStorage(conf); StorageState state = omStorage.getState(); if (state != StorageState.INITIALIZED) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java index c01bd2fe63903..8a0c317858756 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java @@ -121,7 +121,8 @@ public void start(OzoneConfiguration conf) throws IOException, om.join(); } - public boolean init(OzoneConfiguration conf) throws IOException { + public boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException { return OzoneManager.omInit(conf); } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java index d9586e8d814e2..80281693c1b62 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java @@ -142,7 +142,8 @@ public void start(OzoneConfiguration conf) throws IOException, } } - public boolean init(OzoneConfiguration conf) throws IOException { + public boolean init(OzoneConfiguration conf) throws IOException, + AuthenticationException { initCalled = true; if (throwOnInit) { throw new IOException("Simulated Exception"); From e7e5bfa65488e01770fdd50f66d5c88f12e83955 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 13 Jun 2019 12:06:14 +0100 Subject: [PATCH 7/7] Added workaround for failing test and comment explaining why --- .../hadoop/ozone/TestSecureOzoneCluster.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java index 6a3ae68e77fef..247c9d7b4ac84 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java @@ -717,6 +717,26 @@ public void testSecureOmReInit() throws Exception { LogCapturer omLogs = LogCapturer.captureLogs(OzoneManager.getLogger()); omLogs.clearOutput(); + + /** + * As all these processes run inside the same JVM, there are issues around + * the Hadoop UGI if different processes run with different principals. + * In this test, the OM has to contact the SCM to download certs. SCM runs + * as scm/host@REALM, but the OM logs in as om/host@REALM, and then the test + * fails, and the OM is unable to contact the SCM due to kerberos login + * issues. To work around that, have the OM run as the same principal as the + * SCM, and then the test passes. + * + * TODO: Need to look into this further to see if there is a better way to + * address this problem. + */ + String realm = miniKdc.getRealm(); + conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY, + "scm/" + host + "@" + realm); + omKeyTab = new File(workDir, "scm.keytab"); + conf.set(OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY, + omKeyTab.getAbsolutePath()); + initSCM(); try { scm = StorageContainerManager.createSCM(conf);