You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bi...@apache.org on 2012/01/20 16:26:57 UTC

svn commit: r1233952 - in /incubator/accumulo/branches/1.4/src/core/src: main/java/org/apache/accumulo/core/client/admin/ main/java/org/apache/accumulo/core/util/shell/ main/java/org/apache/accumulo/core/util/shell/commands/ test/java/org/apache/accumu...

Author: billie
Date: Fri Jan 20 15:26:55 2012
New Revision: 1233952

URL: http://svn.apache.org/viewvc?rev=1233952&view=rev
Log:
ACCUMULO-311 created listiter command, changed return type of tableOperations().listIterators(tableName)

Added:
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java   (with props)
Modified:
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java Fri Jan 20 15:26:55 2012
@@ -523,7 +523,7 @@ public interface TableOperations {
    * @throws AccumuloException
    * @throws TableNotFoundException
    */
-  public Set<String> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
+  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
   
   /**
    * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java Fri Jan 20 15:26:55 2012
@@ -18,10 +18,8 @@ package org.apache.accumulo.core.client.
 
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -93,19 +91,20 @@ public abstract class TableOperationsHel
   }
   
   @Override
-  public Set<String> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(null, tableName, null);
-    Set<String> result = new HashSet<String>();
-    Set<String> lifecycles = new HashSet<String>();
-    for (IteratorScope scope : IteratorScope.values())
-      lifecycles.add(scope.name().toLowerCase());
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
     for (Entry<String,String> property : this.getProperties(tableName)) {
       String name = property.getKey();
       String[] parts = name.split("\\.");
       if (parts.length == 4) {
-        if (parts[0].equals("table") && parts[1].equals("iterator") && lifecycles.contains(parts[2]))
-          result.add(parts[3]);
+        if (parts[0].equals("table") && parts[1].equals("iterator")) {
+          IteratorScope scope = IteratorScope.valueOf(parts[2]);
+          if (!result.containsKey(parts[3]))
+            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
+          result.get(parts[3]).add(scope);
+        }
       }
     }
     return result;

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java Fri Jan 20 15:26:55 2012
@@ -102,6 +102,7 @@ import org.apache.accumulo.core.util.she
 import org.apache.accumulo.core.util.shell.commands.ImportDirectoryCommand;
 import org.apache.accumulo.core.util.shell.commands.InfoCommand;
 import org.apache.accumulo.core.util.shell.commands.InsertCommand;
+import org.apache.accumulo.core.util.shell.commands.ListIterCommand;
 import org.apache.accumulo.core.util.shell.commands.ListScansCommand;
 import org.apache.accumulo.core.util.shell.commands.MasterStateCommand;
 import org.apache.accumulo.core.util.shell.commands.MaxRowCommand;
@@ -165,7 +166,7 @@ public class Shell {
   private AuthInfo credentials;
   private Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
   private Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
-  private Map<String, Class<? extends Formatter>> tableFormatters = new HashMap<String, Class<? extends Formatter>>();
+  private Map<String,Class<? extends Formatter>> tableFormatters = new HashMap<String,Class<? extends Formatter>>();
   public Map<String,List<IteratorSetting>> scanIteratorOptions = new HashMap<String,List<IteratorSetting>>();
   
   private Token rootToken;
@@ -355,12 +356,12 @@ public class Shell {
         new DeleteTableCommand(), new DeleteUserCommand(), new DropTableCommand(), new DropUserCommand(), new DUCommand(), new EGrepCommand(),
         new ExecfileCommand(), new ExitCommand(), new FlushCommand(), new FormatterCommand(), new GetAuthsCommand(), new GetGroupsCommand(),
         new GetSplitsCommand(), new GrantCommand(), new GrepCommand(), new HelpCommand(), new HiddenCommand(), new HistoryCommand(),
-        new ImportDirectoryCommand(), new InfoCommand(), new InsertCommand(), new ListScansCommand(), new MasterStateCommand(), new MaxRowCommand(),
-        new MergeCommand(), new NoTableCommand(), new OfflineCommand(), new OnlineCommand(), new PasswdCommand(), new QuestionCommand(), new QuitCommand(),
-        new RenameTableCommand(), new RevokeCommand(), new ScanCommand(), new SelectCommand(), new SelectrowCommand(), new SetAuthsCommand(),
-        new SetGroupsCommand(), new SetIterCommand(), new SetScanIterCommand(), new SleepCommand(), new SystemPermissionsCommand(), new TableCommand(),
-        new TablePermissionsCommand(), new TablesCommand(), new TraceCommand(), new UserCommand(), new UserPermissionsCommand(), new UsersCommand(),
-        new WhoAmICommand(),};
+        new ImportDirectoryCommand(), new InfoCommand(), new InsertCommand(), new ListIterCommand(), new ListScansCommand(), new MasterStateCommand(),
+        new MaxRowCommand(), new MergeCommand(), new NoTableCommand(), new OfflineCommand(), new OnlineCommand(), new PasswdCommand(), new QuestionCommand(),
+        new QuitCommand(), new RenameTableCommand(), new RevokeCommand(), new ScanCommand(), new SelectCommand(), new SelectrowCommand(),
+        new SetAuthsCommand(), new SetGroupsCommand(), new SetIterCommand(), new SetScanIterCommand(), new SleepCommand(), new SystemPermissionsCommand(),
+        new TableCommand(), new TablePermissionsCommand(), new TablesCommand(), new TraceCommand(), new UserCommand(), new UserPermissionsCommand(),
+        new UsersCommand(), new WhoAmICommand(),};
     for (Command cmd : external) {
       commandFactory.put(cmd.getName(), cmd);
     }
@@ -462,7 +463,7 @@ public class Shell {
     sb.append("- Debug: ").append(isDebuggingEnabled() ? "on" : "off").append("\n");
     if (!tableFormatters.isEmpty()) {
       sb.append("- Active Formatters");
-      for (Entry<String, Class<? extends Formatter>> entry : tableFormatters.entrySet()) {
+      for (Entry<String,Class<? extends Formatter>> entry : tableFormatters.entrySet()) {
         if (null != entry.getValue()) {
           sb.append("-    Table: ").append(entry.getKey()).append(", ").append(entry.getValue().getName()).append("\n");
         }
@@ -1003,6 +1004,7 @@ public class Shell {
   
   /**
    * Pull the current formatter for the given table and cache it.
+   * 
    * @param tableName
    * @return The formatter class for the given table
    */

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java Fri Jan 20 15:26:55 2012
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig;
 import org.apache.accumulo.core.security.VisibilityConstraint;
 import org.apache.accumulo.core.util.BadArgumentException;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.accumulo.core.util.format.Formatter;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -132,12 +131,13 @@ public class CreateTableCommand extends 
     TimeType timeType = TimeType.MILLIS;
     if (cl.hasOption(createTableOptTimeLogical.getOpt()))
       timeType = TimeType.LOGICAL;
-
     
     // create table
     shellState.getConnector().tableOperations().create(tableName, true, timeType);
-    shellState.getConnector().tableOperations().addSplits(tableName, partitions);
-    shellState.getConnector().tableOperations().addAggregators(tableName, aggregators);
+    if (partitions.size() > 0)
+      shellState.getConnector().tableOperations().addSplits(tableName, partitions);
+    if (aggregators.size() > 0)
+      shellState.getConnector().tableOperations().addAggregators(tableName, aggregators);
     
     shellState.setTableName(tableName); // switch shell to new table
     // context
@@ -179,15 +179,15 @@ public class CreateTableCommand extends 
         shellState.getConnector().tableOperations()
             .setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.getKey() + (max + 1), VisibilityConstraint.class.getName());
     }
-
+    
     // Load custom formatter if set
     if (cl.hasOption(createTableOptFormatter.getOpt())) {
-        String formatterClass = cl.getOptionValue(createTableOptFormatter.getOpt());
-        
-        shellState.setFormatterClass(tableName, AccumuloClassLoader.loadClass(formatterClass, Formatter.class));
-        
-        shellState.getConnector().tableOperations().setProperty(tableName, Property.TABLE_FORMATTER_CLASS.toString(), formatterClass);   
-    }    
+      String formatterClass = cl.getOptionValue(createTableOptFormatter.getOpt());
+      
+      shellState.setFormatterClass(tableName, AccumuloClassLoader.loadClass(formatterClass, Formatter.class));
+      
+      shellState.getConnector().tableOperations().setProperty(tableName, Property.TABLE_FORMATTER_CLASS.toString(), formatterClass);
+    }
     
     return 0;
   }

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java Fri Jan 20 15:26:55 2012
@@ -46,7 +46,7 @@ public class DeleteIterCommand extends C
     }
     
     String name = cl.getOptionValue(nameOpt.getOpt());
-    if (!shellState.getConnector().tableOperations().listIterators(tableName).contains(name)) {
+    if (!shellState.getConnector().tableOperations().listIterators(tableName).containsKey(name)) {
       Shell.log.warn("no iterators found that match your criteria");
       return 0;
     }

Added: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java?rev=1233952&view=auto
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java (added)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java Fri Jan 20 15:26:55 2012
@@ -0,0 +1,124 @@
+/**
+ * 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.accumulo.core.util.shell.commands;
+
+import java.util.EnumMap;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.util.shell.Shell;
+import org.apache.accumulo.core.util.shell.Shell.Command;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+
+public class ListIterCommand extends Command {
+  private Option tableOpt, nameOpt;
+  private Map<IteratorScope,Option> scopeOpts;
+  
+  @Override
+  public int execute(String fullCommand, CommandLine cl, Shell shellState) throws Exception {
+    String tableName;
+    
+    if (cl.hasOption(tableOpt.getOpt())) {
+      tableName = cl.getOptionValue(tableOpt.getOpt());
+      if (!shellState.getConnector().tableOperations().exists(tableName))
+        throw new TableNotFoundException(null, tableName, null);
+    } else {
+      shellState.checkTableState();
+      tableName = shellState.getTableName();
+    }
+    
+    Map<String,EnumSet<IteratorScope>> iterators = shellState.getConnector().tableOperations().listIterators(tableName);
+    
+    if (cl.hasOption(nameOpt.getOpt())) {
+      String name = cl.getOptionValue(nameOpt.getOpt());
+      if (!iterators.containsKey(name)) {
+        Shell.log.warn("no iterators found that match your criteria");
+        return 0;
+      }
+      EnumSet<IteratorScope> scopes = iterators.get(name);
+      iterators.clear();
+      iterators.put(name, scopes);
+    }
+    
+    boolean hasScope = false;
+    for (IteratorScope scope : IteratorScope.values()) {
+      if (cl.hasOption(scopeOpts.get(scope).getOpt()))
+        hasScope = true;
+    }
+    if (!hasScope)
+      throw new IllegalArgumentException("You must select at least one scope to configure");
+    
+    StringBuilder sb = new StringBuilder("-\n");
+    for (String name : iterators.keySet()) {
+      for (IteratorScope scope : iterators.get(name)) {
+        if (cl.hasOption(scopeOpts.get(scope).getOpt())) {
+          IteratorSetting setting = shellState.getConnector().tableOperations().getIteratorSetting(tableName, name, scope);
+          sb.append("-    Iterator ").append(setting.getName()).append(", ").append(scope).append(" scope options:\n");
+          sb.append("-        ").append("iteratorPriority").append(" = ").append(setting.getPriority()).append("\n");
+          sb.append("-        ").append("iteratorClassName").append(" = ").append(setting.getIteratorClass()).append("\n");
+          for (Entry<String,String> optEntry : setting.getProperties().entrySet()) {
+            sb.append("-        ").append(optEntry.getKey()).append(" = ").append(optEntry.getValue()).append("\n");
+          }
+        }
+      }
+    }
+    sb.append("-\n");
+    shellState.getReader().printString(sb.toString());
+    
+    return 0;
+  }
+  
+  public String description() {
+    return "lists table-specific iterators";
+  }
+  
+  @Override
+  public int numArgs() {
+    return 0;
+  }
+  
+  @Override
+  public Options getOptions() {
+    Options o = new Options();
+    
+    tableOpt = new Option(Shell.tableOption, "table", true, "tableName");
+    tableOpt.setArgName("table");
+    
+    nameOpt = new Option("n", "name", true, "iterator to delete");
+    nameOpt.setArgName("itername");
+    
+    scopeOpts = new EnumMap<IteratorScope,Option>(IteratorScope.class);
+    scopeOpts.put(IteratorScope.minc, new Option(IteratorScope.minc.name(), "minor-compaction", false, "applied at minor compaction"));
+    scopeOpts.put(IteratorScope.majc, new Option(IteratorScope.majc.name(), "major-compaction", false, "applied at major compaction"));
+    scopeOpts.put(IteratorScope.scan, new Option(IteratorScope.scan.name(), "scan-time", false, "applied at scan time"));
+    
+    o.addOption(tableOpt);
+    o.addOption(nameOpt);
+    
+    for (Option opt : scopeOpts.values()) {
+      o.addOption(opt);
+    }
+    
+    return o;
+  }
+}

Propchange: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java?rev=1233952&r1=1233951&r2=1233952&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java Fri Jan 20 15:26:55 2012
@@ -214,16 +214,23 @@ public class TableOperationsHelperTest {
     t.check("table", new String[] {"table.iterator.majc.someName=10,foo.bar", "table.iterator.majc.someName.opt.key=value",
         "table.iterator.scan.someName=10,foo.bar",});
     
+    t.removeIterator("table", "someName", EnumSet.of(IteratorScope.scan));
     setting = new IteratorSetting(20, "otherName", "some.classname");
     setting.setScopes(EnumSet.of(IteratorScope.majc));
     setting.addOptions(Collections.singletonMap("key", "value"));
     t.attachIterator("table", setting);
     setting = new IteratorSetting(20, "otherName", "some.classname");
     t.attachIterator("table", setting);
-    Set<String> two = t.listIterators("table");
+    Map<String,EnumSet<IteratorScope>> two = t.listIterators("table");
     Assert.assertEquals(2, two.size());
-    Assert.assertTrue(two.contains("otherName"));
-    Assert.assertTrue(two.contains("someName"));
+    Assert.assertTrue(two.containsKey("otherName"));
+    System.out.println(two.get("otherName"));
+    Assert.assertTrue(two.get("otherName").size() == 2);
+    Assert.assertTrue(two.get("otherName").contains(IteratorScope.majc));
+    Assert.assertTrue(two.get("otherName").contains(IteratorScope.scan));
+    Assert.assertTrue(two.containsKey("someName"));
+    Assert.assertTrue(two.get("someName").size() == 1);
+    Assert.assertTrue(two.get("someName").contains(IteratorScope.majc));
     t.removeIterator("table", "someName", EnumSet.allOf(IteratorScope.class));
     t.check("table", new String[] {"table.iterator.majc.otherName=20,some.classname", "table.iterator.majc.otherName.opt.key=value",
         "table.iterator.scan.otherName=20,some.classname",});