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 2014/09/19 14:18:44 UTC

[1/3] git commit: ACCUMULO-3137 update data version to force a check for outstanding FATE operations, and reorganize the constants that represent upgrade steps.

Repository: accumulo
Updated Branches:
  refs/heads/master 45092aa02 -> 30a605734


ACCUMULO-3137 update data version to force a check for outstanding FATE
operations, and reorganize the constants that represent upgrade steps.


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

Branch: refs/heads/master
Commit: 82af21993c36e801d74f46023219eacb6422631f
Parents: edc9c30
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Sep 17 09:06:13 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Sep 17 09:06:13 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/Accumulo.java    |  4 +-
 .../apache/accumulo/server/ServerConstants.java | 30 ++++++-
 .../java/org/apache/accumulo/master/Master.java | 88 ++++++++++----------
 3 files changed, 73 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/82af2199/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index 516121d..7bb3d71 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -199,14 +199,14 @@ public class Accumulo {
    * @param dataVersion the version that is persisted in the backing Volumes
    */
   public static boolean canUpgradeFromDataVersion(final int dataVersion) {
-    return dataVersion == ServerConstants.DATA_VERSION || dataVersion == ServerConstants.PREV_DATA_VERSION || dataVersion == ServerConstants.TWO_DATA_VERSIONS_AGO;
+    return ServerConstants.CAN_UPGRADE.get(dataVersion);
   }
 
   /**
    * Does the data version number stored in the backing Volumes indicate we need to upgrade something?
    */
   public static boolean persistentVersionNeedsUpgrade(final int accumuloPersistentVersion) {
-    return accumuloPersistentVersion == ServerConstants.TWO_DATA_VERSIONS_AGO || accumuloPersistentVersion == ServerConstants.PREV_DATA_VERSION;
+    return ServerConstants.NEEDS_UPGRADE.get(accumuloPersistentVersion);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/82af2199/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
index 07eb553..fcaa5b7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.server;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -47,11 +48,32 @@ public class ServerConstants {
   public static final Integer WIRE_VERSION = 3;
 
   /**
-   * current version (6) reflects the addition of a separate root table (ACCUMULO-1481) in version 1.6.0
+   * version (7) reflects the change in the representation of trace information in TraceRepo
    */
-  public static final int DATA_VERSION = 6;
-  public static final int PREV_DATA_VERSION = 5;
-  public static final int TWO_DATA_VERSIONS_AGO = 4;
+  public static final int DATA_VERSION = 7;
+  /**
+   * version (6) reflects the addition of a separate root table (ACCUMULO-1481) in version 1.6.0
+   */
+  public static final int MOVE_TO_ROOT_TABLE = 6;
+  /**
+   * version (5) moves delete file markers for the metadata table into the root tablet
+   */
+  public static final int MOVE_DELETE_MARKERS = 5;
+  /**
+   * version (4) moves logging to HDFS in 1.5.0
+   */
+  public static final int LOGGING_TO_HDFS = 4;
+  public static final BitSet CAN_UPGRADE = new BitSet(); 
+  static {
+    for (int i : new int[]{DATA_VERSION, MOVE_TO_ROOT_TABLE, MOVE_DELETE_MARKERS, LOGGING_TO_HDFS}) {
+      CAN_UPGRADE.set(i);
+    }
+  }
+  public static final BitSet NEEDS_UPGRADE = new BitSet();
+  static {
+    NEEDS_UPGRADE.xor(CAN_UPGRADE);
+    NEEDS_UPGRADE.clear(DATA_VERSION);
+  }
 
   private static String[] baseUris = null;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/82af2199/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 0db1e5d..b435b0f 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
@@ -300,56 +300,54 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         final String zooRoot = ZooUtil.getRoot(instance);
 
-        if (accumuloPersistentVersion == ServerConstants.TWO_DATA_VERSIONS_AGO) {
-          log.debug("Handling updates for version " + ServerConstants.TWO_DATA_VERSIONS_AGO);
+        log.debug("Handling updates for version " + accumuloPersistentVersion);
 
-          log.debug("Cleaning out remnants of logger role.");
-          zoo.recursiveDelete(zooRoot + "/loggers", NodeMissingPolicy.SKIP);
-          zoo.recursiveDelete(zooRoot + "/dead/loggers", NodeMissingPolicy.SKIP);
+        log.debug("Cleaning out remnants of logger role.");
+        zoo.recursiveDelete(zooRoot + "/loggers", NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(zooRoot + "/dead/loggers", NodeMissingPolicy.SKIP);
 
-          final byte[] zero = new byte[] {'0'};
-          log.debug("Initializing recovery area.");
-          zoo.putPersistentData(zooRoot + Constants.ZRECOVERY, zero, NodeExistsPolicy.SKIP);
+        final byte[] zero = new byte[] {'0'};
+        log.debug("Initializing recovery area.");
+        zoo.putPersistentData(zooRoot + Constants.ZRECOVERY, zero, NodeExistsPolicy.SKIP);
 
-          for (String id : zoo.getChildren(zooRoot + Constants.ZTABLES)) {
-            log.debug("Prepping table " + id + " for compaction cancellations.");
-            zoo.putPersistentData(zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, zero, NodeExistsPolicy.SKIP);
-          }
+        for (String id : zoo.getChildren(zooRoot + Constants.ZTABLES)) {
+          log.debug("Prepping table " + id + " for compaction cancellations.");
+          zoo.putPersistentData(zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, zero, NodeExistsPolicy.SKIP);
+        }
 
-          @SuppressWarnings("deprecation")
-          String zpath = zooRoot + Constants.ZCONFIG + "/" + Property.TSERV_WAL_SYNC_METHOD.getKey();
-          boolean flushDefault = false;
+        @SuppressWarnings("deprecation")
+        String zpath = zooRoot + Constants.ZCONFIG + "/" + Property.TSERV_WAL_SYNC_METHOD.getKey();
+        // is the entire instance set to use flushing vs sync?
+        boolean flushDefault = false;
+        try {
+          byte data[] = zoo.getData(zpath, null);
+          if (new String(data, StandardCharsets.UTF_8).endsWith("flush")) {
+            flushDefault = true;
+          }
+        } catch (KeeperException.NoNodeException ex) {
+          // skip
+        } 
+        for (String id : zoo.getChildren(zooRoot + Constants.ZTABLES)) {
+          log.debug("Converting table " + id + " WALog setting to Durability");
           try {
-            byte data[] = zoo.getData(zpath, null);
-            if (new String(data, StandardCharsets.UTF_8).endsWith("flush")) {
-              flushDefault = true;
-            }
-          } catch (KeeperException.NoNodeException ex) {
-            // skip
-          } 
-          for (String id : zoo.getChildren(zooRoot + Constants.ZTABLES)) {
-            log.debug("Converting table " + id + " WALog setting to Durability");
-            try {
-              @SuppressWarnings("deprecation")
-              String path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_WALOG_ENABLED.getKey();
-              byte[] data = zoo.getData(path, null);
-              boolean useWAL = Boolean.parseBoolean(new String(data, StandardCharsets.UTF_8));
-              zoo.recursiveDelete(path, NodeMissingPolicy.FAIL);
-              path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_DURABILITY.getKey();
-              if (useWAL) {
-                if (flushDefault) {
-                  zoo.putPersistentData(path, "flush".getBytes(), NodeExistsPolicy.SKIP);
-                } else {
-                  zoo.putPersistentData(path, "sync".getBytes(), NodeExistsPolicy.SKIP);
-                }
+            @SuppressWarnings("deprecation")
+            String path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_WALOG_ENABLED.getKey();
+            byte[] data = zoo.getData(path, null);
+            boolean useWAL = Boolean.parseBoolean(new String(data, StandardCharsets.UTF_8));
+            zoo.recursiveDelete(path, NodeMissingPolicy.FAIL);
+            path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_DURABILITY.getKey();
+            if (useWAL) {
+              if (flushDefault) {
+                zoo.putPersistentData(path, "flush".getBytes(), NodeExistsPolicy.SKIP);
               } else {
-                zoo.putPersistentData(path, "none".getBytes(), NodeExistsPolicy.SKIP);
+                zoo.putPersistentData(path, "sync".getBytes(), NodeExistsPolicy.SKIP);
               }
-            } catch (KeeperException.NoNodeException ex) {
-              // skip it
+            } else {
+              zoo.putPersistentData(path, "none".getBytes(), NodeExistsPolicy.SKIP);
             }
+          } catch (KeeperException.NoNodeException ex) {
+            // skip it
           }
-        
         }
 
         // create initial namespaces
@@ -426,16 +424,20 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           throw new IllegalStateException("Access to Fate should not have been initialized prior to the Master finishing upgrades. Please save all logs and file a bug.");
         }
         Runnable upgradeTask = new Runnable() {
+          int version = accumuloPersistentVersion;
           @Override
           public void run() {
             try {
               log.info("Starting to upgrade metadata table.");
-              if (accumuloPersistentVersion == ServerConstants.TWO_DATA_VERSIONS_AGO) {
+              if (version == ServerConstants.MOVE_DELETE_MARKERS - 1) {
                 log.info("Updating Delete Markers in metadata table for version 1.4");
                 MetadataTableUtil.moveMetaDeleteMarkersFrom14(instance, SystemCredentials.get());
-              } else {
+                version++;
+              }
+              if (version == ServerConstants.MOVE_TO_ROOT_TABLE - 1){
                 log.info("Updating Delete Markers in metadata table.");
                 MetadataTableUtil.moveMetaDeleteMarkers(instance, SystemCredentials.get());
+                version++;
               }
               log.info("Updating persistent data version.");
               Accumulo.updateAccumuloVersion(fs, accumuloPersistentVersion);


[3/3] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: 30a605734855b988ee11ad70be0a790ba900c88d
Parents: 68c3efe 45092aa
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Sep 19 08:18:27 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 19 08:18:27 2014 -0400

----------------------------------------------------------------------
 README                                          |  8 +-
 .../core/client/mapreduce/InputTableConfig.java |  2 +
 .../client/mapreduce/InputTableConfigTest.java  | 20 ++++-
 .../minicluster/impl/ProcessReference.java      |  7 ++
 pom.xml                                         |  2 +-
 .../apache/accumulo/test/AuditMessageIT.java    | 12 ++-
 .../org/apache/accumulo/test/ShellServerIT.java | 89 +++++++++++---------
 .../test/functional/AccumuloInputFormatIT.java  |  9 +-
 .../functional/BalanceAfterCommsFailureIT.java  | 73 ++++++++++++----
 .../functional/SimpleBalancerFairnessIT.java    | 44 +++++++---
 .../test/replication/CyclicReplicationIT.java   |  6 ++
 11 files changed, 189 insertions(+), 83 deletions(-)
----------------------------------------------------------------------



[2/3] git commit: ACCUMULO-3049 close the connections to the tservers

Posted by ec...@apache.org.
ACCUMULO-3049 close the connections to the tservers


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

Branch: refs/heads/master
Commit: 68c3efe8908e0639e1bf55bae2cb6cde50182284
Parents: 82af219
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Sep 19 08:17:58 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 19 08:17:58 2014 -0400

----------------------------------------------------------------------
 .../monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/68c3efe8/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 25803b3..7a724f8 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -536,6 +536,8 @@ public class Monitor {
         }
       } catch (Exception ex) {
         log.debug(ex, ex);
+      } finally {
+        ThriftUtil.returnClient(tserver);
       }
     }
     // Age off old scan information