You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/04/12 19:04:12 UTC

svn commit: r1467356 [1/3] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ hbase...

Author: larsh
Date: Fri Apr 12 17:04:10 2013
New Revision: 1467356

URL: http://svn.apache.org/r1467356
Log:
HBASE-7801 Allow a deferred sync option per Mutation.

Added:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
Modified:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
    hbase/branches/0.95/hbase-protocol/src/main/protobuf/Client.proto
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapper.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java Fri Apr 12 17:04:10 2013
@@ -121,7 +121,7 @@ public class Delete extends Mutation imp
     this.row = d.getRow();
     this.ts = d.getTimeStamp();
     this.familyMap.putAll(d.getFamilyMap());
-    this.writeToWAL = d.writeToWAL;
+    this.durability = d.durability;
   }
 
   /**

Added: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java?rev=1467356&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java (added)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java Fri Apr 12 17:04:10 2013
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Enum describing the durability guarantees for {@link Mutation}
+ * Note that the items must be sorted in order of increasing durability
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum Durability {
+  /**
+   * Use the column family's default setting to determine durability.
+   * This must remain the first option.
+   */
+  USE_DEFAULT,
+  /**
+   * Do not write the Mutation to the WAL
+   */
+  SKIP_WAL,
+  /**
+   * Write the Mutation to the WAL asynchronously
+   */
+  ASYNC_WAL,
+  /**
+   * Write the Mutation to the WAL synchronously.
+   * The data is flushed to the filesystem implementation, but not necessarily to disk.
+   * For HDFS this will flush the data to the designated number of DataNodes.
+   * See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313<a/>
+   */
+  SYNC_WAL,
+  /**
+   * Write the Mutation to the WAL synchronously and force the entries to disk.
+   * (Note: this is currently not supported and will behave identical to {@link #SYNC_WAL})
+   * See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313<a/>
+   */
+  FSYNC_WAL
+}

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Fri Apr 12 17:04:10 2013
@@ -772,7 +772,7 @@ public class HTable implements HTableInt
   public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount)
   throws IOException {
-    return incrementColumnValue(row, family, qualifier, amount, true);
+    return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
   }
 
   /**
@@ -780,7 +780,7 @@ public class HTable implements HTableInt
    */
   @Override
   public long incrementColumnValue(final byte [] row, final byte [] family,
-      final byte [] qualifier, final long amount, final boolean writeToWAL)
+      final byte [] qualifier, final long amount, final Durability durability)
   throws IOException {
     NullPointerException npe = null;
     if (row == null) {
@@ -799,7 +799,7 @@ public class HTable implements HTableInt
             try {
               MutateRequest request = RequestConverter.buildMutateRequest(
                 location.getRegionInfo().getRegionName(), row, family,
-                qualifier, amount, writeToWAL);
+                qualifier, amount, durability);
               PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
               MutateResponse response = server.mutate(rpcController, request);
               Result result =

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Fri Apr 12 17:04:10 2013
@@ -342,11 +342,9 @@ public interface HTableInterface extends
   public Result increment(final Increment increment) throws IOException;
 
   /**
-   * Atomically increments a column value.
+   * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
    * <p>
-   * Equivalent to {@link #incrementColumnValue(byte[], byte[], byte[],
-   * long, boolean) incrementColumnValue}(row, family, qualifier, amount,
-   * <b>true</b>)}
+   * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
    * @param row The row that contains the cell to increment.
    * @param family The column family of the cell to increment.
    * @param qualifier The column qualifier of the cell to increment.
@@ -364,25 +362,19 @@ public interface HTableInterface extends
    * value does not yet exist it is initialized to <code>amount</code> and
    * written to the specified column.
    *
-   * <p>Setting writeToWAL to false means that in a fail scenario, you will lose
-   * any increments that have not been flushed.
+   * <p>Setting durability to {@link Durability#SKIP_WAL} means that in a fail
+   * scenario you will lose any increments that have not been flushed.
    * @param row The row that contains the cell to increment.
    * @param family The column family of the cell to increment.
    * @param qualifier The column qualifier of the cell to increment.
    * @param amount The amount to increment the cell with (or decrement, if the
    * amount is negative).
-   * @param writeToWAL if {@code true}, the operation will be applied to the
-   * Write Ahead Log (WAL).  This makes the operation slower but safer, as if
-   * the call returns successfully, it is guaranteed that the increment will
-   * be safely persisted.  When set to {@code false}, the call may return
-   * successfully before the increment is safely persisted, so it's possible
-   * that the increment be lost in the event of a failure happening before the
-   * operation gets persisted.
+   * @param durability The persistence guarantee for this increment.
    * @return The new value, post increment.
    * @throws IOException if a remote or network exception occurs.
    */
   long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount, boolean writeToWAL) throws IOException;
+      long amount, Durability durability) throws IOException;
 
   /**
    * Tells whether or not 'auto-flush' is turned on.

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Fri Apr 12 17:04:10 2013
@@ -444,9 +444,9 @@ public class HTablePool implements Close
 
     @Override
     public long incrementColumnValue(byte[] row, byte[] family,
-        byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
+        byte[] qualifier, long amount, Durability durability) throws IOException {
       return table.incrementColumnValue(row, family, qualifier, amount,
-          writeToWAL);
+          durability);
     }
 
     @Override

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Fri Apr 12 17:04:10 2013
@@ -1,5 +1,4 @@
 /*
- *
  * 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
@@ -64,7 +63,7 @@ public abstract class Mutation extends O
 
   protected byte [] row = null;
   protected long ts = HConstants.LATEST_TIMESTAMP;
-  protected boolean writeToWAL = true;
+  protected Durability durability = Durability.USE_DEFAULT;
   // A Map sorted by column family.
   protected NavigableMap<byte [], List<? extends Cell>> familyMap =
     new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
@@ -167,19 +166,16 @@ public abstract class Mutation extends O
   }
 
   /**
-   * @return true if edits should be applied to WAL, false if not
+   * Set the durability for this mutation
+   * @param d
    */
-  public boolean getWriteToWAL() {
-    return this.writeToWAL;
+  public void setDurability(Durability d) {
+    this.durability = d;
   }
 
-  /**
-   * Set whether this Delete should be written to the WAL or not.
-   * Not writing the WAL means you may lose edits on server crash.
-   * @param write true if edits should be written to WAL, false if not
-   */
-  public void setWriteToWAL(boolean write) {
-    this.writeToWAL = write;
+  /** Get the current durability */
+  public Durability getDurability() {
+    return this.durability;
   }
 
   /**

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java Fri Apr 12 17:04:10 2013
@@ -86,7 +86,7 @@ public class Put extends Mutation implem
     for(Map.Entry<byte [], List<? extends Cell>> entry: putToCopy.getFamilyMap().entrySet()) {
       this.familyMap.put(entry.getKey(), entry.getValue());
     }
-    this.writeToWAL = putToCopy.writeToWAL;
+    this.durability = putToCopy.durability;
   }
 
   /**

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Fri Apr 12 17:04:10 2013
@@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
@@ -296,6 +297,48 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Convert a protobuf Durability into a client Durability
+   */
+  public static Durability toDurability(
+      final ClientProtos.MutationProto.Durability proto) {
+    switch(proto) {
+    case USE_DEFAULT:
+      return Durability.USE_DEFAULT;
+    case SKIP_WAL:
+      return Durability.SKIP_WAL;
+    case ASYNC_WAL:
+      return Durability.ASYNC_WAL;
+    case SYNC_WAL:
+      return Durability.SYNC_WAL;
+    case FSYNC_WAL:
+      return Durability.FSYNC_WAL;
+    default:
+      return Durability.USE_DEFAULT;
+    }
+  }
+
+  /**
+   * Convert a client Durability into a protbuf Durability
+   */
+  public static ClientProtos.MutationProto.Durability toDurability(
+      final Durability d) {
+    switch(d) {
+    case USE_DEFAULT:
+      return ClientProtos.MutationProto.Durability.USE_DEFAULT;
+    case SKIP_WAL:
+      return ClientProtos.MutationProto.Durability.SKIP_WAL;
+    case ASYNC_WAL:
+      return ClientProtos.MutationProto.Durability.ASYNC_WAL;
+    case SYNC_WAL:
+      return ClientProtos.MutationProto.Durability.SYNC_WAL;
+    case FSYNC_WAL:
+      return ClientProtos.MutationProto.Durability.FSYNC_WAL;
+    default:
+      return ClientProtos.MutationProto.Durability.USE_DEFAULT;
+    }
+  }
+
+  /**
    * Convert a protocol buffer Get to a client Get
    *
    * @param proto the protocol buffer Get to convert
@@ -419,7 +462,7 @@ public final class ProtobufUtil {
         }
       }
     }
-    put.setWriteToWAL(proto.getWriteToWAL());
+    put.setDurability(toDurability(proto.getDurability()));
     for (NameBytesPair attribute: proto.getAttributeList()) {
       put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
     }
@@ -499,7 +542,7 @@ public final class ProtobufUtil {
         }
       }
     }
-    delete.setWriteToWAL(proto.getWriteToWAL());
+    delete.setDurability(toDurability(proto.getDurability()));
     for (NameBytesPair attribute: proto.getAttributeList()) {
       delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
     }
@@ -552,7 +595,7 @@ public final class ProtobufUtil {
         }
       }
     }
-    append.setWriteToWAL(proto.getWriteToWAL());
+    append.setDurability(toDurability(proto.getDurability()));
     for (NameBytesPair attribute: proto.getAttributeList()) {
       append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
     }
@@ -637,7 +680,7 @@ public final class ProtobufUtil {
       }
       increment.setTimeRange(minStamp, maxStamp);
     }
-    increment.setWriteToWAL(proto.getWriteToWAL());
+    increment.setDurability(toDurability(proto.getDurability()));
     return increment;
   }
 
@@ -857,7 +900,7 @@ public final class ProtobufUtil {
     MutationProto.Builder builder = MutationProto.newBuilder();
     builder.setRow(ByteString.copyFrom(increment.getRow()));
     builder.setMutateType(MutationType.INCREMENT);
-    builder.setWriteToWAL(increment.getWriteToWAL());
+    builder.setDurability(toDurability(increment.getDurability()));
     TimeRange timeRange = increment.getTimeRange();
     if (!timeRange.isAllTime()) {
       HBaseProtos.TimeRange.Builder timeRangeBuilder =
@@ -944,7 +987,7 @@ public final class ProtobufUtil {
     MutationProto.Builder builder = MutationProto.newBuilder();
     builder.setRow(ByteString.copyFrom(mutation.getRow()));
     builder.setMutateType(type);
-    builder.setWriteToWAL(mutation.getWriteToWAL());
+    builder.setDurability(toDurability(mutation.getDurability()));
     builder.setTimestamp(mutation.getTimeStamp());
     Map<String, byte[]> attributes = mutation.getAttributesMap();
     if (!attributes.isEmpty()) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Fri Apr 12 17:04:10 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -199,12 +200,12 @@ public final class RequestConverter {
    * @param family
    * @param qualifier
    * @param amount
-   * @param writeToWAL
+   * @param durability
    * @return a mutate request
    */
   public static MutateRequest buildMutateRequest(
       final byte[] regionName, final byte[] row, final byte[] family,
-      final byte [] qualifier, final long amount, final boolean writeToWAL) {
+      final byte [] qualifier, final long amount, final Durability durability) {
     MutateRequest.Builder builder = MutateRequest.newBuilder();
     RegionSpecifier region = buildRegionSpecifier(
       RegionSpecifierType.REGION_NAME, regionName);
@@ -213,7 +214,7 @@ public final class RequestConverter {
     MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
     mutateBuilder.setRow(ByteString.copyFrom(row));
     mutateBuilder.setMutateType(MutationType.INCREMENT);
-    mutateBuilder.setWriteToWAL(writeToWAL);
+    mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
     columnBuilder.setFamily(ByteString.copyFrom(family));
     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();

Modified: hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java (original)
+++ hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java Fri Apr 12 17:04:10 2013
@@ -6401,9 +6401,9 @@ public final class ClientProtos {
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getAttributeOrBuilder(
         int index);
     
-    // optional bool writeToWAL = 6 [default = true];
-    boolean hasWriteToWAL();
-    boolean getWriteToWAL();
+    // optional .MutationProto.Durability durability = 6 [default = USE_DEFAULT];
+    boolean hasDurability();
+    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability getDurability();
     
     // optional .TimeRange timeRange = 7;
     boolean hasTimeRange();
@@ -6442,6 +6442,84 @@ public final class ClientProtos {
       return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MutationProto_fieldAccessorTable;
     }
     
+    public enum Durability
+        implements com.google.protobuf.ProtocolMessageEnum {
+      USE_DEFAULT(0, 0),
+      SKIP_WAL(1, 1),
+      ASYNC_WAL(2, 2),
+      SYNC_WAL(3, 3),
+      FSYNC_WAL(4, 4),
+      ;
+      
+      public static final int USE_DEFAULT_VALUE = 0;
+      public static final int SKIP_WAL_VALUE = 1;
+      public static final int ASYNC_WAL_VALUE = 2;
+      public static final int SYNC_WAL_VALUE = 3;
+      public static final int FSYNC_WAL_VALUE = 4;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static Durability valueOf(int value) {
+        switch (value) {
+          case 0: return USE_DEFAULT;
+          case 1: return SKIP_WAL;
+          case 2: return ASYNC_WAL;
+          case 3: return SYNC_WAL;
+          case 4: return FSYNC_WAL;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<Durability>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<Durability>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<Durability>() {
+              public Durability findValueByNumber(int number) {
+                return Durability.valueOf(number);
+              }
+            };
+      
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final Durability[] VALUES = {
+        USE_DEFAULT, SKIP_WAL, ASYNC_WAL, SYNC_WAL, FSYNC_WAL, 
+      };
+      
+      public static Durability valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+      
+      private final int index;
+      private final int value;
+      
+      private Durability(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:MutationProto.Durability)
+    }
+    
     public enum MutationType
         implements com.google.protobuf.ProtocolMessageEnum {
       APPEND(0, 0),
@@ -6490,7 +6568,7 @@ public final class ClientProtos {
       }
       public static final com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(0);
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(1);
       }
       
       private static final MutationType[] VALUES = {
@@ -6562,7 +6640,7 @@ public final class ClientProtos {
       }
       public static final com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(1);
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(2);
       }
       
       private static final DeleteType[] VALUES = {
@@ -7907,14 +7985,14 @@ public final class ClientProtos {
       return attribute_.get(index);
     }
     
-    // optional bool writeToWAL = 6 [default = true];
-    public static final int WRITETOWAL_FIELD_NUMBER = 6;
-    private boolean writeToWAL_;
-    public boolean hasWriteToWAL() {
+    // optional .MutationProto.Durability durability = 6 [default = USE_DEFAULT];
+    public static final int DURABILITY_FIELD_NUMBER = 6;
+    private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability durability_;
+    public boolean hasDurability() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
-    public boolean getWriteToWAL() {
-      return writeToWAL_;
+    public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability getDurability() {
+      return durability_;
     }
     
     // optional .TimeRange timeRange = 7;
@@ -7946,7 +8024,7 @@ public final class ClientProtos {
       columnValue_ = java.util.Collections.emptyList();
       timestamp_ = 0L;
       attribute_ = java.util.Collections.emptyList();
-      writeToWAL_ = true;
+      durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
       timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
       associatedCellCount_ = 0;
     }
@@ -7990,7 +8068,7 @@ public final class ClientProtos {
         output.writeMessage(5, attribute_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBool(6, writeToWAL_);
+        output.writeEnum(6, durability_.getNumber());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeMessage(7, timeRange_);
@@ -8029,7 +8107,7 @@ public final class ClientProtos {
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(6, writeToWAL_);
+          .computeEnumSize(6, durability_.getNumber());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += com.google.protobuf.CodedOutputStream
@@ -8081,10 +8159,10 @@ public final class ClientProtos {
       }
       result = result && getAttributeList()
           .equals(other.getAttributeList());
-      result = result && (hasWriteToWAL() == other.hasWriteToWAL());
-      if (hasWriteToWAL()) {
-        result = result && (getWriteToWAL()
-            == other.getWriteToWAL());
+      result = result && (hasDurability() == other.hasDurability());
+      if (hasDurability()) {
+        result = result &&
+            (getDurability() == other.getDurability());
       }
       result = result && (hasTimeRange() == other.hasTimeRange());
       if (hasTimeRange()) {
@@ -8125,9 +8203,9 @@ public final class ClientProtos {
         hash = (37 * hash) + ATTRIBUTE_FIELD_NUMBER;
         hash = (53 * hash) + getAttributeList().hashCode();
       }
-      if (hasWriteToWAL()) {
-        hash = (37 * hash) + WRITETOWAL_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getWriteToWAL());
+      if (hasDurability()) {
+        hash = (37 * hash) + DURABILITY_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getDurability());
       }
       if (hasTimeRange()) {
         hash = (37 * hash) + TIMERANGE_FIELD_NUMBER;
@@ -8274,7 +8352,7 @@ public final class ClientProtos {
         } else {
           attributeBuilder_.clear();
         }
-        writeToWAL_ = true;
+        durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
         bitField0_ = (bitField0_ & ~0x00000020);
         if (timeRangeBuilder_ == null) {
           timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
@@ -8355,7 +8433,7 @@ public final class ClientProtos {
         if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.writeToWAL_ = writeToWAL_;
+        result.durability_ = durability_;
         if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000010;
         }
@@ -8445,8 +8523,8 @@ public final class ClientProtos {
             }
           }
         }
-        if (other.hasWriteToWAL()) {
-          setWriteToWAL(other.getWriteToWAL());
+        if (other.hasDurability()) {
+          setDurability(other.getDurability());
         }
         if (other.hasTimeRange()) {
           mergeTimeRange(other.getTimeRange());
@@ -8531,8 +8609,14 @@ public final class ClientProtos {
               break;
             }
             case 48: {
-              bitField0_ |= 0x00000020;
-              writeToWAL_ = input.readBool();
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability value = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(6, rawValue);
+              } else {
+                bitField0_ |= 0x00000020;
+                durability_ = value;
+              }
               break;
             }
             case 58: {
@@ -8996,23 +9080,26 @@ public final class ClientProtos {
         return attributeBuilder_;
       }
       
-      // optional bool writeToWAL = 6 [default = true];
-      private boolean writeToWAL_ = true;
-      public boolean hasWriteToWAL() {
+      // optional .MutationProto.Durability durability = 6 [default = USE_DEFAULT];
+      private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
+      public boolean hasDurability() {
         return ((bitField0_ & 0x00000020) == 0x00000020);
       }
-      public boolean getWriteToWAL() {
-        return writeToWAL_;
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability getDurability() {
+        return durability_;
       }
-      public Builder setWriteToWAL(boolean value) {
+      public Builder setDurability(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
         bitField0_ |= 0x00000020;
-        writeToWAL_ = value;
+        durability_ = value;
         onChanged();
         return this;
       }
-      public Builder clearWriteToWAL() {
+      public Builder clearDurability() {
         bitField0_ = (bitField0_ & ~0x00000020);
-        writeToWAL_ = true;
+        durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
         onChanged();
         return this;
       }
@@ -21128,73 +21215,76 @@ public final class ClientProtos {
       "\006exists\030\002 \003(\010\"\177\n\tCondition\022\013\n\003row\030\001 \002(\014\022" +
       "\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022!\n\013co",
       "mpareType\030\004 \002(\0162\014.CompareType\022\037\n\ncompara" +
-      "tor\030\005 \002(\0132\013.Comparator\"\372\004\n\rMutationProto" +
+      "tor\030\005 \002(\0132\013.Comparator\"\365\005\n\rMutationProto" +
       "\022\013\n\003row\030\001 \001(\014\022/\n\nmutateType\030\002 \001(\0162\033.Muta" +
       "tionProto.MutationType\022/\n\013columnValue\030\003 " +
       "\003(\0132\032.MutationProto.ColumnValue\022\021\n\ttimes" +
       "tamp\030\004 \001(\004\022!\n\tattribute\030\005 \003(\0132\016.NameByte" +
-      "sPair\022\030\n\nwriteToWAL\030\006 \001(\010:\004true\022\035\n\ttimeR" +
-      "ange\030\007 \001(\0132\n.TimeRange\022\033\n\023associatedCell" +
-      "Count\030\010 \001(\005\032\326\001\n\013ColumnValue\022\016\n\006family\030\001 " +
-      "\002(\014\022A\n\016qualifierValue\030\002 \003(\0132).MutationPr",
-      "oto.ColumnValue.QualifierValue\032t\n\016Qualif" +
-      "ierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001" +
-      "(\014\022\021\n\ttimestamp\030\003 \001(\004\022-\n\ndeleteType\030\004 \001(" +
-      "\0162\031.MutationProto.DeleteType\">\n\014Mutation" +
-      "Type\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002" +
-      "\022\n\n\006DELETE\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE" +
-      "_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001" +
-      "\022\021\n\rDELETE_FAMILY\020\002\"r\n\rMutateRequest\022 \n\006" +
-      "region\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mutati" +
-      "on\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition\030\003 ",
-      "\001(\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006res" +
-      "ult\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\307\002" +
-      "\n\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattri" +
-      "bute\030\002 \003(\0132\016.NameBytesPair\022\020\n\010startRow\030\003" +
-      " \001(\014\022\017\n\007stopRow\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007." +
-      "Filter\022\035\n\ttimeRange\030\006 \001(\0132\n.TimeRange\022\026\n" +
-      "\013maxVersions\030\007 \001(\r:\0011\022\031\n\013cacheBlocks\030\010 \001" +
-      "(\010:\004true\022\021\n\tbatchSize\030\t \001(\r\022\025\n\rmaxResult" +
-      "Size\030\n \001(\004\022\022\n\nstoreLimit\030\013 \001(\r\022\023\n\013storeO" +
-      "ffset\030\014 \001(\r\022\"\n\032loadColumnFamiliesOnDeman",
-      "d\030\r \001(\010\"\230\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132" +
-      "\020.RegionSpecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\021" +
-      "\n\tscannerId\030\003 \001(\004\022\024\n\014numberOfRows\030\004 \001(\r\022" +
-      "\024\n\014closeScanner\030\005 \001(\010\022\023\n\013nextCallSeq\030\006 \001" +
-      "(\004\"u\n\014ScanResponse\022\027\n\006result\030\001 \003(\0132\007.Res" +
-      "ult\022\021\n\tscannerId\030\002 \001(\004\022\023\n\013moreResults\030\003 " +
-      "\001(\010\022\013\n\003ttl\030\004 \001(\r\022\027\n\017resultSizeBytes\030\005 \001(" +
-      "\004\"\260\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 \002" +
-      "(\0132\020.RegionSpecifier\0224\n\nfamilyPath\030\002 \003(\013" +
-      "2 .BulkLoadHFileRequest.FamilyPath\022\024\n\014as",
-      "signSeqNum\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family" +
-      "\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRes" +
-      "ponse\022\016\n\006loaded\030\001 \002(\010\"_\n\026CoprocessorServ" +
-      "iceCall\022\013\n\003row\030\001 \002(\014\022\023\n\013serviceName\030\002 \002(" +
-      "\t\022\022\n\nmethodName\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"d" +
-      "\n\031CoprocessorServiceRequest\022 \n\006region\030\001 " +
-      "\002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027.Co" +
-      "processorServiceCall\"]\n\032CoprocessorServi" +
-      "ceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSpeci" +
-      "fier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"B\n\013M",
-      "ultiAction\022 \n\010mutation\030\001 \001(\0132\016.MutationP" +
-      "roto\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014ActionResult\022" +
-      "\026\n\005value\030\001 \001(\0132\007.Result\022!\n\texception\030\002 \001" +
-      "(\0132\016.NameBytesPair\"^\n\014MultiRequest\022 \n\006re" +
-      "gion\030\001 \002(\0132\020.RegionSpecifier\022\034\n\006action\030\002" +
-      " \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 \001(\010\".\n\rMu" +
-      "ltiResponse\022\035\n\006result\030\001 \003(\0132\r.ActionResu" +
-      "lt2\342\002\n\rClientService\022 \n\003get\022\013.GetRequest" +
-      "\032\014.GetResponse\022/\n\010multiGet\022\020.MultiGetReq" +
-      "uest\032\021.MultiGetResponse\022)\n\006mutate\022\016.Muta",
-      "teRequest\032\017.MutateResponse\022#\n\004scan\022\014.Sca" +
-      "nRequest\032\r.ScanResponse\022>\n\rbulkLoadHFile" +
-      "\022\025.BulkLoadHFileRequest\032\026.BulkLoadHFileR" +
-      "esponse\022F\n\013execService\022\032.CoprocessorServ" +
-      "iceRequest\032\033.CoprocessorServiceResponse\022" +
-      "&\n\005multi\022\r.MultiRequest\032\016.MultiResponseB" +
-      "B\n*org.apache.hadoop.hbase.protobuf.gene" +
-      "ratedB\014ClientProtosH\001\210\001\001\240\001\001"
+      "sPair\022:\n\ndurability\030\006 \001(\0162\031.MutationProt" +
+      "o.Durability:\013USE_DEFAULT\022\035\n\ttimeRange\030\007" +
+      " \001(\0132\n.TimeRange\022\033\n\023associatedCellCount\030" +
+      "\010 \001(\005\032\326\001\n\013ColumnValue\022\016\n\006family\030\001 \002(\014\022A\n",
+      "\016qualifierValue\030\002 \003(\0132).MutationProto.Co" +
+      "lumnValue.QualifierValue\032t\n\016QualifierVal" +
+      "ue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\t" +
+      "timestamp\030\003 \001(\004\022-\n\ndeleteType\030\004 \001(\0162\031.Mu" +
+      "tationProto.DeleteType\"W\n\nDurability\022\017\n\013" +
+      "USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL" +
+      "\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014Mutat" +
+      "ionType\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PU" +
+      "T\020\002\022\n\n\006DELETE\020\003\"U\n\nDeleteType\022\026\n\022DELETE_" +
+      "ONE_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSION",
+      "S\020\001\022\021\n\rDELETE_FAMILY\020\002\"r\n\rMutateRequest\022" +
+      " \n\006region\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mut" +
+      "ation\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition" +
+      "\030\003 \001(\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006" +
+      "result\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010" +
+      "\"\307\002\n\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tat" +
+      "tribute\030\002 \003(\0132\016.NameBytesPair\022\020\n\010startRo" +
+      "w\030\003 \001(\014\022\017\n\007stopRow\030\004 \001(\014\022\027\n\006filter\030\005 \001(\013" +
+      "2\007.Filter\022\035\n\ttimeRange\030\006 \001(\0132\n.TimeRange" +
+      "\022\026\n\013maxVersions\030\007 \001(\r:\0011\022\031\n\013cacheBlocks\030",
+      "\010 \001(\010:\004true\022\021\n\tbatchSize\030\t \001(\r\022\025\n\rmaxRes" +
+      "ultSize\030\n \001(\004\022\022\n\nstoreLimit\030\013 \001(\r\022\023\n\013sto" +
+      "reOffset\030\014 \001(\r\022\"\n\032loadColumnFamiliesOnDe" +
+      "mand\030\r \001(\010\"\230\001\n\013ScanRequest\022 \n\006region\030\001 \001" +
+      "(\0132\020.RegionSpecifier\022\023\n\004scan\030\002 \001(\0132\005.Sca" +
+      "n\022\021\n\tscannerId\030\003 \001(\004\022\024\n\014numberOfRows\030\004 \001" +
+      "(\r\022\024\n\014closeScanner\030\005 \001(\010\022\023\n\013nextCallSeq\030" +
+      "\006 \001(\004\"u\n\014ScanResponse\022\027\n\006result\030\001 \003(\0132\007." +
+      "Result\022\021\n\tscannerId\030\002 \001(\004\022\023\n\013moreResults" +
+      "\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022\027\n\017resultSizeBytes\030\005",
+      " \001(\004\"\260\001\n\024BulkLoadHFileRequest\022 \n\006region\030" +
+      "\001 \002(\0132\020.RegionSpecifier\0224\n\nfamilyPath\030\002 " +
+      "\003(\0132 .BulkLoadHFileRequest.FamilyPath\022\024\n" +
+      "\014assignSeqNum\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006fam" +
+      "ily\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFile" +
+      "Response\022\016\n\006loaded\030\001 \002(\010\"_\n\026CoprocessorS" +
+      "erviceCall\022\013\n\003row\030\001 \002(\014\022\023\n\013serviceName\030\002" +
+      " \002(\t\022\022\n\nmethodName\030\003 \002(\t\022\017\n\007request\030\004 \002(" +
+      "\014\"d\n\031CoprocessorServiceRequest\022 \n\006region" +
+      "\030\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027",
+      ".CoprocessorServiceCall\"]\n\032CoprocessorSe" +
+      "rviceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSp" +
+      "ecifier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"B" +
+      "\n\013MultiAction\022 \n\010mutation\030\001 \001(\0132\016.Mutati" +
+      "onProto\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014ActionResu" +
+      "lt\022\026\n\005value\030\001 \001(\0132\007.Result\022!\n\texception\030" +
+      "\002 \001(\0132\016.NameBytesPair\"^\n\014MultiRequest\022 \n" +
+      "\006region\030\001 \002(\0132\020.RegionSpecifier\022\034\n\006actio" +
+      "n\030\002 \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 \001(\010\".\n" +
+      "\rMultiResponse\022\035\n\006result\030\001 \003(\0132\r.ActionR",
+      "esult2\342\002\n\rClientService\022 \n\003get\022\013.GetRequ" +
+      "est\032\014.GetResponse\022/\n\010multiGet\022\020.MultiGet" +
+      "Request\032\021.MultiGetResponse\022)\n\006mutate\022\016.M" +
+      "utateRequest\032\017.MutateResponse\022#\n\004scan\022\014." +
+      "ScanRequest\032\r.ScanResponse\022>\n\rbulkLoadHF" +
+      "ile\022\025.BulkLoadHFileRequest\032\026.BulkLoadHFi" +
+      "leResponse\022F\n\013execService\022\032.CoprocessorS" +
+      "erviceRequest\032\033.CoprocessorServiceRespon" +
+      "se\022&\n\005multi\022\r.MultiRequest\032\016.MultiRespon" +
+      "seBB\n*org.apache.hadoop.hbase.protobuf.g",
+      "eneratedB\014ClientProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -21270,7 +21360,7 @@ public final class ClientProtos {
           internal_static_MutationProto_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_MutationProto_descriptor,
-              new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "WriteToWAL", "TimeRange", "AssociatedCellCount", },
+              new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "Durability", "TimeRange", "AssociatedCellCount", },
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.class,
               org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder.class);
           internal_static_MutationProto_ColumnValue_descriptor =

Modified: hbase/branches/0.95/hbase-protocol/src/main/protobuf/Client.proto
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-protocol/src/main/protobuf/Client.proto?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-protocol/src/main/protobuf/Client.proto (original)
+++ hbase/branches/0.95/hbase-protocol/src/main/protobuf/Client.proto Fri Apr 12 17:04:10 2013
@@ -141,7 +141,7 @@ message MutationProto {
   repeated ColumnValue columnValue = 3;
   optional uint64 timestamp = 4;
   repeated NameBytesPair attribute = 5;
-  optional bool writeToWAL = 6 [default = true];
+  optional Durability durability = 6 [default = USE_DEFAULT];
 
   // For some mutations, a result may be returned, in which case,
   // time range can be specified for potential performance gain
@@ -154,6 +154,14 @@ message MutationProto {
   // 'cell' field above which is non-null when the cells are pb'd.
   optional int32 associatedCellCount = 8;
 
+  enum Durability {
+    USE_DEFAULT  = 0;
+    SKIP_WAL     = 1;
+    ASYNC_WAL    = 2;
+    SYNC_WAL     = 3;
+    FSYNC_WAL    = 4;
+  }
+
   enum MutationType {
     APPEND = 0;
     INCREMENT = 1;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java Fri Apr 12 17:04:10 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -81,7 +82,7 @@ public class ConstraintProcessor extends
 
   @Override
   public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put,
-      WALEdit edit, boolean writeToWAL) throws IOException {
+      WALEdit edit, Durability durability) throws IOException {
     // check the put against the stored constraints
     for (Constraint c : constraints) {
       c.check(put);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java Fri Apr 12 17:04:10 2013
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -231,22 +232,23 @@ public abstract class BaseRegionObserver
 
   @Override
   public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, 
-      final Put put, final WALEdit edit, final boolean writeToWAL) throws IOException {
+      final Put put, final WALEdit edit, final Durability durability) throws IOException {
   }
 
   @Override
   public void postPut(final ObserverContext<RegionCoprocessorEnvironment> e, 
-      final Put put, final WALEdit edit, final boolean writeToWAL) throws IOException {
+      final Put put, final WALEdit edit, final Durability durability) throws IOException {
   }
 
   @Override
-  public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, 
-      final Delete delete, final WALEdit edit, final boolean writeToWAL) throws IOException {
+  public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, final Delete delete,
+      final WALEdit edit, final Durability durability) throws IOException {
   }
 
   @Override
   public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
-      final Delete delete, final WALEdit edit, final boolean writeToWAL) throws IOException {
+      final Delete delete, final WALEdit edit, final Durability durability)
+      throws IOException {
   }
   
   @Override

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Fri Apr 12 17:04:10 2013
@@ -515,10 +515,10 @@ public abstract class CoprocessorHost<E 
       }
 
       public long incrementColumnValue(byte[] row, byte[] family,
-          byte[] qualifier, long amount, boolean writeToWAL)
+          byte[] qualifier, long amount, Durability durability)
           throws IOException {
         return table.incrementColumnValue(row, family, qualifier, amount,
-            writeToWAL);
+            durability);
       }
 
       @Override

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java Fri Apr 12 17:04:10 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -488,11 +489,11 @@ public interface RegionObserver extends 
    * @param c the environment provided by the region server
    * @param put The Put object
    * @param edit The WALEdit object that will be written to the wal
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability Persistence guarantee for this Put
    * @throws IOException if an error occurred on the coprocessor
    */
   void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, 
-      final Put put, final WALEdit edit, final boolean writeToWAL)
+      final Put put, final WALEdit edit, final Durability durability)
     throws IOException;
 
   /**
@@ -503,11 +504,11 @@ public interface RegionObserver extends 
    * @param c the environment provided by the region server
    * @param put The Put object
    * @param edit The WALEdit object for the wal
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability Persistence guarantee for this Put
    * @throws IOException if an error occurred on the coprocessor
    */
   void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, 
-      final Put put, final WALEdit edit, final boolean writeToWAL)
+      final Put put, final WALEdit edit, final Durability durability)
     throws IOException;
 
   /**
@@ -520,11 +521,11 @@ public interface RegionObserver extends 
    * @param c the environment provided by the region server
    * @param delete The Delete object
    * @param edit The WALEdit object for the wal
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability Persistence guarantee for this Delete
    * @throws IOException if an error occurred on the coprocessor
    */
   void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c, 
-      final Delete delete, final WALEdit edit, final boolean writeToWAL)
+      final Delete delete, final WALEdit edit, final Durability durability)
     throws IOException;
 
   /**
@@ -535,11 +536,11 @@ public interface RegionObserver extends 
    * @param c the environment provided by the region server
    * @param delete The Delete object
    * @param edit The WALEdit object for the wal
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability Persistence guarantee for this Delete
    * @throws IOException if an error occurred on the coprocessor
    */
   void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Delete delete, final WALEdit edit, final boolean writeToWAL)
+      final Delete delete, final WALEdit edit, final Durability durability)
     throws IOException;
   
   /**

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java Fri Apr 12 17:04:10 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.De
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -131,7 +132,8 @@ public class MultiTableOutputFormat exte
       // The actions are not immutable, so we defensively copy them
       if (action instanceof Put) {
         Put put = new Put((Put) action);
-        put.setWriteToWAL(useWriteAheadLogging);
+        put.setDurability(useWriteAheadLogging ? Durability.SYNC_WAL
+            : Durability.SKIP_WAL);
         table.put(put);
       } else if (action instanceof Delete) {
         Delete delete = new Delete((Delete) action);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java Fri Apr 12 17:04:10 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.UUID;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 
 import com.google.protobuf.Message;
@@ -48,4 +49,9 @@ implements RowProcessor<S,T> {
   public String getName() {
     return this.getClass().getSimpleName().toLowerCase();
   }
+
+  @Override
+  public Durability useDurability() {
+    return Durability.USE_DEFAULT;
+  }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Apr 12 17:04:10 2013
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.DroppedSnapshotException;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
@@ -184,7 +185,6 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public class HRegion implements HeapSize { // , Writable{
   public static final Log LOG = LogFactory.getLog(HRegion.class);
-  private static final String MERGEDIR = ".merges";
 
   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
       "hbase.hregion.scan.loadColumnFamiliesOnDemand";
@@ -1713,17 +1713,16 @@ public class HRegion implements HeapSize
   //////////////////////////////////////////////////////////////////////////////
   /**
    * @param delete delete object
-   * @param writeToWAL append to the write ahead lock or not
    * @throws IOException read exceptions
    */
-  public void delete(Delete delete, boolean writeToWAL)
+  public void delete(Delete delete)
   throws IOException {
     checkReadOnly();
     checkResources();
     startRegionOperation();
     this.writeRequestsCount.increment();
     try {
-      byte [] row = delete.getRow();
+      delete.getRow();
       // All edits for the given row (across all column families) must happen atomically.
       doBatchMutate(delete, null);
     } finally {
@@ -1738,15 +1737,16 @@ public class HRegion implements HeapSize
   /**
    * This is used only by unit tests. Not required to be a public API.
    * @param familyMap map of family to edits for the given family.
-   * @param writeToWAL
+   * @param clusterId
+   * @param durability
    * @throws IOException
    */
   void delete(NavigableMap<byte[], List<? extends Cell>> familyMap, UUID clusterId,
-      boolean writeToWAL) throws IOException {
+      Durability durability) throws IOException {
     Delete delete = new Delete(FOR_UNIT_TESTS_ONLY);
     delete.setFamilyMap(familyMap);
     delete.setClusterId(clusterId);
-    delete.setWriteToWAL(writeToWAL);
+    delete.setDurability(durability);
     doBatchMutate(delete, null);
   }
 
@@ -1809,16 +1809,7 @@ public class HRegion implements HeapSize
    * @param put
    * @throws IOException
    */
-  public void put(Put put) throws IOException {
-    this.put(put, put.getWriteToWAL());
-  }
-
-  /**
-   * @param put
-   * @param writeToWAL
-   * @throws IOException
-   */
-  public void put(Put put, boolean writeToWAL)
+  public void put(Put put)
   throws IOException {
     checkReadOnly();
 
@@ -1830,13 +1821,6 @@ public class HRegion implements HeapSize
     startRegionOperation();
     this.writeRequestsCount.increment();
     try {
-      // We obtain a per-row lock, so other clients will block while one client
-      // performs an update. The read lock is released by the client calling
-      // #commit or #abort or if the HRegionServer lease on the lock expires.
-      // See HRegionServer#RegionListener for how the expire on HRegionServer
-      // invokes a HRegion#abort.
-      byte [] row = put.getRow();
-
       // All edits for the given row (across all column families) must happen atomically.
       doBatchMutate(put, null);
     } finally {
@@ -1931,13 +1915,13 @@ public class HRegion implements HeapSize
         Pair<Mutation, Integer> nextPair = batchOp.operations[i];
         Mutation m = nextPair.getFirst();
         if (m instanceof Put) {
-          if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
+          if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
             // pre hook says skip this Put
             // mark as success and skip in doMiniBatchMutation
             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
           }
         } else if (m instanceof Delete) {
-          if (coprocessorHost.preDelete((Delete) m, walEdit, m.getWriteToWAL())) {
+          if (coprocessorHost.preDelete((Delete) m, walEdit, m.getDurability())) {
             // pre hook says skip this Delete
             // mark as success and skip in doMiniBatchMutation
             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
@@ -2139,6 +2123,7 @@ public class HRegion implements HeapSize
       // ------------------------------------
       // STEP 4. Build WAL edit
       // ----------------------------------
+      Durability durability = Durability.USE_DEFAULT;
       for (int i = firstIndex; i < lastIndexExclusive; i++) {
         // Skip puts that were determined to be invalid during preprocessing
         if (batchOp.retCodeDetails[i].getOperationStatusCode()
@@ -2148,12 +2133,17 @@ public class HRegion implements HeapSize
         batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
 
         Mutation m = batchOp.operations[i].getFirst();
-        if (!m.getWriteToWAL()) {
+        Durability tmpDur = m.getDurability(); 
+        if (tmpDur.ordinal() > durability.ordinal()) {
+          durability = tmpDur;
+        }
+        if (tmpDur == Durability.SKIP_WAL) {
           if (m instanceof Put) {
             recordPutWithoutWal(m.getFamilyMap());
           }
           continue;
         }
+
         // Add WAL edits by CP
         WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
         if (fromCP != null) {
@@ -2162,7 +2152,6 @@ public class HRegion implements HeapSize
           }
         }
         addFamilyMapToWALEdit(familyMaps[i], walEdit);
-
       }
 
       // -------------------------
@@ -2189,7 +2178,7 @@ public class HRegion implements HeapSize
       // STEP 7. Sync wal.
       // -------------------------
       if (walEdit.size() > 0) {
-        syncOrDefer(txid);
+        syncOrDefer(txid, durability);
       }
       walSyncSuccessful = true;
       // calling the post CP hook for batch mutation
@@ -2221,9 +2210,9 @@ public class HRegion implements HeapSize
           }
           Mutation m = batchOp.operations[i].getFirst();
           if (m instanceof Put) {
-            coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
+            coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
           } else {
-            coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
+            coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability());
           }
         }
       }
@@ -2558,7 +2547,6 @@ public class HRegion implements HeapSize
     Put p = new Put(row);
     p.setFamilyMap(familyMap);
     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
-    p.setWriteToWAL(true);
     doBatchMutate(p, null);
   }
 
@@ -4534,7 +4522,7 @@ public class HRegion implements HeapSize
           }
           // 10. Sync edit log
           if (txid != 0) {
-            syncOrDefer(txid);
+            syncOrDefer(txid, processor.useDurability());
           }
           walSyncSuccessful = true;
         }
@@ -4634,15 +4622,15 @@ public class HRegion implements HeapSize
    * Perform one or more append operations on a row.
    *
    * @param append
-   * @param writeToWAL
    * @return new keyvalues after increment
    * @throws IOException
    */
-  public Result append(Append append, boolean writeToWAL)
+  public Result append(Append append)
       throws IOException {
     byte[] row = append.getRow();
     checkRow(row, "append");
     boolean flush = false;
+    boolean writeToWAL = append.getDurability() != Durability.SKIP_WAL;
     WALEdit walEdits = null;
     List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
@@ -4775,7 +4763,8 @@ public class HRegion implements HeapSize
         releaseRowLock(lid);
       }
       if (writeToWAL) {
-        syncOrDefer(txid); // sync the transaction log outside the rowlock
+        // sync the transaction log outside the rowlock
+        syncOrDefer(txid, append.getDurability());
       }
     } finally {
       if (w != null) {
@@ -4800,16 +4789,16 @@ public class HRegion implements HeapSize
   /**
    * Perform one or more increment operations on a row.
    * @param increment
-   * @param writeToWAL
    * @return new keyvalues after increment
    * @throws IOException
    */
-  public Result increment(Increment increment, boolean writeToWAL)
+  public Result increment(Increment increment)
   throws IOException {
     byte [] row = increment.getRow();
     checkRow(row, "increment");
     TimeRange tr = increment.getTimeRange();
     boolean flush = false;
+    boolean writeToWAL = increment.getDurability() != Durability.SKIP_WAL;
     WALEdit walEdits = null;
     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.size());
     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
@@ -4918,7 +4907,8 @@ public class HRegion implements HeapSize
         releaseRowLock(lid);
       }
       if (writeToWAL) {
-        syncOrDefer(txid); // sync the transaction log outside the rowlock
+        // sync the transaction log outside the rowlock
+        syncOrDefer(txid, increment.getDurability());
       }
     } finally {
       if (w != null) {
@@ -5340,9 +5330,32 @@ public class HRegion implements HeapSize
    * @param txid should sync up to which transaction
    * @throws IOException If anything goes wrong with DFS
    */
-  private void syncOrDefer(long txid) throws IOException {
-    if (this.getRegionInfo().isMetaRegion() || !isDeferredLogSyncEnabled()) {
+  private void syncOrDefer(long txid, Durability durability) throws IOException {
+    if (this.getRegionInfo().isMetaRegion()) {
       this.log.sync(txid);
+    } else {
+      switch(durability) {
+      case USE_DEFAULT:
+        // do what CF defaults to
+        if (!isDeferredLogSyncEnabled()) {
+          this.log.sync(txid);
+        }
+        break;
+      case SKIP_WAL:
+        // nothing do to
+        break;
+      case ASYNC_WAL:
+        // defer the sync, unless we globally can't
+        if (this.deferredLogSyncDisabled) {
+          this.log.sync(txid);
+        }
+        break;
+      case SYNC_WAL:
+      case FSYNC_WAL:
+        // sync the WAL edit (SYNC and FSYNC treated the same for now)
+        this.log.sync(txid);
+        break;
+      }
     }
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Apr 12 17:04:10 2013
@@ -2761,7 +2761,7 @@ public class HRegionServer implements Cl
             processed = result;
           }
         } else {
-          region.delete(delete, delete.getWriteToWAL());
+          region.delete(delete);
           processed = Boolean.TRUE;
         }
         break;
@@ -3706,7 +3706,7 @@ public class HRegionServer implements Cl
       r = region.getCoprocessorHost().preAppend(append);
     }
     if (r == null) {
-      r = region.append(append, append.getWriteToWAL());
+      r = region.append(append);
       if (region.getCoprocessorHost() != null) {
         region.getCoprocessorHost().postAppend(append, r);
       }
@@ -3733,7 +3733,7 @@ public class HRegionServer implements Cl
       r = region.getCoprocessorHost().preIncrement(increment);
     }
     if (r == null) {
-      r = region.increment(increment, increment.getWriteToWAL());
+      r = region.increment(increment);
       if (region.getCoprocessorHost() != null) {
         r = region.getCoprocessorHost().postIncrement(increment, r);
       }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java Fri Apr 12 17:04:10 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValueU
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProcessorProtos.MultiRowMutationProcessorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProcessorProtos.MultiRowMutationProcessorResponse;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -86,7 +87,7 @@ MultiRowMutationProcessorResponse> {
             + m.getClass().getName());
       }
       for (List<? extends Cell> cells: m.getFamilyMap().values()) {
-        boolean writeToWAL = m.getWriteToWAL();
+        boolean writeToWAL = m.getDurability() != Durability.SKIP_WAL;
         for (Cell cell : cells) {
           KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
           mutationKvs.add(kv);
@@ -104,14 +105,14 @@ MultiRowMutationProcessorResponse> {
     if (coprocessorHost != null) {
       for (Mutation m : mutations) {
         if (m instanceof Put) {
-          if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
+          if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
             // by pass everything
             return;
           }
         } else if (m instanceof Delete) {
           Delete d = (Delete) m;
           region.prepareDelete(d);
-          if (coprocessorHost.preDelete(d, walEdit, d.getWriteToWAL())) {
+          if (coprocessorHost.preDelete(d, walEdit, d.getDurability())) {
             // by pass everything
             return;
           }
@@ -126,9 +127,9 @@ MultiRowMutationProcessorResponse> {
     if (coprocessorHost != null) {
       for (Mutation m : mutations) {
         if (m instanceof Put) {
-          coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
+          coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
         } else if (m instanceof Delete) {
-          coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
+          coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability());
         }
       }
     }
@@ -143,4 +144,16 @@ MultiRowMutationProcessorResponse> {
   public void initialize(MultiRowMutationProcessorRequest msg) {
     //nothing
   }
+
+  @Override
+  public Durability useDurability() {
+    // return true when at least one mutation requested a WAL flush (default)
+    Durability durability = Durability.USE_DEFAULT;
+    for (Mutation m : mutations) {
+      if (m.getDurability().ordinal() > durability.ordinal()) {
+        durability = m.getDurability();
+      }
+    }
+    return durability;
+  }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java Fri Apr 12 17:04:10 2013
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -882,19 +883,19 @@ public class RegionCoprocessorHost
   /**
    * @param put The Put object
    * @param edit The WALEdit object.
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability The durability used
    * @return true if default processing should be bypassed
    * @exception IOException Exception
    */
   public boolean prePut(Put put, WALEdit edit,
-      final boolean writeToWAL) throws IOException {
+      final Durability durability) throws IOException {
     boolean bypass = false;
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
         ctx = ObserverContext.createAndPrepare(env, ctx);
         try {
-          ((RegionObserver)env.getInstance()).prePut(ctx, put, edit, writeToWAL);
+          ((RegionObserver)env.getInstance()).prePut(ctx, put, edit, durability);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         }
@@ -910,17 +911,17 @@ public class RegionCoprocessorHost
   /**
    * @param put The Put object
    * @param edit The WALEdit object.
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability The durability used
    * @exception IOException Exception
    */
   public void postPut(Put put, WALEdit edit,
-      final boolean writeToWAL) throws IOException {
+      final Durability durability) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
         ctx = ObserverContext.createAndPrepare(env, ctx);
         try {
-          ((RegionObserver)env.getInstance()).postPut(ctx, put, edit, writeToWAL);
+          ((RegionObserver)env.getInstance()).postPut(ctx, put, edit, durability);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         }
@@ -934,19 +935,19 @@ public class RegionCoprocessorHost
   /**
    * @param delete The Delete object
    * @param edit The WALEdit object.
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability The durability used
    * @return true if default processing should be bypassed
    * @exception IOException Exception
    */
   public boolean preDelete(Delete delete, WALEdit edit,
-      final boolean writeToWAL) throws IOException {
+      final Durability durability) throws IOException {
     boolean bypass = false;
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
         ctx = ObserverContext.createAndPrepare(env, ctx);
         try {
-          ((RegionObserver)env.getInstance()).preDelete(ctx, delete, edit, writeToWAL);
+          ((RegionObserver)env.getInstance()).preDelete(ctx, delete, edit, durability);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         }
@@ -962,17 +963,17 @@ public class RegionCoprocessorHost
   /**
    * @param delete The Delete object
    * @param edit The WALEdit object.
-   * @param writeToWAL true if the change should be written to the WAL
+   * @param durability The durability used
    * @exception IOException Exception
    */
   public void postDelete(Delete delete, WALEdit edit,
-      final boolean writeToWAL) throws IOException {
+      final Durability durability) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
         ctx = ObserverContext.createAndPrepare(env, ctx);
         try {
-          ((RegionObserver)env.getInstance()).postDelete(ctx, delete, edit, writeToWAL);
+          ((RegionObserver)env.getInstance()).postDelete(ctx, delete, edit, durability);
         } catch (Throwable e) {
           handleCoprocessorThrowable(env, e);
         }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java Fri Apr 12 17:04:10 2013
@@ -25,6 +25,7 @@ import java.util.UUID;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 
 import com.google.protobuf.ByteString;
@@ -133,4 +134,9 @@ public interface RowProcessor<S extends 
    * @throws IOException
    */
   void initialize(S msg) throws IOException;
+
+  /**
+   * @return The {@link Durability} to use
+   */
+  Durability useDurability();
 }
\ No newline at end of file

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1467356&r1=1467355&r2=1467356&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Fri Apr 12 17:04:10 2013
@@ -706,7 +706,7 @@ public class RemoteHTable implements HTa
   }
 
   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount, boolean writeToWAL) throws IOException {
+      long amount, Durability durability) throws IOException {
     throw new IOException("incrementColumnValue not supported");
   }