Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (working copy)
@@ -121,7 +121,7 @@
this.row = d.getRow();
this.ts = d.getTimeStamp();
this.familyMap.putAll(d.getFamilyMap());
- this.writeToWAL = d.writeToWAL;
+ this.durability = d.durability;
}
/**
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java (revision 0)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java (working copy)
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * Enum describing the write guarantees for {@link Mutation}
+ */
+public enum Durability {
+ /**
+ * Use the column family's default setting to determine durability
+ */
+ 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.
+ */
+ SYNC_WAL,
+ /**
+ * Write the Mutation to the WAL synchronously and force the entries to disk.
+ * (currently that same as {@link #SYNC_WAL})
+ */
+ FSYNC_WAL
+}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (working copy)
@@ -772,7 +772,7 @@
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 @@
*/
@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 @@
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 =
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (working copy)
@@ -342,11 +342,9 @@
public Result increment(final Increment increment) throws IOException;
/**
- * Atomically increments a column value.
+ * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
*
- * Equivalent to {@link #incrementColumnValue(byte[], byte[], byte[],
- * long, boolean) incrementColumnValue}(row, family, qualifier, amount,
- * true)}
+ * 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 @@
* value does not yet exist it is initialized to amount and
* written to the specified column.
*
- *
Setting writeToWAL to false means that in a fail scenario, you will lose
- * any increments that have not been flushed.
+ *
Setting writeGuarantee 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 writeGuarantee 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 writeGuarantee) throws IOException;
/**
* Tells whether or not 'auto-flush' is turned on.
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (working copy)
@@ -444,9 +444,9 @@
@Override
public long incrementColumnValue(byte[] row, byte[] family,
- byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
+ byte[] qualifier, long amount, Durability writeGuarantee) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount,
- writeToWAL);
+ writeGuarantee);
}
@Override
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (working copy)
@@ -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 @@
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> familyMap =
new TreeMap>(Bytes.BYTES_COMPARATOR);
@@ -167,19 +166,16 @@
}
/**
- * @return true if edits should be applied to WAL, false if not
+ * Set the write guarantee for this mutation
+ * @param wg
*/
- 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 write guarantee */
+ public Durability getDurability() {
+ return this.durability;
}
/**
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (working copy)
@@ -86,7 +86,7 @@
for(Map.Entry> entry: putToCopy.getFamilyMap().entrySet()) {
this.familyMap.put(entry.getKey(), entry.getValue());
}
- this.writeToWAL = putToCopy.writeToWAL;
+ this.durability = putToCopy.durability;
}
/**
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (working copy)
@@ -67,6 +67,7 @@
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;
@@ -284,6 +285,48 @@
}
/**
+ * 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
@@ -407,7 +450,7 @@
}
}
}
- put.setWriteToWAL(proto.getWriteToWAL());
+ put.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute: proto.getAttributeList()) {
put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
@@ -487,7 +530,7 @@
}
}
}
- delete.setWriteToWAL(proto.getWriteToWAL());
+ delete.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute: proto.getAttributeList()) {
delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
@@ -540,7 +583,7 @@
}
}
}
- append.setWriteToWAL(proto.getWriteToWAL());
+ append.setDurability(toDurability(proto.getDurability()));
for (NameBytesPair attribute: proto.getAttributeList()) {
append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
@@ -625,7 +668,7 @@
}
increment.setTimeRange(minStamp, maxStamp);
}
- increment.setWriteToWAL(proto.getWriteToWAL());
+ increment.setDurability(toDurability(proto.getDurability()));
return increment;
}
@@ -845,7 +888,7 @@
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 =
@@ -932,7 +975,7 @@
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 attributes = mutation.getAttributesMap();
if (!attributes.isEmpty()) {
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision 1465321)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (working copy)
@@ -36,6 +36,7 @@
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 @@
* @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 @@
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();
Index: hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
===================================================================
--- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java (revision 1465321)
+++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java (working copy)
@@ -6401,9 +6401,9 @@
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 @@
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
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ 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 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 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 @@
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 @@
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 @@
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 @@
}
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 @@
}
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 @@
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 @@
} 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 @@
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 @@
}
}
}
- if (other.hasWriteToWAL()) {
- setWriteToWAL(other.getWriteToWAL());
+ if (other.hasDurability()) {
+ setDurability(other.getDurability());
}
if (other.hasTimeRange()) {
mergeTimeRange(other.getTimeRange());
@@ -8531,8 +8609,14 @@
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 @@
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 @@
"\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 @@
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 =
Index: hbase-protocol/src/main/protobuf/Client.proto
===================================================================
--- hbase-protocol/src/main/protobuf/Client.proto (revision 1465321)
+++ hbase-protocol/src/main/protobuf/Client.proto (working copy)
@@ -141,7 +141,7 @@
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 @@
// '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;
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/ConstraintProcessor.java (working copy)
@@ -27,6 +27,7 @@
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 @@
@Override
public void prePut(ObserverContext e, Put put,
- WALEdit edit, boolean writeToWAL) throws IOException {
+ WALEdit edit, Durability writeGuarantee) throws IOException {
// check the put against the stored constraints
for (Constraint c : constraints) {
c.check(put);
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (working copy)
@@ -33,6 +33,7 @@
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 @@
@Override
public void prePut(final ObserverContext e,
- final Put put, final WALEdit edit, final boolean writeToWAL) throws IOException {
+ final Put put, final WALEdit edit, final Durability writeGuarantee) throws IOException {
}
@Override
public void postPut(final ObserverContext e,
- final Put put, final WALEdit edit, final boolean writeToWAL) throws IOException {
+ final Put put, final WALEdit edit, final Durability writeGuarantee) throws IOException {
}
@Override
- public void preDelete(final ObserverContext e,
- final Delete delete, final WALEdit edit, final boolean writeToWAL) throws IOException {
+ public void preDelete(final ObserverContext e, final Delete delete,
+ final WALEdit edit, final Durability writeGuarantee) throws IOException {
}
@Override
public void postDelete(final ObserverContext e,
- final Delete delete, final WALEdit edit, final boolean writeToWAL) throws IOException {
+ final Delete delete, final WALEdit edit, final Durability writeGuarantee)
+ throws IOException {
}
@Override
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (working copy)
@@ -515,10 +515,10 @@
}
public long incrementColumnValue(byte[] row, byte[] family,
- byte[] qualifier, long amount, boolean writeToWAL)
+ byte[] qualifier, long amount, Durability writeGuarantee)
throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount,
- writeToWAL);
+ writeGuarantee);
}
@Override
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (working copy)
@@ -32,6 +32,7 @@
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 @@
* @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 writeGuarantee Persistence guarantee for this Put
* @throws IOException if an error occurred on the coprocessor
*/
void prePut(final ObserverContext c,
- final Put put, final WALEdit edit, final boolean writeToWAL)
+ final Put put, final WALEdit edit, final Durability writeGuarantee)
throws IOException;
/**
@@ -503,11 +504,11 @@
* @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 writeGuarantee Persistence guarantee for this Put
* @throws IOException if an error occurred on the coprocessor
*/
void postPut(final ObserverContext c,
- final Put put, final WALEdit edit, final boolean writeToWAL)
+ final Put put, final WALEdit edit, final Durability writeGuarantee)
throws IOException;
/**
@@ -520,11 +521,11 @@
* @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 writeGuarantee Persistence guarantee for this Delete
* @throws IOException if an error occurred on the coprocessor
*/
void preDelete(final ObserverContext c,
- final Delete delete, final WALEdit edit, final boolean writeToWAL)
+ final Delete delete, final WALEdit edit, final Durability writeGuarantee)
throws IOException;
/**
@@ -535,11 +536,11 @@
* @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 writeGuarantee Persistence guarantee for this Delete
* @throws IOException if an error occurred on the coprocessor
*/
void postDelete(final ObserverContext c,
- final Delete delete, final WALEdit edit, final boolean writeToWAL)
+ final Delete delete, final WALEdit edit, final Durability writeGuarantee)
throws IOException;
/**
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java (working copy)
@@ -32,6 +32,7 @@
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 @@
// 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);
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java (working copy)
@@ -21,6 +21,7 @@
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 @@
public String getName() {
return this.getClass().getSimpleName().toLowerCase();
}
+
+ @Override
+ public Durability useWriteGuarantee() {
+ return Durability.SYNC_WAL;
+ }
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy)
@@ -66,7 +66,6 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -78,7 +77,6 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@@ -90,6 +88,7 @@
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;
@@ -185,7 +184,6 @@
@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";
@@ -1401,7 +1399,6 @@
// We also set the memstore size to zero here before we allow updates
// again so its value will represent the size of the updates received
// during the flush
- long sequenceId = -1L;
MultiVersionConsistencyControl.WriteEntry w = null;
// We have to take a write lock during snapshot, or else a write could
@@ -1668,17 +1665,16 @@
//////////////////////////////////////////////////////////////////////////////
/**
* @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 {
@@ -1697,11 +1693,11 @@
* @throws IOException
*/
void delete(NavigableMap> familyMap, UUID clusterId,
- boolean writeToWAL) throws IOException {
+ Durability writeGuarantee) throws IOException {
Delete delete = new Delete(FOR_UNIT_TESTS_ONLY);
delete.setFamilyMap(familyMap);
delete.setClusterId(clusterId);
- delete.setWriteToWAL(writeToWAL);
+ delete.setDurability(writeGuarantee);
doBatchMutate(delete, null);
}
@@ -1764,16 +1760,7 @@
* @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();
@@ -1785,13 +1772,6 @@
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 {
@@ -1886,13 +1866,13 @@
Pair 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;
@@ -2094,6 +2074,7 @@
// ------------------------------------
// STEP 4. Build WAL edit
// ----------------------------------
+ Durability wg = 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()
@@ -2103,12 +2084,16 @@
batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
Mutation m = batchOp.operations[i].getFirst();
- if (!m.getWriteToWAL()) {
+ Durability tmpWg = m.getDurability();
+ if (tmpWg == Durability.SKIP_WAL) {
if (m instanceof Put) {
recordPutWithoutWal(m.getFamilyMap());
}
continue;
+ } else if (tmpWg.ordinal() > wg.ordinal()) {
+ wg = tmpWg;
}
+
// Add WAL edits by CP
WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
if (fromCP != null) {
@@ -2144,7 +2129,7 @@
// STEP 7. Sync wal.
// -------------------------
if (walEdit.size() > 0) {
- syncOrDefer(txid);
+ syncOrDefer(txid, wg);
}
walSyncSuccessful = true;
// calling the post CP hook for batch mutation
@@ -2176,9 +2161,9 @@
}
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());
}
}
}
@@ -2513,7 +2498,6 @@
Put p = new Put(row);
p.setFamilyMap(familyMap);
p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
- p.setWriteToWAL(true);
doBatchMutate(p, null);
}
@@ -4532,7 +4516,7 @@
}
// 10. Sync edit log
if (txid != 0) {
- syncOrDefer(txid);
+ syncOrDefer(txid, processor.useWriteGuarantee());
}
walSyncSuccessful = true;
}
@@ -4632,15 +4616,15 @@
* 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 allKVs = new ArrayList(append.size());
Map> tempMemstore = new HashMap>();
@@ -4773,7 +4757,8 @@
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) {
@@ -4798,16 +4783,16 @@
/**
* 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 allKVs = new ArrayList(increment.size());
Map> tempMemstore = new HashMap>();
@@ -4916,7 +4901,8 @@
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) {
@@ -5338,9 +5324,32 @@
* @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 wg) throws IOException {
+ if (this.getRegionInfo().isMetaRegion()) {
this.log.sync(txid);
+ } else {
+ switch(wg) {
+ 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;
+ }
}
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (working copy)
@@ -2761,7 +2761,7 @@
processed = result;
}
} else {
- region.delete(delete, delete.getWriteToWAL());
+ region.delete(delete);
processed = Boolean.TRUE;
}
break;
@@ -3706,7 +3706,7 @@
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 @@
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);
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java (working copy)
@@ -29,6 +29,7 @@
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 @@
+ 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 @@
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 @@
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 @@
public void initialize(MultiRowMutationProcessorRequest msg) {
//nothing
}
+
+ @Override
+ public Durability useWriteGuarantee() {
+ // return true when at least one mutation requested a WAL flush (default)
+ Durability wg = Durability.USE_DEFAULT;
+ for (Mutation m : mutations) {
+ if (m.getDurability().ordinal() > wg.ordinal()) {
+ wg = m.getDurability();
+ }
+ }
+ return wg;
+ }
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (working copy)
@@ -48,6 +48,7 @@
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 @@
/**
* @param put The Put object
* @param edit The WALEdit object.
- * @param writeToWAL true if the change should be written to the WAL
+ * @param writeGuarantee The WriteGuarantee 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 writeGuarantee) throws IOException {
boolean bypass = false;
ObserverContext 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, writeGuarantee);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
@@ -910,17 +911,17 @@
/**
* @param put The Put object
* @param edit The WALEdit object.
- * @param writeToWAL true if the change should be written to the WAL
+ * @param writeGuarantee The WriteGuarantee used
* @exception IOException Exception
*/
public void postPut(Put put, WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
ObserverContext 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, writeGuarantee);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
@@ -934,19 +935,19 @@
/**
* @param delete The Delete object
* @param edit The WALEdit object.
- * @param writeToWAL true if the change should be written to the WAL
+ * @param writeGuarantee The WriteGuarantee 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 writeGuarantee) throws IOException {
boolean bypass = false;
ObserverContext 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, writeGuarantee);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
@@ -962,17 +963,17 @@
/**
* @param delete The Delete object
* @param edit The WALEdit object.
- * @param writeToWAL true if the change should be written to the WAL
+ * @param writeGuarantee The WriteGuarantee used
* @exception IOException Exception
*/
public void postDelete(Delete delete, WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
ObserverContext 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, writeGuarantee);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java (working copy)
@@ -25,6 +25,7 @@
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 @@
* @throws IOException
*/
void initialize(S msg) throws IOException;
+
+ /**
+ * @return The {@link Durability} to use
+ */
+ Durability useWriteGuarantee();
}
\ No newline at end of file
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (working copy)
@@ -706,7 +706,7 @@
}
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
- long amount, boolean writeToWAL) throws IOException {
+ long amount, Durability writeGuarantee) throws IOException {
throw new IOException("incrementColumnValue not supported");
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (working copy)
@@ -46,6 +46,7 @@
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.*;
import org.apache.hadoop.hbase.exceptions.CoprocessorException;
import org.apache.hadoop.hbase.filter.CompareFilter;
@@ -901,7 +902,7 @@
@Override
public void prePut(final ObserverContext c,
- final Put put, final WALEdit edit, final boolean writeToWAL)
+ final Put put, final WALEdit edit, final Durability writeGuarantee)
throws IOException {
requirePermission("put", Permission.Action.WRITE, c.getEnvironment(),
put.getFamilyMap());
@@ -909,7 +910,7 @@
@Override
public void postPut(final ObserverContext c,
- final Put put, final WALEdit edit, final boolean writeToWAL) {
+ final Put put, final WALEdit edit, final Durability writeGuarantee) {
if (aclRegion) {
updateACL(c.getEnvironment(), put.getFamilyMap());
}
@@ -917,7 +918,7 @@
@Override
public void preDelete(final ObserverContext c,
- final Delete delete, final WALEdit edit, final boolean writeToWAL)
+ final Delete delete, final WALEdit edit, final Durability writeGuarantee)
throws IOException {
requirePermission("delete", Permission.Action.WRITE, c.getEnvironment(),
delete.getFamilyMap());
@@ -925,7 +926,7 @@
@Override
public void postDelete(final ObserverContext c,
- final Delete delete, final WALEdit edit, final boolean writeToWAL)
+ final Delete delete, final WALEdit edit, final Durability writeGuarantee)
throws IOException {
if (aclRegion) {
updateACL(c.getEnvironment(), delete.getFamilyMap());
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (working copy)
@@ -63,6 +63,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
@@ -976,7 +977,8 @@
} else {
delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
}
- delete.setWriteToWAL(m.writeToWAL);
+ delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
+ : Durability.SKIP_WAL);
} else {
if(famAndQf.length == 1) {
put.add(famAndQf[0], HConstants.EMPTY_BYTE_ARRAY,
@@ -987,7 +989,7 @@
m.value != null ? getBytes(m.value)
: HConstants.EMPTY_BYTE_ARRAY);
}
- put.setWriteToWAL(m.writeToWAL);
+ put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
}
}
if (!delete.isEmpty())
@@ -1034,7 +1036,8 @@
} else {
delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
}
- delete.setWriteToWAL(m.writeToWAL);
+ delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
+ : Durability.SKIP_WAL);
} else {
if(famAndQf.length == 1) {
put.add(famAndQf[0], HConstants.EMPTY_BYTE_ARRAY,
@@ -1045,7 +1048,7 @@
m.value != null ? getBytes(m.value)
: HConstants.EMPTY_BYTE_ARRAY);
}
- put.setWriteToWAL(m.writeToWAL);
+ put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
}
}
if (!delete.isEmpty())
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (working copy)
@@ -154,7 +154,7 @@
out = new Put(in.getRow());
}
- out.setWriteToWAL(in.isWriteToWal());
+ out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
for (TColumnValue columnValue : in.getColumnValues()) {
if (columnValue.isSetTimestamp()) {
@@ -228,7 +228,7 @@
out = new Delete(in.getRow());
}
}
- out.setWriteToWAL(in.isWriteToWal());
+ out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
return out;
}
@@ -322,7 +322,7 @@
for (TColumnIncrement column : in.getColumns()) {
out.addColumn(column.getFamily(), column.getQualifier(), column.getAmount());
}
- out.setWriteToWAL(in.isWriteToWal());
+ out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
return out;
}
}
Index: hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
===================================================================
--- hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java (revision 1465321)
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java (working copy)
@@ -224,7 +224,7 @@
Delete delete = new Delete(regioninfo.getRegionName(),
System.currentTimeMillis());
- meta.delete(delete, true);
+ meta.delete(delete);
}
/*
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (working copy)
@@ -445,7 +445,7 @@
public void delete(Delete delete, boolean writeToWAL)
throws IOException {
- this.region.delete(delete, writeToWAL);
+ this.region.delete(delete);
}
public Result get(Get get) throws IOException {
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (working copy)
@@ -65,6 +65,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.TableExistsException;
import org.apache.hadoop.hbase.exceptions.TableNotEnabledException;
@@ -1307,7 +1308,7 @@
k[2] = b3;
Put put = new Put(k);
put.add(f, null, k);
- if (r.getLog() == null) put.setWriteToWAL(false);
+ if (r.getLog() == null) put.setDurability(Durability.SKIP_WAL);
r.put(put);
rowCount++;
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (working copy)
@@ -51,6 +51,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.filter.PageFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.filter.Filter;
@@ -1230,7 +1231,7 @@
Put put = new Put(row);
byte[] value = generateValue(this.rand);
put.add(FAMILY_NAME, QUALIFIER_NAME, value);
- put.setWriteToWAL(writeToWAL);
+ put.setDurability(writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
table.put(put);
}
}
@@ -1361,7 +1362,7 @@
Put put = new Put(format(i));
byte[] value = generateValue(this.rand);
put.add(FAMILY_NAME, QUALIFIER_NAME, value);
- put.setWriteToWAL(writeToWAL);
+ put.setDurability(writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
table.put(put);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java (working copy)
@@ -24,6 +24,7 @@
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.util.Bytes;
/**
@@ -242,7 +243,7 @@
final long ts)
throws IOException {
Put put = new Put(ROW, ts);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILY_NAME, QUALIFIER_NAME, bytes);
loader.put(put);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java (working copy)
@@ -47,6 +47,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.DataOutputBuffer;
@@ -331,7 +332,7 @@
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
@@ -398,7 +399,7 @@
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = MetaEditor.makePutFromRegionInfo(hri);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
meta.put(put);
LOG.info("createMultiRegionsWithPBSerialization: PUT inserted " + hri.toString());
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java (working copy)
@@ -248,7 +248,7 @@
byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
Put put = new Put(key);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
for (byte[] family: families) {
put.add(family, qualifier, value);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (working copy)
@@ -518,7 +518,7 @@
System.out.println(String.format("Saving row: %s, with value %s", row,
value));
Put put = new Put(Bytes.toBytes(row));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(Bytes.toBytes("trans-blob"), null, Bytes
.toBytes("value for blob"));
put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
@@ -734,7 +734,7 @@
};
for(int i=0;i<10;i++) {
Put put = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILY, QUALIFIERS[i], VALUE);
ht.put(put);
}
@@ -770,7 +770,7 @@
};
for(int i=0;i<10;i++) {
Put put = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILY, QUALIFIERS[i], VALUE);
ht.put(put);
}
@@ -1995,7 +1995,7 @@
for (int i = 0; i < 10; i++) {
byte [] bytes = Bytes.toBytes(i);
put = new Put(bytes);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILIES[0], QUALIFIER, bytes);
ht.put(put);
}
@@ -2103,7 +2103,7 @@
for(int i=0;i c,
final Put put, final WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
Map> familyMap = put.getFamilyMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
@@ -347,7 +348,7 @@
@Override
public void postPut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
Map> familyMap = put.getFamilyMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
@@ -380,7 +381,7 @@
@Override
public void preDelete(final ObserverContext c,
final Delete delete, final WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
Map> familyMap = delete.getFamilyMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
@@ -394,7 +395,7 @@
@Override
public void postDelete(final ObserverContext c,
final Delete delete, final WALEdit edit,
- final boolean writeToWAL) throws IOException {
+ final Durability writeGuarantee) throws IOException {
Map> familyMap = delete.getFamilyMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java (working copy)
@@ -27,6 +27,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.client.coprocessor.LongColumnInterpreter;
import org.apache.hadoop.hbase.filter.Filter;
@@ -86,12 +87,12 @@
*/
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
Long l = new Long(i);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(l));
table.put(put);
Put p2 = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
p2.add(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(l)), Bytes
.toBytes(l * 10));
table.put(p2);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java (working copy)
@@ -26,6 +26,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.client.coprocessor.BigDecimalColumnInterpreter;
import org.apache.hadoop.hbase.filter.Filter;
@@ -83,12 +84,12 @@
*/
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
BigDecimal bd = new BigDecimal(i);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(bd));
table.put(put);
Put p2 = new Put(ROWS[i]);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
p2.add(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(bd)),
Bytes.toBytes(bd.multiply(new BigDecimal("0.10"))));
table.put(p2);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java (working copy)
@@ -37,6 +37,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -205,7 +206,7 @@
public static class TestCoprocessor extends BaseRegionObserver {
@Override
public void prePut(final ObserverContext e,
- final Put put, final WALEdit edit, final boolean writeToWAL)
+ final Put put, final WALEdit edit, final Durability writeGuarantee)
throws IOException {
Map> familyMap = put.getFamilyMap();
if (familyMap.containsKey(test)) {
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java (working copy)
@@ -53,6 +53,7 @@
import org.apache.hadoop.hbase.client.ResultScanner;
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.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
@@ -399,7 +400,7 @@
for (long i=1; i<=10; i++) {
byte[] iBytes = Bytes.toBytes(i);
Put put = new Put(iBytes);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(A, A, iBytes);
table.put(put);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java (working copy)
@@ -33,6 +33,7 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -50,7 +51,7 @@
@Override
public void postPut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL)
+ final Durability writeGuarantee)
throws IOException {
id = System.currentTimeMillis();
try {
@@ -65,7 +66,7 @@
@Override
public void postPut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL)
+ final Durability writeGuarantee)
throws IOException {
id = System.currentTimeMillis();
try {
@@ -81,7 +82,7 @@
@Override
public void postPut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL)
+ final Durability writeGuarantee)
throws IOException {
id = System.currentTimeMillis();
try {
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java (working copy)
@@ -28,6 +28,7 @@
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -100,7 +101,7 @@
@Override
public void prePut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL) {
+ final Durability writeGuarantee) {
String tableName = c.getEnvironment().getRegion().getRegionInfo().getTableNameAsString();
if (TABLE_NAME.equals(tableName)) {
throw new NullPointerException("Buggy coprocessor");
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java (working copy)
@@ -26,6 +26,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -50,7 +51,7 @@
@Override
public void prePut(final ObserverContext c,
final Put put, final WALEdit edit,
- final boolean writeToWAL) {
+ final Durability writeGuarantee) {
String tableName =
c.getEnvironment().getRegion().getRegionInfo().getTableNameAsString();
if (tableName.equals("observed_table")) {
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (working copy)
@@ -30,6 +30,7 @@
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
@@ -67,7 +68,7 @@
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
@@ -129,7 +130,7 @@
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java (working copy)
@@ -35,6 +35,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
@@ -183,7 +184,7 @@
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column : columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (working copy)
@@ -40,6 +40,7 @@
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
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.FilterList.Operator;
import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -143,7 +144,7 @@
// Insert first half
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
}
@@ -151,7 +152,7 @@
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
}
@@ -164,7 +165,7 @@
// Insert second half (reverse families)
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
}
@@ -172,7 +173,7 @@
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
}
@@ -184,13 +185,13 @@
Delete d = new Delete(ROW);
d.deleteColumns(FAMILIES[0], QUALIFIERS_ONE[1]);
d.deleteColumns(FAMILIES[1], QUALIFIERS_ONE[1]);
- this.region.delete(d, false);
+ this.region.delete(d);
}
for(byte [] ROW : ROWS_TWO) {
Delete d = new Delete(ROW);
d.deleteColumns(FAMILIES[0], QUALIFIERS_TWO[1]);
d.deleteColumns(FAMILIES[1], QUALIFIERS_TWO[1]);
- this.region.delete(d, false);
+ this.region.delete(d);
}
colsPerRow -= 2;
@@ -199,13 +200,13 @@
Delete d = new Delete(ROWS_ONE[1]);
d.deleteColumns(FAMILIES[0], QUALIFIER);
d.deleteColumns(FAMILIES[1], QUALIFIER);
- this.region.delete(d, false);
+ this.region.delete(d);
}
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
Delete d = new Delete(ROWS_TWO[1]);
d.deleteColumns(FAMILIES[0], QUALIFIER);
d.deleteColumns(FAMILIES[1], QUALIFIER);
- this.region.delete(d, false);
+ this.region.delete(d);
}
numRows -= 2;
}
@@ -222,7 +223,7 @@
// create new rows and column family to show how reseek works..
for (byte[] ROW : ROWS_THREE) {
Put p = new Put(ROW);
- p.setWriteToWAL(true);
+ p.setDurability(Durability.SKIP_WAL);
for (byte[] QUALIFIER : QUALIFIERS_THREE) {
p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
@@ -231,7 +232,7 @@
}
for (byte[] ROW : ROWS_FOUR) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (byte[] QUALIFIER : QUALIFIERS_FOUR) {
p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
}
@@ -243,7 +244,7 @@
// Insert second half (reverse families)
for (byte[] ROW : ROWS_THREE) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (byte[] QUALIFIER : QUALIFIERS_THREE) {
p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
}
@@ -251,7 +252,7 @@
}
for (byte[] ROW : ROWS_FOUR) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (byte[] QUALIFIER : QUALIFIERS_FOUR) {
p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
}
@@ -1317,7 +1318,7 @@
for(KeyValue kv : srcKVs) {
Put put = new Put(kv.getRow()).add(kv);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
this.region.put(put);
}
@@ -1500,7 +1501,7 @@
public void testColumnPaginationFilter() throws Exception {
// Test that the filter skips multiple column versions.
Put p = new Put(ROWS_ONE[0]);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
p.add(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]);
this.region.put(p);
this.region.flushcache();
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java (working copy)
@@ -30,6 +30,7 @@
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
@@ -69,7 +70,7 @@
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
@@ -131,7 +132,7 @@
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
double rand = Math.random();
@@ -188,7 +189,7 @@
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java (working copy)
@@ -39,6 +39,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@@ -140,7 +141,7 @@
put.add(CF_BYTES, getQualifier(j),
getValue(batchId, i, j));
}
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
table.put(put);
}
table.close();
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java (working copy)
@@ -38,6 +38,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -117,7 +118,7 @@
for (Long ts : tsList) {
Put put = new Put(key.get());
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILY_NAME, COLUMN_NAME, ts, Bytes.toBytes(true));
table.put(put);
}
@@ -157,7 +158,7 @@
private void prepareTest(final HTable table) throws IOException {
for (Map.Entry entry : TIMESTAMP.entrySet()) {
Put put = new Put(KEY);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false));
table.put(put);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapper.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapper.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TsvImporterCustomTestMapper.java (working copy)
@@ -21,6 +21,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.KeyValue;
@@ -56,7 +57,7 @@
ImmutableBytesWritable rowKey =
new ImmutableBytesWritable(Bytes.toBytes(valueTokens[0]));
Put put = new Put(rowKey.copyBytes());
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
//The value should look like this: VALUE1 or VALUE2. Let's multiply
//the integer by 3
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java (working copy)
@@ -31,6 +31,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -491,7 +492,7 @@
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
p.add(getTestFamily(), getTestQualifier(), row);
t.put(p);
rows++;
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java (working copy)
@@ -42,6 +42,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
import org.apache.hadoop.hbase.executor.EventType;
@@ -390,7 +391,7 @@
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
p.add(getTestFamily(), getTestQualifier(), row);
t.put(p);
rows++;
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java (working copy)
@@ -105,11 +105,11 @@
MutationProto proto = mutateBuilder.build();
// default fields
- assertEquals(true, proto.getWriteToWAL());
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
// set the default value for equal comparison
mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setWriteToWAL(true);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Append append = ProtobufUtil.toAppend(proto, null);
@@ -152,11 +152,11 @@
MutationProto proto = mutateBuilder.build();
// default fields
- assertEquals(true, proto.getWriteToWAL());
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
// set the default value for equal comparison
mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setWriteToWAL(true);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Delete delete = ProtobufUtil.toDelete(proto);
@@ -196,11 +196,11 @@
MutationProto proto = mutateBuilder.build();
// default fields
- assertEquals(true, proto.getWriteToWAL());
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
// set the default value for equal comparison
mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setWriteToWAL(true);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Increment increment = ProtobufUtil.toIncrement(proto, null);
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(increment));
@@ -231,11 +231,11 @@
MutationProto proto = mutateBuilder.build();
// default fields
- assertEquals(true, proto.getWriteToWAL());
+ assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
// set the default value for equal comparison
mutateBuilder = MutationProto.newBuilder(proto);
- mutateBuilder.setWriteToWAL(true);
+ mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Put put = ProtobufUtil.toPut(proto);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java (working copy)
@@ -24,6 +24,7 @@
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
@@ -32,7 +33,7 @@
/**
* A region server that will OOME.
- * Everytime {@link #put(regionName, Put)} is called, we add
+ * Everytime {@link #put(regionName, Durability)} is called, we add
* keep around a reference to the batch. Use this class to test OOME extremes.
* Needs to be started manually as in
* ${HBASE_HOME}/bin/hbase ./bin/hbase org.apache.hadoop.hbase.OOMERegionServer start.
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java (working copy)
@@ -93,11 +93,11 @@
a.setReturnResults(false);
a.add(fam1, qual1, Bytes.toBytes(v1));
a.add(fam1, qual2, Bytes.toBytes(v2));
- assertNull(region.append(a, true));
+ assertNull(region.append(a));
a = new Append(row);
a.add(fam1, qual1, Bytes.toBytes(v2));
a.add(fam1, qual2, Bytes.toBytes(v1));
- Result result = region.append(a, true);
+ Result result = region.append(a);
assertEquals(0, Bytes.compareTo(Bytes.toBytes(v1+v2), result.getValue(fam1, qual1)));
assertEquals(0, Bytes.compareTo(Bytes.toBytes(v2+v1), result.getValue(fam1, qual2)));
}
@@ -210,7 +210,7 @@
inc.addColumn(fam1, qual1, amount);
inc.addColumn(fam1, qual2, amount*2);
inc.addColumn(fam2, qual3, amount*3);
- region.increment(inc, true);
+ region.increment(inc);
// verify: Make sure we only see completed increments
Get g = new Get(row);
@@ -246,7 +246,7 @@
a.add(fam1, qual1, val);
a.add(fam1, qual2, val);
a.add(fam2, qual3, val);
- region.append(a, true);
+ region.append(a);
Get g = new Get(row);
Result result = region.get(g);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java (working copy)
@@ -39,6 +39,7 @@
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -128,7 +129,7 @@
long versionEnd) throws IOException {
byte columnBytes[] = Bytes.toBytes(col);
Put put = new Put(Bytes.toBytes(row));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
for (long version = versionStart; version <= versionEnd; version++) {
put.add(cf, columnBytes, version, genValue(row, col, version));
@@ -188,7 +189,7 @@
del.deleteFamily(Bytes.toBytes(family + "_ROWCOL"), version);
del.deleteFamily(Bytes.toBytes(family + "_ROW"), version);
del.deleteFamily(Bytes.toBytes(family + "_NONE"), version);
- region.delete(del, true);
+ region.delete(del);
}
private static void verifyData(KeyValue kv, String expectedRow,
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java (working copy)
@@ -34,6 +34,7 @@
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -92,7 +93,7 @@
for (String value : values) {
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column : allColumns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv =
@@ -203,7 +204,7 @@
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (String column : allColumns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv =
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (working copy)
@@ -48,6 +48,7 @@
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.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
@@ -135,7 +136,7 @@
do {
List results = new ArrayList();
boolean result = s.next(results);
- r.delete(new Delete(results.get(0).getRow()), false);
+ r.delete(new Delete(results.get(0).getRow()));
if (!result) break;
} while(true);
s.close();
@@ -253,7 +254,7 @@
Delete delete = new Delete(secondRowBytes, System.currentTimeMillis());
byte [][] famAndQf = {COLUMN_FAMILY, null};
delete.deleteFamily(famAndQf[0]);
- r.delete(delete, true);
+ r.delete(delete);
// Assert deleted.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
@@ -424,7 +425,7 @@
// the compaction threshold of 3 store files. Compacting these store files
// should result in a compacted store file that has no references to the
// deleted row.
- r.delete(delete, true);
+ r.delete(delete);
// Make sure that we have only deleted family2 from secondRowBytes
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
@@ -501,7 +502,7 @@
for (int i = 0; i < compactionThreshold; i++) {
HRegionIncommon loader = new HRegionIncommon(r);
Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for (int j = 0; j < jmax; j++) {
p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
}
@@ -539,7 +540,7 @@
Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
byte [][] famAndQf = {COLUMN_FAMILY, null};
delete.deleteFamily(famAndQf[0]);
- r.delete(delete, true);
+ r.delete(delete);
}
r.flushcache();
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java (working copy)
@@ -37,6 +37,7 @@
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.util.Bytes;
import org.junit.experimental.categories.Category;
@@ -80,8 +81,8 @@
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = MetaEditor.makePutFromRegionInfo(hri);
- put.setWriteToWAL(false);
- mr.put(put, false);
+ put.setDurability(Durability.SKIP_WAL);
+ mr.put(put);
}
}
InternalScanner s = mr.getScanner(new Scan());
@@ -111,7 +112,7 @@
try {
List keys = new ArrayList();
while (s.next(keys)) {
- mr.delete(new Delete(keys.get(0).getRow()), false);
+ mr.delete(new Delete(keys.get(0).getRow()));
keys.clear();
}
} finally {
@@ -206,7 +207,7 @@
Delete d = new Delete(T20);
d.deleteColumn(c0, c0);
- region.delete(d, false);
+ region.delete(d);
r = region.getClosestRowBefore(T20, c0);
assertTrue(Bytes.equals(T10, r.getRow()));
@@ -220,7 +221,7 @@
d = new Delete(T30);
d.deleteColumn(c0, c0);
- region.delete(d, false);
+ region.delete(d);
r = region.getClosestRowBefore(T30, c0);
assertTrue(Bytes.equals(T10, r.getRow()));
@@ -256,7 +257,7 @@
// in memory; make sure we get back t10 again.
d = new Delete(T20);
d.deleteColumn(c1, c1);
- region.delete(d, false);
+ region.delete(d);
r = region.getClosestRowBefore(T30, c0);
assertTrue(Bytes.equals(T10, r.getRow()));
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (working copy)
@@ -65,6 +65,7 @@
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.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.exceptions.WrongRegionException;
@@ -174,7 +175,7 @@
RegionScanner scanner1 = region.getScanner(scan);
Delete delete = new Delete(Bytes.toBytes("r1"));
- region.delete(delete, false);
+ region.delete(delete);
region.flushcache();
// open the second scanner
@@ -524,10 +525,11 @@
this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
boolean exceptionCaught = false;
Append append = new Append(Bytes.toBytes("somerow"));
+ append.setDurability(Durability.SKIP_WAL);
append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"),
Bytes.toBytes("somevalue"));
try {
- region.append(append, false);
+ region.append(append);
} catch (IOException e) {
exceptionCaught = true;
} finally {
@@ -542,9 +544,10 @@
this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
boolean exceptionCaught = false;
Increment inc = new Increment(Bytes.toBytes("somerow"));
+ inc.setDurability(Durability.SKIP_WAL);
inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
try {
- region.increment(inc, false);
+ region.increment(inc);
} catch (IOException e) {
exceptionCaught = true;
} finally {
@@ -568,7 +571,7 @@
break;
Delete delete = new Delete(results.get(0).getRow());
delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
- r.delete(delete, false);
+ r.delete(delete);
results.clear();
} while (more);
assertEquals("Did not perform correct number of deletes", 3, count);
@@ -619,7 +622,7 @@
System.out.println(String.format("Saving row: %s, with value %s", row,
value));
Put put = new Put(Bytes.toBytes(row));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(Bytes.toBytes("trans-blob"), null,
Bytes.toBytes("value for blob"));
put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
@@ -1110,7 +1113,7 @@
Delete delete = new Delete(row1);
delete.deleteColumn(fam1, qual);
delete.deleteColumn(fam1, qual);
- region.delete(delete, false);
+ region.delete(delete);
Get get = new Get(row1);
get.addFamily(fam1);
@@ -1144,7 +1147,7 @@
NavigableMap> deleteMap =
new TreeMap>(Bytes.BYTES_COMPARATOR);
deleteMap.put(family, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);
} catch (Exception e) {
assertTrue("Family " +new String(family)+ " does not exist", false);
}
@@ -1156,7 +1159,7 @@
NavigableMap> deleteMap =
new TreeMap>(Bytes.BYTES_COMPARATOR);
deleteMap.put(family, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);
} catch (Exception e) {
ok = true;
}
@@ -1198,7 +1201,7 @@
// ok now delete a split:
Delete delete = new Delete(row);
delete.deleteColumns(fam, splitA);
- region.delete(delete, true);
+ region.delete(delete);
// assert some things:
Get get = new Get(row).addColumn(fam, serverinfo);
@@ -1223,7 +1226,7 @@
// Now delete all... then test I can add stuff back
delete = new Delete(row);
- region.delete(delete, false);
+ region.delete(delete);
assertEquals(0, region.get(get).size());
region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
@@ -1253,7 +1256,7 @@
// now delete something in the present
Delete delete = new Delete(row);
- region.delete(delete, true);
+ region.delete(delete);
// make sure we still see our data
Get get = new Get(row).addColumn(fam, serverinfo);
@@ -1262,7 +1265,7 @@
// delete the future row
delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3);
- region.delete(delete, true);
+ region.delete(delete);
// make sure it is gone
get = new Get(row).addColumn(fam, serverinfo);
@@ -1292,7 +1295,7 @@
// add data with LATEST_TIMESTAMP, put without WAL
Put put = new Put(row);
put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
- region.put(put, false);
+ region.put(put);
// Make sure it shows up with an actual timestamp
Get get = new Get(row).addColumn(fam, qual);
@@ -1308,7 +1311,7 @@
row = Bytes.toBytes("row2");
put = new Put(row);
put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
- region.put(put, true);
+ region.put(put);
// Make sure it shows up with an actual timestamp
get = new Get(row).addColumn(fam, qual);
@@ -1346,11 +1349,11 @@
try {
// no TS specified == use latest. should not error
region.put(new Put(row).add(fam, Bytes.toBytes("qual"), Bytes
- .toBytes("value")), false);
+ .toBytes("value")));
// TS out of range. should error
region.put(new Put(row).add(fam, Bytes.toBytes("qual"),
System.currentTimeMillis() + 2000,
- Bytes.toBytes("value")), false);
+ Bytes.toBytes("value")));
fail("Expected IOE for TS out of configured timerange");
} catch (FailedSanityCheckException ioe) {
LOG.debug("Received expected exception", ioe);
@@ -1377,7 +1380,7 @@
Delete delete = new Delete(rowA);
delete.deleteFamily(fam1);
- region.delete(delete, true);
+ region.delete(delete);
// now create data.
Put put = new Put(rowA);
@@ -1428,7 +1431,7 @@
region.put(put);
// now delete the value:
- region.delete(delete, true);
+ region.delete(delete);
// ok put data:
@@ -1484,7 +1487,7 @@
NavigableMap> deleteMap =
new TreeMap>(Bytes.BYTES_COMPARATOR);
deleteMap.put(fam1, kvs);
- region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+ region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);
// extract the key values out the memstore:
// This is kinda hacky, but better than nothing...
@@ -2785,7 +2788,7 @@
boolean toggle=true;
for (long i = 0; i < numRows; i++) {
Put put = new Put(Bytes.toBytes(i));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(family, qual1, Bytes.toBytes(i % 10));
region.put(put);
@@ -3005,7 +3008,7 @@
for (int r = 0; r < numRows; r++) {
byte[] row = Bytes.toBytes("row" + r);
Put put = new Put(row);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
for (byte[] family : families) {
for (byte[] qualifier : qualifiers) {
@@ -3018,7 +3021,7 @@
if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
System.out.println("put iteration = " + numPutsFinished);
Delete delete = new Delete(row, (long)numPutsFinished-30);
- region.delete(delete, true);
+ region.delete(delete);
}
numPutsFinished++;
}
@@ -3192,7 +3195,7 @@
Delete delete = new Delete(Bytes.toBytes(1L), 1L);
//delete.deleteColumn(family, qual1);
- region.delete(delete, true);
+ region.delete(delete);
put = new Put(Bytes.toBytes(2L));
put.add(family, qual1, 2L, Bytes.toBytes(2L));
@@ -3250,7 +3253,7 @@
for (int i = 0; i < duplicate_multiplier; i ++) {
for (int j = 0; j < num_unique_rows; j++) {
Put put = new Put(Bytes.toBytes("row" + j));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(fam1, qf1, version++, val1);
region.put(put);
}
@@ -3304,7 +3307,7 @@
byte row[] = Bytes.toBytes("row:" + 0);
byte column[] = Bytes.toBytes("column:" + 0);
Put put = new Put(row);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
for (long idx = 1; idx <= 4; idx++) {
put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
}
@@ -3360,7 +3363,7 @@
region.flushcache();
Delete del = new Delete(row);
- region.delete(del, true);
+ region.delete(del);
region.flushcache();
// Get remaining rows (should have none)
@@ -3545,7 +3548,7 @@
inc.addColumn(family, qualifier, ONE);
count++;
try {
- region.increment(inc, true);
+ region.increment(inc);
} catch (IOException e) {
e.printStackTrace();
break;
@@ -3636,7 +3639,7 @@
app.add(family, qualifier, CHAR);
count++;
try {
- region.append(app, true);
+ region.append(app);
} catch (IOException e) {
e.printStackTrace();
break;
@@ -3772,7 +3775,7 @@
throws IOException {
for(int i=startRow; i e,
final Put put, final WALEdit edit,
- final boolean writeToWAL)
+ final Durability writeGuarantee)
throws IOException {
nCount++;
}
@Override
public void postDelete(final ObserverContext c,
final Delete delete, final WALEdit edit,
- final boolean writeToWAL)
+ final Durability writeGuarantee)
throws IOException {
nDelete++;
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java (working copy)
@@ -36,6 +36,7 @@
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.rest.client.Client;
import org.apache.hadoop.hbase.rest.client.Cluster;
import org.apache.hadoop.hbase.rest.client.Response;
@@ -87,7 +88,7 @@
k[1] = b2;
k[2] = b3;
Put put = new Put(k);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(famAndQf[0], famAndQf[1], k);
table.put(put);
count++;
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java (working copy)
@@ -38,6 +38,7 @@
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
@@ -137,7 +138,7 @@
// Insert first half
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
}
@@ -145,7 +146,7 @@
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
}
@@ -155,7 +156,7 @@
// Insert second half (reverse families)
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
}
@@ -163,7 +164,7 @@
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
- p.setWriteToWAL(false);
+ p.setDurability(Durability.SKIP_WAL);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java (working copy)
@@ -34,6 +34,7 @@
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.rest.client.Client;
import org.apache.hadoop.hbase.rest.client.Cluster;
import org.apache.hadoop.hbase.rest.client.Response;
@@ -94,7 +95,7 @@
k[1] = b2;
k[2] = b3;
Put put = new Put(k);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(famAndQf[0], famAndQf[1], k);
table.put(put);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java (working copy)
@@ -32,6 +32,7 @@
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.exceptions.SnapshotDoesNotExistException;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -235,7 +236,7 @@
byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
Put put = new Put(key);
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
for (byte[] family: families) {
put.add(family, qualifier, value);
}
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (working copy)
@@ -45,6 +45,7 @@
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.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -215,7 +216,7 @@
// since it is loaded by a different class loader
@Override
public void prePut(final ObserverContext c, final Put put,
- final WALEdit edit, final boolean writeToWAL) throws IOException {
+ final WALEdit edit, final Durability writeGuarantee) throws IOException {
if (put.getAttribute("ttl") != null) {
Cell cell = put.getFamilyMap().values().iterator().next().get(0);
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (working copy)
@@ -68,6 +68,7 @@
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.io.hfile.TestHFile;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
@@ -165,7 +166,7 @@
// When we find a diff RS, change the assignment and break
if (startCode != sn.getStartcode()) {
Put put = new Put(res.getRow());
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort()));
put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Index: hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
===================================================================
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java (revision 1465321)
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java (working copy)
@@ -33,6 +33,7 @@
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -141,7 +142,7 @@
LOG.info("Created region " + region.getRegionNameAsString());
for(int i = firstRow; i < firstRow + nrows; i++) {
Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
- put.setWriteToWAL(false);
+ put.setDurability(Durability.SKIP_WAL);
put.add(COLUMN_NAME, null, VALUE);
region.put(put);
if (i % 10000 == 0) {