You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/09/24 19:17:25 UTC

svn commit: r1389490 [2/2] - in /accumulo/branches/ACCUMULO-259: ./ core/ server/ server/src/main/java/org/apache/accumulo/server/client/ server/src/main/java/org/apache/accumulo/server/master/ server/src/main/java/org/apache/accumulo/server/master/tab...

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java?rev=1389490&r1=1389489&r2=1389490&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java Mon Sep 24 17:17:23 2012
@@ -52,19 +52,8 @@ public class TableOp extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    boolean userExists = SecurityHelper.getTabUserExists(state);
-    Connector conn;
-    try {
-      conn = state.getInstance().getConnector(SecurityHelper.getTabUserName(state), SecurityHelper.getTabUserPass(state));
-    } catch (AccumuloSecurityException ae) {
-      if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
-        if (userExists)
-          throw new AccumuloException("User didn't exist when they should (or worse- password mismatch)", ae);
-        else
-          return;
-      }
-      throw new AccumuloException("Unexpected exception!", ae);
-    }
+    Connector conn = WalkingSecurity.get(state).getTableConnector();
+
     String action = props.getProperty("action", "_random");
     TablePermission tp;
     if ("_random".equalsIgnoreCase(action)) {
@@ -74,19 +63,17 @@ public class TableOp extends Test {
       tp = TablePermission.valueOf(action);
     }
     
-    boolean tableExists = SecurityHelper.getTableExists(state);
-    boolean hasPerm = SecurityHelper.getTabPerm(state, SecurityHelper.getTabUserName(state), tp);
-    
-    String tableName = state.getString("secTableName");
-    boolean ambiguousZone;
+    boolean tableExists = WalkingSecurity.get(state).getTableExists();
+    String tableName = WalkingSecurity.get(state).getTableName();
     
     switch (tp) {
-      case READ:
-        Authorizations auths = SecurityHelper.getUserAuths(state, SecurityHelper.getTabUserName(state));
-        boolean canRead = SecurityHelper.getTabPerm(state, SecurityHelper.getTabUserName(state), TablePermission.READ);
-        ambiguousZone = SecurityHelper.inAmbiguousZone(state, SecurityHelper.getTabUserName(state), tp);
+      case READ: {
+        boolean canRead = WalkingSecurity.get(state).canScan(WalkingSecurity.get(state).getTabAuthInfo(), tableName);
+        Authorizations auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
+        boolean ambiguousZone = WalkingSecurity.get(state).inAmbiguousZone(conn.whoami(), tp);
+
         try {
-          Scanner scan = conn.createScanner(tableName, conn.securityOperations().getUserAuthorizations(SecurityHelper.getTabUserName(state)));
+          Scanner scan = conn.createScanner(tableName, conn.securityOperations().getUserAuthorizations(conn.whoami()));
           int seen = 0;
           Iterator<Entry<Key,Value>> iter = scan.iterator();
           while (iter.hasNext()) {
@@ -98,7 +85,7 @@ public class TableOp extends Test {
           }
           if (!canRead && !ambiguousZone)
             throw new AccumuloException("Was able to read when I shouldn't have had the perm with connection user " + conn.whoami() + " table " + tableName);
-          for (Entry<String,Integer> entry : SecurityHelper.getAuthsMap(state).entrySet()) {
+          for (Entry<String,Integer> entry : WalkingSecurity.get(state).getAuthsMap().entrySet()) {
             if (auths.contains(entry.getKey().getBytes()))
               seen = seen - entry.getValue();
           }
@@ -128,13 +115,14 @@ public class TableOp extends Test {
         }
         
         break;
+      }
       case WRITE:
-        // boolean canWrite = SecurityHelper.getTabPerm(state, SecurityHelper.getTabUserName(state), TablePermission.WRITE);
-        ambiguousZone = SecurityHelper.inAmbiguousZone(state, SecurityHelper.getTabUserName(state), tp);
+        // boolean canWrite = WalkingSecurity.get(state).canWrite(WalkingSecurity.get(state).getTabAuthInfo(), tableName);
+        boolean ambiguousZone = WalkingSecurity.get(state).inAmbiguousZone(conn.whoami(), tp);
 
-        String key = SecurityHelper.getLastKey(state) + "1";
+        String key = WalkingSecurity.get(state).getLastKey() + "1";
         Mutation m = new Mutation(new Text(key));
-        for (String s : SecurityHelper.getAuthsArray()) {
+        for (String s : WalkingSecurity.get(state).getAuthsArray()) {
           m.put(new Text(), new Text(), new ColumnVisibility(s), new Value("value".getBytes()));
         }
         BatchWriter writer;
@@ -156,6 +144,7 @@ public class TableOp extends Test {
           if (ambiguousZone) {
             Thread.sleep(1000);
             try {
+              writer = conn.createBatchWriter(tableName, 9000l, 0l, 1);
               writer.addMutation(m);
               writer.close();
             } catch (MutationsRejectedException mre2) {
@@ -164,19 +153,19 @@ public class TableOp extends Test {
           }
         }
         if (works)
-          for (String s : SecurityHelper.getAuthsArray())
-            SecurityHelper.increaseAuthMap(state, s, 1);
+          for (String s : WalkingSecurity.get(state).getAuthsArray())
+            WalkingSecurity.get(state).increaseAuthMap(s, 1);
         break;
       case BULK_IMPORT:
-        key = SecurityHelper.getLastKey(state) + "1";
+        key = WalkingSecurity.get(state).getLastKey() + "1";
         SortedSet<Key> keys = new TreeSet<Key>();
-        for (String s : SecurityHelper.getAuthsArray()) {
+        for (String s : WalkingSecurity.get(state).getAuthsArray()) {
           Key k = new Key(key, "", "", s);
           keys.add(k);
         }
         Path dir = new Path("/tmp", "bulk_" + UUID.randomUUID().toString());
         Path fail = new Path(dir.toString() + "_fail");
-        FileSystem fs = SecurityHelper.getFs(state);
+        FileSystem fs = WalkingSecurity.get(state).getFs();
         FileSKVWriter f = FileOperations.getInstance().openWriter(dir + "/securityBulk." + RFile.EXTENSION, fs, fs.getConf(),
             AccumuloConfiguration.getDefaultConfiguration());
         f.startDefaultLocalityGroup();
@@ -192,25 +181,26 @@ public class TableOp extends Test {
           return;
         } catch (AccumuloSecurityException ae) {
           if (ae.getErrorCode().equals(SecurityErrorCode.PERMISSION_DENIED)) {
-            if (hasPerm)
+            if (WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabAuthInfo(), tableName))
               throw new AccumuloException("Bulk Import failed when it should have worked: " + tableName);
             return;
           } else if (ae.getErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
-            if (SecurityHelper.sysUserPassTransient(state))
+            if (WalkingSecurity.get(state).userPassTransient(conn.whoami()))
               return;
           }
           throw new AccumuloException("Unexpected exception!", ae);
         }
-        for (String s : SecurityHelper.getAuthsArray())
-          SecurityHelper.increaseAuthMap(state, s, 1);
+        for (String s : WalkingSecurity.get(state).getAuthsArray())
+          WalkingSecurity.get(state).increaseAuthMap(s, 1);
         fs.delete(dir, true);
         fs.delete(fail, true);
 
-        if (!hasPerm)
+        if (!WalkingSecurity.get(state).canBulkImport(WalkingSecurity.get(state).getTabAuthInfo(), tableName))
           throw new AccumuloException("Bulk Import succeeded when it should have failed: " + dir + " table " + tableName);
         break;
       case ALTER_TABLE:
-        AlterTable.renameTable(conn, state, tableName, tableName + "plus", hasPerm, tableExists);
+        AlterTable.renameTable(conn, state, tableName, tableName + "plus",
+            WalkingSecurity.get(state).canAlterTable(WalkingSecurity.get(state).getTabAuthInfo(), tableName), tableExists);
         break;
       
       case GRANT:

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java?rev=1389490&r1=1389489&r2=1389490&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java Mon Sep 24 17:17:23 2012
@@ -39,13 +39,13 @@ public class Validate extends Test {
   public static void validate(State state, Logger log) throws Exception {
     Connector conn = state.getConnector();
     
-    boolean tableExists = SecurityHelper.getTableExists(state);
-    boolean cloudTableExists = conn.tableOperations().list().contains(SecurityHelper.getTableName(state));
+    boolean tableExists = WalkingSecurity.get(state).getTableExists();
+    boolean cloudTableExists = conn.tableOperations().list().contains(WalkingSecurity.get(state).getTableName());
     if (tableExists != cloudTableExists)
       throw new AccumuloException("Table existance out of sync");
     
-    boolean tableUserExists = SecurityHelper.getTabUserExists(state);
-    boolean cloudTableUserExists = conn.securityOperations().listUsers().contains(SecurityHelper.getTabUserName(state));
+    boolean tableUserExists = WalkingSecurity.get(state).userExists(WalkingSecurity.get(state).getTabUserName());
+    boolean cloudTableUserExists = conn.securityOperations().listUsers().contains(WalkingSecurity.get(state).getTabUserName());
     if (tableUserExists != cloudTableUserExists)
       throw new AccumuloException("Table User existance out of sync");
     
@@ -55,17 +55,16 @@ public class Validate extends Test {
     props.setProperty("target", "table");
     Authenticate.authenticate(conn, state, props);
     
-    boolean tabUserExists = SecurityHelper.getTabUserExists(state);
-    for (String user : new String[] {SecurityHelper.getSysUserName(state), SecurityHelper.getTabUserName(state)}) {
+    for (String user : new String[] {WalkingSecurity.get(state).getSysUserName(), WalkingSecurity.get(state).getTabUserName()}) {
       for (SystemPermission sp : SystemPermission.values()) {
-        boolean hasSp = SecurityHelper.getSysPerm(state, user, sp);
+        boolean hasSp = WalkingSecurity.get(state).hasSystemPermission(user, sp);
         boolean accuHasSp;
         try {
           accuHasSp = conn.securityOperations().hasSystemPermission(user, sp);
           log.debug("Just checked to see if user " + user + " has system perm " + sp.name() + " with answer " + accuHasSp);
         } catch (AccumuloSecurityException ae) {
           if (ae.getErrorCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {
-            if (tabUserExists)
+            if (tableUserExists)
               throw new AccumuloException("Got user DNE error when they should", ae);
             else
               continue;
@@ -77,14 +76,14 @@ public class Validate extends Test {
       }
       
       for (TablePermission tp : TablePermission.values()) {
-        boolean hasTp = SecurityHelper.getTabPerm(state, user, tp);
+        boolean hasTp = WalkingSecurity.get(state).hasTablePermission(user, WalkingSecurity.get(state).getTableName(), tp);
         boolean accuHasTp;
         try {
-          accuHasTp = conn.securityOperations().hasTablePermission(user, SecurityHelper.getTableName(state), tp);
+          accuHasTp = conn.securityOperations().hasTablePermission(user, WalkingSecurity.get(state).getTableName(), tp);
           log.debug("Just checked to see if user " + user + " has table perm " + tp.name() + " with answer " + accuHasTp);
         } catch (AccumuloSecurityException ae) {
           if (ae.getErrorCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {
-            if (tabUserExists)
+            if (tableUserExists)
               throw new AccumuloException("Got user DNE error when they should", ae);
             else
               continue;
@@ -102,13 +101,13 @@ public class Validate extends Test {
       
     }
     
-    Authorizations auths = SecurityHelper.getUserAuths(state, SecurityHelper.getTabUserName(state));
+    Authorizations auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
     Authorizations accuAuths;
     try {
-      accuAuths = conn.securityOperations().getUserAuthorizations(SecurityHelper.getTabUserName(state));
+      accuAuths = conn.securityOperations().getUserAuthorizations(WalkingSecurity.get(state).getTabUserName());
     } catch (AccumuloSecurityException ae) {
       if (ae.getErrorCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {
-        if (tabUserExists)
+        if (tableUserExists)
           throw new AccumuloException("Table user didn't exist when they should.", ae);
         else
           return;

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1389490&r1=1389489&r2=1389490&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Mon Sep 24 17:17:23 2012
@@ -50,8 +50,8 @@ import org.apache.accumulo.server.conf.S
 import org.apache.accumulo.server.constraints.MetadataConstraints;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
 import org.apache.accumulo.server.master.state.tables.TableManager;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityConstants;
-import org.apache.accumulo.server.security.SecurityOperationImpl;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.tabletserver.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -412,11 +412,19 @@ public class Initialize {
   }
   
   private static void initSecurity(String iid, byte[] rootpass) throws AccumuloSecurityException, ThriftSecurityException {
-    SecurityOperationImpl.getInstance(iid).initializeSecurity(SecurityConstants.getSystemCredentials(), ROOT_USER, rootpass);
+    AuditedSecurityOperation.getInstance(iid).initializeSecurity(SecurityConstants.getSystemCredentials(), ROOT_USER, rootpass);
   }
   
   protected static void initMetadataConfig() throws IOException {
     try {
+      Configuration conf = CachedConfiguration.getInstance();
+      int max = conf.getInt("dfs.replication.max", 512);
+      // Hadoop 0.23 switched the min value configuration name
+      int min = Math.max(conf.getInt("dfs.replication.min", 1), conf.getInt("dfs.namenode.replication.min", 1));
+      if (max < 5)
+        setMetadataReplication(max, "max");
+      if (min > 5)
+        setMetadataReplication(min, "min");
       for (Entry<String,String> entry : initialMetadataConf.entrySet())
         if (!TablePropUtil.setTableProperty(Constants.METADATA_TABLE_ID, entry.getKey(), entry.getValue()))
           throw new IOException("Cannot create per-table property " + entry.getKey());
@@ -426,6 +434,18 @@ public class Initialize {
     }
   }
   
+  private static void setMetadataReplication(int replication, String reason) throws IOException {
+    String rep = getConsoleReader().readLine(
+        "Your HDFS replication " + reason
+            + " is not compatible with our default !METADATA replication of 5. What do you want to set your !METADATA replication to? (" + replication + ") ");
+    if (rep == null || rep.length() == 0)
+      rep = Integer.toString(replication);
+    else
+      // Lets make sure it's a number
+      Integer.parseInt(rep);
+    initialMetadataConf.put(Property.TABLE_FILE_REPLICATION.getKey(), rep);
+  }
+
   public static boolean isInitialized(FileSystem fs) throws IOException {
     return (fs.exists(ServerConstants.getInstanceIdLocation()) || fs.exists(ServerConstants.getDataVersionLocation()));
   }

Propchange: accumulo/branches/ACCUMULO-259/src/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src:r1362561-1363473
  Merged /accumulo/branches/1.4/src/src:r1363430