You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2015/01/12 18:38:13 UTC

svn commit: r1651151 - in /hive/branches/HIVE-8065: itests/util/src/main/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/processors/ ql/src/test/org/apache/hadoop/hive/ql/processors/

Author: brock
Date: Mon Jan 12 17:38:13 2015
New Revision: 1651151

URL: http://svn.apache.org/r1651151
Log:
HIVE-9346 - Enable the unit tests for the TestCommandProcessorFactory after adding crypto command for the test purpose (Ferdinand Xu via Brock)

Added:
    hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
Removed:
    hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/EncryptionProcessor.java
Modified:
    hive/branches/HIVE-8065/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
    hive/branches/HIVE-8065/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java

Modified: hive/branches/HIVE-8065/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/HIVE-8065/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1651151&r1=1651150&r2=1651151&view=diff
==============================================================================
--- hive/branches/HIVE-8065/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/HIVE-8065/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java Mon Jan 12 17:38:13 2015
@@ -40,6 +40,7 @@ import java.io.Serializable;
 import java.io.StringWriter;
 import java.lang.RuntimeException;
 import java.net.URL;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -85,6 +86,10 @@ import org.apache.hadoop.hive.ql.parse.P
 import org.apache.hadoop.hive.ql.parse.ParseException;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.processors.CommandProcessor;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.processors.HiveCommand;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -94,10 +99,6 @@ import org.apache.tools.ant.BuildExcepti
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.hadoop.hive.ql.processors.EncryptionProcessor;
-import org.apache.hadoop.hive.ql.processors.CommandProcessor;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.processors.HiveCommand;
 
 import com.google.common.collect.ImmutableList;
 
@@ -117,6 +118,7 @@ public class QTestUtil {
   private static final String QTEST_LEAVE_FILES = "QTEST_LEAVE_FILES";
   private final String defaultInitScript = "q_test_init.sql";
   private final String defaultCleanupScript = "q_test_cleanup.sql";
+  private final String[] testOnlyCommands = new String[]{"crypto"};
 
   private String testWarehouse;
   private final String testFiles;
@@ -966,7 +968,9 @@ public class QTestUtil {
     String wareHouseDir = SessionState.get().getConf().getVar(ConfVars.METASTOREWAREHOUSE)
         .replaceAll("^[a-zA-Z]+://.*?:\\d+", "");
     commandArgs = commandArgs.replaceAll("\\$\\{hiveconf:hive\\.metastore\\.warehouse\\.dir\\}",
-        wareHouseDir);
+      wareHouseDir);
+
+    enableTestOnlyCmd(SessionState.get().getConf());
 
     try {
       CommandProcessor proc = getTestCommand(commandName);
@@ -987,22 +991,25 @@ public class QTestUtil {
     }
   }
 
-  private CommandProcessor getTestCommand(final String commandName) {
+  private CommandProcessor getTestCommand(final String commandName) throws SQLException {
     HiveCommand testCommand = HiveCommand.find(new String[]{commandName}, HiveCommand.ONLY_FOR_TESTING);
+
     if (testCommand == null) {
       return null;
     }
 
-    switch (testCommand) {
-      case CRYPTO:
-        if (hes == null) {
-          throw new RuntimeException("HDFS encryption is not initialized for testing.");
-        }
+    return CommandProcessorFactory
+      .getForHiveCommandInternal(new String[]{commandName}, SessionState.get().getConf(),
+        testCommand.isOnlyForTesting());
+  }
 
-        return new EncryptionProcessor(hes, conf);
-      default:
-        throw new IllegalArgumentException("Unknown test command: " + commandName);
+  private void enableTestOnlyCmd(HiveConf conf){
+    StringBuilder securityCMDs = new StringBuilder(conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST));
+    for(String c : testOnlyCommands){
+      securityCMDs.append(",");
+      securityCMDs.append(c);
     }
+    conf.set(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.toString(), securityCMDs.toString());
   }
 
   private boolean isCommandUsedForTesting(final String command) {

Modified: hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java?rev=1651151&r1=1651150&r2=1651151&view=diff
==============================================================================
--- hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java (original)
+++ hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java Mon Jan 12 17:38:13 2015
@@ -29,6 +29,7 @@ import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
@@ -49,8 +50,14 @@ public final class CommandProcessorFacto
   }
 
   public static CommandProcessor getForHiveCommand(String[] cmd, HiveConf conf)
-      throws SQLException {
-    HiveCommand hiveCommand = HiveCommand.find(cmd);
+    throws SQLException {
+    return getForHiveCommandInternal(cmd, conf, false);
+  }
+
+  public static CommandProcessor getForHiveCommandInternal(String[] cmd, HiveConf conf,
+                                                           boolean testOnly)
+    throws SQLException {
+    HiveCommand hiveCommand = HiveCommand.find(cmd, testOnly);
     if (hiveCommand == null || isBlank(cmd[0])) {
       return null;
     }
@@ -58,7 +65,8 @@ public final class CommandProcessorFacto
       conf = new HiveConf();
     }
     Set<String> availableCommands = new HashSet<String>();
-    for (String availableCommand : conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST).split(",")) {
+    for (String availableCommand : conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST)
+      .split(",")) {
       availableCommands.add(availableCommand.toLowerCase().trim());
     }
     if (!availableCommands.contains(cmd[0].trim().toLowerCase())) {
@@ -82,6 +90,12 @@ public final class CommandProcessorFacto
         return new CompileProcessor();
       case RELOAD:
         return new ReloadProcessor();
+      case CRYPTO:
+        try {
+          return new CryptoProcessor(SessionState.get().getHdfsEncryptionShim(), conf);
+        } catch (HiveException e) {
+          throw new SQLException("Fail to start the command processor due to the exception: ", e);
+        }
       default:
         throw new AssertionError("Unknown HiveCommand " + hiveCommand);
     }

Added: hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
URL: http://svn.apache.org/viewvc/hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java?rev=1651151&view=auto
==============================================================================
--- hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java (added)
+++ hive/branches/HIVE-8065/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java Mon Jan 12 17:38:13 2015
@@ -0,0 +1,184 @@
+/**
+ * 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.hive.ql.processors;
+
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.HadoopShims;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * This class processes HADOOP commands used for HDFS encryption. It is meant to be run
+ * only by Hive unit & queries tests.
+ */
+public class CryptoProcessor implements CommandProcessor {
+  public static final Log LOG = LogFactory.getLog(CryptoProcessor.class.getName());
+
+  private HadoopShims.HdfsEncryptionShim encryptionShim;
+
+  private Options CREATE_KEY_OPTIONS;
+  private Options DELETE_KEY_OPTIONS;
+  private Options CREATE_ZONE_OPTIONS;
+
+  private int DEFAULT_BIT_LENGTH = 128;
+
+  private HiveConf conf;
+
+  public CryptoProcessor(HadoopShims.HdfsEncryptionShim encryptionShim, HiveConf conf) {
+    this.encryptionShim = encryptionShim;
+    this.conf = conf;
+
+    CREATE_KEY_OPTIONS = new Options();
+    CREATE_KEY_OPTIONS.addOption(OptionBuilder.hasArg().withLongOpt("keyName").isRequired().create());
+    CREATE_KEY_OPTIONS.addOption(OptionBuilder.hasArg().withLongOpt("bitLength").create());   // optional
+
+    DELETE_KEY_OPTIONS = new Options();
+    DELETE_KEY_OPTIONS.addOption(OptionBuilder.hasArg().withLongOpt("keyName").isRequired().create());
+
+    CREATE_ZONE_OPTIONS = new Options();
+    CREATE_ZONE_OPTIONS.addOption(OptionBuilder.hasArg().withLongOpt("keyName").isRequired().create());
+    CREATE_ZONE_OPTIONS.addOption(OptionBuilder.hasArg().withLongOpt("path").isRequired().create());
+  }
+
+  private CommandLine parseCommandArgs(final Options opts, String[] args) throws ParseException {
+    CommandLineParser parser = new GnuParser();
+    return parser.parse(opts, args);
+  }
+
+  private CommandProcessorResponse returnErrorResponse(final String errmsg) {
+    return new CommandProcessorResponse(1, "Encryption Processor Helper Failed:" + errmsg, null);
+  }
+
+  private void writeTestOutput(final String msg) {
+    SessionState.get().out.println(msg);
+  }
+
+  @Override
+  public void init() {
+  }
+
+  @Override
+  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+    String[] args = command.split("\\s+");
+
+    if (args.length < 1) {
+      return returnErrorResponse("Command arguments are empty.");
+    }
+
+    if (encryptionShim == null) {
+      return returnErrorResponse("Hadoop encryption shim is not initialized.");
+    }
+
+    String action = args[0];
+    String params[] = Arrays.copyOfRange(args, 1, args.length);
+
+    try {
+      if (action.equalsIgnoreCase("create_key")) {
+        createEncryptionKey(params);
+      } else if (action.equalsIgnoreCase("create_zone")) {
+        createEncryptionZone(params);
+      } else if (action.equalsIgnoreCase("delete_key")) {
+        deleteEncryptionKey(params);
+      } else {
+        return returnErrorResponse("Unknown command action: " + action);
+      }
+    } catch (Exception e) {
+      return returnErrorResponse(e.getMessage());
+    }
+
+    return new CommandProcessorResponse(0);
+  }
+
+  /**
+   * Creates an encryption key using the parameters passed through the 'create_key' action.
+   *
+   * @param params Parameters passed to the 'create_key' command action.
+   * @throws Exception If key creation failed.
+   */
+  private void createEncryptionKey(String[] params) throws Exception {
+    CommandLine args = parseCommandArgs(CREATE_KEY_OPTIONS, params);
+
+    String keyName = args.getOptionValue("keyName");
+    String bitLength = args.getOptionValue("bitLength", Integer.toString(DEFAULT_BIT_LENGTH));
+
+    try {
+      encryptionShim.createKey(keyName, new Integer(bitLength));
+    } catch (Exception e) {
+      throw new Exception("Cannot create encryption key: " + e.getMessage());
+    }
+
+    writeTestOutput("Encryption key created: '" + keyName + "'");
+  }
+
+  /**
+   * Creates an encryption zone using the parameters passed through the 'create_zone' action.
+   *
+   * @param params Parameters passed to the 'create_zone' command action.
+   * @throws Exception If zone creation failed.
+   */
+  private void createEncryptionZone(String[] params) throws Exception {
+    CommandLine args = parseCommandArgs(CREATE_ZONE_OPTIONS, params);
+
+    String keyName = args.getOptionValue("keyName");
+    Path cryptoZone = new Path(args.getOptionValue("path"));
+    if (cryptoZone == null) {
+      throw new Exception("Cannot create encryption zone: Invalid path '"
+          + args.getOptionValue("path") + "'");
+    }
+
+    try {
+      encryptionShim.createEncryptionZone(cryptoZone, keyName);
+    } catch (IOException e) {
+      throw new Exception("Cannot create encryption zone: " + e.getMessage());
+    }
+
+    writeTestOutput("Encryption zone created: '" + cryptoZone + "' using key: '" + keyName + "'");
+  }
+
+  /**
+   * Deletes an encryption key using the parameters passed through the 'delete_key' action.
+   *
+   * @param params Parameters passed to the 'delete_key' command action.
+   * @throws Exception If key deletion failed.
+   */
+  private void deleteEncryptionKey(String[] params) throws Exception {
+    CommandLine args = parseCommandArgs(DELETE_KEY_OPTIONS, params);
+
+    String keyName = args.getOptionValue("keyName");
+    try {
+      encryptionShim.deleteKey(keyName);
+    } catch (IOException e) {
+      throw new Exception("Cannot delete encryption key: " + e.getMessage());
+    }
+
+    writeTestOutput("Encryption key deleted: '" + keyName + "'");
+  }
+}

Modified: hive/branches/HIVE-8065/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/HIVE-8065/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java?rev=1651151&r1=1651150&r2=1651151&view=diff
==============================================================================
--- hive/branches/HIVE-8065/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java (original)
+++ hive/branches/HIVE-8065/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java Mon Jan 12 17:38:13 2015
@@ -20,15 +20,17 @@ package org.apache.hadoop.hive.ql.proces
 
 import java.sql.SQLException;
 
-import junit.framework.Assert;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestCommandProcessorFactory {
 
+  private final String[] testOnlyCommands = new String[]{"crypto"};
+
   private HiveConf conf;
 
   @Before
@@ -38,27 +40,39 @@ public class TestCommandProcessorFactory
 
   @Test
   public void testInvalidCommands() throws Exception {
-    Assert.assertNull("Null should have returned null", CommandProcessorFactory.getForHiveCommand(null, conf));
-    Assert.assertNull("Blank should have returned null", CommandProcessorFactory.getForHiveCommand(new String[]{" "}, conf));
-    Assert.assertNull("set role should have returned null", CommandProcessorFactory.getForHiveCommand(new String[]{"set role"}, conf));
-    Assert.assertNull("SQL should have returned null", CommandProcessorFactory.getForHiveCommand(new String[]{"SELECT * FROM TABLE"}, conf));
+    Assert.assertNull("Null should have returned null",
+      CommandProcessorFactory.getForHiveCommand(null, conf));
+    Assert.assertNull("Blank should have returned null",
+      CommandProcessorFactory.getForHiveCommand(new String[]{" "}, conf));
+    Assert.assertNull("Set role should have returned null",
+      CommandProcessorFactory.getForHiveCommand(new String[]{"set role"}, conf));
+    Assert.assertNull("SQL should have returned null",
+      CommandProcessorFactory.getForHiveCommand(new String[]{"SELECT * FROM TABLE"}, conf));
+    Assert.assertNull("Test only command should have returned null",
+      CommandProcessorFactory.getForHiveCommand(new String[]{"CRYPTO --listZones"}, conf));
   }
+
   @Test
   public void testAvailableCommands() throws Exception {
+    enableTestOnlyCmd(conf);
     SessionState.start(conf);
+
     for (HiveCommand command : HiveCommand.values()) {
       String cmd = command.name();
-      Assert.assertNotNull("Cmd " + cmd + " not return null", CommandProcessorFactory.getForHiveCommand(new String[]{cmd}, conf));
-    }
-    for (HiveCommand command : HiveCommand.values()) {
-      String cmd = command.name().toLowerCase();
-      Assert.assertNotNull("Cmd " + cmd + " not return null", CommandProcessorFactory.getForHiveCommand(new String[]{cmd}, conf));
+      String cmdInLowerCase = cmd.toLowerCase();
+      Assert.assertNotNull("Cmd " + cmd + " not return null",
+        CommandProcessorFactory
+          .getForHiveCommandInternal(new String[]{cmd}, conf, command.isOnlyForTesting()));
+      Assert.assertNotNull("Cmd " + cmd + " not return null",
+        CommandProcessorFactory.getForHiveCommandInternal(
+          new String[]{cmdInLowerCase}, conf, command.isOnlyForTesting()));
     }
     conf.set(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.toString(), "");
     for (HiveCommand command : HiveCommand.values()) {
       String cmd = command.name();
       try {
-        CommandProcessorFactory.getForHiveCommand(new String[]{cmd}, conf);
+        CommandProcessorFactory
+          .getForHiveCommandInternal(new String[]{cmd}, conf, command.isOnlyForTesting());
         Assert.fail("Expected SQLException for " + cmd + " as available commands is empty");
       } catch (SQLException e) {
         Assert.assertEquals("Insufficient privileges to execute " + cmd, e.getMessage());
@@ -66,4 +80,13 @@ public class TestCommandProcessorFactory
       }
     }
   }
+
+  private void enableTestOnlyCmd(HiveConf conf){
+    StringBuilder securityCMDs = new StringBuilder(conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST));
+    for(String c : testOnlyCommands){
+      securityCMDs.append(",");
+      securityCMDs.append(c);
+    }
+    conf.set(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.toString(), securityCMDs.toString());
+  }
 }