Browse Source

Reverting previous accidental commit r1130306

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.22@1130309 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 14 years ago
parent
commit
e4e0538081

+ 0 - 3
CHANGES.txt

@@ -51,9 +51,6 @@ Release 0.22.0 - Unreleased
 
   IMPROVEMENTS
 
-    HADOOP-7001.  Configuration changes can occur via the Reconfigurable
-    interface. (Patrick Kline via dhruba)
-
     HADOOP-6644. util.Shell getGROUPS_FOR_USER_COMMAND method name 
     - should use common naming convention (boryas)
 

+ 1 - 11
src/java/org/apache/hadoop/conf/Configuration.java

@@ -634,22 +634,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
   
-  /**
-   * Unset a previously set property.
-   */
-  public synchronized void unset(String name) {
-    name = handleDeprecation(name);
-
-    getOverlay().remove(name);
-    getProps().remove(name);
-  }
-
   /**
    * Sets a property if it is currently unset.
    * @param name the property name
    * @param value the new value
    */
-  public synchronized void setIfUnset(String name, String value) {
+  public void setIfUnset(String name, String value) {
     if (get(name) == null) {
       set(name, value);
     }

+ 0 - 57
src/java/org/apache/hadoop/conf/Reconfigurable.java

@@ -1,57 +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.conf;
-
-import java.util.Collection;
-
-/**
- * Something whose {@link Configuration} can be changed at run time.
- */
-public interface Reconfigurable extends Configurable {
-
-  /**
-   * Change a configuration property on this object to the value specified.
-   *
-   * Change a configuration property on this object to the value specified 
-   * and return the previous value that the configuration property was set to
-   * (or null if it was not previously set). If newVal is null, set the property
-   * to its default value;
-   *
-   * If the property cannot be changed, throw a 
-   * {@link ReconfigurationException}.
-   */
-  public String reconfigureProperty(String property, String newVal) 
-    throws ReconfigurationException;
-
-  /**
-   * Return whether a given property is changeable at run time.
-   *
-   * If isPropertyReconfigurable returns true for a property,
-   * then changeConf should not throw an exception when changing
-   * this property.
-   */
-  public boolean isPropertyReconfigurable(String property);
-
-  /**
-   * Return all the properties that can be changed at run time.
-   */
-  public Collection<String> getReconfigurableProperties();
-
-
-}

+ 0 - 114
src/java/org/apache/hadoop/conf/ReconfigurableBase.java

@@ -1,114 +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.conf;
-
-import org.apache.commons.logging.*;
-
-import java.util.Collection;
-
-/**
- * Utility base class for implementing the Reconfigurable interface.
- *
- * Subclasses should override reconfigurePropertyImpl to change individual
- * properties and getReconfigurableProperties to get all properties that
- * can be changed at run time.
- */
-public abstract class ReconfigurableBase 
-  extends Configured implements Reconfigurable {
-  
-  private static final Log LOG =
-    LogFactory.getLog(ReconfigurableBase.class);
-
-  /**
-   * Construct a ReconfigurableBase.
-   */
-  public ReconfigurableBase() {
-    super(new Configuration());
-  }
-
-  /**
-   * Construct a ReconfigurableBase with the {@link Configuration}
-   * conf.
-   */
-  public ReconfigurableBase(Configuration conf) {
-    super((conf == null) ? new Configuration() : conf);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * This method makes the change to this objects {@link Configuration}
-   * and calls reconfigurePropertyImpl to update internal data structures.
-   * This method cannot be overridden, subclasses should instead override
-   * reconfigureProperty.
-   */
-  @Override
-  public final String reconfigureProperty(String property, String newVal) 
-    throws ReconfigurationException {
-    if (isPropertyReconfigurable(property)) {
-      LOG.info("changing property " + property + " to " + newVal);
-      String oldVal;
-      synchronized(getConf()) {
-        oldVal = getConf().get(property);
-        reconfigurePropertyImpl(property, newVal);
-        if (newVal != null) {
-          getConf().set(property, newVal);
-        } else {
-          getConf().unset(property);
-        }
-      }
-      return oldVal;
-    } else {
-      throw new ReconfigurationException(property, newVal,
-                                             getConf().get(property));
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * Subclasses must override this.
-   */
-  @Override 
-  public abstract Collection<String> getReconfigurableProperties();
-
-
-  /**
-   * {@inheritDoc}
-   *
-   * Subclasses may wish to override this with a more efficient implementation.
-   */
-  @Override
-  public boolean isPropertyReconfigurable(String property) {
-    return getReconfigurableProperties().contains(property);
-  }
-
-  /**
-   * Change a configuration property.
-   *
-   * Subclasses must override this. This method applies the change to
-   * all internal data structures derived from the configuration property
-   * that is being changed. If this object owns other Reconfigurable objects
-   * reconfigureProperty should be called recursively to make sure that
-   * to make sure that the configuration of these objects is updated.
-   */
-  protected abstract void reconfigurePropertyImpl(String property, String newVal) 
-    throws ReconfigurationException;
-
-}

+ 0 - 104
src/java/org/apache/hadoop/conf/ReconfigurationException.java

@@ -1,104 +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.conf;
-
-
-/**
- * Exception indicating that configuration property cannot be changed
- * at run time.
- */
-public class ReconfigurationException extends Exception {
-
-  private static final long serialVersionUID = 1L;
-
-  private String property;
-  private String newVal;
-  private String oldVal;
-
-  /**
-   * Construct the exception message.
-   */
-  private static String constructMessage(String property, 
-                                         String newVal, String oldVal) {
-    String message = "Could not change property " + property;
-    if (oldVal != null) {
-      message += " from \'" + oldVal;
-    }
-    if (newVal != null) {
-      message += "\' to \'" + newVal + "\'";
-    }
-    return message;
-  }
-
-  
-  /**
-   * Create a new instance of {@link ReconfigurationException}.
-   */
-  public ReconfigurationException() {
-    super("Could not change configuration.");
-    this.property = null;
-    this.newVal = null;
-    this.oldVal = null;
-  }
-
-  /**
-   * Create a new instance of {@link ReconfigurationException}.
-   */
-  public ReconfigurationException(String property, 
-                                  String newVal, String oldVal,
-                                  Throwable cause) {
-    super(constructMessage(property, newVal, oldVal), cause);
-    this.property = property;
-    this.newVal = newVal;
-    this.oldVal = oldVal;
-  }
-
-  /**
-   * Create a new instance of {@link ReconfigurationException}.
-   */
-  public ReconfigurationException(String property, 
-                                  String newVal, String oldVal) {
-    super(constructMessage(property, newVal, oldVal));
-    this.property = property;
-    this.newVal = newVal;
-    this.oldVal = oldVal;
-  }
-
-  /**
-   * Get property that cannot be changed.
-   */
-  public String getProperty() {
-    return property;
-  }
-
-  /**
-   * Get value to which property was supposed to be changed.
-   */
-  public String getNewValue() {
-    return newVal;
-  }
-
-  /**
-   * Get old value of property that cannot be changed.
-   */
-  public String getOldValue() {
-    return oldVal;
-  }
-
-}

+ 0 - 248
src/java/org/apache/hadoop/conf/ReconfigurationServlet.java

@@ -1,248 +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.conf;
-
-import org.apache.commons.logging.*;
-
-import org.apache.commons.lang.StringEscapeUtils;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Enumeration;
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * A servlet for changing a node's configuration.
- *
- * Reloads the configuration file, verifies whether changes are
- * possible and asks the admin to approve the change.
- *
- */
-public class ReconfigurationServlet extends HttpServlet {
-  
-  private static final long serialVersionUID = 1L;
-
-  private static final Log LOG =
-    LogFactory.getLog(ReconfigurationServlet.class);
-
-  // the prefix used to fing the attribute holding the reconfigurable 
-  // for a given request
-  //
-  // we get the attribute prefix + servlet path
-  public static final String CONF_SERVLET_RECONFIGURABLE_PREFIX =
-    "conf.servlet.reconfigurable.";
-  
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void init() throws ServletException {
-    super.init();
-  }
-
-  private Reconfigurable getReconfigurable(HttpServletRequest req) {
-    LOG.info("servlet path: " + req.getServletPath());
-    LOG.info("getting attribute: " + CONF_SERVLET_RECONFIGURABLE_PREFIX +
-             req.getServletPath());
-    return (Reconfigurable)
-      this.getServletContext().getAttribute(CONF_SERVLET_RECONFIGURABLE_PREFIX +
-                                            req.getServletPath());
-  }
-
-  private void printHeader(PrintWriter out, String nodeName) {
-    out.print("<html><head>");
-    out.printf("<title>%s Reconfiguration Utility</title>\n", 
-               StringEscapeUtils.escapeHtml(nodeName));
-    out.print("</head><body>\n");
-    out.printf("<h1>%s Reconfiguration Utility</h1>\n",
-               StringEscapeUtils.escapeHtml(nodeName));
-  }
-
-  private void printFooter(PrintWriter out) {
-    out.print("</body></html>\n");
-  }
-  
-  /**
-   * Print configuration options that can be changed.
-   */
-  private void printConf(PrintWriter out, Reconfigurable reconf) {
-    Configuration oldConf = reconf.getConf();
-    Configuration newConf = new Configuration();
-
-    Collection<ReconfigurationUtil.PropertyChange> changes = 
-      ReconfigurationUtil.getChangedProperties(newConf, 
-                                               oldConf);
-
-    boolean changeOK = true;
-    
-    out.println("<form action=\"\" method=\"post\">");
-    out.println("<table border=\"1\">");
-    out.println("<tr><th>Property</th><th>Old value</th>");
-    out.println("<th>New value </th><th></th></tr>");
-    for (ReconfigurationUtil.PropertyChange c: changes) {
-      out.print("<tr><td>");
-      if (!reconf.isPropertyReconfigurable(c.prop)) {
-        out.print("<font color=\"red\">" + 
-                  StringEscapeUtils.escapeHtml(c.prop) + "</font>");
-        changeOK = false;
-      } else {
-        out.print(StringEscapeUtils.escapeHtml(c.prop));
-        out.print("<input type=\"hidden\" name=\"" +
-                  StringEscapeUtils.escapeHtml(c.prop) + "\" value=\"" +
-                  StringEscapeUtils.escapeHtml(c.newVal) + "\"/>");
-      }
-      out.print("</td><td>" +
-                (c.oldVal == null ? "<it>default</it>" : 
-                 StringEscapeUtils.escapeHtml(c.oldVal)) +
-                "</td><td>" +
-                (c.newVal == null ? "<it>default</it>" : 
-                 StringEscapeUtils.escapeHtml(c.newVal)) +
-                "</td>");
-      out.print("</tr>\n");
-    }
-    out.println("</table>");
-    if (!changeOK) {
-      out.println("<p><font color=\"red\">WARNING: properties marked red" +
-                  " will not be changed until the next restart.</font></p>");
-    }
-    out.println("<input type=\"submit\" value=\"Apply\" />");
-    out.println("</form>");
-  }
-
-  @SuppressWarnings("unchecked")
-  private Enumeration<String> getParams(HttpServletRequest req) {
-    return (Enumeration<String>) req.getParameterNames();
-  }
-
-  /**
-   * Apply configuratio changes after admin has approved them.
-   */
-  private void applyChanges(PrintWriter out, Reconfigurable reconf,
-                            HttpServletRequest req) 
-    throws IOException, ReconfigurationException {
-    Configuration oldConf = reconf.getConf();
-    Configuration newConf = new Configuration();
-
-    Enumeration<String> params = getParams(req);
-
-    synchronized(oldConf) {
-      while (params.hasMoreElements()) {
-        String rawParam = params.nextElement();
-        String param = StringEscapeUtils.unescapeHtml(rawParam);
-        String value =
-          StringEscapeUtils.unescapeHtml(req.getParameter(rawParam));
-        if (value != null) {
-          if (value.equals(newConf.getRaw(param)) || value.equals("default") ||
-              value.equals("null") || value.equals("")) {
-            if ((value.equals("default") || value.equals("null") || 
-                 value.equals("")) && 
-                oldConf.getRaw(param) != null) {
-              out.println("<p>Changed \"" + 
-                          StringEscapeUtils.escapeHtml(param) + "\" from \"" +
-                          StringEscapeUtils.escapeHtml(oldConf.getRaw(param)) +
-                          "\" to default</p>");
-              reconf.reconfigureProperty(param, null);
-            } else if (!value.equals("default") && !value.equals("null") &&
-                       !value.equals("") && 
-                       (oldConf.getRaw(param) == null || 
-                        !oldConf.getRaw(param).equals(value))) {
-              // change from default or value to different value
-              if (oldConf.getRaw(param) == null) {
-                out.println("<p>Changed \"" + 
-                            StringEscapeUtils.escapeHtml(param) + 
-                            "\" from default to \"" +
-                            StringEscapeUtils.escapeHtml(value) + "\"</p>");
-              } else {
-                out.println("<p>Changed \"" + 
-                            StringEscapeUtils.escapeHtml(param) + "\" from \"" +
-                            StringEscapeUtils.escapeHtml(oldConf.
-                                                         getRaw(param)) +
-                            "\" to \"" +
-                            StringEscapeUtils.escapeHtml(value) + "\"</p>");
-              }
-              reconf.reconfigureProperty(param, value);
-            } else {
-              LOG.info("property " + param + " unchanged");
-            }
-          } else {
-            // parameter value != newConf value
-            out.println("<p>\"" + StringEscapeUtils.escapeHtml(param) + 
-                        "\" not changed because value has changed from \"" +
-                        StringEscapeUtils.escapeHtml(value) + "\" to \"" +
-                        StringEscapeUtils.escapeHtml(newConf.getRaw(param)) +
-                        "\" since approval</p>");
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  protected void doGet(HttpServletRequest req, HttpServletResponse resp)
-    throws ServletException, IOException {
-    LOG.info("GET");
-    PrintWriter out = resp.getWriter();
-    
-    Reconfigurable reconf = getReconfigurable(req);
-    String nodeName = reconf.getClass().getCanonicalName();
-
-    printHeader(out, nodeName);
-    printConf(out, reconf);
-    printFooter(out);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  protected void doPost(HttpServletRequest req, HttpServletResponse resp)
-    throws ServletException, IOException {
-    LOG.info("POST");
-    PrintWriter out = resp.getWriter();
-
-    Reconfigurable reconf = getReconfigurable(req);
-    String nodeName = reconf.getClass().getCanonicalName();
-
-    printHeader(out, nodeName);
-
-    try { 
-      applyChanges(out, reconf, req);
-    } catch (ReconfigurationException e) {
-      resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, 
-                     StringUtils.stringifyException(e));
-      return;
-    }
-
-    out.println("<p><a href=\"" + req.getServletPath() + "\">back</a></p>");
-    printFooter(out);
-  }
-
-}

+ 0 - 66
src/java/org/apache/hadoop/conf/ReconfigurationUtil.java

@@ -1,66 +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.conf;
-
-import java.util.Map;
-import java.util.Collection;
-import java.util.HashMap;
-
-public class ReconfigurationUtil {
-
-  public static class PropertyChange {
-    public String prop;
-    public String oldVal;
-    public String newVal;
-
-    public PropertyChange(String prop, String newVal, String oldVal) {
-      this.prop = prop;
-      this.newVal = newVal;
-      this.oldVal = oldVal;
-    }
-  }
-
-  public static Collection<PropertyChange> 
-    getChangedProperties(Configuration newConf, Configuration oldConf) {
-    Map<String, PropertyChange> changes = new HashMap<String, PropertyChange>();
-
-    // iterate over old configuration
-    for (Map.Entry<String, String> oldEntry: oldConf) {
-      String prop = oldEntry.getKey();
-      String oldVal = oldEntry.getValue();
-      String newVal = newConf.getRaw(prop);
-      
-      if (newVal == null || !newVal.equals(oldVal)) {
-        changes.put(prop, new PropertyChange(prop, newVal, oldVal));
-      }
-    }
-    
-    // now iterate over new configuration
-    // (to look for properties not present in old conf)
-    for (Map.Entry<String, String> newEntry: newConf) {
-      String prop = newEntry.getKey();
-      String newVal = newEntry.getValue();
-      if (oldConf.get(prop) == null) {
-        changes.put(prop, new PropertyChange(prop, newVal, null));
-      }
-    } 
-
-    return changes.values();
-  }
-}

+ 0 - 293
src/test/core/org/apache/hadoop/conf/TestReconfiguration.java

@@ -1,293 +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.conf;
-
-import junit.framework.TestCase;
-
-import org.junit.Test;
-import org.junit.Before;
-
-import java.util.Collection;
-import java.util.Arrays;
-
-public class TestReconfiguration extends TestCase {
-  private Configuration conf1;
-  private Configuration conf2;
-
-  private static final String PROP1 = "test.prop.one";
-  private static final String PROP2 = "test.prop.two";
-  private static final String PROP3 = "test.prop.three";
-  private static final String PROP4 = "test.prop.four";
-  private static final String PROP5 = "test.prop.five";
-
-  private static final String VAL1 = "val1";
-  private static final String VAL2 = "val2";
-
-  @Before
-  public void setUp () {
-    conf1 = new Configuration();
-    conf2 = new Configuration();
-    
-    // set some test properties
-    conf1.set(PROP1, VAL1);
-    conf1.set(PROP2, VAL1);
-    conf1.set(PROP3, VAL1);
-
-    conf2.set(PROP1, VAL1); // same as conf1
-    conf2.set(PROP2, VAL2); // different value as conf1
-    // PROP3 not set in conf2
-    conf2.set(PROP4, VAL1); // not set in conf1
-
-  }
-  
-  /**
-   * Test ReconfigurationUtil.getChangedProperties.
-   */
-  @Test
-  public void testGetChangedProperties() {
-    Collection<ReconfigurationUtil.PropertyChange> changes = 
-      ReconfigurationUtil.getChangedProperties(conf2, conf1);
-
-    assertTrue(changes.size() == 3);
-
-    boolean changeFound = false;
-    boolean unsetFound = false;
-    boolean setFound = false;
-
-    for (ReconfigurationUtil.PropertyChange c: changes) {
-      if (c.prop.equals(PROP2) && c.oldVal != null && c.oldVal.equals(VAL1) &&
-          c.newVal != null && c.newVal.equals(VAL2)) {
-        changeFound = true;
-      } else if (c.prop.equals(PROP3) && c.oldVal != null && c.oldVal.equals(VAL1) &&
-          c.newVal == null) {
-        unsetFound = true;
-      } else if (c.prop.equals(PROP4) && c.oldVal == null &&
-          c.newVal != null && c.newVal.equals(VAL1)) {
-        setFound = true;
-      } 
-    }
-    
-    assertTrue(changeFound && unsetFound && setFound);
-  }
-
-  /**
-   * a simple reconfigurable class
-   */
-  public static class ReconfigurableDummy extends ReconfigurableBase 
-  implements Runnable {
-    public volatile boolean running = true;
-
-    public ReconfigurableDummy(Configuration conf) {
-      super(conf);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override 
-    public Collection<String> getReconfigurableProperties() {
-      return Arrays.asList(PROP1, PROP2, PROP4);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public synchronized void reconfigurePropertyImpl(String property, 
-                                                     String newVal) {
-      // do nothing
-    }
-    
-    /**
-     * Run until PROP1 is no longer VAL1.
-     */
-    @Override
-    public void run() {
-      while (running && getConf().get(PROP1).equals(VAL1)) {
-        try {
-          Thread.sleep(1);
-        } catch (InterruptedException ignore) {
-          // do nothing
-        }
-      }
-    }
-
-  }
-
-  /**
-   * Test reconfiguring a Reconfigurable.
-   */
-  @Test
-  public void testReconfigure() {
-    ReconfigurableDummy dummy = new ReconfigurableDummy(conf1);
-
-    assertTrue(dummy.getConf().get(PROP1).equals(VAL1));
-    assertTrue(dummy.getConf().get(PROP2).equals(VAL1));
-    assertTrue(dummy.getConf().get(PROP3).equals(VAL1));
-    assertTrue(dummy.getConf().get(PROP4) == null);
-    assertTrue(dummy.getConf().get(PROP5) == null);
-
-    assertTrue(dummy.isPropertyReconfigurable(PROP1));
-    assertTrue(dummy.isPropertyReconfigurable(PROP2));
-    assertFalse(dummy.isPropertyReconfigurable(PROP3));
-    assertTrue(dummy.isPropertyReconfigurable(PROP4));
-    assertFalse(dummy.isPropertyReconfigurable(PROP5));
-
-    // change something to the same value as before
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP1, VAL1);
-        assertTrue(dummy.getConf().get(PROP1).equals(VAL1));
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertFalse(exceptionCaught);
-    }
-
-    // change something to null
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP1, null);
-        assertTrue(dummy.getConf().get(PROP1) == null);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertFalse(exceptionCaught);
-    }
-
-    // change something to a different value than before
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP1, VAL2);
-        assertTrue(dummy.getConf().get(PROP1).equals(VAL2));
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertFalse(exceptionCaught);
-    }
-
-    // set unset property to null
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP4, null);
-        assertTrue(dummy.getConf().get(PROP4) == null);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertFalse(exceptionCaught);
-    }
-
-    // set unset property
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP4, VAL1);
-        assertTrue(dummy.getConf().get(PROP4).equals(VAL1));
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertFalse(exceptionCaught);
-    }
-
-    // try to set unset property to null (not reconfigurable)
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP5, null);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertTrue(exceptionCaught);
-    }
-
-    // try to set unset property to value (not reconfigurable)
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP5, VAL1);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertTrue(exceptionCaught);
-    }
-
-    // try to change property to value (not reconfigurable)
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP3, VAL2);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertTrue(exceptionCaught);
-    }
-
-    // try to change property to null (not reconfigurable)
-    {
-      boolean exceptionCaught = false;
-      try {
-        dummy.reconfigureProperty(PROP3, null);
-      } catch (ReconfigurationException e) {
-        exceptionCaught = true;
-      }
-      assertTrue(exceptionCaught);
-    }
-  }
-
-  /**
-   * Test whether configuration changes are visible in another thread.
-   */
-  @Test
-  public void testThread() throws ReconfigurationException { 
-    ReconfigurableDummy dummy = new ReconfigurableDummy(conf1);
-    assertTrue(dummy.getConf().get(PROP1).equals(VAL1));
-    Thread dummyThread = new Thread(dummy);
-    dummyThread.start();
-    try {
-      Thread.sleep(500);
-    } catch (InterruptedException ignore) {
-      // do nothing
-    }
-    dummy.reconfigureProperty(PROP1, VAL2);
-
-    long endWait = System.currentTimeMillis() + 2000;
-    while (dummyThread.isAlive() && System.currentTimeMillis() < endWait) {
-      try {
-        Thread.sleep(50);
-      } catch (InterruptedException ignore) {
-        // do nothing
-      }
-    }
-
-    assertFalse(dummyThread.isAlive());
-    dummy.running = false;
-    try {
-      dummyThread.join();
-    } catch (InterruptedException ignore) {
-      // do nothing
-    }
-    assertTrue(dummy.getConf().get(PROP1).equals(VAL2));
-    
-  }
-
-}