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 2008/08/13 00:08:06 UTC

svn commit: r685340 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/rest/

Author: stack
Date: Tue Aug 12 15:08:04 2008
New Revision: 685340

URL: http://svn.apache.org/viewvc?rev=685340&view=rev
Log:
HBASE-795  More Table operation in TableHandler for REST interface

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=685340&r1=685339&r2=685340&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Aug 12 15:08:04 2008
@@ -25,6 +25,8 @@
    HBASE-812  Compaction needs little better skip algo (Daniel Leffel via Stack)
    HBASE-806  Change HbaseMapWritable and RowResult to implement SortedMap
               instead of Map (Jonathan Gray via Stack)
+   HBASE-795  More Table operation in TableHandler for REST interface
+              (Sishen Freecity via Stack)
 
   NEW FEATURES
    HBASE-787  Postgresql to HBase table replication example (Tim Sell via Stack)

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java?rev=685340&r1=685339&r2=685340&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java Tue Aug 12 15:08:04 2008
@@ -22,7 +22,6 @@
 import java.io.IOException;
 
 import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
@@ -63,8 +62,10 @@
   private static final Log LOG = LogFactory.getLog(Dispatcher.class.getName());
   private MetaHandler metaHandler;
   private TableHandler tableHandler;
+  private RowHandler rowHandler;
   private ScannerHandler scannerHandler;
 
+  private static final String TABLES = "tables";
   private static final String SCANNER = "scanner";
   private static final String ROW = "row";
       
@@ -85,6 +86,7 @@
       admin = new HBaseAdmin(conf);
       metaHandler = new MetaHandler(conf, admin);
       tableHandler = new TableHandler(conf, admin);
+      rowHandler = new RowHandler(conf, admin);
       scannerHandler = new ScannerHandler(conf, admin);
     } catch(Exception e){
       throw new ServletException(e);
@@ -100,9 +102,14 @@
       // the entire instance.
       metaHandler.doGet(request, response, pathSegments);
     } else {
-      // otherwise, it must be a GET request suitable for the
-      // table handler.
-      tableHandler.doGet(request, response, pathSegments);
+      if (pathSegments.length >= 2 && pathSegments[0].length() > 0 && pathSegments[1].toLowerCase().equals(ROW)) {
+        // if it has table name and row path segments
+        rowHandler.doGet(request, response, pathSegments);
+      } else {
+        // otherwise, it must be a GET request suitable for the
+        // table handler.
+        tableHandler.doGet(request, response, pathSegments);
+      }
     }
   }
 
@@ -110,15 +117,26 @@
   throws IOException, ServletException {
     String [] pathSegments = getPathSegments(request);
     
-    // there should be at least two path segments (table name and row or scanner)
-    if (pathSegments.length >= 2 && pathSegments[0].length() > 0) {
-      if (pathSegments[1].toLowerCase().equals(SCANNER) &&
-          pathSegments.length >= 2) {
-        scannerHandler.doPost(request, response, pathSegments);
-        return;
-      } else if (pathSegments[1].toLowerCase().equals(ROW) && pathSegments.length >= 3) {
-        tableHandler.doPost(request, response, pathSegments);
-        return;
+    if (pathSegments.length == 1 && pathSegments[0].toLowerCase().equals(TABLES)) {
+      tableHandler.doPost(request, response, pathSegments);
+      return;
+    } else {
+      // there should be at least two path segments (table name and row or
+      // scanner)
+      if (pathSegments.length >= 2 && pathSegments[0].length() > 0) {
+        if (pathSegments[1].toLowerCase().equals(SCANNER)
+            && pathSegments.length >= 2) {
+          scannerHandler.doPost(request, response, pathSegments);
+          return;
+        } else if (pathSegments[1].toLowerCase().equals(ROW)
+            && pathSegments.length >= 3) {
+          rowHandler.doPost(request, response, pathSegments);
+          return;
+        } else if (pathSegments[0].toLowerCase().equals(TABLES) && pathSegments[1].length() > 0 
+            && (pathSegments[2].toLowerCase().equals(TableHandler.DISABLE) || pathSegments[2].toLowerCase().equals(TableHandler.ENABLE))) {
+          tableHandler.doPost(request, response, pathSegments);
+          return;
+        }
       }
     }
 
@@ -130,7 +148,13 @@
   protected void doPut(HttpServletRequest request, HttpServletResponse response)
   throws ServletException, IOException {
     // Equate PUT with a POST.
-    doPost(request, response);
+    String [] pathSegments = getPathSegments(request);
+    
+    if (pathSegments.length == 2 && pathSegments[0].toLowerCase().equals(TABLES) && pathSegments[1].length() > 0) {
+      tableHandler.doPut(request, response, pathSegments);
+    } else {
+      doPost(request, response);
+    }
   }
 
   protected void doDelete(HttpServletRequest request,
@@ -138,18 +162,21 @@
   throws IOException, ServletException {
     String [] pathSegments = getPathSegments(request);
     
-    // must be at least two path segments (table name and row or scanner)
-    if (pathSegments.length >= 2 && pathSegments[0].length() > 0) {
-      // DELETE to a scanner requires at least three path segments
+    if (pathSegments.length == 2 && pathSegments[0].toLowerCase().equals(TABLES) && pathSegments[1].length() > 0) {
+      tableHandler.doDelete(request, response, pathSegments);
+      return;
+    } else if (pathSegments.length >= 3 && pathSegments[0].length() > 0) {
+      // must be at least two path segments (table name and row or scanner)
       if (pathSegments[1].toLowerCase().equals(SCANNER) &&
           pathSegments.length == 3 && pathSegments[2].length() > 0) {
+        // DELETE to a scanner requires at least three path segments
         scannerHandler.doDelete(request, response, pathSegments);
         return;
       } else if (pathSegments[1].toLowerCase().equals(ROW) &&
           pathSegments.length >= 3) {
-        tableHandler.doDelete(request, response, pathSegments);
+        rowHandler.doDelete(request, response, pathSegments);
         return;
-      } 
+      }
     }
     
     // if we reach this point, then no handler exists for this request.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java?rev=685340&r1=685339&r2=685340&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java Tue Aug 12 15:08:04 2008
@@ -55,6 +55,7 @@
   protected static final String CONTENT_TYPE = "content-type";
   protected static final String ROW = "row";
   protected static final String REGIONS = "regions";
+  protected static final String TABLES = "tables";
   
   protected final Log LOG = LogFactory.getLog(this.getClass());
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java?rev=685340&r1=685339&r2=685340&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java Tue Aug 12 15:08:04 2008
@@ -35,7 +35,7 @@
 
 /**
  * MetaHandler fields all requests for metadata at the instance level. At the
- * momment this is only GET requests to /.
+ * moment this is only GET requests to /.
  */
 public class MetaHandler extends GenericHandler {
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java?rev=685340&r1=685339&r2=685340&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java Tue Aug 12 15:08:04 2008
@@ -21,12 +21,6 @@
 
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.URLDecoder;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
@@ -36,14 +30,11 @@
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
@@ -57,7 +48,9 @@
  * this handler.
  */
 public class TableHandler extends GenericHandler {
-     
+  public static final String DISABLE = "disable";
+  public static final String ENABLE = "enable";
+  
   public TableHandler(HBaseConfiguration conf, HBaseAdmin admin) 
   throws ServletException{
     super(conf, admin);
@@ -76,10 +69,6 @@
         // get a region list
         getTableRegions(table, request, response);
       }
-      else if (pathSegments[1].toLowerCase().equals(ROW)) {
-        // get a row
-        getRow(table, request, response, pathSegments);
-      }
       else{
         doNotFound(response, "Not handled in TableHandler");
       }
@@ -89,252 +78,31 @@
   public void doPost(HttpServletRequest request, HttpServletResponse response, 
     String[] pathSegments)
   throws ServletException, IOException {
-    putRow(request, response, pathSegments);
+    if (pathSegments.length == 1) {
+      // if it's a creation operation
+      putTable(request, response, pathSegments);
+    } else {
+      // if it's a disable operation or enable operation
+      String tableName = pathSegments[1];      
+      if (pathSegments[2].toLowerCase().equals(DISABLE)) {
+        admin.disableTable(tableName);
+      } else if (pathSegments[2].toLowerCase().equals(ENABLE)) {
+        admin.enableTable(tableName);
+      }
+      response.setStatus(202);
+    }
   }
-  
+
   public void doPut(HttpServletRequest request, HttpServletResponse response, 
     String[] pathSegments)
   throws ServletException, IOException {
-    doPost(request, response, pathSegments);
+    updateTable(request, response, pathSegments);
   }
   
   public void doDelete(HttpServletRequest request, HttpServletResponse response, 
     String[] pathSegments)
   throws ServletException, IOException {
-    deleteRow(request, response, pathSegments);
-  }
-  
-  /*
-   * @param request
-   * @param response
-   * @param pathSegments info path split on the '/' character.  First segment
-   * is the tablename, second is 'row', and third is the row id.
-   * @throws IOException
-   * Retrieve a row in one of several output formats.
-   */
-  private void getRow(HTable table, final HttpServletRequest request,
-    final HttpServletResponse response, final String [] pathSegments)
-  throws IOException {
-    // pull the row key out of the path
-    String row = URLDecoder.decode(pathSegments[2], HConstants.UTF8_ENCODING);
-
-    String timestampStr = null;
-    if (pathSegments.length == 4) {
-      // A timestamp has been supplied.
-      timestampStr = pathSegments[3];
-      if (timestampStr.equals("timestamps")) {
-        // Not supported in hbase just yet. TODO
-        doMethodNotAllowed(response, "Not yet supported by hbase");
-        return;
-      }
-    }
-    
-    String[] columns = request.getParameterValues(COLUMN);
-        
-    if (columns == null || columns.length == 0) {
-      // They want full row returned. 
-
-      // Presumption is that this.table has already been focused on target table.
-      Map<byte [], Cell> result = timestampStr == null ? 
-        table.getRow(Bytes.toBytes(row)) 
-        : table.getRow(Bytes.toBytes(row), Long.parseLong(timestampStr));
-        
-      if (result == null || result.size() == 0) {
-        doNotFound(response, "Row not found!");
-      } else {
-        switch (ContentType.getContentType(request.getHeader(ACCEPT))) {
-        case XML:
-          outputRowXml(response, result);
-          break;
-        case MIME:
-        default:
-          doNotAcceptable(response, "Unsupported Accept Header Content: " +
-            request.getHeader(CONTENT_TYPE));
-        }
-      }
-    } else {
-      Map<byte [], Cell> prefiltered_result = table.getRow(Bytes.toBytes(row));
-    
-      if (prefiltered_result == null || prefiltered_result.size() == 0) {
-        doNotFound(response, "Row not found!");
-      } else {
-        // create a Set from the columns requested so we can
-        // efficiently filter the actual found columns
-        Set<String> requested_columns_set = new HashSet<String>();
-        for(int i = 0; i < columns.length; i++){
-          requested_columns_set.add(columns[i]);
-        }
-  
-        // output map that will contain the filtered results
-        Map<byte [], Cell> m =
-          new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
-
-        // get an array of all the columns retrieved
-        Set<byte []> columns_retrieved = prefiltered_result.keySet();
-
-        // copy over those cells with requested column names
-        for(byte [] current_column: columns_retrieved) {
-          if (requested_columns_set.contains(Bytes.toString(current_column))) {
-            m.put(current_column, prefiltered_result.get(current_column));            
-          }
-        }
-        
-        switch (ContentType.getContentType(request.getHeader(ACCEPT))) {
-          case XML:
-            outputRowXml(response, m);
-            break;
-          case MIME:
-          default:
-            doNotAcceptable(response, "Unsupported Accept Header Content: " +
-              request.getHeader(CONTENT_TYPE));
-        }
-      }
-    }
-  }
-  
-  /*
-   * Output a row encoded as XML.
-   * @param response
-   * @param result
-   * @throws IOException
-   */
-  private void outputRowXml(final HttpServletResponse response,
-      final Map<byte [], Cell> result)
-  throws IOException {
-    setResponseHeader(response, result.size() > 0? 200: 204,
-        ContentType.XML.toString());
-    XMLOutputter outputter = getXMLOutputter(response.getWriter());
-    outputter.startTag(ROW);
-    outputColumnsXml(outputter, result);
-    outputter.endTag();
-    outputter.endDocument();
-    outputter.getWriter().close();
-  }
-
-  /*
-   * @param response
-   * @param result
-   * Output the results contained in result as a multipart/related response.
-   */
-  // private void outputRowMime(final HttpServletResponse response,
-  //     final Map<Text, Cell> result)
-  // throws IOException {
-  //   response.setStatus(result.size() > 0? 200: 204);
-  //   // This code ties me to the jetty server.
-  //   MultiPartResponse mpr = new MultiPartResponse(response);
-  //   // Content type should look like this for multipart:
-  //   // Content-type: multipart/related;start="<ro...@example.jaxws.sun.com>";type="application/xop+xml";boundary="uuid:94ebf1e6-7eb5-43f1-85f4-2615fc40c5d6";start-info="text/xml"
-  //   String ct = ContentType.MIME.toString() + ";charset=\"UTF-8\";boundary=\"" +
-  //     mpr.getBoundary() + "\"";
-  //   // Setting content type is broken.  I'm unable to set parameters on the
-  //   // content-type; They get stripped.  Can't set boundary, etc.
-  //   // response.addHeader("Content-Type", ct);
-  //   response.setContentType(ct);
-  //   outputColumnsMime(mpr, result);
-  //   mpr.close();
-  // }
-  
-  /*
-   * @param request
-   * @param response
-   * @param pathSegments
-   * Do a put based on the client request.
-   */
-  private void putRow(final HttpServletRequest request,
-    final HttpServletResponse response, final String [] pathSegments)
-  throws IOException, ServletException {
-    HTable table = getTable(pathSegments[0]);
-
-    // pull the row key out of the path
-    String row = URLDecoder.decode(pathSegments[2], HConstants.UTF8_ENCODING);
-    
-    switch(ContentType.getContentType(request.getHeader(CONTENT_TYPE))) {
-      case XML:
-        putRowXml(table, row, request, response, pathSegments);
-        break;
-      case MIME:
-        doNotAcceptable(response, "Don't support multipart/related yet...");
-        break;
-      default:
-        doNotAcceptable(response, "Unsupported Accept Header Content: " +
-          request.getHeader(CONTENT_TYPE));
-    }
-  }
-
-  /*
-   * @param request
-   * @param response
-   * @param pathSegments
-   * Decode supplied XML and do a put to Hbase.
-   */
-  private void putRowXml(HTable table, String row, 
-    final HttpServletRequest request, final HttpServletResponse response, 
-    final String [] pathSegments)
-  throws IOException, ServletException{
-
-    DocumentBuilderFactory docBuilderFactory 
-      = DocumentBuilderFactory.newInstance();  
-    //ignore all comments inside the xml file
-    docBuilderFactory.setIgnoringComments(true);
-
-    DocumentBuilder builder = null;
-    Document doc = null;
-    
-    String timestamp = pathSegments.length >= 4 ? pathSegments[3] : null;
-    
-    try{
-      builder = docBuilderFactory.newDocumentBuilder();
-      doc = builder.parse(request.getInputStream());
-    } catch (javax.xml.parsers.ParserConfigurationException e) {
-      throw new ServletException(e);
-    } catch (org.xml.sax.SAXException e){
-      throw new ServletException(e);
-    }
-
-    BatchUpdate batchUpdate;
-    
-    try{
-      // start an update
-      batchUpdate = timestamp == null ? 
-        new BatchUpdate(row) : new BatchUpdate(row, Long.parseLong(timestamp));
-
-      // set the columns from the xml
-      NodeList columns = doc.getElementsByTagName("column");
-
-      for(int i = 0; i < columns.getLength(); i++){
-        // get the current column element we're working on
-        Element column = (Element)columns.item(i);
-
-        // extract the name and value children
-        Node name_node = column.getElementsByTagName("name").item(0);
-        String name = name_node.getFirstChild().getNodeValue();
-
-        Node value_node = column.getElementsByTagName("value").item(0);
-
-        byte[] value = new byte[0];
-        
-        // for some reason there's no value here. probably indicates that
-        // the consumer passed a null as the cell value.
-        if(value_node.getFirstChild() != null && 
-          value_node.getFirstChild().getNodeValue() != null){
-          // decode the base64'd value
-          value = org.apache.hadoop.hbase.util.Base64.decode(
-            value_node.getFirstChild().getNodeValue());
-        }
-
-        // put the value
-        batchUpdate.put(name, value);
-      }
-
-      // commit the update
-      table.commit(batchUpdate);
-      
-      // respond with a 200
-      response.setStatus(200);      
-    }
-    catch(Exception e){
-      throw new ServletException(e);
-    }
+    deleteTable(request, response, pathSegments);
   }
 
   /*
@@ -438,41 +206,158 @@
     }
   }
   
-  /*
-   * @param request
-   * @param response
-   * @param pathSegments
-   * Delete some or all cells for a row.
-   */
-   private void deleteRow(final HttpServletRequest request,
-    final HttpServletResponse response, final String [] pathSegments)
+  private void putTable(HttpServletRequest request,
+    HttpServletResponse response, String[] pathSegments) 
+  throws IOException, ServletException {
+    switch(ContentType.getContentType(request.getHeader(CONTENT_TYPE))) {
+      case XML:
+        putTableXml(request, response, pathSegments);
+        break;
+      case MIME:
+        doNotAcceptable(response, "Don't support multipart/related yet...");
+        break;
+      default:
+        doNotAcceptable(response, "Unsupported Accept Header Content: " +
+            request.getHeader(CONTENT_TYPE));
+    }
+  } 
+  
+  private void updateTable(HttpServletRequest request,
+    HttpServletResponse response, String[] pathSegments) 
+  throws IOException, ServletException {
+    switch(ContentType.getContentType(request.getHeader(CONTENT_TYPE))) {
+      case XML:
+        updateTableXml(request, response, pathSegments);
+        break;
+      case MIME:
+        doNotAcceptable(response, "Don't support multipart/related yet...");
+        break;
+      default:
+        doNotAcceptable(response, "Unsupported Accept Header Content: " +
+            request.getHeader(CONTENT_TYPE));
+    }
+  }
+  
+  private void deleteTable(HttpServletRequest request,
+      HttpServletResponse response, String[] pathSegments) throws IOException {
+    String tableName = pathSegments[1];
+    admin.deleteTable(tableName);
+    response.setStatus(202);
+  }  
+  
+  private void putTableXml(HttpServletRequest 
+    request, HttpServletResponse response, String[] pathSegments)
   throws IOException, ServletException {
-    // grab the table we're operating on
-    HTable table = getTable(getTableName(pathSegments));
+    DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
+        .newInstance();
+    // ignore all comments inside the xml file
+    docBuilderFactory.setIgnoringComments(true);
+
+    DocumentBuilder builder = null;
+    Document doc = null;
+
+    try {
+      builder = docBuilderFactory.newDocumentBuilder();
+      doc = builder.parse(request.getInputStream());
+    } catch (javax.xml.parsers.ParserConfigurationException e) {
+      throw new ServletException(e);
+    } catch (org.xml.sax.SAXException e) {
+      throw new ServletException(e);
+    }
     
-    // pull the row key out of the path
-    String row = URLDecoder.decode(pathSegments[2], HConstants.UTF8_ENCODING);
+    try {
+      Node name_node = doc.getElementsByTagName("name").item(0);
+      String table_name = name_node.getFirstChild().getNodeValue();
+      
+      HTableDescriptor htd = new HTableDescriptor(table_name);
+      NodeList columnfamily_nodes = doc.getElementsByTagName("columnfamily");
+      for (int i = 0; i < columnfamily_nodes.getLength(); i++) {
+        Element columnfamily = (Element)columnfamily_nodes.item(i);
+        htd.addFamily(putColumnFamilyXml(columnfamily));
+      }
+      admin.createTable(htd);      
+    } catch (Exception e) {
+      throw new ServletException(e);
+    }
+  }
 
-    String[] columns = request.getParameterValues(COLUMN);
-        
-    // hack - we'll actually test for the presence of the timestamp parameter
-    // eventually
-    boolean timestamp_present = false;
-    if(timestamp_present){ // do a timestamp-aware delete
-      doMethodNotAllowed(response, "DELETE with a timestamp not implemented!");
-    }
-    else{ // ignore timestamps
-      if(columns == null || columns.length == 0){
-        // retrieve all the columns
-        doMethodNotAllowed(response,
-          "DELETE without specified columns not implemented!");
-      } else{
-        // delete each column in turn      
-        for(int i = 0; i < columns.length; i++){
-          table.deleteAll(row, columns[i]);
-        }
+  private void updateTableXml(HttpServletRequest request,
+      HttpServletResponse response, String[] pathSegments) throws IOException,
+      ServletException {
+    DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
+        .newInstance();
+    // ignore all comments inside the xml file
+    docBuilderFactory.setIgnoringComments(true);
+
+    DocumentBuilder builder = null;
+    Document doc = null;
+
+    try {
+      builder = docBuilderFactory.newDocumentBuilder();
+      doc = builder.parse(request.getInputStream());
+    } catch (javax.xml.parsers.ParserConfigurationException e) {
+      throw new ServletException(e);
+    } catch (org.xml.sax.SAXException e) {
+      throw new ServletException(e);
+    }
+
+    try {
+      String tableName = pathSegments[1];
+
+      NodeList columnfamily_nodes = doc.getElementsByTagName("columnfamily");
+      for (int i = 0; i < columnfamily_nodes.getLength(); i++) {
+        Element columnfamily = (Element) columnfamily_nodes.item(i);
+        HColumnDescriptor hcd = putColumnFamilyXml(columnfamily);
+        admin.modifyColumn(tableName, hcd.getNameAsString(), hcd);
       }
-      response.setStatus(202);
+    } catch (Exception e) {
+      throw new ServletException(e);
     }
-  } 
+  }
+  
+  private HColumnDescriptor putColumnFamilyXml(Element columnfamily) {
+    Node name_node = columnfamily.getElementsByTagName("name").item(0);
+    String colname = name_node.getFirstChild().getNodeValue();
+    
+    if (colname.indexOf(":") == -1) {
+      colname += ":";
+    }
+    
+    int max_versions = HColumnDescriptor.DEFAULT_VERSIONS;
+    NodeList max_versions_list = columnfamily.getElementsByTagName("max-versions");
+    if (max_versions_list.getLength() > 0) {
+      max_versions = Integer.parseInt(max_versions_list.item(0).getFirstChild().getNodeValue());
+    }
+    CompressionType compression = HColumnDescriptor.DEFAULT_COMPRESSION;
+    NodeList compression_list = columnfamily.getElementsByTagName("compression");
+    if (compression_list.getLength() > 0) {
+      compression = CompressionType.valueOf(compression_list.item(0).getFirstChild().getNodeValue());
+    }
+    boolean in_memory = HColumnDescriptor.DEFAULT_IN_MEMORY;
+    NodeList in_memory_list = columnfamily.getElementsByTagName("in-memory");
+    if (in_memory_list.getLength() > 0) {
+      in_memory = Boolean.valueOf(in_memory_list.item(0).getFirstChild().getNodeValue());
+    }
+    boolean block_cache = HColumnDescriptor.DEFAULT_BLOCKCACHE;
+    NodeList block_cache_list = columnfamily.getElementsByTagName("block-cache");
+    if (block_cache_list.getLength() > 0) {
+      block_cache = Boolean.valueOf(block_cache_list.item(0).getFirstChild().getNodeValue());
+    }
+    int max_cell_size = HColumnDescriptor.DEFAULT_LENGTH;
+    NodeList max_cell_size_list = columnfamily.getElementsByTagName("max-cell-size");
+    if (max_cell_size_list.getLength() > 0) {
+      max_cell_size = Integer.valueOf(max_cell_size_list.item(0).getFirstChild().getNodeValue());
+    }
+    int ttl = HColumnDescriptor.DEFAULT_TTL;
+    NodeList ttl_list = columnfamily.getElementsByTagName("time-to-live");
+    if (ttl_list.getLength() > 0) {
+      ttl = Integer.valueOf(ttl_list.item(0).getFirstChild().getNodeValue());
+    }
+    boolean bloomfilter = HColumnDescriptor.DEFAULT_BLOOMFILTER;
+    NodeList bloomfilter_list = columnfamily.getElementsByTagName("bloomfilter");
+    if (bloomfilter_list.getLength() > 0) {
+      bloomfilter = Boolean.valueOf(bloomfilter_list.item(0).getFirstChild().getNodeValue());
+    }
+    return new HColumnDescriptor(Bytes.toBytes(colname), max_versions, compression, in_memory, block_cache, max_cell_size, ttl, bloomfilter);
+  }
 }