You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/04/05 21:13:41 UTC

svn commit: r1310017 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/executor/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/master...

Author: stack
Date: Thu Apr  5 19:13:39 2012
New Revision: 1310017

URL: http://svn.apache.org/viewvc?rev=1310017&view=rev
Log:
HBASE-5715 Revert 'Instant schema alter' for now, HBASE-4213

Removed:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterSchemaChangeTracker.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/SchemaChangeTracker.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestInstantSchemaChange.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestInstantSchemaChangeFailover.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestInstantSchemaChangeSplit.java
Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
    hbase/branches/0.94/src/main/resources/hbase-default.xml
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/InstantSchemaChangeTestBase.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java Thu Apr  5 19:13:39 2012
@@ -305,10 +305,12 @@ public class LocalHBaseCluster {
    */
   public HMaster getActiveMaster() {
     for (JVMClusterUtil.MasterThread mt : masterThreads) {
-      // Ensure that the current active master is not stopped.
-      // We don't want to return a stopping master as an active master.
-      if (mt.getMaster().isActiveMaster()  && !mt.getMaster().isStopped()) {
-        return mt.getMaster();
+      if (mt.getMaster().isActiveMaster()) {
+        // Ensure that the current active master is not stopped.
+        // We don't want to return a stopping master as an active master.
+        if (mt.getMaster().isActiveMaster()  && !mt.getMaster().isStopped()) {
+          return mt.getMaster();
+        }
       }
     }
     return null;

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java Thu Apr  5 19:13:39 2012
@@ -142,12 +142,13 @@ public abstract class EventHandler imple
      * Constructor
      */
     EventType(int value) {}
-    public boolean isSchemaChangeEvent() {
+    public boolean isOnlineSchemaChangeSupported() {
       return (
-          this.equals(EventType.C_M_ADD_FAMILY) ||
-          this.equals(EventType.C_M_DELETE_FAMILY) ||
-          this.equals(EventType.C_M_MODIFY_FAMILY) ||
-          this.equals(EventType.C_M_MODIFY_TABLE));
+        this.equals(EventType.C_M_ADD_FAMILY) ||
+        this.equals(EventType.C_M_DELETE_FAMILY) ||
+        this.equals(EventType.C_M_MODIFY_FAMILY) ||
+        this.equals(EventType.C_M_MODIFY_TABLE)
+      );
     }
   }
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Thu Apr  5 19:13:39 2012
@@ -266,12 +266,4 @@ public interface HMasterInterface extend
    * @return array of HTableDescriptor
    */
   public HTableDescriptor[] getHTableDescriptors(List<String> tableNames);
-
-  /**
-   * Returns the current running status of load balancer.
-   * @return True if LoadBalancer is running now else False.
-   */
-  public boolean isLoadBalancerRunning();
-
-
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu Apr  5 19:13:39 2012
@@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.client.Me
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
 import org.apache.hadoop.hbase.ipc.HBaseRPC;
@@ -85,6 +84,7 @@ import org.apache.hadoop.hbase.master.ha
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
+import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -101,7 +101,6 @@ import org.apache.hadoop.hbase.util.Vers
 import org.apache.hadoop.hbase.zookeeper.ClusterId;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
-import org.apache.hadoop.hbase.zookeeper.MasterSchemaChangeTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -174,10 +173,7 @@ Server {
   private CatalogTracker catalogTracker;
   // Cluster status zk tracker and local setter
   private ClusterStatusTracker clusterStatusTracker;
-
-  // Schema change tracker
-  private MasterSchemaChangeTracker schemaChangeTracker;
-
+  
   // buffer for "fatal error" notices from region servers
   // in the cluster. This is only used for assisting
   // operations/debugging.
@@ -205,18 +201,12 @@ Server {
 
   private CatalogJanitor catalogJanitorChore;
   private LogCleaner logCleaner;
-  private Thread schemaJanitorChore;
 
   private MasterCoprocessorHost cpHost;
   private final ServerName serverName;
 
   private TableDescriptors tableDescriptors;
 
-  // Whether or not schema alter changes go through ZK or not.
-  private boolean supportInstantSchemaChanges = false;
-
-  private volatile boolean loadBalancerRunning = false;
-
   // Time stamps for when a hmaster was started and when it became active
   private long masterStartTime;
   private long masterActiveTime;
@@ -290,17 +280,6 @@ Server {
     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this, true);
     this.rpcServer.startThreads();
     this.metrics = new MasterMetrics(getServerName().toString());
-    // initialize instant schema change settings
-    this.supportInstantSchemaChanges = conf.getBoolean(
-        "hbase.instant.schema.alter.enabled", false);
-    if (supportInstantSchemaChanges) {
-      LOG.info("Instant schema change enabled. All schema alter operations will " +
-          "happen through ZK.");
-    }
-   else {
-      LOG.info("Instant schema change disabled. All schema alter operations will " +
-          "happen normally.");
-    }
   }
 
   /**
@@ -440,12 +419,6 @@ Server {
     boolean wasUp = this.clusterStatusTracker.isClusterUp();
     if (!wasUp) this.clusterStatusTracker.setClusterUp();
 
-    // initialize schema change tracker
-    this.schemaChangeTracker = new MasterSchemaChangeTracker(getZooKeeper(),
-        this, this,
-        conf.getInt("hbase.instant.schema.alter.timeout", 60000));
-    this.schemaChangeTracker.start();
-
     LOG.info("Server active/primary master; " + this.serverName +
         ", sessionid=0x" +
         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
@@ -566,9 +539,6 @@ Server {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     Threads.setDaemonThreadRunning(catalogJanitorChore.getThread());
 
-    // Schema janitor chore.
-    this.schemaJanitorChore = getAndStartSchemaJanitorChore(this);
-
     registerMBean();
 
     status.markComplete("Initialization successful");
@@ -744,15 +714,6 @@ Server {
     return this.tableDescriptors;
   }
 
-  @Override
-  public MasterSchemaChangeTracker getSchemaChangeTracker() {
-    return this.schemaChangeTracker;
-  }
-
-  public RegionServerTracker getRegionServerTracker() {
-    return this.regionServerTracker;
-  }
-
   /** @return InfoServer object. Maybe null.*/
   public InfoServer getInfoServer() {
     return this.infoServer;
@@ -855,28 +816,7 @@ Server {
     if (this.executorService != null) this.executorService.shutdown();
   }
 
-  /**
-   * Start the schema janitor. This Janitor will periodically sweep the failed/expired schema
-   * changes.
-   * @param master
-   * @return
-   */
-  private Thread getAndStartSchemaJanitorChore(final HMaster master) {
-    String name = master.getServerName() + "-SchemaJanitorChore";
-    int schemaJanitorPeriod =
-      master.getConfiguration().getInt("hbase.instant.schema.janitor.period", 120000);
-    // Start up the schema janitor chore
-    Chore chore = new Chore(name, schemaJanitorPeriod, master) {
-      @Override
-      protected void chore() {
-        master.getSchemaChangeTracker().handleFailedOrExpiredSchemaChanges();
-      }
-    };
-    return Threads.setDaemonThreadRunning(chore.getThread());
-  }
-
-
-  private Thread getAndStartBalancerChore(final HMaster master) {
+  private static Thread getAndStartBalancerChore(final HMaster master) {
     String name = master.getServerName() + "-BalancerChore";
     int balancerPeriod =
       master.getConfiguration().getInt("hbase.balancer.period", 300000);
@@ -897,10 +837,6 @@ Server {
     if (this.catalogJanitorChore != null) {
       this.catalogJanitorChore.interrupt();
     }
-    if (this.schemaJanitorChore != null) {
-      this.schemaJanitorChore.interrupt();
-    }
-
   }
 
   @Override
@@ -971,15 +907,6 @@ Server {
     return balancerCutoffTime;
   }
 
-
-  /**
-   * Check whether the Load Balancer is currently running.
-   * @return true if the Load balancer is currently running.
-   */
-  public boolean isLoadBalancerRunning() {
-    return loadBalancerRunning;
-  }
-
   @Override
   public boolean balance() {
     // If balance not true, don't run balancer.
@@ -987,33 +914,23 @@ Server {
     // Do this call outside of synchronized block.
     int maximumBalanceTime = getBalancerCutoffTime();
     long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
-    boolean balancerRan = false;
+    boolean balancerRan;
     synchronized (this.balancer) {
-      if (loadBalancerRunning) {
-        LOG.debug("Load balancer is currently running. Skipping the current execution.");
-        return false;
-      }
-
       // Only allow one balance run at at time.
       if (this.assignmentManager.isRegionsInTransition()) {
         LOG.debug("Not running balancer because " +
-            this.assignmentManager.getRegionsInTransition().size() +
-            " region(s) in transition: " +
-            org.apache.commons.lang.StringUtils.
+          this.assignmentManager.getRegionsInTransition().size() +
+          " region(s) in transition: " +
+          org.apache.commons.lang.StringUtils.
             abbreviate(this.assignmentManager.getRegionsInTransition().toString(), 256));
         return false;
       }
       if (this.serverManager.areDeadServersInProgress()) {
         LOG.debug("Not running balancer because processing dead regionserver(s): " +
-            this.serverManager.getDeadServers());
+          this.serverManager.getDeadServers());
         return false;
       }
-      if (schemaChangeTracker.isSchemaChangeInProgress()) {
-        LOG.debug("Schema change operation is in progress. Waiting for " +
-        "it to complete before running the load balancer.");
-        return false;
-      }
-      loadBalancerRunning = true;
+
       if (this.cpHost != null) {
         try {
           if (this.cpHost.preBalance()) {
@@ -1048,7 +965,7 @@ Server {
               // if performing next balance exceeds cutoff time, exit the loop
               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
-                maximumBalanceTime);
+              maximumBalanceTime);
             break;
           }
         }
@@ -1061,7 +978,6 @@ Server {
           LOG.error("Error invoking master coprocessor postBalance()", ioe);
         }
       }
-      loadBalancerRunning = false;
     }
     return balancerRan;
   }
@@ -1211,9 +1127,7 @@ Server {
     if (cpHost != null) {
       cpHost.preDeleteTable(tableName);
     }
-    this.executorService.submit(new DeleteTableHandler(tableName, this, this, this,
-        supportInstantSchemaChanges));
-
+    this.executorService.submit(new DeleteTableHandler(tableName, this, this));
     if (cpHost != null) {
       cpHost.postDeleteTable(tableName);
     }
@@ -1225,49 +1139,13 @@ Server {
    * @return Pair indicating the number of regions updated Pair.getFirst is the
    *         regions that are yet to be updated Pair.getSecond is the total number
    *         of regions of the table
+   * @throws IOException 
    */
   public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
   throws IOException {
-    if (supportInstantSchemaChanges) {
-      return getAlterStatusFromSchemaChangeTracker(tableName);
-    }
     return this.assignmentManager.getReopenStatus(tableName);
   }
 
-  /**
-   * Used by the client to identify if all regions have the schema updates
-   *
-   * @param tableName
-   * @return Pair indicating the status of the alter command
-   * @throws IOException
-   */
-  private Pair<Integer, Integer> getAlterStatusFromSchemaChangeTracker(byte[] tableName)
-      throws IOException {
-    MasterSchemaChangeTracker.MasterAlterStatus alterStatus = null;
-    try {
-      alterStatus =
-          this.schemaChangeTracker.getMasterAlterStatus(Bytes.toString(tableName));
-    } catch (KeeperException ke) {
-      LOG.error("KeeperException while getting schema alter status for table = "
-      + Bytes.toString(tableName), ke);
-    }
-    if (alterStatus != null) {
-      LOG.debug("Getting AlterStatus from SchemaChangeTracker for table = "
-          + Bytes.toString(tableName) + " Alter Status = "
-          + alterStatus.toString());
-      int numberPending = alterStatus.getNumberOfRegionsToProcess() -
-          alterStatus.getNumberOfRegionsProcessed();
-      return new Pair<Integer, Integer>(alterStatus.getNumberOfRegionsProcessed(),
-          alterStatus.getNumberOfRegionsToProcess());
-    } else {
-      LOG.debug("MasterAlterStatus is NULL for table = "
-          + Bytes.toString(tableName));
-      // should we throw IOException here as it makes more sense?
-      return new Pair<Integer, Integer>(0,0);
-    }
-  }
-
-
   public void addColumn(byte [] tableName, HColumnDescriptor column)
   throws IOException {
     checkInitialized();
@@ -1276,8 +1154,7 @@ Server {
         return;
       }
     }
-    new TableAddFamilyHandler(tableName, column, this, this,
-        this, supportInstantSchemaChanges).process();
+    new TableAddFamilyHandler(tableName, column, this, this).process();
     if (cpHost != null) {
       cpHost.postAddColumn(tableName, column);
     }
@@ -1291,8 +1168,7 @@ Server {
         return;
       }
     }
-    new TableModifyFamilyHandler(tableName, descriptor, this, this,
-        this, supportInstantSchemaChanges).process();
+    new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
     if (cpHost != null) {
       cpHost.postModifyColumn(tableName, descriptor);
     }
@@ -1306,8 +1182,7 @@ Server {
         return;
       }
     }
-    new TableDeleteFamilyHandler(tableName, c, this, this,
-        this, supportInstantSchemaChanges).process();
+    new TableDeleteFamilyHandler(tableName, c, this, this).process();
     if (cpHost != null) {
       cpHost.postDeleteColumn(tableName, c);
     }
@@ -1319,7 +1194,7 @@ Server {
       cpHost.preEnableTable(tableName);
     }
     this.executorService.submit(new EnableTableHandler(this, tableName,
-        catalogTracker, assignmentManager, false));
+      catalogTracker, assignmentManager, false));
 
     if (cpHost != null) {
       cpHost.postEnableTable(tableName);
@@ -1381,33 +1256,14 @@ Server {
     if (cpHost != null) {
       cpHost.preModifyTable(tableName, htd);
     }
-    this.executorService.submit(new ModifyTableHandler(tableName, htd, this,
-      this, this, supportInstantSchemaChanges));
+    this.executorService.submit(new ModifyTableHandler(tableName, htd, this, this));
     if (cpHost != null) {
       cpHost.postModifyTable(tableName, htd);
     }
   }
 
-  private boolean isOnlineSchemaChangeAllowed() {
-    return conf.getBoolean(
-        "hbase.online.schema.update.enable", false);
-  }
-  
   @Override
-  public void checkTableModifiable(final byte [] tableName,
-                                   EventHandler.EventType eventType)
-  throws IOException {
-    preCheckTableModifiable(tableName);
-    if (!eventType.isSchemaChangeEvent() ||
-        !isOnlineSchemaChangeAllowed()) {
-      if (!getAssignmentManager().getZKTable().
-          isDisabledTable(Bytes.toString(tableName))) {
-        throw new TableNotDisabledException(tableName);
-      }
-    }
-  }
-
-  private void preCheckTableModifiable(final byte[] tableName)
+  public void checkTableModifiable(final byte [] tableName)
   throws IOException {
     String tableNameStr = Bytes.toString(tableName);
     if (isCatalogTable(tableName)) {
@@ -1416,6 +1272,10 @@ Server {
     if (!MetaReader.tableExists(getCatalogTracker(), tableNameStr)) {
       throw new TableNotFoundException(tableNameStr);
     }
+    if (!getAssignmentManager().getZKTable().
+        isDisabledTable(Bytes.toString(tableName))) {
+      throw new TableNotDisabledException(tableName);
+    }
   }
 
   public void clearFromTransition(HRegionInfo hri) {

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Thu Apr  5 19:13:39 2012
@@ -512,7 +512,7 @@ public class MasterFileSystem {
    */
   public HTableDescriptor addColumn(byte[] tableName, HColumnDescriptor hcd)
       throws IOException {
-    LOG.debug("AddColumn. Table = " + Bytes.toString(tableName) + " HCD = " +
+    LOG.info("AddColumn. Table = " + Bytes.toString(tableName) + " HCD = " +
       hcd.toString());
     HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
     if (htd == null) {

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java Thu Apr  5 19:13:39 2012
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.zookeeper.MasterSchemaChangeTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
 
 /**
@@ -54,15 +53,12 @@ public interface MasterServices extends 
   public ExecutorService getExecutorService();
 
   /**
-   * Check table modifiable. i.e not ROOT or META and offlined for all commands except
-   * alter commands
-   * @param tableName
-   * @param eventType
-   * @throws IOException
+   * Check table is modifiable; i.e. exists and is offline.
+   * @param tableName Name of table to check.
+   * @throws TableNotDisabledException
+   * @throws TableNotFoundException 
    */
-  public void checkTableModifiable(final byte [] tableName,
-                                   EventHandler.EventType eventType)
-      throws IOException;
+  public void checkTableModifiable(final byte [] tableName) throws IOException;
 
   /**
    * Create a table using the given table definition.
@@ -79,20 +75,7 @@ public interface MasterServices extends 
   public TableDescriptors getTableDescriptors();
 
   /**
-   * Get Master Schema change tracker
-   * @return
-   */
-  public MasterSchemaChangeTracker getSchemaChangeTracker();
-
-  /**
-   * Return the Region server tracker.
-   * @return RegionServerTracker
-   */
-  public RegionServerTracker getRegionServerTracker();
-
-  /**
    * @return true if master enables ServerShutdownHandler;
    */
   public boolean isServerShutdownHandlerEnabled();
-
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Thu Apr  5 19:13:39 2012
@@ -343,15 +343,6 @@ public class ServerManager {
     }
   }
 
-  /**
-   * Exclude a RS from any pending schema change process.
-   * @param serverName
-   */
-  private void excludeRegionServerFromSchemaChanges(final ServerName serverName) {
-    this.services.getSchemaChangeTracker()
-        .excludeRegionServerForSchemaChanges(serverName.getServerName());
-  }
-
   /*
    * Expire the passed server.  Add it to list of deadservers and queue a
    * shutdown processing.
@@ -363,7 +354,6 @@ public class ServerManager {
       this.deadNotExpiredServers.add(serverName);
       return;
     }
-    excludeRegionServerFromSchemaChanges(serverName);
     if (!this.onlineServers.containsKey(serverName)) {
       LOG.warn("Received expiration of " + serverName +
         " but server is not currently online");

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java Thu Apr  5 19:13:39 2012
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -38,11 +37,9 @@ public class DeleteTableHandler extends 
   private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class);
 
   public DeleteTableHandler(byte [] tableName, Server server,
-      final MasterServices masterServices, HMasterInterface masterInterface,
-      boolean instantChange)
+      final MasterServices masterServices)
   throws IOException {
-    super(EventType.C_M_DELETE_TABLE, tableName, server, masterServices,
-        masterInterface, instantChange);
+    super(EventType.C_M_DELETE_TABLE, tableName, server, masterServices);
     // The next call fails if no such table.
     getTableDescriptor();
   }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java Thu Apr  5 19:13:39 2012
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.MasterServices;
 
 public class ModifyTableHandler extends TableEventHandler {
@@ -33,11 +32,9 @@ public class ModifyTableHandler extends 
 
   public ModifyTableHandler(final byte [] tableName,
       final HTableDescriptor htd, final Server server,
-      final MasterServices masterServices, final HMasterInterface masterInterface,
-      boolean instantModify)
+      final MasterServices masterServices)
   throws IOException {
-    super(EventType.C_M_MODIFY_TABLE, tableName, server, masterServices,
-        masterInterface, instantModify);
+    super(EventType.C_M_MODIFY_TABLE, tableName, server, masterServices);
     // Check table exists.
     getTableDescriptor();
     // This is the new schema we are going to write out as this modification.

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java Thu Apr  5 19:13:39 2012
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.MasterServices;
 
 /**
@@ -38,10 +37,8 @@ public class TableAddFamilyHandler exten
   private final HColumnDescriptor familyDesc;
 
   public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc,
-      Server server, final MasterServices masterServices,
-      HMasterInterface masterInterface, boolean instantChange) throws IOException {
-    super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices,
-        masterInterface, instantChange);
+      Server server, final MasterServices masterServices) throws IOException {
+    super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices);
     HTableDescriptor htd = getTableDescriptor();
     if (htd.hasFamily(familyDesc.getName())) {
       throw new InvalidFamilyOperationException("Family '" +

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java Thu Apr  5 19:13:39 2012
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -37,10 +36,8 @@ public class TableDeleteFamilyHandler ex
   private final byte [] familyName;
 
   public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName,
-      Server server, final MasterServices masterServices,
-      HMasterInterface masterInterface, boolean instantChange) throws IOException {
-    super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices,
-        masterInterface, instantChange);
+      Server server, final MasterServices masterServices) throws IOException {
+    super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices);
     HTableDescriptor htd = getTableDescriptor();
     this.familyName = hasColumnFamily(htd, familyName);
   }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java Thu Apr  5 19:13:39 2012
@@ -35,19 +35,13 @@ import org.apache.hadoop.hbase.InvalidFa
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.BulkReOpen;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.MasterSchemaChangeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZKAssign;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 import com.google.common.collect.Lists;
@@ -63,22 +57,32 @@ import com.google.common.collect.Maps;
 public abstract class TableEventHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(TableEventHandler.class);
   protected final MasterServices masterServices;
-  protected HMasterInterface master = null;
   protected final byte [] tableName;
   protected final String tableNameStr;
-  protected boolean instantAction = false;
 
   public TableEventHandler(EventType eventType, byte [] tableName, Server server,
-      MasterServices masterServices, HMasterInterface masterInterface,
-      boolean instantSchemaChange)
+      MasterServices masterServices)
   throws IOException {
     super(server, eventType);
     this.masterServices = masterServices;
     this.tableName = tableName;
-    this.masterServices.checkTableModifiable(tableName, eventType);
+    try {
+      this.masterServices.checkTableModifiable(tableName);
+    } catch (TableNotDisabledException ex)  {
+      if (isOnlineSchemaChangeAllowed()
+          && eventType.isOnlineSchemaChangeSupported()) {
+        LOG.debug("Ignoring table not disabled exception " +
+            "for supporting online schema changes.");
+      }	else {
+        throw ex;
+      }
+    }
     this.tableNameStr = Bytes.toString(this.tableName);
-    this.instantAction = instantSchemaChange;
-    this.master = masterInterface;
+  }
+
+  private boolean isOnlineSchemaChangeAllowed() {
+    return this.server.getConfiguration().getBoolean(
+      "hbase.online.schema.update.enable", false);
   }
 
   @Override
@@ -90,7 +94,16 @@ public abstract class TableEventHandler 
         MetaReader.getTableRegions(this.server.getCatalogTracker(),
           tableName);
       handleTableOperation(hris);
-      handleSchemaChanges(hris);
+      if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
+          getAssignmentManager().getZKTable().
+          isEnabledTable(Bytes.toString(tableName))) {
+        if (reOpenAllRegions(hris)) {
+          LOG.info("Completed table operation " + eventType + " on table " +
+              Bytes.toString(tableName));
+        } else {
+          LOG.warn("Error on reopening the regions");
+        }
+      }
     } catch (IOException e) {
       LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
     } catch (KeeperException e) {
@@ -98,47 +111,13 @@ public abstract class TableEventHandler 
     }
   }
 
-  private void handleSchemaChanges(List<HRegionInfo> regions)
-      throws IOException {
-    if (instantAction && regions != null && !regions.isEmpty()) {
-      handleInstantSchemaChanges(regions);
-    } else {
-      handleRegularSchemaChanges(regions);
-    }
-  }
-
-
-  /**
-   * Perform schema changes only if the table is in enabled state.
-   * @return
-   */
-  private boolean canPerformSchemaChange() {
-    return (eventType.isSchemaChangeEvent() && this.masterServices.
-        getAssignmentManager().getZKTable().
-        isEnabledTable(Bytes.toString(tableName)));
-  }
-
-  private void handleRegularSchemaChanges(List<HRegionInfo> regions)
-      throws IOException {
-    if (canPerformSchemaChange()) {
-      this.masterServices.getAssignmentManager().setRegionsToReopen(regions);
-      if (reOpenAllRegions(regions)) {
-        LOG.info("Completed table operation " + eventType + " on table " +
-            Bytes.toString(tableName));
-      } else {
-        LOG.warn("Error on reopening the regions");
-      }
-    }
-  }
-
   public boolean reOpenAllRegions(List<HRegionInfo> regions) throws IOException {
     boolean done = false;
     LOG.info("Bucketing regions by region server...");
     HTable table = new HTable(masterServices.getConfiguration(), tableName);
     TreeMap<ServerName, List<HRegionInfo>> serverToRegions = Maps
         .newTreeMap();
-    NavigableMap<HRegionInfo, ServerName> hriHserverMapping
-        = table.getRegionLocations();
+    NavigableMap<HRegionInfo, ServerName> hriHserverMapping = table.getRegionLocations();
     List<HRegionInfo> reRegions = new ArrayList<HRegionInfo>();
     for (HRegionInfo hri : regions) {
       ServerName rsLocation = hriHserverMapping.get(hri);
@@ -181,91 +160,6 @@ public abstract class TableEventHandler 
   }
 
   /**
-   * Check whether any of the regions from the list of regions is undergoing a split.
-   * We simply check whether there is a unassigned node for any of the region and if so
-   * we return as true.
-   * @param regionInfos
-   * @return
-   */
-  private boolean isSplitInProgress(List<HRegionInfo> regionInfos) {
-    for (HRegionInfo hri : regionInfos) {
-      ZooKeeperWatcher zkw = this.masterServices.getZooKeeper();
-      String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
-      try {
-        if (ZKUtil.checkExists(zkw, node) != -1) {
-          LOG.debug("Region " + hri.getRegionNameAsString() + " is unassigned. Assuming" +
-          " that it is undergoing a split");
-          return true;
-        }
-      } catch (KeeperException ke) {
-        LOG.debug("KeeperException while determining splits in progress.", ke);
-        // Assume no splits happening?
-        return false;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Wait for region split transaction in progress (if any)
-   * @param regions
-   * @param status
-   */
-  private void waitForInflightSplit(List<HRegionInfo> regions, MonitoredTask status) {
-    while (isSplitInProgress(regions)) {
-      try {
-        status.setStatus("Alter Schema is waiting for split region to complete.");
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
-  protected void handleInstantSchemaChanges(List<HRegionInfo> regions) {
-    if (regions == null || regions.isEmpty()) {
-      LOG.debug("Region size is null or empty. Ignoring alter request.");
-      return;
-    }
-    MonitoredTask status = TaskMonitor.get().createStatus(
-        "Handling alter table request for table = " + tableNameStr);
-    if (canPerformSchemaChange()) {
-      boolean prevBalanceSwitch = false;
-      try {
-        // turn off load balancer synchronously
-        prevBalanceSwitch = master.synchronousBalanceSwitch(false);
-        waitForInflightSplit(regions, status);
-        MasterSchemaChangeTracker masterSchemaChangeTracker =
-          this.masterServices.getSchemaChangeTracker();
-        masterSchemaChangeTracker
-        .createSchemaChangeNode(Bytes.toString(tableName),
-            regions.size());
-        while(!masterSchemaChangeTracker.doesSchemaChangeNodeExists(
-            Bytes.toString(tableName))) {
-          try {
-            Thread.sleep(50);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-          }
-        }
-        status.markComplete("Created ZK node for handling the alter table request for table = "
-            + tableNameStr);
-      } catch (KeeperException e) {
-        LOG.warn("Instant schema change failed for table " + tableNameStr, e);
-        status.setStatus("Instant schema change failed for table " + tableNameStr
-            + " Cause = " + e.getCause());
-
-      } catch (IOException ioe) {
-        LOG.warn("Instant schema change failed for table " + tableNameStr, ioe);
-        status.setStatus("Instant schema change failed for table " + tableNameStr
-            + " Cause = " + ioe.getCause());
-      } finally {
-        master.synchronousBalanceSwitch(prevBalanceSwitch);
-      }
-    }
-  }
-
-  /**
    * @return Table descriptor for this table
    * @throws TableExistsException
    * @throws FileNotFoundException

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java Thu Apr  5 19:13:39 2012
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ipc.HMasterInterface;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -39,10 +38,8 @@ public class TableModifyFamilyHandler ex
 
   public TableModifyFamilyHandler(byte[] tableName,
       HColumnDescriptor familyDesc, Server server,
-      final MasterServices masterServices,
-      HMasterInterface masterInterface, boolean instantChange) throws IOException {
-    super(EventType.C_M_MODIFY_FAMILY, tableName, server, masterServices,
-        masterInterface, instantChange);
+      final MasterServices masterServices) throws IOException {
+    super(EventType.C_M_MODIFY_FAMILY, tableName, server, masterServices);
     HTableDescriptor htd = getTableDescriptor();
     hasColumnFamily(htd, familyDesc.getName());
     this.familyDesc = familyDesc;

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Thu Apr  5 19:13:39 2012
@@ -155,29 +155,12 @@ public class CompactSplitThread implemen
     return false;
   }
 
-  /**
-   * Wait for mid-flight schema alter requests. (if any). We don't want to execute a split
-   * when a schema alter is in progress as we end up in an inconsistent state.
-   * @param tableName
-   */
-  private void waitForInflightSchemaChange(String tableName) {
-    while (this.server.getSchemaChangeTracker()
-        .isSchemaChangeInProgress(tableName)) {
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
   public synchronized void requestSplit(final HRegion r, byte[] midKey) {
     if (midKey == null) {
       LOG.debug("Region " + r.getRegionNameAsString() +
         " not splittable because midkey=null");
       return;
     }
-    waitForInflightSchemaChange(r.getRegionInfo().getTableNameAsString());
     try {
       this.splits.execute(new SplitRequest(r, midKey, this.server));
       if (LOG.isDebugEnabled()) {

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Apr  5 19:13:39 2012
@@ -147,7 +147,6 @@ import org.apache.hadoop.hbase.util.Slee
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
-import org.apache.hadoop.hbase.zookeeper.SchemaChangeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -292,9 +291,6 @@ public class HRegionServer implements HR
   // Cluster Status Tracker
   private ClusterStatusTracker clusterStatusTracker;
 
-  // Schema change Tracker
-  private SchemaChangeTracker schemaChangeTracker;
-
   // Log Splitting Worker
   private SplitLogWorker splitLogWorker;
 
@@ -597,11 +593,6 @@ public class HRegionServer implements HR
     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf,
       this, this.conf.getInt("hbase.regionserver.catalog.timeout", Integer.MAX_VALUE));
     catalogTracker.start();
-
-    // Schema change tracker
-    this.schemaChangeTracker = new SchemaChangeTracker(this.zooKeeper,
-        this, this);
-    this.schemaChangeTracker.start();
   }
 
   /**
@@ -2899,26 +2890,9 @@ public class HRegionServer implements HR
     splitRegion(regionInfo, null);
   }
 
-  /**
-   * Wait for mid-flight schema change requests. (if any)
-   * @param tableName
-   */
-  private void waitForSchemaChange(String tableName) {
-    while (schemaChangeTracker.isSchemaChangeInProgress(tableName)) {
-      try {
-        LOG.debug("Schema alter is inprogress for table = " + tableName
-            + " Waiting for alter to complete before a split");
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
   @Override
   public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
       throws NotServingRegionException, IOException {
-    waitForSchemaChange(Bytes.toString(regionInfo.getTableName()));
     checkOpen();
     HRegion region = getRegion(regionInfo.getRegionName());
     region.flushcache();
@@ -3670,58 +3644,27 @@ public class HRegionServer implements HR
   }
 
   /**
-  * Refresh schema changes for given region.
-  * @param hRegion HRegion to refresh
-  * @throws IOException
-  */
- public void refreshRegion(HRegion hRegion) throws IOException {
-
-   if (hRegion != null) {
+   * Gets the online regions of the specified table.
+   * This method looks at the in-memory onlineRegions.  It does not go to <code>.META.</code>.
+   * Only returns <em>online</em> regions.  If a region on this table has been
+   * closed during a disable, etc., it will not be included in the returned list.
+   * So, the returned list may not necessarily be ALL regions in this table, its
+   * all the ONLINE regions in the table.
+   * @param tableName
+   * @return Online regions from <code>tableName</code>
+   */
+   public List<HRegion> getOnlineRegions(byte[] tableName) {
+     List<HRegion> tableRegions = new ArrayList<HRegion>();
      synchronized (this.onlineRegions) {
-       HRegionInfo regionInfo = hRegion.getRegionInfo();
-       // Close the region
-       hRegion.close();
-       // Remove from online regions
-       removeFromOnlineRegions(regionInfo.getEncodedName());
-       // Get new HTD
-       HTableDescriptor htd = this.tableDescriptors.get(regionInfo.getTableName());
-       LOG.debug("HTD for region = " + regionInfo.getRegionNameAsString()
-           + " Is = " + htd );
-       HRegion region =
-         HRegion.openHRegion(hRegion.getRegionInfo(), htd, hlog, conf,
-             this, null);
-       // Add new region to the onlineRegions
-       addToOnlineRegions(region);
+       for (HRegion region: this.onlineRegions.values()) {
+         HRegionInfo regionInfo = region.getRegionInfo();
+         if(Bytes.equals(regionInfo.getTableName(), tableName)) {
+           tableRegions.add(region);
+         }
+       }
      }
+     return tableRegions;
    }
- }
-
- /**
-  * Gets the online regions of the specified table.
-  * This method looks at the in-memory onlineRegions.  It does not go to <code>.META.</code>.
-  * Only returns <em>online</em> regions.  If a region on this table has been
-  * closed during a disable, etc., it will not be included in the returned list.
-  * So, the returned list may not necessarily be ALL regions in this table, its
-  * all the ONLINE regions in the table.
-  * @param tableName
-  * @return Online regions from <code>tableName</code>
-  */
-  public List<HRegion> getOnlineRegions(byte[] tableName) {
-    List<HRegion> tableRegions = new ArrayList<HRegion>();
-    synchronized (this.onlineRegions) {
-      for (HRegion region: this.onlineRegions.values()) {
-        HRegionInfo regionInfo = region.getRegionInfo();
-        if(Bytes.equals(regionInfo.getTableName(), tableName)) {
-          tableRegions.add(region);
-        }
-      }
-    }
-    return tableRegions;
-  }
-
-  public SchemaChangeTracker getSchemaChangeTracker() {
-    return this.schemaChangeTracker;
-  }
 
   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
   public String[] getCoprocessors() {
@@ -3739,5 +3682,4 @@ public class HRegionServer implements HR
         mxBeanInfo);
     LOG.info("Registered RegionServer MXBean");
   }
-
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java Thu Apr  5 19:13:39 2012
@@ -52,18 +52,12 @@ interface OnlineRegions extends Server {
    * null if named region is not member of the online regions.
    */
   public HRegion getFromOnlineRegions(String encodedRegionName);
-  /**
-   * Get all online regions of a table in this RS.
-   * @param tableName
-   * @return List of HRegion
-   * @throws java.io.IOException
-   */
-  public List<HRegion> getOnlineRegions(byte[] tableName) throws IOException;
-
-  /**
-   * Refresh a given region updating it with latest HTD info.
-   * @param hRegion
-   */
-  public void refreshRegion(HRegion hRegion) throws IOException;
 
-}
+   /**
+    * Get all online regions of a table in this RS.
+    * @param tableName
+    * @return List of HRegion
+    * @throws java.io.IOException
+    */
+   public List<HRegion> getOnlineRegions(byte[] tableName) throws IOException;
+}
\ No newline at end of file

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Thu Apr  5 19:13:39 2012
@@ -102,8 +102,6 @@ public class ZooKeeperWatcher implements
   public String clusterIdZNode;
   // znode used for log splitting work assignment
   public String splitLogZNode;
-  // znode used to record table schema changes
-  public String schemaZNode;
 
   // Certain ZooKeeper nodes need to be world-readable
   public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
@@ -166,7 +164,6 @@ public class ZooKeeperWatcher implements
       ZKUtil.createAndFailSilent(this, drainingZNode);
       ZKUtil.createAndFailSilent(this, tableZNode);
       ZKUtil.createAndFailSilent(this, splitLogZNode);
-      ZKUtil.createAndFailSilent(this, schemaZNode);
       ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode);
     } catch (KeeperException e) {
       throw new ZooKeeperConnectionException(
@@ -219,8 +216,6 @@ public class ZooKeeperWatcher implements
         conf.get("zookeeper.znode.clusterId", "hbaseid"));
     splitLogZNode = ZKUtil.joinZNode(baseZNode,
         conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
-    schemaZNode = ZKUtil.joinZNode(baseZNode,
-                conf.get("zookeeper.znode.schema", "schema"));
   }
 
   /**

Modified: hbase/branches/0.94/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/resources/hbase-default.xml?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/resources/hbase-default.xml (original)
+++ hbase/branches/0.94/src/main/resources/hbase-default.xml Thu Apr  5 19:13:39 2012
@@ -780,39 +780,15 @@
     </description>
   </property>
   <property>
-    <name>hbase.coprocessor.abortonerror</name>
-    <value>false</value>
-    <description>
-    Set to true to cause the hosting server (master or regionserver) to
-    abort if a coprocessor throws a Throwable object that is not IOException or
-    a subclass of IOException. Setting it to true might be useful in development
-    environments where one wants to terminate the server as soon as possible to
-    simplify coprocessor failure analysis.
-    </description>
-  </property>
-  <property>
-    <name>hbase.instant.schema.alter.enabled</name>
-    <value>false</value>
-    <description>Whether or not to handle alter schema changes instantly or not.
-    If enabled, all schema change alter operations will be instant, as the master will not
-    explicitly unassign/assign the impacted regions and instead will rely on Region servers to
-    refresh their schema changes. If enabled, the schema alter requests will survive
-    master or RS failures.
-    </description>
-  </property>
-  <property>
-    <name>hbase.instant.schema.janitor.period</name>
-    <value>120000</value>
-    <description>The Schema Janitor process wakes up every millis and sweeps all
-    expired/failed schema change requests.
-    </description>
-  </property>
-  <property>
-    <name>hbase.instant.schema.alter.timeout</name>
-    <value>60000</value>
-    <description>Timeout in millis after which any pending schema alter request will be
-    considered as failed.
-    </description>
+      <name>hbase.coprocessor.abortonerror</name>
+      <value>false</value>
+      <description>
+      Set to true to cause the hosting server (master or regionserver) to
+      abort if a coprocessor throws a Throwable object that is not IOException or
+      a subclass of IOException. Setting it to true might be useful in development
+      environments where one wants to terminate the server as soon as possible to
+      simplify coprocessor failure analysis.
+      </description>
   </property>
   <property>
     <name>hbase.online.schema.update.enable</name>

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/InstantSchemaChangeTestBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/InstantSchemaChangeTestBase.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/InstantSchemaChangeTestBase.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/InstantSchemaChangeTestBase.java Thu Apr  5 19:13:39 2012
@@ -1,169 +0,0 @@
-/**
- * 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.hadoop.hbase.client;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.LargeTests;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.zookeeper.MasterSchemaChangeTracker;
-import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class InstantSchemaChangeTestBase {
-
-  final Log LOG = LogFactory.getLog(getClass());
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  protected HBaseAdmin admin;
-  protected static MiniHBaseCluster miniHBaseCluster = null;
-  protected Configuration conf;
-  protected static MasterSchemaChangeTracker msct = null;
-
-  protected final byte [] row = Bytes.toBytes("row");
-  protected final byte [] qualifier = Bytes.toBytes("qualifier");
-  final byte [] value = Bytes.toBytes("value");
-
-  @Before
-  public void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
-    TEST_UTIL.getConfiguration().setBoolean("hbase.instant.schema.alter.enabled", true);
-    TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
-    TEST_UTIL.getConfiguration().setInt("hbase.instant.schema.janitor.period", 10000);
-    TEST_UTIL.getConfiguration().setInt("hbase.instant.schema.alter.timeout", 30000);
-    //
-    miniHBaseCluster = TEST_UTIL.startMiniCluster(2,5);
-    msct = TEST_UTIL.getHBaseCluster().getMaster().getSchemaChangeTracker();
-    this.admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
-
-  }
-
-  @After
-  public void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * Find the RS that is currently holding our online region.
-   * @param tableName
-   * @return
-   */
-  protected HRegionServer findRSWithOnlineRegionFor(String tableName) {
-    List<JVMClusterUtil.RegionServerThread> rsThreads =
-        miniHBaseCluster.getLiveRegionServerThreads();
-    for (JVMClusterUtil.RegionServerThread rsT : rsThreads) {
-      HRegionServer rs = rsT.getRegionServer();
-      List<HRegion> regions = rs.getOnlineRegions(Bytes.toBytes(tableName));
-      if (regions != null && !regions.isEmpty()) {
-        return rs;
-      }
-    }
-    return null;
-  }
-
-  protected void waitForSchemaChangeProcess(final String tableName)
-      throws KeeperException, InterruptedException {
-    waitForSchemaChangeProcess(tableName, 10000);
-  }
-
-  /**
-   * This a pretty low cost signalling mechanism. It is quite possible that we will
-   * miss out the ZK node creation signal as in some cases the schema change process
-   * happens rather quickly and our thread waiting for ZK node creation might wait forver.
-   * The fool-proof strategy would be to directly listen for ZK events.
-   * @param tableName
-   * @throws KeeperException
-   * @throws InterruptedException
-   */
-  protected void waitForSchemaChangeProcess(final String tableName, final long waitTimeMills)
-      throws KeeperException, InterruptedException {
-    LOG.info("Waiting for ZK node creation for table = " + tableName);
-    final MasterSchemaChangeTracker msct =
-    TEST_UTIL.getHBaseCluster().getMaster().getSchemaChangeTracker();
-    final Runnable r = new Runnable() {
-      public void run() {
-        try {
-          while(!msct.doesSchemaChangeNodeExists(tableName)) {
-            try {
-              Thread.sleep(50);
-            } catch (InterruptedException e) {
-              Thread.currentThread().interrupt();
-            }
-          }
-        } catch (KeeperException ke) {
-            ke.printStackTrace();
-        }
-        LOG.info("Waiting for ZK node deletion for table = " + tableName);
-        try {
-          while(msct.doesSchemaChangeNodeExists(tableName)) {
-            try {
-              Thread.sleep(50);
-            } catch (InterruptedException e) {
-               Thread.currentThread().interrupt();
-            }
-          }
-        }  catch (KeeperException ke) {
-            ke.printStackTrace();
-        }
-      }
-    };
-    Thread t = new Thread(r);
-    t.start();
-    if (waitTimeMills > 0) {
-      t.join(waitTimeMills);
-    }  else {
-      t.join(10000);
-    }
-  }
-
-  protected HTable createTableAndValidate(String tableName) throws IOException {
-    conf = TEST_UTIL.getConfiguration();
-    LOG.info("Start createTableAndValidate()");
-    TEST_UTIL.getHBaseCluster().getMaster().getSchemaChangeTracker();
-    HTableDescriptor[] tables = admin.listTables();
-    int numTables = 0;
-    if (tables != null) {
-      numTables = tables.length;
-    }
-    HTable ht = TEST_UTIL.createTable(Bytes.toBytes(tableName),
-      HConstants.CATALOG_FAMILY);
-    tables = this.admin.listTables();
-    assertEquals(numTables + 1, tables.length);
-    LOG.info("created table = " + tableName);
-    return ht;
-  }
-
-}
\ No newline at end of file

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Thu Apr  5 19:13:39 2012
@@ -51,15 +51,12 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hbase.zookeeper.MasterSchemaChangeTracker;
-import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -156,6 +153,11 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public void checkTableModifiable(byte[] tableName) throws IOException {
+      //no-op
+    }
+
+    @Override
     public void createTable(HTableDescriptor desc, byte[][] splitKeys)
         throws IOException {
       // no-op
@@ -171,11 +173,6 @@ public class TestCatalogJanitor {
       return null;
     }
 
-    public void checkTableModifiable(byte[] tableName,
-                                     EventHandler.EventType eventType)
-        throws IOException {
-    }
-
     @Override
     public MasterFileSystem getMasterFileSystem() {
       return this.mfs;
@@ -263,14 +260,6 @@ public class TestCatalogJanitor {
       };
     }
 
-    public MasterSchemaChangeTracker getSchemaChangeTracker() {
-      return null;
-    }
-
-    public RegionServerTracker getRegionServerTracker() {
-      return null;
-    }
-
     @Override
     public boolean isServerShutdownHandlerEnabled() {
       return true;

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1310017&r1=1310016&r2=1310017&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Thu Apr  5 19:13:39 2012
@@ -63,9 +63,6 @@ public class MockRegionServerServices im
     return null;
   }
 
-  public void refreshRegion(HRegion hRegion) throws IOException {
-  }
-
   @Override
   public void addToOnlineRegions(HRegion r) {
     this.regions.put(r.getRegionInfo().getEncodedName(), r);