You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2012/01/20 19:06:34 UTC

svn commit: r1234045 - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/rest/ main/java/org/apache/hadoop/hbase/rest/transform/ test/java/org/apache/hadoop/hbase/rest/

Author: apurtell
Date: Fri Jan 20 18:06:34 2012
New Revision: 1234045

URL: http://svn.apache.org/viewvc?rev=1234045&view=rev
Log:
HBASE-5228. [REST] Rip out transform feature

Removed:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/transform/
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestTransform.java
Modified:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java?rev=1234045&r1=1234044&r2=1234045&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java Fri Jan 20 18:06:34 2012
@@ -19,8 +19,6 @@
  */
 package org.apache.hadoop.hbase.rest;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.rest.ResourceBase;
 import org.apache.hadoop.hbase.rest.RowSpec;
@@ -28,7 +26,6 @@ import org.apache.hadoop.hbase.rest.Tabl
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
-import org.apache.hadoop.hbase.rest.transform.Transform;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
@@ -38,10 +35,8 @@ import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
 import java.io.IOException;
-import java.util.ArrayList;
 
 public class MultiRowResource extends ResourceBase {
-  private static final Log LOG = LogFactory.getLog(MultiRowResource.class);
   public static final String ROW_KEYS_PARAM_NAME = "row";
 
   TableResource tableResource;
@@ -87,12 +82,9 @@ public class MultiRowResource extends Re
         KeyValue value = null;
         RowModel rowModel = new RowModel(rk);
 
-
         while ((value = generator.next()) != null) {
-          byte[] family = value.getFamily();
-          byte[] qualifier = value.getQualifier();
-          byte[] data = tableResource.transform(family, qualifier, value.getValue(), Transform.Direction.OUT);
-          rowModel.addCell(new CellModel(family, qualifier, value.getTimestamp(), data));
+          rowModel.addCell(new CellModel(value.getFamily(), value.getQualifier(),
+            value.getTimestamp(), value.getValue()));
         }
 
         model.addRow(rowModel);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java?rev=1234045&r1=1234044&r2=1234045&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java Fri Jan 20 18:06:34 2012
@@ -44,12 +44,10 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
-import org.apache.hadoop.hbase.rest.transform.Transform;
 import org.apache.hadoop.hbase.util.Bytes;
 
 public class RowResource extends ResourceBase {
@@ -99,12 +97,8 @@ public class RowResource extends Resourc
           rowKey = value.getRow();
           rowModel = new RowModel(rowKey);
         }
-        byte[] family = value.getFamily();
-        byte[] qualifier = value.getQualifier();
-        byte[] data = tableResource.transform(family, qualifier,
-          value.getValue(), Transform.Direction.OUT);
-        rowModel.addCell(new CellModel(family, qualifier,
-          value.getTimestamp(), data));
+        rowModel.addCell(new CellModel(value.getFamily(), value.getQualifier(),
+          value.getTimestamp(), value.getValue()));
         if (++count > rowspec.getMaxValues()) {
           break;
         }
@@ -137,11 +131,7 @@ public class RowResource extends Resourc
         throw new WebApplicationException(Response.Status.NOT_FOUND);
       }
       KeyValue value = generator.next();
-      byte[] family = value.getFamily();
-      byte[] qualifier = value.getQualifier();
-      byte[] data = tableResource.transform(family, qualifier,
-        value.getValue(), Transform.Direction.OUT);
-      ResponseBuilder response = Response.ok(data);
+      ResponseBuilder response = Response.ok(value.getValue());
       response.header("X-Timestamp", value.getTimestamp());
       return response.build();
     } catch (IOException e) {
@@ -182,13 +172,9 @@ public class RowResource extends Resourc
           }
           byte [][] parts = KeyValue.parseColumn(col);
           if (parts.length == 2 && parts[1].length > 0) {
-            put.add(parts[0], parts[1], cell.getTimestamp(),
-              tableResource.transform(parts[0], parts[1], cell.getValue(),
-                Transform.Direction.IN));
+            put.add(parts[0], parts[1], cell.getTimestamp(), cell.getValue());
           } else {
-            put.add(parts[0], null, cell.getTimestamp(),
-              tableResource.transform(parts[0], null, cell.getValue(),
-                Transform.Direction.IN));
+            put.add(parts[0], null, cell.getTimestamp(), cell.getValue());
           }
         }
         puts.add(put);
@@ -251,13 +237,9 @@ public class RowResource extends Resourc
       Put put = new Put(row);
       byte parts[][] = KeyValue.parseColumn(column);
       if (parts.length == 2 && parts[1].length > 0) {
-        put.add(parts[0], parts[1], timestamp,
-          tableResource.transform(parts[0], parts[1], message,
-            Transform.Direction.IN));
+        put.add(parts[0], parts[1], timestamp, message);
       } else {
-        put.add(parts[0], null, timestamp,
-          tableResource.transform(parts[0], null, message,
-            Transform.Direction.IN));
+        put.add(parts[0], null, timestamp, message);
       }
       table = pool.getTable(tableResource.getName());
       table.put(put);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java?rev=1234045&r1=1234044&r2=1234045&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java Fri Jan 20 18:06:34 2012
@@ -21,177 +21,19 @@
 package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import javax.ws.rs.Encoded;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.QueryParam;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.rest.transform.NullTransform;
-import org.apache.hadoop.hbase.rest.transform.Transform;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.StringUtils;
 
 public class TableResource extends ResourceBase {
-  private static final Log LOG = LogFactory.getLog(TableResource.class);
-
-  /**
-   * HCD attributes starting with this string are considered transform
-   * directives
-   */
-  private static final String DIRECTIVE_KEY = "Transform$";
-
-  /**
-   * Transform directives are of the form <tt>&lt;qualifier&gt;:&lt;class&gt;</tt>
-   * where <tt>qualifier</tt> is a string for exact matching or '*' as a wildcard
-   * that will match anything; and <tt>class</tt> is either the fully qualified
-   * class name of a transform implementation or can be the short name of a
-   * transform in the <tt>org.apache.hadoop.hbase.rest.transform package</tt>.
-   */
-  private static final Pattern DIRECTIVE_PATTERN =
-    Pattern.compile("([^\\:]+)\\:([^\\,]+)\\,?");
-  private static final Transform defaultTransform = new NullTransform();
-  private static final
-    Map<String,Map<byte[],Map<byte[],Transform>>> transformMap =
-      new ConcurrentHashMap<String,Map<byte[],Map<byte[],Transform>>>();
-  private static final Map<String,Long> lastCheckedMap =
-    new ConcurrentHashMap<String,Long>();
-
-  /**
-   * @param table the table
-   * @param family the column family
-   * @param qualifier the column qualifier, or null
-   * @return the transformation specified for the given family or qualifier, if
-   * any, otherwise the default
-   */
-  static Transform getTransform(String table, byte[] family, byte[] qualifier) {
-    if (qualifier == null) {
-      qualifier = HConstants.EMPTY_BYTE_ARRAY;
-    }
-    Map<byte[],Map<byte[],Transform>> familyMap = transformMap.get(table);
-    if (familyMap != null) {
-      Map<byte[],Transform> columnMap = familyMap.get(family);
-      if (columnMap != null) {
-        Transform t = columnMap.get(qualifier);
-        // check as necessary if there is a wildcard entry
-        if (t == null) {
-          t = columnMap.get(HConstants.EMPTY_BYTE_ARRAY);
-        }
-        // if we found something, return it, otherwise we will return the
-        // default by falling through
-        if (t != null) {
-          return t;
-        }
-      }
-    }
-    return defaultTransform;
-  }
-
-  synchronized static void setTransform(String table, byte[] family,
-      byte[] qualifier, Transform transform) {
-    Map<byte[],Map<byte[],Transform>> familyMap = transformMap.get(table);
-    if (familyMap == null) {
-      familyMap =  new ConcurrentSkipListMap<byte[],Map<byte[],Transform>>(
-          Bytes.BYTES_COMPARATOR);
-      transformMap.put(table, familyMap);
-    }
-    Map<byte[],Transform> columnMap = familyMap.get(family);
-    if (columnMap == null) {
-      columnMap = new ConcurrentSkipListMap<byte[],Transform>(
-          Bytes.BYTES_COMPARATOR);
-      familyMap.put(family, columnMap);
-    }
-    // if transform is null, remove any existing entry
-    if (transform != null) {
-      columnMap.put(qualifier, transform);
-    } else {
-      columnMap.remove(qualifier);
-    }
-  }
 
   String table;
 
   /**
-   * Scan the table schema for transform directives. These are column family
-   * attributes containing a comma-separated list of elements of the form
-   * <tt>&lt;qualifier&gt;:&lt;transform-class&gt;</tt>, where qualifier
-   * can be a string for exact matching or '*' as a wildcard to match anything.
-   * The attribute key must begin with the string "Transform$".
-   */
-  void scanTransformAttrs() throws IOException {
-    try {
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-      HTableDescriptor htd = admin.getTableDescriptor(Bytes.toBytes(table));
-      for (HColumnDescriptor hcd: htd.getFamilies()) {
-        for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
-            hcd.getValues().entrySet()) {
-          // does the key start with the transform directive tag?
-          String key = Bytes.toString(e.getKey().get());
-          if (!key.startsWith(DIRECTIVE_KEY)) {
-            // no, skip
-            continue;
-          }
-          // match a comma separated list of one or more directives
-          byte[] value = e.getValue().get();
-          Matcher m = DIRECTIVE_PATTERN.matcher(Bytes.toString(value));
-          while (m.find()) {
-            byte[] qualifier = HConstants.EMPTY_BYTE_ARRAY;
-            String s = m.group(1);
-            if (s.length() > 0 && !s.equals("*")) {
-              qualifier = Bytes.toBytes(s);
-            }
-            boolean retry = false;
-            String className = m.group(2);
-            while (true) {
-              try {
-                // if a transform was previously configured for the qualifier,
-                // this will simply replace it
-                setTransform(table, hcd.getName(), qualifier,
-                  (Transform)Class.forName(className).newInstance());
-                break;
-              } catch (InstantiationException ex) {
-                LOG.error(StringUtils.stringifyException(ex));
-                if (retry) {
-                  break;
-                }
-                retry = true;
-              } catch (IllegalAccessException ex) {
-                LOG.error(StringUtils.stringifyException(ex));
-                if (retry) {
-                  break;
-                }
-                retry = true;
-              } catch (ClassNotFoundException ex) {
-                if (retry) {
-                  LOG.error(StringUtils.stringifyException(ex));
-                  break;
-                }
-                className = "org.apache.hadoop.hbase.rest.transform." + className;
-                retry = true;
-              }
-            }
-          }
-        }
-      }
-    } catch (TableNotFoundException e) {
-      // ignore
-    }
-  }
-
-  /**
    * Constructor
    * @param table
    * @throws IOException
@@ -199,22 +41,6 @@ public class TableResource extends Resou
   public TableResource(String table) throws IOException {
     super();
     this.table = table;
-    // Scanning the table schema is too expensive to do for every operation.
-    // Do it once per minute by default.
-    // Setting hbase.rest.transform.check.interval to <= 0 disables rescanning.
-    long now = System.currentTimeMillis();
-    Long lastChecked = lastCheckedMap.get(table);
-    if (lastChecked != null) {
-      long interval = servlet.getConfiguration()
-        .getLong("hbase.rest.transform.check.interval", 60000);
-      if (interval > 0 && (now - lastChecked.longValue()) > interval) {
-        scanTransformAttrs();
-        lastCheckedMap.put(table, now);
-      }
-    } else {
-      scanTransformAttrs();
-      lastCheckedMap.put(table, now);
-    }
   }
 
   /** @return the table name */
@@ -228,25 +54,11 @@ public class TableResource extends Resou
    */
   boolean exists() throws IOException {
     HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-    return admin.tableExists(table);
-  }
-
-  /**
-   * Apply any configured transformations to the value
-   * @param family
-   * @param qualifier
-   * @param value
-   * @param direction
-   * @return
-   * @throws IOException
-   */
-  byte[] transform(byte[] family, byte[] qualifier, byte[] value,
-      Transform.Direction direction) throws IOException {
-    Transform t = getTransform(table, family, qualifier);
-    if (t != null) {
-      return t.transform(value, direction);
+    try {
+      return admin.tableExists(table);
+    } finally {
+      admin.close();
     }
-    return value;
   }
 
   @Path("exists")