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 2013/10/30 23:51:13 UTC

git commit: ACCUMULO-1715 - Applying and updating Michael Berman's patch

Updated Branches:
  refs/heads/master 9d6da9698 -> bc24925b2


ACCUMULO-1715 - Applying and updating Michael Berman's patch


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/bc24925b
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/bc24925b
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/bc24925b

Branch: refs/heads/master
Commit: bc24925b2fdc387384d30f324ff5964b1296ba53
Parents: 9d6da96
Author: John Vines <jv...@gmail.com>
Authored: Wed Oct 30 18:50:32 2013 -0400
Committer: John Vines <jv...@gmail.com>
Committed: Wed Oct 30 18:50:32 2013 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/Admin.java  | 185 +++++++++++--------
 .../org/apache/accumulo/test/DumpConfigIT.java  |   8 +-
 2 files changed, 119 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bc24925b/server/src/main/java/org/apache/accumulo/server/util/Admin.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/src/main/java/org/apache/accumulo/server/util/Admin.java
index fe84e52..2f5576c 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -22,6 +22,7 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.text.MessageFormat;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -44,6 +45,8 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.security.SystemCredentials;
@@ -54,43 +57,44 @@ import org.apache.log4j.Logger;
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
+import com.google.common.collect.Lists;
 import com.google.common.net.HostAndPort;
 
 public class Admin {
   private static final Logger log = Logger.getLogger(Admin.class);
-  
+
   static class AdminOpts extends ClientOpts {
     @Parameter(names = {"-f", "--force"}, description = "force the given server to stop by removing its lock")
     boolean force = false;
   }
-  
+
   @Parameters(commandDescription = "stop the tablet server on the given hosts")
   static class StopCommand {
     @Parameter(description = "<host> {<host> ... }")
     List<String> args = new ArrayList<String>();
   }
-  
+
   @Parameters(commandDescription = "Ping tablet servers.  If no arguments, pings all.")
   static class PingCommand {
     @Parameter(description = "{<host> ... }")
     List<String> args = new ArrayList<String>();
   }
-  
+
   @Parameters(commandDescription = "print tablets that are offline in online tables")
   static class CheckTabletsCommand {
     @Parameter(names = "--fixFiles", description = "Remove dangling file pointers")
     boolean fixFiles = false;
-    
+
     @Parameter(names = {"-t", "--table"}, description = "Table to check, if not set checks all tables")
     String table = null;
   }
-  
+
   @Parameters(commandDescription = "stop the master")
   static class StopMasterCommand {}
-  
+
   @Parameters(commandDescription = "stop all the servers")
   static class StopAllCommand {}
-  
+
   @Parameters(commandDescription = "list Accumulo instances in zookeeper")
   static class ListInstancesCommand {
     @Parameter(names = "--print-errors", description = "display errors while listing instances")
@@ -98,38 +102,40 @@ public class Admin {
     @Parameter(names = "--print-all", description = "print information for all instances, not just those with names")
     boolean printAll = false;
   }
-  
+
   @Parameters(commandDescription = "print out non-default configuration settings")
   static class DumpConfigCommand {
-    @Parameter(names={"-t", "--tables"}, description = "print per-table configuration")
+    @Parameter(names = {"-t", "--tables"}, description = "print per-table configuration")
     List<String> tables = new ArrayList<String>();
-    @Parameter(names={"-a", "--all"}, description = "print the system and all table configurations")
+    @Parameter(names = {"-a", "--all"}, description = "print the system and all table configurations")
     boolean allConfiguration = false;
-    @Parameter(names={"-s", "--system"}, description = "print the system configuration")
+    @Parameter(names = {"-s", "--system"}, description = "print the system configuration")
     boolean systemConfiguration = false;
-    @Parameter(names={"-d", "--directory"}, description = "directory to place config files")
+    @Parameter(names = {"-p", "--permissions"}, description = "print user permissions (must be used in combination with -a, -s, or -t)")
+    boolean userPermissions = false;
+    @Parameter(names = {"-d", "--directory"}, description = "directory to place config files")
     String directory = null;
   }
-  
+
   public static void main(String[] args) {
     boolean everything;
-    
+
     AdminOpts opts = new AdminOpts();
     JCommander cl = new JCommander(opts);
     cl.setProgramName(Admin.class.getName());
-    
+
     CheckTabletsCommand checkTabletsCommand = new CheckTabletsCommand();
     cl.addCommand("checkTablets", checkTabletsCommand);
-    
+
     ListInstancesCommand listIntancesOpts = new ListInstancesCommand();
     cl.addCommand("listInstances", listIntancesOpts);
-    
+
     PingCommand pingCommand = new PingCommand();
     cl.addCommand("ping", pingCommand);
-    
+
     DumpConfigCommand dumpConfigCommand = new DumpConfigCommand();
     cl.addCommand("dumpConfig", dumpConfigCommand);
-    
+
     StopCommand stopOpts = new StopCommand();
     cl.addCommand("stop", stopOpts);
     StopAllCommand stopAllOpts = new StopAllCommand();
@@ -137,13 +143,13 @@ public class Admin {
     StopMasterCommand stopMasterOpts = new StopMasterCommand();
     cl.addCommand("stopMaster", stopMasterOpts);
     cl.parse(args);
-    
+
     if (opts.help || cl.getParsedCommand() == null) {
       cl.usage();
       return;
     }
     Instance instance = opts.getInstance();
-    
+
     try {
       String principal;
       AuthenticationToken token;
@@ -154,9 +160,9 @@ public class Admin {
         principal = opts.principal;
         token = opts.getToken();
       }
-      
+
       int rc = 0;
-      
+
       if (cl.getParsedCommand().equals("listInstances")) {
         ListInstances.listInstances(instance.getZooKeepers(), listIntancesOpts.printAll, listIntancesOpts.printErrors);
       } else if (cl.getParsedCommand().equals("ping")) {
@@ -174,20 +180,20 @@ public class Admin {
           if (RemoveEntriesForMissingFiles.checkTable(instance, principal, token, checkTabletsCommand.table, checkTabletsCommand.fixFiles) != 0)
             rc = 6;
         }
-        
+
       } else if (cl.getParsedCommand().equals("stop")) {
         stopTabletServer(instance, new Credentials(principal, token), stopOpts.args, opts.force);
       } else if (cl.getParsedCommand().equals("dumpConfig")) {
         printConfig(instance, principal, token, dumpConfigCommand);
       } else {
         everything = cl.getParsedCommand().equals("stopAll");
-        
+
         if (everything)
           flushAll(instance, principal, token);
-        
+
         stopServer(instance, new Credentials(principal, token), everything);
       }
-      
+
       if (rc != 0)
         System.exit(rc);
     } catch (AccumuloException e) {
@@ -201,18 +207,18 @@ public class Admin {
       System.exit(3);
     }
   }
-  
+
   private static int ping(Instance instance, String principal, AuthenticationToken token, List<String> args) throws AccumuloException,
-  AccumuloSecurityException {
-    
+      AccumuloSecurityException {
+
     InstanceOperations io = instance.getConnector(principal, token).instanceOperations();
-    
+
     if (args.size() == 0) {
       args = io.getTabletServers();
     }
-    
+
     int unreachable = 0;
-    
+
     for (String tserver : args) {
       try {
         io.ping(tserver);
@@ -222,23 +228,23 @@ public class Admin {
         unreachable++;
       }
     }
-    
+
     System.out.printf("\n%d of %d tablet servers unreachable\n\n", unreachable, args.size());
     return unreachable;
   }
-  
+
   /**
    * flushing during shutdown is a performance optimization, its not required. The method will make an attempt to initiate flushes of all tables and give up if
    * it takes too long.
    * 
    */
   private static void flushAll(final Instance instance, final String principal, final AuthenticationToken token) throws AccumuloException,
-  AccumuloSecurityException {
-    
+      AccumuloSecurityException {
+
     final AtomicInteger flushesStarted = new AtomicInteger(0);
-    
+
     Runnable flushTask = new Runnable() {
-      
+
       @Override
       public void run() {
         try {
@@ -257,31 +263,31 @@ public class Admin {
         }
       }
     };
-    
+
     Thread flusher = new Thread(flushTask);
     flusher.setDaemon(true);
     flusher.start();
-    
+
     long start = System.currentTimeMillis();
     try {
       flusher.join(3000);
     } catch (InterruptedException e) {}
-    
+
     while (flusher.isAlive() && System.currentTimeMillis() - start < 15000) {
       int flushCount = flushesStarted.get();
       try {
         flusher.join(1000);
       } catch (InterruptedException e) {}
-      
+
       if (flushCount == flushesStarted.get()) {
         // no progress was made while waiting for join... maybe its stuck, stop waiting on it
         break;
       }
     }
   }
-  
+
   private static void stopServer(final Instance instance, final Credentials credentials, final boolean tabletServersToo) throws AccumuloException,
-  AccumuloSecurityException {
+      AccumuloSecurityException {
     MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
@@ -289,9 +295,9 @@ public class Admin {
       }
     });
   }
-  
+
   private static void stopTabletServer(final Instance instance, final Credentials creds, List<String> servers, final boolean force) throws AccumuloException,
-  AccumuloSecurityException {
+      AccumuloSecurityException {
     for (String server : servers) {
       HostAndPort address = AddressUtil.parseAddress(server);
       final String finalServer = address.toString();
@@ -304,20 +310,24 @@ public class Admin {
       });
     }
   }
-  
+
   private static final String ACCUMULO_SITE_BACKUP_FILE = "accumulo-site.xml.bak";
-  private static final MessageFormat format = new MessageFormat("config -t {0} -s {1}\n");
-  
+  private static final String PERMISSION_FILE_SUFFIX = "_perm.cfg";
+  private static final MessageFormat configFormat = new MessageFormat("config -t {0} -s {1}\n");
+  private static final MessageFormat sysPermFormat = new MessageFormat("grant System.{0} -s -u {1}\n");
+  private static final MessageFormat tablePermFormat = new MessageFormat("grant Table.{0} -t {1} -u {2}\n");
+
   private static DefaultConfiguration defaultConfig;
   private static Map<String,String> siteConfig, systemConfig;
-  
+  private static List<String> localUsers;
+
   public static void printConfig(Instance instance, String principal, AuthenticationToken token, DumpConfigCommand opts) throws Exception {
-    
+
     File outputDirectory = null;
     if (opts.directory != null) {
       outputDirectory = new File(opts.directory);
       if (!outputDirectory.isDirectory()) {
-        throw new IllegalArgumentException(opts.directory +  " does not exist on the local filesystem.");
+        throw new IllegalArgumentException(opts.directory + " does not exist on the local filesystem.");
       }
       if (!outputDirectory.canWrite()) {
         throw new IllegalArgumentException(opts.directory + " is not writable");
@@ -327,39 +337,44 @@ public class Admin {
     defaultConfig = AccumuloConfiguration.getDefaultConfiguration();
     siteConfig = connector.instanceOperations().getSiteConfiguration();
     systemConfig = connector.instanceOperations().getSystemConfiguration();
+    if (opts.userPermissions) {
+      localUsers = Lists.newArrayList(connector.securityOperations().listLocalUsers());
+      Collections.sort(localUsers);
+    }
     if (opts.allConfiguration) {
-      printSystemConfiguration(outputDirectory);
+      printSystemConfiguration(connector, outputDirectory, opts.userPermissions);
       SortedSet<String> tableNames = connector.tableOperations().list();
       for (String tableName : tableNames) {
-        printTableConfiguration(connector, tableName, outputDirectory);
+        printTableConfiguration(connector, tableName, outputDirectory, opts.userPermissions);
       }
-      
+
     } else {
       if (opts.systemConfiguration) {
-        printSystemConfiguration(outputDirectory);
+        printSystemConfiguration(connector, outputDirectory, opts.userPermissions);
       }
-      
+
       for (String tableName : opts.tables) {
-        printTableConfiguration(connector, tableName, outputDirectory);          
+        printTableConfiguration(connector, tableName, outputDirectory, opts.userPermissions);
       }
     }
   }
-  
-  private static String getDefaultConfigValue (String key) {
-    if (null == key) return null;
-    
+
+  private static String getDefaultConfigValue(String key) {
+    if (null == key)
+      return null;
+
     String defaultValue = null;
     try {
       Property p = Property.getPropertyByKey(key);
       if (null == p)
         return defaultValue;
       defaultValue = defaultConfig.get(p);
-    } catch (IllegalArgumentException e) {
-    }
+    } catch (IllegalArgumentException e) {}
     return defaultValue;
   }
-  
-  private static void printSystemConfiguration(File outputDirectory) throws IOException {
+
+  private static void printSystemConfiguration(Connector connector, File outputDirectory, boolean userPermissions) throws IOException, AccumuloException,
+      AccumuloSecurityException {
     Configuration conf = new Configuration(false);
     for (Entry<String,String> prop : siteConfig.entrySet()) {
       String defaultValue = getDefaultConfigValue(prop.getKey());
@@ -380,22 +395,48 @@ public class Admin {
     } finally {
       fos.close();
     }
+    if (userPermissions) {
+      File permScript = new File(outputDirectory, "system" + PERMISSION_FILE_SUFFIX);
+      FileWriter writer = new FileWriter(permScript);
+      for (String principal : localUsers) {
+        for (SystemPermission perm : SystemPermission.values()) {
+          if (connector.securityOperations().hasSystemPermission(principal, perm)) {
+            writer.write(sysPermFormat.format(new String[] {perm.name(), principal}));
+          }
+        }
+      }
+      writer.close();
+    }
   }
-  
-  private static void printTableConfiguration(Connector connector, String tableName, File outputDirectory) throws AccumuloException, TableNotFoundException, IOException {
+
+  private static void printTableConfiguration(Connector connector, String tableName, File outputDirectory, boolean userPermissions) throws AccumuloException,
+      TableNotFoundException, IOException, AccumuloSecurityException {
     Iterable<Entry<String,String>> tableConfig = connector.tableOperations().getProperties(tableName);
-    File tableBackup = new File(outputDirectory, tableName+".cfg");
+    File tableBackup = new File(outputDirectory, tableName + ".cfg");
     FileWriter writer = new FileWriter(tableBackup);
     for (Entry<String,String> prop : tableConfig) {
       if (prop.getKey().startsWith(Property.TABLE_PREFIX.getKey())) {
         String defaultValue = getDefaultConfigValue(prop.getKey());
         if (defaultValue == null || !defaultValue.equals(prop.getValue())) {
           if (!prop.getValue().equals(siteConfig.get(prop.getKey())) && !prop.getValue().equals(systemConfig.get(prop.getKey()))) {
-            writer.write(format.format(new String[] {tableName, prop.getKey() + "=" + prop.getValue() } ));
+            writer.write(configFormat.format(new String[] {tableName, prop.getKey() + "=" + prop.getValue()}));
           }
         }
       }
     }
     writer.close();
+
+    if (userPermissions) {
+      File permScript = new File(outputDirectory, tableName + PERMISSION_FILE_SUFFIX);
+      FileWriter permWriter = new FileWriter(permScript);
+      for (String principal : localUsers) {
+        for (TablePermission perm : TablePermission.values()) {
+          if (connector.securityOperations().hasTablePermission(principal, tableName, perm)) {
+            permWriter.write(tablePermFormat.format(new String[] {perm.name(), tableName, principal}));
+          }
+        }
+      }
+      permWriter.close();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bc24925b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
index c653751..349f709 100644
--- a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
@@ -48,13 +48,17 @@ public class DumpConfigIT extends ConfigurableMacIT {
   public void test() throws Exception {
     File siteFileBackup = new File(folder.getRoot(), "accumulo-site.xml.bak");
     assertFalse(siteFileBackup.exists());
-    assertEquals(0, exec(Admin.class, new String[] {"dumpConfig", "-a", "-d", folder.getRoot().getPath()}).waitFor());
+    assertEquals(0, exec(Admin.class, new String[] {"dumpConfig", "-a", "-p", "-d", folder.getRoot().getPath()}).waitFor());
     assertTrue(siteFileBackup.exists());
     String site = FunctionalTestUtils.readAll(new FileInputStream(siteFileBackup));
     assertTrue(site.contains(Property.TABLE_FILE_BLOCK_SIZE.getKey()));
     assertTrue(site.contains("1234567"));
     String meta = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), MetadataTable.NAME + ".cfg")));
     assertTrue(meta.contains(Property.TABLE_FILE_REPLICATION.getKey()));
+    String systemPerm = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), "system_perm.cfg")));
+    assertTrue(systemPerm.contains("grant System.ALTER_USER -s -u root"));
+    String metaPerm = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), "!METADATA_perm.cfg")));
+    assertTrue(metaPerm.contains("grant Table.READ -t !METADATA -u root"));
+    assertFalse(metaPerm.contains("grant Table.DROP -t !METADATA -u root"));
   }
-
 }