You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2012/01/25 21:29:21 UTC

svn commit: r1235917 - in /incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server: logger/LogReader.java test/randomwalk/bulk/BulkPlusOne.java

Author: kturner
Date: Wed Jan 25 20:29:21 2012
New Revision: 1235917

URL: http://svn.apache.org/viewvc?rev=1235917&view=rev
Log:
ACCUMULO-334 Added ability to filter on tablet and row regex to LogReader.  Made bulk test print when a bulk import finishes.

Modified:
    incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/logger/LogReader.java
    incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/bulk/BulkPlusOne.java

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/logger/LogReader.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/logger/LogReader.java?rev=1235917&r1=1235916&r2=1235917&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/logger/LogReader.java (original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/logger/LogReader.java Wed Jan 25 20:29:21 2012
@@ -18,7 +18,12 @@ package org.apache.accumulo.server.logge
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
+import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -38,7 +43,7 @@ import org.apache.hadoop.io.Text;
 
 public class LogReader {
   public static void usage() {
-    System.err.println("Usage : " + LogReader.class.getName() + " [-r <row>] [-m <maxColumns] <log file>");
+    System.err.println("Usage : " + LogReader.class.getName() + " [-r <row>] [-m <maxColumns] [-t <key extent>] [-p <row regex>] <log file>");
   }
   
   /**
@@ -55,9 +60,13 @@ public class LogReader {
     FileSystem local = TraceFileSystem.wrap(FileSystem.getLocal(conf));
     Option rowOpt = new Option("r", "--row", true, "search for a specific row");
     Option maxOpt = new Option("m", "--max-mutations", true, "the maximum number of mutations to print per log entry");
+    Option tabletOpt = new Option("t", "--tablet", true, "key extent");
+    Option rowPatternOpt = new Option("p", "--row-pattern", true, "search for a row that matches the given regex");
     Options options = new Options();
     options.addOption(rowOpt);
     options.addOption(maxOpt);
+    options.addOption(tabletOpt);
+    options.addOption(rowPatternOpt);
     CommandLine cl;
     try {
       cl = new BasicParser().parse(options, args);
@@ -66,6 +75,8 @@ public class LogReader {
       return;
     }
     
+    Matcher rowMatcher = null;
+    KeyExtent ke = null;
     Text row = null;
     int max = 5;
     String[] files = cl.getArgs();
@@ -77,7 +88,18 @@ public class LogReader {
       row = new Text(cl.getOptionValue(rowOpt.getOpt()));
     if (cl.hasOption(maxOpt.getOpt()))
       max = Integer.parseInt(cl.getOptionValue(maxOpt.getOpt()));
+    if (cl.hasOption(tabletOpt.getOpt())) {
+      String extent = cl.getOptionValue(tabletOpt.getOpt());
+      String sa[] = extent.split(";");
+      ke = new KeyExtent(new Text(sa[0]), new Text(sa[1]), new Text(sa[2]));
+    }
+    if (cl.hasOption(rowPatternOpt.getOpt())) {
+      Pattern pattern = Pattern.compile(cl.getOptionValue(rowPatternOpt.getOpt()));
+      rowMatcher = pattern.matcher("");
+    }
     
+    Set<Integer> tabletIds = new HashSet<Integer>();
+
     for (String file : files) {
       
       Path path = new Path(file);
@@ -88,40 +110,61 @@ public class LogReader {
         // read log entries from a simple hdfs file
         org.apache.hadoop.io.SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(file), conf);
         while (reader.next(key, value)) {
-          printLogEvent(key, value, row, max);
+          printLogEvent(key, value, row, rowMatcher, ke, tabletIds, max);
         }
       } else if (local.isFile(path)) {
         // read log entries from a simple file
         org.apache.hadoop.io.SequenceFile.Reader reader = new SequenceFile.Reader(local, new Path(file), conf);
         while (reader.next(key, value)) {
-          printLogEvent(key, value, row, max);
+          printLogEvent(key, value, row, rowMatcher, ke, tabletIds, max);
         }
       } else {
         try {
           // read the log entries sorted in a map file
           MultiReader input = new MultiReader(fs, conf, file);
           while (input.next(key, value)) {
-            printLogEvent(key, value, row, max);
+            printLogEvent(key, value, row, rowMatcher, ke, tabletIds, max);
           }
         } catch (FileNotFoundException ex) {
           SequenceFile.Reader input = new SequenceFile.Reader(local, new Path(file), conf);
           while (input.next(key, value)) {
-            printLogEvent(key, value, row, max);
+            printLogEvent(key, value, row, rowMatcher, ke, tabletIds, max);
           }
         }
       }
     }
   }
   
-  public static void printLogEvent(LogFileKey key, LogFileValue value, Text row, int maxMutations) {
-    if (row != null) {
+  public static void printLogEvent(LogFileKey key, LogFileValue value, Text row, Matcher rowMatcher, KeyExtent ke, Set<Integer> tabletIds, int maxMutations) {
+    
+    if (ke != null) {
+      if (key.event == LogEvents.DEFINE_TABLET) {
+        if (key.tablet.equals(ke)) {
+          tabletIds.add(key.tid);
+        } else {
+          return;
+        }
+      } else if (!tabletIds.contains(key.tid)) {
+        return;
+      }
+    }
+
+    if (row != null || rowMatcher != null) {
       if (key.event == LogEvents.MUTATION || key.event == LogEvents.MANY_MUTATIONS) {
         boolean found = false;
         for (Mutation m : value.mutations) {
-          if (new Text(m.getRow()).equals(row)) {
+          if (row != null && new Text(m.getRow()).equals(row)) {
             found = true;
             break;
           }
+          
+          if (rowMatcher != null) {
+            rowMatcher.reset(new String(m.getRow()));
+            if (rowMatcher.matches()) {
+              found = true;
+              break;
+            }
+          }
         }
         
         if (!found)

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/bulk/BulkPlusOne.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/bulk/BulkPlusOne.java?rev=1235917&r1=1235916&r2=1235917&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/bulk/BulkPlusOne.java (original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/bulk/BulkPlusOne.java Wed Jan 25 20:29:21 2012
@@ -101,6 +101,7 @@ public class BulkPlusOne extends BulkTes
     FileStatus[] failures = fs.listStatus(fail);
     if (failures != null && failures.length > 0)
       throw new Exception("Failures " + Arrays.asList(failures) + " found importing files from " + dir);
+    log.debug("Finished bulk import, start rows " + printRows + " last row " + String.format(FMT, LOTS - 1) + " marker " + markerColumnFamily);
   }
   
   @Override