You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hcatalog-commits@incubator.apache.org by ma...@apache.org on 2011/05/04 20:39:27 UTC

svn commit: r1099599 - in /incubator/hcatalog/trunk/src: java/org/apache/hcatalog/cli/HCatDriver.java java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java

Author: macyang
Date: Wed May  4 20:39:27 2011
New Revision: 1099599

URL: http://svn.apache.org/viewvc?rev=1099599&view=rev
Log:
HCATALOG:19 with https://issues.apache.org/jira/secure/attachment/12478147/exim.cli.patch
(Krishna via Mac)

Added:
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java
Modified:
    incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/HCatDriver.java
    incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java

Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/HCatDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/HCatDriver.java?rev=1099599&r1=1099598&r2=1099599&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/HCatDriver.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/HCatDriver.java Wed May  4 20:39:27 2011
@@ -37,24 +37,25 @@ public class HCatDriver extends Driver {
   @Override
   public CommandProcessorResponse run(String command) {
 
-    int ret = -1;
-	try {
-		ret = super.run(command).getResponseCode();
-	} catch (CommandNeedRetryException e) {
-		return new CommandProcessorResponse(ret, e.toString(), "");
-	}
+    CommandProcessorResponse cpr = null;
+    try {
+      cpr = super.run(command);
+    } catch (CommandNeedRetryException e) {
+      return new CommandProcessorResponse(-1, e.toString(), "");
+    }
 
     SessionState ss = SessionState.get();
 
-    if (ret == 0){
+    if (cpr.getResponseCode() == 0){
       // Only attempt to do this, if cmd was successful.
-      ret = setFSPermsNGrp(ss);
+      int rc = setFSPermsNGrp(ss);
+      cpr = new CommandProcessorResponse(rc);
     }
     // reset conf vars
     ss.getConf().set(HCatConstants.HCAT_CREATE_DB_NAME, "");
     ss.getConf().set(HCatConstants.HCAT_CREATE_TBL_NAME, "");
 
-    return new CommandProcessorResponse(ret);
+    return cpr;
   }
 
   private int setFSPermsNGrp(SessionState ss) {
@@ -62,6 +63,10 @@ public class HCatDriver extends Driver {
     Configuration conf =ss.getConf();
 
     String tblName = conf.get(HCatConstants.HCAT_CREATE_TBL_NAME,"");
+    if (tblName.isEmpty()) {
+      tblName = conf.get("import.destination.table", "");
+      conf.set("import.destination.table", "");
+    }
     String dbName = conf.get(HCatConstants.HCAT_CREATE_DB_NAME, "");
     String grp = conf.get(HCatConstants.HCAT_GROUP,null);
     String permsStr = conf.get(HCatConstants.HCAT_PERMS,null);

Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java?rev=1099599&r1=1099598&r2=1099599&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java Wed May  4 20:39:27 2011
@@ -20,6 +20,8 @@ package org.apache.hcatalog.cli.Semantic
 import java.io.Serializable;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.metastore.Warehouse;
@@ -43,6 +45,8 @@ public class HCatSemanticAnalyzer extend
   private AbstractSemanticAnalyzerHook hook;
   private ASTNode ast;
 
+  private static final Log LOG = LogFactory.getLog(HCatSemanticAnalyzer.class);
+
   @Override
   public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context, ASTNode ast)
       throws SemanticException {
@@ -91,6 +95,11 @@ public class HCatSemanticAnalyzer extend
         return ast;
       }
 
+    // allow export/import operations
+    case HiveParser.TOK_EXPORT:
+    case HiveParser.TOK_IMPORT:
+      return ast;
+
     // In all other cases, throw an exception. Its a white-list of allowed operations.
     default:
       throw new SemanticException("Operation not supported.");
@@ -148,6 +157,19 @@ public class HCatSemanticAnalyzer extend
         // here easily. So, it is especially handled in CreateTable post hook.
         break;
 
+      case HiveParser.TOK_EXPORT:
+        String tableName = BaseSemanticAnalyzer.getUnescapedName(((ASTNode) ast.getChild(0).getChild(0)));
+        LOG.debug("Export for table " + tableName);
+        authorize(tableName, context, FsAction.READ, false);
+        break;
+
+      case HiveParser.TOK_IMPORT:
+        LOG.debug("Import into location " + context.getConf().get("import.destination.dir"));
+        AuthUtils.authorize(new Path(context.getConf().get("import.destination.dir")),
+                    FsAction.WRITE, context.getConf());
+        break;
+
+
       default:
         throw new HCatException(ErrorType.ERROR_INTERNAL_EXCEPTION, "Unexpected token: "+ast.getToken());
       }

Added: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java?rev=1099599&view=auto
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java (added)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java Wed May  4 20:39:27 2011
@@ -0,0 +1,171 @@
+/**
+ * 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.hcatalog.cli;
+
+import java.io.IOException;
+import java.net.URI;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hcatalog.MiniCluster;
+import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import org.apache.hcatalog.common.HCatConstants;
+
+
+public class TestEximSemanticAnalysis extends TestCase {
+
+  private final MiniCluster cluster = MiniCluster.buildCluster();
+  private HiveConf hcatConf;
+  private HCatDriver hcatDriver;
+  private Warehouse wh;
+  private static final Log LOG = LogFactory.getLog(TestEximSemanticAnalysis.class);
+
+  @Override
+  protected void setUp() throws Exception {
+
+    hcatConf = new HiveConf(this.getClass());
+    hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
+    hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
+    hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
+    hcatConf.set("fs.pfile.impl", "org.apache.hadoop.fs.ProxyLocalFileSystem");
+    URI fsuri = cluster.getFileSystem().getUri();
+    Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(), "/user/hive/warehouse");
+    hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString());
+    hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString());
+    wh = new Warehouse(hcatConf);
+    SessionState.start(new CliSessionState(hcatConf));
+
+    hcatDriver = new HCatDriver();
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+  }
+
+  public void testExportPerms() throws IOException, MetaException {
+
+    hcatDriver.run("drop table junit_sem_analysis");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE");
+    assertEquals(0, response.getResponseCode());
+    Path whPath = wh.getDefaultTablePath("default", "junit_sem_analysis");
+    cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwx-wx"));
+    cluster.getFileSystem().setOwner(whPath, "nosuchuser", "nosuchgroup");
+
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+    response = hcatDriver
+        .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'");
+
+    assertEquals(10, response.getResponseCode());
+    assertEquals(
+        "FAILED: Error in semantic analysis: org.apache.hcatalog.common.HCatException : 3000 : Permission denied",
+        response.getErrorMessage());
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+    response = hcatDriver.run("drop table junit_sem_analysis");
+    if (response.getResponseCode() != 0) {
+      System.err.println(response.getErrorMessage());
+      fail("Drop table failed");
+    }
+  }
+
+  public void testImportPerms() throws IOException, MetaException {
+
+    hcatDriver.run("drop table junit_sem_analysis");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE");
+    assertEquals(0, response.getResponseCode());
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+    response = hcatDriver
+        .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'");
+    assertEquals(0, response.getResponseCode());
+    response = hcatDriver.run("drop table junit_sem_analysis");
+    assertEquals(0, response.getResponseCode());
+    response = hcatDriver
+        .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE");
+    assertEquals(0, response.getResponseCode());
+    Path whPath = wh.getDefaultTablePath("default", "junit_sem_analysis");
+    cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwxr-x"));
+    cluster.getFileSystem().setOwner(whPath, "nosuchuser", "nosuchgroup");
+
+    response = hcatDriver
+        .run("import table junit_sem_analysis from 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'");
+
+    assertEquals(10, response.getResponseCode());
+    assertEquals(
+        "FAILED: Error in semantic analysis: org.apache.hcatalog.common.HCatException : 3000 : Permission denied",
+        response.getErrorMessage());
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+
+    cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwxrwx"));
+    response = hcatDriver.run("drop table junit_sem_analysis");
+    if (response.getResponseCode() != 0) {
+      System.err.println(response.getErrorMessage());
+      fail("Drop table failed");
+    }
+  }
+
+  public void testImportSetPermsGroup() throws IOException, MetaException {
+
+    hcatDriver.run("drop table junit_sem_analysis");
+    hcatDriver.run("drop table junit_sem_analysis_imported");
+    CommandProcessorResponse response = hcatDriver
+        .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE");
+    assertEquals(0, response.getResponseCode());
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+    response = hcatDriver
+        .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'");
+    assertEquals(0, response.getResponseCode());
+    response = hcatDriver.run("drop table junit_sem_analysis");
+    assertEquals(0, response.getResponseCode());
+
+    hcatConf.set(HCatConstants.HCAT_PERMS, "-rwxrw-r--");
+    hcatConf.set(HCatConstants.HCAT_GROUP, "nosuchgroup");
+
+    response = hcatDriver
+        .run("import table junit_sem_analysis_imported from 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'");
+    assertEquals(0, response.getResponseCode());
+
+    Path whPath = wh.getDefaultTablePath("default", "junit_sem_analysis_imported");
+    assertEquals(FsPermission.valueOf("-rwxrw-r--"), cluster.getFileSystem().getFileStatus(whPath).getPermission());
+    assertEquals("nosuchgroup", cluster.getFileSystem().getFileStatus(whPath).getGroup());
+
+    Runtime.getRuntime().exec("rm -rf /tmp/hcat");
+
+    response = hcatDriver.run("drop table junit_sem_analysis_imported");
+    if (response.getResponseCode() != 0) {
+      System.err.println(response.getErrorMessage());
+      fail("Drop table failed");
+    }
+  }
+
+
+}
+