You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/11/05 19:44:12 UTC

[1/3] git commit: ACCUMULO-1832 prep for command to re-write uris

Updated Branches:
  refs/heads/master 8f87c720d -> 6239604c1


ACCUMULO-1832 prep for command to re-write uris


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

Branch: refs/heads/master
Commit: e86a8d063e68ff94fb6aaf71a778f3b8c6ba4a9b
Parents: 109c179
Author: Eric Newton <er...@gmail.com>
Authored: Tue Nov 5 13:41:46 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Tue Nov 5 13:41:46 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  6 +-
 .../core/tabletserver/log/LogEntry.java         | 36 +++++++++
 .../core/util/shell/commands/OnlineCommand.java |  6 +-
 .../master/state/MetaDataTableScanner.java      |  4 +-
 .../master/state/ZooTabletStateStore.java       |  6 +-
 .../accumulo/server/util/MetadataTableUtil.java | 79 ++------------------
 .../server/util/OfflineMetadataScanner.java     |  2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        |  3 +-
 .../java/org/apache/accumulo/master/Master.java | 10 +--
 .../accumulo/master/TabletGroupWatcher.java     |  2 +-
 .../org/apache/accumulo/tserver/Tablet.java     |  4 +-
 .../apache/accumulo/tserver/TabletServer.java   |  4 +-
 12 files changed, 67 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 1b652f9..85d9fa2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -1177,8 +1177,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
         range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
       else
         range = new Range(startRow, lastRow);
-      
-      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      String metaTable = MetadataTable.NAME;
+      if (tableId.equals(MetadataTable.ID))
+        metaTable = RootTable.NAME;
+      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(metaTable, Authorizations.EMPTY);
       scanner = new IsolatedScanner(scanner);
       TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 7864f2d..fc2da4b 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -21,9 +21,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.util.StringUtil;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
 
 public class LogEntry {
   public KeyExtent extent;
@@ -71,4 +76,35 @@ public class LogEntry {
     this.logSet = logSet;
   }
   
+  static private final Text EMPTY_TEXT = new Text();
+
+  public static LogEntry fromKeyValue(Key key, Value value) {
+    LogEntry result = new LogEntry();
+    result.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
+    String[] parts = key.getColumnQualifier().toString().split("/", 2);
+    result.server = parts[0];
+    result.filename = parts[1];
+    parts = value.toString().split("\\|");
+    result.tabletId = Integer.parseInt(parts[1]);
+    result.logSet = Arrays.asList(parts[0].split(";"));
+    result.timestamp = key.getTimestamp();
+    return result;
+  }
+  
+  public Text getRow() {
+    return extent.getMetadataEntry();
+  }
+  
+  public Text getColumnFamily() {
+    return MetadataSchema.TabletsSection.LogColumnFamily.NAME;
+  }
+  
+  public Text getColumnQualifier() {
+    return new Text(server + "/" + filename);
+  }
+  
+  public Value getValue() {
+    return new Value((StringUtil.join(logSet, ";") + "|" + tabletId).getBytes());
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
index bbf2b72..7b6ebe2 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.util.shell.commands;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -37,8 +37,8 @@ public class OnlineCommand extends TableOperation {
   
   @Override
   protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (tableName.equals(MetadataTable.NAME)) {
-      Shell.log.info("  The " + MetadataTable.NAME + " is always online.");
+    if (tableName.equals(RootTable.NAME)) {
+      Shell.log.info("  The " + RootTable.NAME + " is always online.");
     } else {
       shellState.getConnector().tableOperations().online(tableName, wait);
       Shell.log.info("Online of table " + tableName + (wait ? " completed." : " initiated..."));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index bae614d..5471fac 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -50,7 +50,7 @@ public class MetaDataTableScanner implements Iterator<TabletLocationState> {
   private static final Logger log = Logger.getLogger(MetaDataTableScanner.class);
   
   BatchScanner mdScanner = null;
-  Iterator<Entry<Key,Value>> iter;
+  Iterator<Entry<Key,Value>> iter = null;
   
   public MetaDataTableScanner(Instance instance, Credentials credentials, Range range, CurrentState state) {
     this(instance, credentials, range, state, MetadataTable.NAME);
@@ -67,6 +67,8 @@ public class MetaDataTableScanner implements Iterator<TabletLocationState> {
     } catch (Exception ex) {
       if (mdScanner != null)
         mdScanner.close();
+      iter = null;
+      mdScanner = null;
       throw new RuntimeException(ex);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index 2c5b709..c792917 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -23,9 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.master.state.TabletLocationState;
-import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.log4j.Logger;
 
@@ -84,7 +82,7 @@ public class ZooTabletStateStore extends TabletStateStore {
           for (String entry : store.getChildren(RootTable.ZROOT_TABLET_WALOGS)) {
             byte[] logInfo = store.get(RootTable.ZROOT_TABLET_WALOGS + "/" + entry);
             if (logInfo != null) {
-              MetadataTableUtil.LogEntry logEntry = new MetadataTableUtil.LogEntry();
+              LogEntry logEntry = new LogEntry();
               logEntry.fromBytes(logInfo);
               logs.add(logEntry.logSet);
               log.debug("root tablet logSet " + logEntry.logSet);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 6979268..cbe42b7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -18,8 +18,6 @@ package org.apache.accumulo.server.util;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -66,11 +64,11 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Lo
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -88,8 +86,6 @@ import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
@@ -467,55 +463,6 @@ public class MetadataTableUtil {
     bw.close();
   }
 
-  public static class LogEntry {
-    public KeyExtent extent;
-    public long timestamp;
-    public String server;
-    public String filename;
-    public int tabletId;
-    public Collection<String> logSet;
-
-    @Override
-    public String toString() {
-      return extent.toString() + " " + filename + " (" + tabletId + ")";
-    }
-
-    public String getName() {
-      return server + "/" + filename;
-    }
-
-    public byte[] toBytes() throws IOException {
-      DataOutputBuffer out = new DataOutputBuffer();
-      extent.write(out);
-      out.writeLong(timestamp);
-      out.writeUTF(server);
-      out.writeUTF(filename.toString());
-      out.write(tabletId);
-      out.write(logSet.size());
-      for (String s : logSet) {
-        out.writeUTF(s);
-      }
-      return Arrays.copyOf(out.getData(), out.getLength());
-    }
-
-    public void fromBytes(byte bytes[]) throws IOException {
-      DataInputBuffer inp = new DataInputBuffer();
-      inp.reset(bytes, bytes.length);
-      extent = new KeyExtent();
-      extent.readFields(inp);
-      timestamp = inp.readLong();
-      server = inp.readUTF();
-      filename = inp.readUTF();
-      tabletId = inp.read();
-      int count = inp.read();
-      ArrayList<String> logSet = new ArrayList<String>(count);
-      for (int i = 0; i < count; i++)
-        logSet.add(inp.readUTF());
-      this.logSet = logSet;
-    }
-
-  }
-
   static String getZookeeperLogLocation() {
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
   }
@@ -542,26 +489,12 @@ public class MetadataTableUtil {
         UtilWaitThread.sleep(1000);
       }
     } else {
-      String value = StringUtil.join(entry.logSet, ";") + "|" + entry.tabletId;
-      Mutation m = new Mutation(entry.extent.getMetadataEntry());
-      m.put(LogColumnFamily.NAME, new Text(entry.server + "/" + entry.filename), new Value(value.getBytes()));
+      Mutation m = new Mutation(entry.getRow());
+      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
       update(credentials, zooLock, m, entry.extent);
     }
   }
 
-  public static LogEntry entryFromKeyValue(Key key, Value value) {
-    MetadataTableUtil.LogEntry e = new MetadataTableUtil.LogEntry();
-    e.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
-    String[] parts = key.getColumnQualifier().toString().split("/", 2);
-    e.server = parts[0];
-    e.filename = parts[1];
-    parts = value.toString().split("\\|");
-    e.tabletId = Integer.parseInt(parts[1]);
-    e.logSet = Arrays.asList(parts[0].split(";"));
-    e.timestamp = key.getTimestamp();
-    return e;
-  }
-
   public static String getRootTabletDir() throws IOException {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
@@ -605,7 +538,7 @@ public class MetadataTableUtil {
         }
 
         if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
-          result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
+          result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
         } else if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
           DataFileValue dfv = new DataFileValue(entry.getValue().get());
           sizes.put(new FileRef(fs, entry.getKey()), dfv);
@@ -632,7 +565,7 @@ public class MetadataTableUtil {
         Text row = entry.getKey().getRow();
         if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
           if (row.equals(pattern)) {
-            result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
+            result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
           }
         }
       }
@@ -723,7 +656,7 @@ public class MetadataTableUtil {
         return rootTableEntries.next();
       }
       Entry<Key,Value> entry = metadataEntries.next();
-      return entryFromKeyValue(entry.getKey(), entry.getValue());
+      return LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/base/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java b/server/base/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
index cefec22..cdb18a5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
@@ -49,13 +49,13 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.util.MetadataTableUtil.LogEntry;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 776d68a..98b7bd6 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.util.AddressUtil;
@@ -225,7 +226,7 @@ public class GarbageCollectWriteAheadLogs {
   private int removeMetadataEntries(Map<Path,String> fileToServerMap, Set<Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
       InterruptedException {
     int count = 0;
-    Iterator<MetadataTableUtil.LogEntry> iterator = MetadataTableUtil.getLogEntries(SystemCredentials.get());
+    Iterator<LogEntry> iterator = MetadataTableUtil.getLogEntries(SystemCredentials.get());
     while (iterator.hasNext()) {
       for (String entry : iterator.next().logSet) {
         String parts[] = entry.split("/", 2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 8a1a9b2..887f527 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -428,9 +428,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return result;
   }
 
-  private void checkNotMetadataID(String tableId, TableOperation operation) throws ThriftTableOperationException {
-    if (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) {
-      String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
+  private void checkNotRootID(String tableId, TableOperation operation) throws ThriftTableOperationException {
+    if (RootTable.ID.equals(tableId)) {
+      String why = "Table name cannot be == " + RootTable.NAME;
       log.warn(why);
       throw new ThriftTableOperationException(tableId, null, operation, TableOperationExceptionType.OTHER, why);
     }
@@ -908,7 +908,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         }
         case ONLINE: {
           final String tableId = ByteBufferUtil.toString(arguments.get(0));
-          checkNotMetadataID(tableId, TableOperation.ONLINE);
+          checkNotRootID(tableId, TableOperation.ONLINE);
 
           if (!security.canOnlineOfflineTable(c, tableId, op))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -918,7 +918,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         }
         case OFFLINE: {
           final String tableId = ByteBufferUtil.toString(arguments.get(0));
-          checkNotMetadataID(tableId, TableOperation.OFFLINE);
+          checkNotRootID(tableId, TableOperation.OFFLINE);
 
           if (!security.canOnlineOfflineTable(c, tableId, op))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index d82ee00..8609105 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -55,11 +55,11 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.master.Master.TabletGoalState;
 import org.apache.accumulo.master.state.MergeStats;
 import org.apache.accumulo.master.state.TableCounts;
 import org.apache.accumulo.master.state.TableStats;
+import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 818f978..ee3b243 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -89,6 +89,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Sc
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
@@ -114,7 +115,6 @@ import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.accumulo.server.util.FileUtil;
 import org.apache.accumulo.server.util.MasterMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.MetadataTableUtil.LogEntry;
 import org.apache.accumulo.server.util.TabletOperations;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
@@ -1183,7 +1183,7 @@ public class Tablet {
         Key key = entry.getKey();
         if (key.getRow().equals(row)) {
           if (key.getColumnFamily().equals(LogColumnFamily.NAME)) {
-            logEntries.add(MetadataTableUtil.entryFromKeyValue(key, entry.getValue()));
+            logEntries.add(LogEntry.fromKeyValue(key, entry.getValue()));
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e86a8d06/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 2ba25ba..e7ac7fc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -116,6 +116,7 @@ import org.apache.accumulo.core.security.AuthorizationContainer;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -173,7 +174,6 @@ import org.apache.accumulo.server.util.FileSystemMonitor;
 import org.apache.accumulo.server.util.Halt;
 import org.apache.accumulo.server.util.MasterMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.MetadataTableUtil.LogEntry;
 import org.apache.accumulo.server.util.TServerUtils;
 import org.apache.accumulo.server.util.TServerUtils.ServerAddress;
 import org.apache.accumulo.server.util.time.RelativeTime;
@@ -3004,7 +3004,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     List<String> logSet = new ArrayList<String>();
     for (DfsLogger log : logs)
       logSet.add(log.getFileName());
-    MetadataTableUtil.LogEntry entry = new MetadataTableUtil.LogEntry();
+    LogEntry entry = new LogEntry();
     entry.extent = extent;
     entry.tabletId = id;
     entry.timestamp = now;


[2/3] git commit: ACCUMULO-1832 class for re-writting uris

Posted by ec...@apache.org.
ACCUMULO-1832 class for re-writting uris


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

Branch: refs/heads/master
Commit: 5d7f6a940e7866a3b79fd563b7008ad5faec4f85
Parents: e86a8d0
Author: Eric Newton <er...@gmail.com>
Authored: Tue Nov 5 13:43:42 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Tue Nov 5 13:43:42 2013 -0500

----------------------------------------------------------------------
 server/base/pom.xml                             |  10 +-
 .../apache/accumulo/utils/NamespaceRename.java  | 211 +++++++++++++++++++
 2 files changed, 216 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5d7f6a94/server/base/pom.xml
----------------------------------------------------------------------
diff --git a/server/base/pom.xml b/server/base/pom.xml
index 6f5f4a7..1e3164f 100644
--- a/server/base/pom.xml
+++ b/server/base/pom.xml
@@ -100,13 +100,13 @@
       <artifactId>jetty</artifactId>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5d7f6a94/server/extras/src/main/java/org/apache/accumulo/utils/NamespaceRename.java
----------------------------------------------------------------------
diff --git a/server/extras/src/main/java/org/apache/accumulo/utils/NamespaceRename.java b/server/extras/src/main/java/org/apache/accumulo/utils/NamespaceRename.java
new file mode 100644
index 0000000..bc0c834
--- /dev/null
+++ b/server/extras/src/main/java/org/apache/accumulo/utils/NamespaceRename.java
@@ -0,0 +1,211 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.ColumnFQ;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+
+public class NamespaceRename {
+  
+  static final Logger log = LoggerFactory.getLogger(NamespaceRename.class);
+  
+  static class Opts {
+    @Parameter(names = {"--old", "-o"}, required = true)
+    String oldName = null;
+    @Parameter(names = {"--new", "-n"}, required = true)
+    String newName = null;
+  }
+  
+  /**
+   * Utility to recovery from a name node restoration at a new location. For example, if you had been using "nn1" and the machine died but you were able to
+   * restore the service on a different machine, "nn2" you could rewrite the metadata using
+   * <pre>
+   * accumulo org.apache.accumulo.server.util.NamespaceRename --old hdfs://nn1:9001 --new hdfs://nn2:9001
+   * </pre>
+   * @param args
+   * @throws Exception 
+   */
+  public static void main(String[] args) throws Exception {
+    Opts opts = new Opts();
+    JCommander cmdline = new JCommander(opts);
+    cmdline.parse(args);
+    log.info("Checking current configuration");
+    AccumuloConfiguration configuration = ServerConfiguration.getSiteConfiguration();
+    checkConfiguration(opts, configuration);
+    Instance instance = HdfsZooInstance.getInstance();
+    log.info("Waiting for HDFS and Zookeeper to be ready");
+    VolumeManager fs = VolumeManagerImpl.get();
+    Accumulo.waitForZookeeperAndHdfs(fs);
+    log.info("Putting servers in SAFE_MODE");
+    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    zoo.putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, MasterGoalState.SAFE_MODE.toString().getBytes(), NodeExistsPolicy.OVERWRITE);
+    log.info("Updating root table write-ahead logs");
+    updateZookeeper(opts, instance, zoo);
+    log.info("Updating file references in the root table");
+    updateMetaTable(opts, instance, RootTable.NAME);
+    log.info("Updating file references in the metadata table");
+    updateMetaTable(opts, instance, MetadataTable.NAME);
+    zoo.putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, MasterGoalState.NORMAL.toString().getBytes(), NodeExistsPolicy.OVERWRITE);
+    log.info("Namespace " + opts.oldName + " has been renamed " + opts.newName);
+  }
+  
+  static final ColumnFQ DIRECTORY_COLUMN = MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
+
+  private static void updateMetaTable(Opts opts, Instance instance, String tableName) throws Exception,
+      MutationsRejectedException {
+    log.info("Waiting for " + tableName + " to come online");
+    Connector conn = getConnector(instance);
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+    DIRECTORY_COLUMN.fetch(scanner);
+    scanner.iterator().hasNext();
+    log.info(tableName + " is online");
+    BatchWriter bw = conn.createBatchWriter(tableName,  new BatchWriterConfig());
+    for (Entry<Key,Value> entry : scanner) {
+      Key key = entry.getKey();
+      Mutation m = new Mutation(key.getRow());
+      if (DIRECTORY_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier())) {
+        m.putDelete(key.getColumnFamily(), key.getColumnQualifier(), key.getTimestamp());
+        String newName = rename(entry.getValue().toString(), opts);
+        m.put(key.getColumnFamily(), key.getColumnQualifier(), new Value(newName.getBytes()));
+        bw.addMutation(m);
+      } else if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME)) {
+        m.putDelete(key.getColumnFamily(), key.getColumnQualifier(), key.getTimestamp());
+        String newName = rename(key.getColumnQualifier().toString(), opts);
+        m.put(key.getColumnFamily(), new Text(newName), entry.getValue());
+        bw.addMutation(m);
+      } else if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.LogColumnFamily.NAME)) {
+        m.putDelete(key.getColumnFamily(), key.getColumnQualifier(), key.getTimestamp());
+        LogEntry update = convert(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()), opts);
+        m.put(update.getColumnFamily(), update.getColumnQualifier(), update.getValue());
+        bw.addMutation(m);
+      }
+    }
+    bw.close();
+  }
+
+  static private LogEntry convert(LogEntry entry, Opts opts) {
+    entry.filename = rename(entry.filename, opts);
+    List<String> logSet = new ArrayList<String>();
+    for (String log : entry.logSet) {
+      logSet.add(rename(log, opts));
+    }
+    entry.logSet = logSet;
+    return entry;
+  }
+  
+  private static Connector getConnector(Instance instance) throws AccumuloException, AccumuloSecurityException {
+    return instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
+  }
+
+  private static void updateZookeeper(Opts opts, Instance instance, ZooReaderWriter zoo) throws KeeperException, InterruptedException,
+      IOException {
+    String root = ZooUtil.getRoot(instance);
+    String rootTabletLocation = root + RootTable.ZROOT_TABLET_WALOGS;
+    for (String walogName : zoo.getChildren(rootTabletLocation)) {
+      LogEntry entry = new LogEntry();
+      String logZPath = rootTabletLocation + "/" + walogName;
+      byte[] data = zoo.getData(logZPath, null);
+      entry.fromBytes(data);
+      entry = convert(entry, opts);
+      zoo.putPersistentData(logZPath, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
+    }
+    String dirPath = root + RootTable.ZROOT_TABLET_PATH;
+    byte[] dir = zoo.getData(dirPath, null);
+    String newDir = rename(new String(dir), opts);
+    zoo.putPersistentData(dirPath, newDir.getBytes(), NodeExistsPolicy.OVERWRITE);
+  }
+
+  private static String rename(String filename, Opts opts) {
+    if (filename.startsWith(opts.oldName))
+      return opts.newName + filename.substring(opts.oldName.length(), filename.length());
+    return filename;
+  }
+
+  private static void checkConfiguration(Opts opts, AccumuloConfiguration configuration) throws IOException {
+    if (opts.oldName.endsWith("/"))
+      throw new RuntimeException(opts.oldName + " ends with a slash, do not include it");
+    if (opts.newName.endsWith("/"))
+      throw new RuntimeException(opts.newName + " ends with a slash, do not include it");
+    String volumes = configuration.get(Property.INSTANCE_VOLUMES);
+    if (volumes != null && !volumes.isEmpty()) {
+      Set<String> volumeSet = new HashSet<String>(Arrays.asList(volumes.split(",")));
+      if (volumeSet.contains(opts.oldName))
+        throw new RuntimeException(Property.INSTANCE_VOLUMES.getKey() + " is set to " + volumes + " which still contains the old name " + opts.oldName);
+      if (!volumeSet.contains(opts.newName))
+        throw new RuntimeException(Property.INSTANCE_VOLUMES.getKey() + " is set to " + volumes + " which does not contain the new name " + opts.oldName);
+      return;
+    } else {
+      String uri = configuration.get(Property.INSTANCE_DFS_URI);
+      if (uri != null && !uri.isEmpty()) {
+        if (!uri.startsWith(opts.newName))
+          throw new RuntimeException(Property.INSTANCE_DFS_DIR.getKey() + " is set to " + uri + " which is not in " + opts.newName);
+        return;
+      }
+    }
+    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    if (!fs.getUri().toString().equals(opts.newName))
+      throw new RuntimeException("Default filesystem is " + fs.getUri() + " and the new name is " + opts.newName + ". Update your hadoop dfs configuration.");
+  }
+  
+}


[3/3] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by ec...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 6239604c19a789d81d0b964e31018b4067900364
Parents: 8f87c72 5d7f6a9
Author: Eric Newton <er...@gmail.com>
Authored: Tue Nov 5 13:44:15 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Tue Nov 5 13:44:15 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |   6 +-
 .../core/tabletserver/log/LogEntry.java         |  36 ++++
 .../core/util/shell/commands/OnlineCommand.java |   6 +-
 server/base/pom.xml                             |  10 +-
 .../master/state/MetaDataTableScanner.java      |   4 +-
 .../master/state/ZooTabletStateStore.java       |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java |  79 +------
 .../server/util/OfflineMetadataScanner.java     |   2 +-
 .../apache/accumulo/utils/NamespaceRename.java  | 211 +++++++++++++++++++
 .../gc/GarbageCollectWriteAheadLogs.java        |   3 +-
 .../java/org/apache/accumulo/master/Master.java |  10 +-
 .../accumulo/master/TabletGroupWatcher.java     |   2 +-
 .../org/apache/accumulo/tserver/Tablet.java     |   4 +-
 .../apache/accumulo/tserver/TabletServer.java   |   4 +-
 14 files changed, 283 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6239604c/server/base/pom.xml
----------------------------------------------------------------------