You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 19:43:53 UTC

svn commit: r1181951 [1/3] - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/thrift/ main/java/org/apache/ha...

Author: nspiegelberg
Date: Tue Oct 11 17:43:52 2011
New Revision: 1181951

URL: http://svn.apache.org/viewvc?rev=1181951&view=rev
Log:
Backport thrift multiget/checkmutate to 89

Summary: added multiget/checkmutate functions to thrift and also added multiget
support to regionserver and client connection handling backend. It uses a
Template, which minimizes code changes but is pretty hacky. Suggestions welcome!
Now with regenerated 0.20 code from a thrift compiler that actually worked!

Test Plan: Made tiny unit test addition for now; needs very extensive testing
when used! However, effects on existing, non-multiget applications are
negligible.

Reviewers: kranganathan, mbautin

Reviewed By: kranganathan

CC: hisham, dhruba, jgray, dgoode, kranganathan

Differential Revision: 296169

Task ID: 622045

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/Get.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
    hbase/branches/0.89/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/Get.java Tue Oct 11 17:43:52 2011
@@ -66,7 +66,7 @@ import java.util.TreeSet;
  * <p>
  * To add a filter, execute {@link #setFilter(Filter) setFilter}.
  */
-public class Get extends Operation implements Writable {
+public class Get extends Operation implements Writable, Row, Comparable<Row> {
   private static final byte GET_VERSION = (byte)2;
 
   private byte [] row = null;
@@ -369,6 +369,10 @@ public class Get extends Operation imple
     return map;
   }
 
+  public int compareTo(Row p) {
+    return Bytes.compareTo(this.getRow(), p.getRow());
+  }
+
   //Writable
   public void readFields(final DataInput in)
   throws IOException {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Tue Oct 11 17:43:52 2011
@@ -197,6 +197,20 @@ public interface HConnection {
   public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable) 
   throws IOException, RuntimeException;
 
+  /**
+   * Process a batch of Gets. Does the retries.
+   *
+   * @param list
+   *          A batch of Gets to process.
+   * @param tableName
+   *          The name of the table
+   * @return Count of committed Puts. On fault, < list.size().
+   * @throws IOException
+   *           if a remote or network exception occurs
+   */
+  public Result[] processBatchOfGets(List<Get> actions,
+      final byte[] tableName)
+ throws IOException;
 
   /**
    * Process a batch of Puts. Does the retries.

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Tue Oct 11 17:43:52 2011
@@ -1289,7 +1289,7 @@ public class HConnectionManager {
      * Helper class for batch updates.
      * Holds code shared doing batch puts and batch deletes.
      */
-    private abstract class Batch {
+    private abstract class Batch<T> {
       final HConnection c;
 
       private Batch(final HConnection c) {
@@ -1306,7 +1306,7 @@ public class HConnectionManager {
        * @throws RuntimeException other undefined exception
        */
       abstract int doCall(final List<? extends Row> currentList,
-        final byte [] row, final byte [] tableName)
+          final byte[] row, final byte[] tableName, T ret)
       throws IOException, RuntimeException;
 
       /**
@@ -1316,7 +1316,7 @@ public class HConnectionManager {
        * @return Count of how many added or -1 if all added.
        * @throws IOException if a remote or network exception occurs
        */
-      int process(final List<? extends Row> list, final byte[] tableName)
+      int process(final List<? extends Row> list, final byte[] tableName, T ret)
       throws IOException {
         byte [] region = getRegionName(tableName, list.get(0).getRow(), false);
         byte [] currentRegion = region;
@@ -1324,6 +1324,9 @@ public class HConnectionManager {
         boolean retryOnlyOne = false;
         List<Row> currentList = new ArrayList<Row>();
         int i, tries;
+        if (list.size() > 1) {
+          Collections.sort(list);
+        }
         for (i = 0, tries = 0; i < list.size() && tries < numRetries; i++) {
           Row row = list.get(i);
           currentList.add(row);
@@ -1334,7 +1337,7 @@ public class HConnectionManager {
             region = getRegionName(tableName, list.get(i + 1).getRow(), false);
           }
           if (!Bytes.equals(currentRegion, region) || isLastRow || retryOnlyOne) {
-            int index = doCall(currentList, row.getRow(), tableName);
+            int index = doCall(currentList, row.getRow(), tableName, ret);
             // index is == -1 if all processed successfully, else its index
             // of last record successfully processed.
             if (index != -1) {
@@ -1370,7 +1373,7 @@ public class HConnectionManager {
        * @return Region name that holds passed row <code>r</code>
        * @throws IOException
        */
-      private byte [] getRegionName(final byte [] t, final byte [] r,
+      private byte[] getRegionName(final byte[] t, final byte[] r,
         final boolean re)
       throws IOException {
         HRegionLocation location = getRegionLocationForRowWithRetries(t, r, re);
@@ -1383,7 +1386,7 @@ public class HConnectionManager {
        * @param tries
        * @return New value for tries.
        */
-      private int doBatchPause(final byte [] currentRegion, final int tries) {
+      private int doBatchPause(final byte[] currentRegion, final int tries) {
         int localTries = tries;
         long sleepTime = getPauseTime(tries);
         if (LOG.isDebugEnabled()) {
@@ -1405,12 +1408,11 @@ public class HConnectionManager {
       final byte[] tableName)
     throws IOException {
       if (list.isEmpty()) return 0;
-      if (list.size() > 1) Collections.sort(list);
-      Batch b = new Batch(this) {
+      Batch<Object> b = new Batch<Object>(this) {
         @SuppressWarnings("unchecked")
         @Override
         int doCall(final List<? extends Row> currentList, final byte [] row,
-          final byte [] tableName)
+            final byte[] tableName, Object whatevs)
         throws IOException, RuntimeException {
           final List<Put> puts = (List<Put>)currentList;
           return getRegionServerWithRetries(new ServerCallable<Integer>(this.c,
@@ -1421,19 +1423,51 @@ public class HConnectionManager {
           });
         }
       };
-      return b.process(list, tableName);
+      return b.process(list, tableName, new Object());
+    }
+
+    public Result[] processBatchOfGets(final List<Get> list,
+        final byte[] tableName)
+        throws IOException {
+      if (list.isEmpty()) {
+        return null;
+      }
+
+      final List<Get> origList = new ArrayList<Get>(list);
+      Batch<Result[]> b = new Batch<Result[]>(this) {
+        @SuppressWarnings("unchecked")
+        @Override
+        int doCall(final List<? extends Row> currentList,
+            final byte[] row,
+            final byte[] tableName, Result[] res) throws IOException,
+            RuntimeException {
+          final List<Get> gets = (List<Get>) currentList;
+          Result[] tmp = getRegionServerWithRetries(new ServerCallable<Result[]>(
+              this.c, tableName, row) {
+            public Result[] call() throws IOException {
+              return server.get(location.getRegionInfo().getRegionName(), gets);
+            }
+          });
+          for (int i = 0; i < tmp.length; i++) {
+            res[origList.indexOf(gets.get(i))] = tmp[i];
+          }
+          return tmp.length == currentList.size() ? -1 : tmp.length;
+        }
+      };
+      Result[] results = new Result[list.size()];
+      b.process(list, tableName, results);
+      return results;
     }
 
     public int processBatchOfDeletes(final List<Delete> list,
       final byte[] tableName)
     throws IOException {
       if (list.isEmpty()) return 0;
-      if (list.size() > 1) Collections.sort(list);
-      Batch b = new Batch(this) {
+      Batch<Object> b = new Batch<Object>(this) {
         @SuppressWarnings("unchecked")
         @Override
         int doCall(final List<? extends Row> currentList, final byte [] row,
-          final byte [] tableName)
+            final byte[] tableName, Object whatevs)
         throws IOException, RuntimeException {
           final List<Delete> deletes = (List<Delete>)currentList;
           return getRegionServerWithRetries(new ServerCallable<Integer>(this.c,
@@ -1445,7 +1479,7 @@ public class HConnectionManager {
             });
           }
         };
-        return b.process(list, tableName);
+      return b.process(list, tableName, new Object());
       }
 
     void close(boolean stopProxy) {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java Tue Oct 11 17:43:52 2011
@@ -519,6 +519,10 @@ public class HTable implements HTableInt
     );
   }
 
+  public Result[] get(List<Get> gets) throws IOException {
+    return connection.processBatchOfGets(gets, tableName);
+  }
+
   public void delete(final Delete delete)
   throws IOException {
     connection.getRegionServerWithRetries(

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java Tue Oct 11 17:43:52 2011
@@ -111,6 +111,9 @@ public interface HRegionInterface extend
    */
   public Result get(byte [] regionName, Get get) throws IOException;
 
+  public Result[] get(byte[] regionName, List<Get> gets)
+      throws IOException;
+
   /**
    * Perform exists operation.
    * @param regionName name of region to get from

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Oct 11 17:43:52 2011
@@ -1928,7 +1928,8 @@ public class HRegion implements HeapSize
         result = get(get);
 
         boolean matches = false;
-        if (result.size() == 0 && expectedValue.length == 0) {
+        if (result.size() == 0
+            && (expectedValue == null || expectedValue.length == 0)) {
           matches = true;
         } else if (result.size() == 1) {
           //Compare the expected value with the actual value

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Oct 11 17:43:52 2011
@@ -1868,6 +1868,25 @@ public class HRegionServer implements HR
   }
 
   @Override
+  public Result[] get(byte[] regionName, List<Get> gets)
+      throws IOException {
+    checkOpen();
+    requestCount.addAndGet(gets.size());
+    Result[] rets = new Result[gets.size()];
+    try {
+      HRegion region = getRegion(regionName);
+      int i = 0;
+      for (Get get : gets) {
+        rets[i] = region.get(get, getLockFromId(get.getLockId()));
+        i++;
+      }
+      return rets;
+    } catch(Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
+    }
+  }
+
+  @Override
   public boolean exists(byte [] regionName, Get get) throws IOException {
     checkOpen();
     requestCount.incrementAndGet();

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Tue Oct 11 17:43:52 2011
@@ -398,6 +398,56 @@ public class ThriftServer {
       deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP);
     }
 
+    @Override
+    public List<TRowResult> getRows(byte[] tableName, List<byte[]> rows)
+        throws IOError {
+      return getRowsWithColumnsTs(tableName, rows, null,
+          HConstants.LATEST_TIMESTAMP);
+    }
+
+    @Override
+    public List<TRowResult> getRowsWithColumns(byte[] tableName,
+        List<byte[]> rows, List<byte[]> columns) throws IOError {
+      return getRowsWithColumnsTs(tableName, rows, columns,
+          HConstants.LATEST_TIMESTAMP);
+    }
+
+    @Override
+    public List<TRowResult> getRowsTs(byte[] tableName, List<byte[]> rows,
+        long timestamp) throws IOError {
+      return getRowsWithColumnsTs(tableName, rows, null, timestamp);
+    }
+
+    @Override
+    public List<TRowResult> getRowsWithColumnsTs(byte[] tableName,
+        List<byte[]> rows, List<byte[]> columns, long timestamp) throws IOError {
+      try {
+        List<Get> gets = new ArrayList<Get>(rows.size());
+        HTable table = getTable(tableName);
+        // For now, don't support ragged gets, with different columns per row
+        // Probably pretty sensible indefinitely anyways.
+        for (byte[] row : rows) {
+          Get get = new Get(row);
+          if (columns != null) {
+            for (byte[] column : columns) {
+              byte[][] famAndQf = KeyValue.parseColumn(column);
+              if (famAndQf.length == 1) {
+                get.addFamily(famAndQf[0]);
+              } else {
+                get.addColumn(famAndQf[0], famAndQf[1]);
+              }
+            }
+            get.setTimeRange(Long.MIN_VALUE, timestamp);
+          }
+          gets.add(get);
+        }
+        Result[] result = table.get(gets);
+        return ThriftUtilities.rowResultFromHBase(result);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+
     public void deleteAllTs(byte[] tableName, byte[] row, byte[] column,
         long timestamp) throws IOError {
       try {
@@ -560,6 +610,76 @@ public class ThriftServer {
       }
     }
 
+    /**
+     * Warning; the puts and deletes are NOT atomic together and so a lot of
+     * weird things can happen if you expect that to be the case!!
+     *
+     * A valueCheck of null means that the row can't exist before being put.
+     * This is kind of a stupid thing to enforce when deleting, for obvious
+     * reasons.
+     */
+    @Override
+    public boolean checkAndMutateRow(byte[] tableName, byte[] row,
+        byte[] columnCheck, byte[] valueCheck, List<Mutation> mutations)
+        throws IOError, IllegalArgument {
+      return checkAndMutateRowTs(tableName, row, columnCheck, valueCheck,
+          mutations, HConstants.LATEST_TIMESTAMP);
+    }
+
+    @Override
+    public boolean checkAndMutateRowTs(byte[] tableName, byte[] row,
+        byte[] columnCheck, byte[] valueCheck,
+        List<Mutation> mutations,
+        long timestamp) throws IOError, IllegalArgument {
+      HTable table;
+      try {
+        table = getTable(tableName);
+        Put put = new Put(row, timestamp, null);
+
+        Delete delete = new Delete(row);
+
+        for (Mutation m : mutations) {
+          byte[][] famAndQf = KeyValue.parseColumn(m.column);
+          if (m.isDelete) {
+            if (famAndQf.length == 1) {
+              delete.deleteFamily(famAndQf[0], timestamp);
+            } else {
+              delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
+            }
+          } else {
+            if (famAndQf.length == 1) {
+              put.add(famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, m.value);
+            } else {
+              put.add(famAndQf[0], famAndQf[1], m.value);
+            }
+          }
+        }
+        byte[][] famAndQfCheck = KeyValue.parseColumn(columnCheck);
+
+        if (!delete.isEmpty() && !put.isEmpty()) {
+          // can't do both, not atomic, not good idea!
+          throw new IllegalArgumentException(
+              "Single Thrift CheckAndMutate call cannot do both puts and deletes.");
+        }
+        if (!delete.isEmpty()) {
+          return table.checkAndDelete(row, famAndQfCheck[0],
+                  famAndQfCheck.length != 1 ? famAndQfCheck[1]
+                      : HConstants.EMPTY_BYTE_ARRAY, valueCheck, delete);
+        }
+        if (!put.isEmpty()) {
+          return table.checkAndPut(row, famAndQfCheck[0],
+                  famAndQfCheck.length != 1 ? famAndQfCheck[1]
+                      : HConstants.EMPTY_BYTE_ARRAY, valueCheck, put);
+        }
+        throw new IllegalArgumentException(
+            "Thrift CheckAndMutate call must do either put or delete.");
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgument(e.getMessage());
+      }
+    }
+
     @Deprecated
     public long atomicIncrement(byte[] tableName, byte[] row, byte[] column,
         long amount) throws IOError, IllegalArgument, TException {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java Tue Oct 11 17:43:52 2011
@@ -1,30 +1,29 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.thrift.*;
-import org.apache.thrift.meta_data.FieldMetaData;
-import org.apache.thrift.meta_data.FieldValueMetaData;
+import org.apache.thrift.meta_data.*;
 import org.apache.thrift.protocol.*;
 
-import java.util.*;
-
 /**
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java Tue Oct 11 17:43:52 2011
@@ -1,19 +1,7 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java?rev=1181951&r1=1181950&r2=1181951&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java Tue Oct 11 17:43:52 2011
@@ -1,19 +1,7 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;