You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/03/31 07:23:22 UTC
[hbase] branch HBASE-25713 updated: hbase-wal module compiles
This is an automated email from the ASF dual-hosted git repository.
stack pushed a commit to branch HBASE-25713
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/HBASE-25713 by this push:
new c62ab8e hbase-wal module compiles
c62ab8e is described below
commit c62ab8e5f2347aa843944bebea0c0694808c3f1d
Author: stack <st...@apache.org>
AuthorDate: Wed Mar 31 00:19:42 2021 -0700
hbase-wal module compiles
---
hbase-client/pom.xml | 4 ++
.../org/apache/hadoop/hbase/client/RegionInfo.java | 11 +++
.../MultiVersionConcurrencyControl.java | 5 +-
.../TestMultiVersionConcurrencyControl.java | 0
hbase-coprocessor/pom.xml | 4 ++
.../hbase/coprocessor/MetricsCoprocessor.java | 2 +-
.../hbase/mapreduce/TestWALRecordReader.java | 13 ++--
hbase-server/pom.xml | 4 ++
.../apache/hadoop/hbase/regionserver/HRegion.java | 2 +-
.../hbase/regionserver/wal/AbstractFSWAL.java | 33 ++++-----
.../hadoop/hbase/regionserver/wal/AsyncFSWAL.java | 14 ++--
.../hadoop/hbase/regionserver/wal/FSHLog.java | 25 +++----
.../hadoop/hbase/regionserver/wal/FSWALEntry.java | 15 +----
.../hadoop/hbase/regionserver/wal/WALUtil.java | 23 +++----
.../coprocessor/CurrentCoprocessorMethods.java | 2 +-
.../hadoop/hbase/wal/DisabledWALProvider.java | 23 ++++---
.../coprocessor/SampleRegionWALCoprocessor.java | 13 ++--
.../hbase/coprocessor/TestCoprocessorMetrics.java | 11 ---
...ObserverForAddingMutationsFromCoprocessors.java | 26 +++-----
.../hadoop/hbase/coprocessor/TestWALObserver.java | 12 ++--
.../hadoop/hbase/master/AbstractTestDLS.java | 2 +-
.../hadoop/hbase/regionserver/TestBulkLoad.java | 8 +--
.../regionserver/TestCompactionAfterBulkLoad.java | 2 +-
.../hadoop/hbase/regionserver/TestHRegion.java | 21 +++---
.../regionserver/TestHRegionReplayEvents.java | 12 ++--
.../hadoop/hbase/regionserver/TestWALLockup.java | 8 +--
.../hbase/regionserver/wal/AbstractTestFSWAL.java | 12 ++--
.../regionserver/wal/AbstractTestWALReplay.java | 57 ++++------------
.../hbase/regionserver/wal/TestAsyncFSWAL.java | 3 +-
.../hbase/regionserver/wal/TestLogRollAbort.java | 2 +-
.../hbase/regionserver/wal/TestLogRolling.java | 1 -
.../regionserver/wal/TestLogRollingNoCluster.java | 7 +-
.../regionserver/wal/TestWALActionsListener.java | 5 +-
.../TestReplicationEmptyWALRecovery.java | 4 +-
.../replication/TestReplicationSmallTests.java | 5 +-
...stRegionReplicaReplicationEndpointNoMaster.java | 20 +-----
.../regionserver/TestReplicationSink.java | 61 ++++++-----------
.../regionserver/TestReplicationSourceManager.java | 21 ++----
.../regionserver/TestWALEntryStream.java | 6 +-
.../org/apache/hadoop/hbase/wal/FaultyFSLog.java | 9 +--
.../hadoop/hbase/wal/TestFSHLogProvider.java | 10 +--
.../org/apache/hadoop/hbase/wal/TestSecureWAL.java | 2 +-
.../apache/hadoop/hbase/wal/TestWALFactory.java | 78 +++++++++++-----------
.../hadoop/hbase/wal/TestWALReaderOnSecureWAL.java | 2 +-
.../apache/hadoop/hbase/wal/TestWALRootDir.java | 10 +--
.../org/apache/hadoop/hbase/wal/TestWALSplit.java | 27 +++++---
.../hadoop/hbase/wal/WALPerformanceEvaluation.java | 7 +-
{hbase-coprocessor => hbase-wal}/pom.xml | 18 ++++-
.../hadoop/hbase/coprocessor/WALCoprocessor.java | 6 +-
.../coprocessor/WALCoprocessorEnvironment.java | 4 +-
.../hadoop/hbase/coprocessor/WALObserver.java | 29 +-------
.../hadoop/hbase/regionserver/SequenceId.java | 0
.../hbase/regionserver/wal/CompressionContext.java | 6 +-
.../regionserver/wal/FailedLogCloseException.java | 9 +--
.../hbase/regionserver/wal/WALActionsListener.java | 19 +++---
.../hbase/regionserver/wal/WALCellCodec.java | 0
.../hbase/regionserver/wal/WALCoprocessorHost.java | 43 ++++--------
.../regionserver/WALFileLengthProvider.java | 2 -
.../main/java/org/apache/hadoop/hbase/wal/WAL.java | 36 ++++------
.../java/org/apache/hadoop/hbase/wal/WALEdit.java | 40 ++++-------
.../java/org/apache/hadoop/hbase/wal/WALKey.java | 13 ++--
.../org/apache/hadoop/hbase/wal/WALKeyImpl.java | 15 +----
.../regionserver/wal/TestCustomWALCellCodec.java | 0
pom.xml | 13 ++++
64 files changed, 362 insertions(+), 535 deletions(-)
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 8f61522..288993c 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -204,6 +204,10 @@
</exclusion>
</exclusions>
</dependency>
+ <dependency>
+ <artifactId>hbase-wal</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ </dependency>
</dependencies>
<profiles>
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index e105dbc..427a49c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -851,4 +851,15 @@ public interface RegionInfo extends Comparable<RegionInfo> {
default int compareTo(RegionInfo other) {
return RegionInfo.COMPARATOR.compare(this, other);
}
+
+ /**
+ * Empty row key is not allowed in mutations because it is both the start key and the end key
+ * so we return the smallest byte[] that is bigger (in lex comparison) than byte[0] if startkey
+ * is empty.
+ * @return {@link #getStartKey()} or if length == 0, new byte [] {0}
+ */
+ default byte[] getNonEmptyStartKey() {
+ byte[] startKey = getStartKey();
+ return startKey.length == 0? new byte [] {0}: startKey;
+ }
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
rename to hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
index d821eec..bcab681 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
@@ -23,11 +23,10 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* Manages the read/write consistency. This provides an interface for readers to determine what
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
similarity index 100%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
rename to hbase-common/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index 0bcfcef..2998751 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -98,5 +98,9 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
rename to hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
index a77a0fe..d8a1bcb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
+++ b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
@@ -20,10 +20,10 @@
package org.apache.hadoop.hbase.coprocessor;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.MetricRegistries;
import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility class for tracking metrics for various types of coprocessors. Each coprocessor instance
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 0fa73f6..76fd384 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALFactory;
@@ -134,10 +133,10 @@ public class TestWALRecordReader {
long ts = System.currentTimeMillis();
WALEdit edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
- log.appendData(info, getWalKeyImpl(ts, scopes), edit);
+ log.appendData(getWalKeyImpl(ts, scopes), edit);
edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
- log.appendData(info, getWalKeyImpl(ts+1, scopes), edit);
+ log.appendData(getWalKeyImpl(ts+1, scopes), edit);
log.sync();
Threads.sleep(10);
LOG.info("Before 1st WAL roll " + log.toString());
@@ -149,10 +148,10 @@ public class TestWALRecordReader {
edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
- log.appendData(info, getWalKeyImpl(ts1+1, scopes), edit);
+ log.appendData(getWalKeyImpl(ts1+1, scopes), edit);
edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
- log.appendData(info, getWalKeyImpl(ts1+2, scopes), edit);
+ log.appendData(getWalKeyImpl(ts1+2, scopes), edit);
log.sync();
log.shutdown();
walfactory.shutdown();
@@ -196,7 +195,7 @@ public class TestWALRecordReader {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
System.currentTimeMillis(), value));
- long txid = log.appendData(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
+ long txid = log.appendData(getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
log.sync(txid);
Thread.sleep(1); // make sure 2nd log gets a later timestamp
@@ -205,7 +204,7 @@ public class TestWALRecordReader {
edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), System.currentTimeMillis(), value));
- txid = log.appendData(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
+ txid = log.appendData(getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
log.sync(txid);
log.shutdown();
walfactory.shutdown();
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 93051e0..410d7fc 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -258,6 +258,10 @@
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-wal</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
<artifactId>hbase-coprocessor</artifactId>
</dependency>
<dependency>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 9220b2f..6f3193c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7907,7 +7907,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
WriteEntry writeEntry = null;
try {
- long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
+ long txid = this.wal.appendData(walKey, walEdit);
// Call sync on our edit.
if (txid != 0) {
sync(txid, durability);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 6c7f008..8345560 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -23,7 +23,6 @@ import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRe
import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-
import com.lmax.disruptor.RingBuffer;
import java.io.FileNotFoundException;
import java.io.IOException;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
import org.apache.hadoop.hbase.ipc.RpcServer;
@@ -82,13 +80,12 @@ import org.apache.hadoop.hbase.wal.WALProvider.WriterBase;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.util.StringUtils;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.htrace.core.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
/**
* Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
* WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
@@ -265,6 +262,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
protected volatile boolean closed = false;
protected final AtomicBoolean shutdown = new AtomicBoolean(false);
+
/**
* WAL Comparator; it compares the timestamp (log filenum), present in the log file name. Throws
* an IllegalArgumentException if used to compare paths from different wals.
@@ -273,12 +271,11 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
(o1, o2) -> Long.compare(getFileNumFromFileName(o1), getFileNumFromFileName(o2));
private static final class WalProps {
-
/**
* Map the encoded region name to the highest sequence id.
* <p/>Contains all the regions it has an entry for.
*/
- public final Map<byte[], Long> encodedName2HighestSequenceId;
+ private final Map<byte[], Long> encodedName2HighestSequenceId;
/**
* The log file size. Notice that the size may not be accurate if we do asynchronous close in
@@ -1007,7 +1004,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
// Coprocessor hook.
- coprocessorHost.preWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit());
+ coprocessorHost.preWALWrite(entry.getKey(), entry.getEdit());
if (!listeners.isEmpty()) {
for (WALActionsListener i : listeners) {
i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
@@ -1023,7 +1020,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
entry.isInMemStore());
}
- coprocessorHost.postWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit());
+ coprocessorHost.postWALWrite(entry.getKey(), entry.getEdit());
// Update metrics.
postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
numEntries.incrementAndGet();
@@ -1071,12 +1068,12 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
}
- protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
+ protected final long stampSequenceIdAndPublishToRingBuffer(WALKeyImpl key,
WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
throws IOException {
if (this.closed) {
- throw new IOException(
- "Cannot append; log is closed, regionName = " + hri.getRegionNameAsString());
+ throw new IOException("WAL closed; cannot append; encodedRegionName=" +
+ Bytes.toString(key.getEncodedRegionName()));
}
MutableLong txidHolder = new MutableLong();
MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin(() -> {
@@ -1086,7 +1083,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
.filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
try (TraceScope scope = TraceUtil.createTrace(implClassName + ".append")) {
- FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
+ FSWALEntry entry = new FSWALEntry(txid, key, edits, inMemstore, rpcCall);
entry.stampRegionSequenceId(we);
ringBuffer.get(txid).load(entry);
} finally {
@@ -1123,14 +1120,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
@Override
- public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
- return append(info, key, edits, true);
+ public long appendData(WALKeyImpl key, WALEdit edits) throws IOException {
+ return append(key, edits, true);
}
@Override
- public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
- throws IOException {
- return append(info, key, edits, false);
+ public long appendMarker(WALKeyImpl key, WALEdit edits) throws IOException {
+ return append(key, edits, false);
}
/**
@@ -1150,7 +1146,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
* passed in WALKey <code>walKey</code> parameter. Be warned that the WriteEntry is not
* immediately available on return from this method. It WILL be available subsequent to a sync of
* this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
- * @param info the regioninfo associated with append
* @param key Modified by this call; we add to it this edits region edit/sequence id.
* @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
* sequence id that is after all currently appended edits.
@@ -1163,7 +1158,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
* in it.
*/
- protected abstract long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+ protected abstract long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
throws IOException;
protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 66149a3..5576a77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.ERROR;
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
-
import com.lmax.disruptor.RingBuffer;
import com.lmax.disruptor.Sequence;
import com.lmax.disruptor.Sequencer;
@@ -44,7 +43,6 @@ import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Supplier;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -60,14 +58,14 @@ import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.htrace.core.TraceScope;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.io.netty.channel.Channel;
import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
import org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor;
+import org.apache.htrace.core.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
@@ -595,9 +593,9 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}
@Override
- protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+ protected long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
throws IOException {
- long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
+ long txid = stampSequenceIdAndPublishToRingBuffer(key, edits, inMemstore,
waitingConsumePayloads);
if (shouldScheduleConsumer()) {
consumeExecutor.execute(consumer);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 7e12097..c8879aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRe
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.LOW_REPLICATION;
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SLOW_SYNC;
-
import com.lmax.disruptor.BlockingWaitStrategy;
import com.lmax.disruptor.EventHandler;
import com.lmax.disruptor.ExceptionHandler;
@@ -65,8 +64,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
/**
* The default implementation of FSWAL.
*/
@@ -109,8 +106,10 @@ public class FSHLog extends AbstractFSWAL<Writer> {
// We use ring buffer sequence as txid of FSWALEntry and SyncFuture.
private static final Logger LOG = LoggerFactory.getLogger(FSHLog.class);
- private static final String TOLERABLE_LOW_REPLICATION = "hbase.regionserver.hlog.tolerable.lowreplication";
- private static final String LOW_REPLICATION_ROLL_LIMIT = "hbase.regionserver.hlog.lowreplication.rolllimit";
+ private static final String TOLERABLE_LOW_REPLICATION =
+ "hbase.regionserver.hlog.tolerable.lowreplication";
+ private static final String LOW_REPLICATION_ROLL_LIMIT =
+ "hbase.regionserver.hlog.lowreplication.rolllimit";
private static final int DEFAULT_LOW_REPLICATION_ROLL_LIMIT = 5;
private static final String ROLL_ERRORS_TOLERATED = "hbase.regionserver.logroll.errors.tolerated";
private static final int DEFAULT_ROLL_ERRORS_TOLERATED = 2;
@@ -119,7 +118,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
private static final String MAX_BATCH_COUNT = "hbase.regionserver.wal.sync.batch.count";
private static final int DEFAULT_MAX_BATCH_COUNT = 200;
- private static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = "hbase.wal.fshlog.wait.on.shutdown.seconds";
+ private static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS =
+ "hbase.wal.fshlog.wait.on.shutdown.seconds";
private static final int DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = 5;
/**
@@ -491,9 +491,9 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
@Override
- protected long append(final RegionInfo hri, final WALKeyImpl key, final WALEdit edits,
+ protected long append(final WALKeyImpl key, final WALEdit edits,
final boolean inMemstore) throws IOException {
- return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
+ return stampSequenceIdAndPublishToRingBuffer(key, edits, inMemstore,
disruptor.getRingBuffer());
}
@@ -865,8 +865,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
* To start up the drama, Thread A creates an instance of this class each time it would do this
* zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only).
* Thread B notices the new instance (via reading a volatile reference or how ever) and it starts
- * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it cannot proceed
- * until the Thread B 'safe point' is attained. Thread A will be held inside in
+ * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it
+ * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
* {@link #waitSafePoint(SyncFuture)} until Thread B reaches the 'safe point'. Once there, Thread B frees
* Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 'safe
* point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it blocks
@@ -1052,8 +1052,9 @@ public class FSHLog extends AbstractFSWAL<Writer> {
// Failed append. Record the exception.
this.exception = e;
// invoking cleanupOutstandingSyncsOnException when append failed with exception,
- // it will cleanup existing sync requests recorded in syncFutures but not offered to SyncRunner yet,
- // so there won't be any sync future left over if no further truck published to disruptor.
+ // it will cleanup existing sync requests recorded in syncFutures but not offered to
+ // SyncRunner yet, so there won't be any sync future left over if no further truck
+ // published to disruptor.
cleanupOutstandingSyncsOnException(sequence,
this.exception instanceof DamagedWALException ? this.exception
: new DamagedWALException("On sync", this.exception));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index ca51ec0..710fc28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -25,16 +25,14 @@ import java.util.TreeSet;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.ServerCall;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.yetus.audience.InterfaceAudience;
-
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A WAL Entry for {@link AbstractFSWAL} implementation. Immutable.
@@ -61,8 +59,6 @@ class FSWALEntry extends Entry {
* Set if this is a meta edit and it is of close region type.
*/
private final transient boolean closeRegion;
-
- private final transient RegionInfo regionInfo;
private final transient Set<byte[]> familyNames;
private final transient ServerCall<?> rpcCall;
@@ -70,12 +66,11 @@ class FSWALEntry extends Entry {
* @param inMemstore If true, then this is a data edit, one that came from client. If false, it
* is a meta edit made by the hbase system itself and is for the WAL only.
*/
- FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit, final RegionInfo regionInfo,
- final boolean inMemstore, ServerCall<?> rpcCall) {
+ FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit, final boolean inMemstore,
+ ServerCall<?> rpcCall) {
super(key, edit);
this.inMemstore = inMemstore;
this.closeRegion = !inMemstore && edit.isRegionCloseMarker();
- this.regionInfo = regionInfo;
this.txid = txid;
if (inMemstore) {
// construct familyNames here to reduce the work of log sinker.
@@ -117,10 +112,6 @@ class FSWALEntry extends Entry {
return closeRegion;
}
- RegionInfo getRegionInfo() {
- return this.regionInfo;
- }
-
/**
* @return The transaction id of this edit.
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 19f6b96..f90c04d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
import java.util.Map;
import java.util.NavigableMap;
import java.util.function.Function;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -68,10 +67,10 @@ public class WALUtil {
* @param mvcc Used by WAL to get sequence Id for the waledit.
*/
public static WALKeyImpl writeCompactionMarker(WAL wal,
- NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
- MultiVersionConcurrencyControl mvcc) throws IOException {
- WALKeyImpl walKey =
- writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc, null);
+ NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
+ MultiVersionConcurrencyControl mvcc) throws IOException {
+ WALKeyImpl walKey = writeMarker(wal, replicationScope,
+ hri, WALEdit.createCompaction(hri.getNonEmptyStartKey(), c), mvcc, null);
if (LOG.isTraceEnabled()) {
LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
}
@@ -87,7 +86,7 @@ public class WALUtil {
RegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKeyImpl walKey = doFullMarkerAppendTransaction(wal, replicationScope, hri,
- WALEdit.createFlushWALEdit(hri, f), mvcc, null, sync);
+ WALEdit.createFlushWALEdit(hri.getNonEmptyStartKey(), f), mvcc, null, sync);
if (LOG.isTraceEnabled()) {
LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
}
@@ -103,7 +102,7 @@ public class WALUtil {
MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
- WALEdit.createRegionEventWALEdit(hri, r), mvcc, null);
+ WALEdit.createRegionEventWALEdit(hri.getNonEmptyStartKey(), r), mvcc, null);
if (LOG.isTraceEnabled()) {
LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
}
@@ -125,7 +124,7 @@ public class WALUtil {
final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
throws IOException {
WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
- WALEdit.createBulkLoadEvent(hri, desc), mvcc, null);
+ WALEdit.createBulkLoadEvent(hri.getNonEmptyStartKey(), desc), mvcc, null);
if (LOG.isTraceEnabled()) {
LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
}
@@ -133,8 +132,8 @@ public class WALUtil {
}
private static WALKeyImpl writeMarker(final WAL wal,
- NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, WALEdit edit, MultiVersionConcurrencyControl mvcc,
- Map<String, byte[]> extendedAttributes)
+ NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, WALEdit edit,
+ MultiVersionConcurrencyControl mvcc, Map<String, byte[]> extendedAttributes)
throws IOException {
// If sync == true in below, then timeout is not used; safe to pass UNSPECIFIED_TIMEOUT
return doFullMarkerAppendTransaction(wal, replicationScope, hri, edit, mvcc,
@@ -156,9 +155,9 @@ public class WALUtil {
// TODO: Pass in current time to use?
WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(),
System.currentTimeMillis(), mvcc, replicationScope, extendedAttributes);
- long trx = MultiVersionConcurrencyControl.NONE;
+ long trx;
try {
- trx = wal.appendMarker(hri, walKey, edit);
+ trx = wal.appendMarker(walKey, edit);
if (sync) {
wal.sync(trx);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java
index 265cf51..88da2ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java
@@ -1,4 +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
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index 6c215f8..5a9241e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -1,4 +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
@@ -162,18 +162,17 @@ class DisabledWALProvider implements WALProvider {
}
@Override
- public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
- return append(info, key, edits, true);
+ public long appendData(WALKeyImpl key, WALEdit edits) throws IOException {
+ return append(key, edits, true);
}
@Override
- public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
+ public long appendMarker(WALKeyImpl key, WALEdit edits)
throws IOException {
- return append(info, key, edits, false);
+ return append(key, edits, false);
}
- private long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
- throws IOException {
+ private long append(WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException {
WriteEntry writeEntry = key.getMvcc().begin();
if (!edits.isReplay()) {
for (Cell cell : edits.getCells()) {
@@ -196,8 +195,10 @@ class DisabledWALProvider implements WALProvider {
}
@Override
- public void updateStore(byte[] encodedRegionName, byte[] familyName,
- Long sequenceid, boolean onlyIfGreater) { return; }
+ public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid,
+ boolean onlyIfGreater) {
+ return;
+ }
@Override
public void sync() {
@@ -221,7 +222,9 @@ class DisabledWALProvider implements WALProvider {
@Override
public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> flushedFamilyNames) {
- if (closed.get()) return null;
+ if (closed.get()) {
+ return null;
+ }
return HConstants.NO_SEQNUM;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
index 1da31da..c80e790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
@@ -23,7 +23,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
-
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
@@ -92,17 +91,17 @@ public class SampleRegionWALCoprocessor implements WALCoprocessor, RegionCoproce
return Optional.of(this);
}
- @Override
- public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+ // What to do here? TODO?
+ public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env, WALKey logKey,
+ WALEdit logEdit) throws IOException {
postWALWriteCalled = true;
}
- @Override
+ // What to do here? TODO?
public void preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+ WALKey logKey, WALEdit logEdit) throws IOException {
// check table name matches or not.
- if (!Bytes.equals(info.getTable().toBytes(), this.tableName)) {
+ if (!Bytes.equals(logKey.getTableName().toBytes(), this.tableName)) {
return;
}
preWALWriteCalled = true;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index 24f74d8..cc4ebe3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
-
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@@ -64,8 +63,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@@ -76,7 +73,6 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
@@ -177,13 +173,6 @@ public class TestCoprocessorMetrics {
private Counter walEditsCount;
@Override
- public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
- RegionInfo info, WALKey logKey,
- WALEdit logEdit) throws IOException {
- walEditsCount.increment();
- }
-
- @Override
public void start(CoprocessorEnvironment env) throws IOException {
if (env instanceof WALCoprocessorEnvironment) {
MetricRegistry registry =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
index 21435d9..64eb6b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -1,4 +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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
-
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.TableName;
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.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@@ -56,8 +55,6 @@ import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-
@Category(MediumTests.class)
public class TestRegionObserverForAddingMutationsFromCoprocessors {
@@ -107,7 +104,6 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
/**
* Test various multiput operations.
- * @throws Exception
*/
@Test
public void testMulti() throws Exception {
@@ -225,9 +221,9 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
Mutation mut = miniBatchOp.getOperation(0);
List<Cell> cells = mut.getFamilyCellMap().get(test);
Put[] puts = new Put[] {
- new Put(Bytes.toBytes("cpPut")).addColumn(test, dummy, cells.get(0).getTimestamp(),
- Bytes.toBytes("cpdummy")).setTTL(mut.getTTL())
- };
+ new Put(Bytes.toBytes("cpPut")).addColumn(test, dummy, cells.get(0).getTimestamp(),
+ Bytes.toBytes("cpdummy")).setTTL(mut.getTTL())
+ };
LOG.info("Putting:" + Arrays.toString(puts));
miniBatchOp.addOperationsFromCP(0, puts);
}
@@ -317,9 +313,9 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
if (mut instanceof Delete) {
List<Cell> cells = mut.getFamilyCellMap().get(test);
Delete[] deletes = new Delete[] {
- // delete only 2 rows
- new Delete(row1, cells.get(0).getTimestamp()),
- new Delete(row2, cells.get(0).getTimestamp()),
+ // delete only 2 rows
+ new Delete(row1, cells.get(0).getTimestamp()),
+ new Delete(row2, cells.get(0).getTimestamp()),
};
LOG.info("Deleting:" + Arrays.toString(deletes));
miniBatchOp.addOperationsFromCP(0, deletes);
@@ -335,10 +331,10 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
return Optional.of(this);
}
- @Override
+ // WHAT TO DO HERE?
public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
- if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
+ WALKey logKey, WALEdit logEdit) throws IOException {
+ if (logKey.getTableName().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
savedEdit = logEdit;
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index 6fbd3b5..9a10924 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -1,4 +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
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
-
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
@@ -239,7 +238,7 @@ public class TestWALObserver {
// it's where WAL write cp should occur.
long now = EnvironmentEdgeManager.currentTime();
// we use HLogKey here instead of WALKeyImpl directly to support legacy coprocessors.
- long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now,
+ long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now,
new MultiVersionConcurrencyControl(), scopes), edit);
log.sync(txid);
@@ -290,7 +289,7 @@ public class TestWALObserver {
assertFalse(cp.isPostWALWriteCalled());
final long now = EnvironmentEdgeManager.currentTime();
- long txid = log.appendData(hri,
+ long txid = log.appendData(
new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes),
new WALEdit());
log.sync(txid);
@@ -338,8 +337,7 @@ public class TestWALObserver {
addWALEdits(tableName, hri, TEST_ROW, fam, countPerFamily,
EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
}
- wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
- edit);
+ wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
// sync to fs.
wal.sync();
@@ -454,7 +452,7 @@ public class TestWALObserver {
edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes));
// uses WALKeyImpl instead of HLogKey on purpose. will only work for tests where we don't care
// about legacy coprocessors
- txid = wal.appendData(hri,
+ txid = wal.appendData(
new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, ee.currentTime(), mvcc), edit);
}
if (-1 != txid) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
index 24fdbc4..ef12911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
@@ -372,7 +372,7 @@ public abstract class AbstractTestDLS {
// HBaseTestingUtility.createMultiRegions use 5 bytes key
byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
e.add(new KeyValue(row, COLUMN_FAMILY, qualifier, System.currentTimeMillis(), value));
- log.appendData(curRegionInfo, new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(),
+ log.appendData(new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(),
tableName, System.currentTimeMillis(), mvcc), e);
if (0 == i % syncEvery) {
log.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 3a934b7..d6a28ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -63,7 +63,7 @@ public class TestBulkLoad extends TestBulkloadBase {
storeFileName = (new Path(storeFileName)).getName();
List<String> storeFileNames = new ArrayList<>();
storeFileNames.add(storeFileName);
- when(log.appendMarker(any(), any(),
+ when(log.appendMarker(any(),
argThat(bulkLogWalEdit(WALEdit.BULK_LOAD, tableName.toBytes(), familyName, storeFileNames))))
.thenAnswer(new Answer() {
@Override
@@ -89,7 +89,7 @@ public class TestBulkLoad extends TestBulkloadBase {
@Test
public void shouldBulkLoadSingleFamilyHLog() throws IOException {
- when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+ when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
.thenAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) {
@@ -109,7 +109,7 @@ public class TestBulkLoad extends TestBulkloadBase {
@Test
public void shouldBulkLoadManyFamilyHLog() throws IOException {
when(log.appendMarker(any(),
- any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
+ argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) {
WALKeyImpl walKey = invocation.getArgument(1);
@@ -128,7 +128,7 @@ public class TestBulkLoad extends TestBulkloadBase {
@Test
public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
- when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+ when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
.thenAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
index c736513..b6b7235 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
@@ -89,7 +89,7 @@ public class TestCompactionAfterBulkLoad extends TestBulkloadBase {
conf.setBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, true);
when(regionServerServices.getConfiguration()).thenReturn(conf);
when(regionServerServices.getCompactionRequestor()).thenReturn(compactionRequester);
- when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+ when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
.thenAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index de8118d..1e08e92 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -900,8 +900,8 @@ public class TestHRegion {
long time = System.nanoTime();
WALEdit edit = null;
if (i == maxSeqId) {
- edit = WALEdit.createCompaction(region.getRegionInfo(),
- CompactionDescriptor.newBuilder()
+ edit = WALEdit.createCompaction(region.getRegionInfo().getNonEmptyStartKey(),
+ CompactionDescriptor.newBuilder()
.setTableName(ByteString.copyFrom(tableName.getName()))
.setFamilyName(ByteString.copyFrom(regionName))
.setEncodedRegionName(ByteString.copyFrom(regionName))
@@ -1012,8 +1012,8 @@ public class TestHRegion {
long time = System.nanoTime();
writer.append(new WAL.Entry(new WALKeyImpl(regionName, tableName, 10, time,
- HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(region.getRegionInfo(),
- compactionDescriptor)));
+ HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(
+ region.getRegionInfo().getNonEmptyStartKey(), compactionDescriptor)));
writer.close();
// close the region now, and reopen again
@@ -5416,7 +5416,6 @@ public class TestHRegion {
/**
* Test case to check put function with memstore flushing for same row, same ts
- * @throws Exception
*/
@Test
public void testPutWithMemStoreFlush() throws Exception {
@@ -5547,7 +5546,7 @@ public class TestHRegion {
region.put(put);
// verify append called or not
- verify(wal, expectAppend ? times(1) : never()).appendData((HRegionInfo) any(),
+ verify(wal, expectAppend ? times(1) : never()).appendData(
(WALKeyImpl) any(), (WALEdit) any());
// verify sync called or not
@@ -6682,7 +6681,7 @@ public class TestHRegion {
region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
TEST_UTIL.getConfiguration(), rss, null);
- verify(wal, times(1)).appendMarker(any(RegionInfo.class), any(WALKeyImpl.class),
+ verify(wal, times(1)).appendMarker(any(WALKeyImpl.class),
editCaptor.capture());
WALEdit edit = editCaptor.getValue();
@@ -6755,7 +6754,7 @@ public class TestHRegion {
*/
private WAL mockWAL() throws IOException {
WAL wal = mock(WAL.class);
- when(wal.appendData(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class)))
+ when(wal.appendData(any(WALKeyImpl.class), any(WALEdit.class)))
.thenAnswer(new Answer<Long>() {
@Override
public Long answer(InvocationOnMock invocation) throws Throwable {
@@ -6765,7 +6764,7 @@ public class TestHRegion {
return 1L;
}
});
- when(wal.appendMarker(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class))).
+ when(wal.appendMarker(any(WALKeyImpl.class), any(WALEdit.class))).
thenAnswer(new Answer<Long>() {
@Override
public Long answer(InvocationOnMock invocation) throws Throwable {
@@ -6807,8 +6806,8 @@ public class TestHRegion {
region.close(false);
// 2 times, one for region open, the other close region
- verify(wal, times(2)).appendMarker(any(RegionInfo.class),
- (WALKeyImpl) any(WALKeyImpl.class), editCaptor.capture());
+ verify(wal, times(2)).appendMarker((WALKeyImpl)any(WALKeyImpl.class),
+ editCaptor.capture());
WALEdit edit = editCaptor.getAllValues().get(1);
assertNotNull(edit);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 7b62119..968f619 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -368,8 +368,7 @@ public class TestHRegionReplayEvents {
if (entry == null) {
break;
}
- FlushDescriptor flushDesc
- = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
+ FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
CompactionDescriptor compactionDesc
= WALEdit.getCompaction(entry.getEdit().getCells().get(0));
if (flushDesc != null) {
@@ -470,8 +469,7 @@ public class TestHRegionReplayEvents {
if (entry == null) {
break;
}
- FlushDescriptor flushDesc
- = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
+ FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
if (flushDesc != null) {
// first verify that everything is replayed and visible before flush event replay
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
@@ -1164,7 +1162,7 @@ public class TestHRegionReplayEvents {
// test for region open and close
secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null);
- verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+ verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
any(WALEdit.class));
// test for replay prepare flush
@@ -1180,11 +1178,11 @@ public class TestHRegionReplayEvents {
primaryRegion.getRegionInfo().getRegionName()))
.build());
- verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+ verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
any(WALEdit.class));
secondaryRegion.close();
- verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+ verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
any(WALEdit.class));
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 09d6934..daeb9bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertTrue;
-
import java.io.IOException;
import java.util.NavigableMap;
import java.util.TreeMap;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -66,7 +66,6 @@ import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
/**
* Testing for lock up of FSHLog.
@@ -155,8 +154,9 @@ public class TestWALLockup {
// Don't countdown latch until someone waiting on it otherwise, the above
// afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll
// be stuck; test won't go down
- while (this.latch.getCount() <= 0)
+ while (this.latch.getCount() <= 0) {
Threads.sleep(1);
+ }
this.latch.countDown();
}
}
@@ -253,7 +253,7 @@ public class TestWALLockup {
LOG.info("SET throwing of exception on append");
dodgyWAL.throwException = true;
// This append provokes a WAL roll request
- dodgyWAL.appendData(region.getRegionInfo(), key, edit);
+ dodgyWAL.appendData(key, edit);
boolean exception = false;
try {
dodgyWAL.sync(false);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index 6d0771e..9aeac2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
@@ -1,4 +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
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@@ -187,7 +186,7 @@ public abstract class AbstractTestFSWAL {
WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), htd.getTableName(),
SequenceId.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
HConstants.NO_NONCE, mvcc, scopes);
- log.appendData(hri, key, cols);
+ log.appendData(key, cols);
}
log.sync();
}
@@ -203,7 +202,6 @@ public abstract class AbstractTestFSWAL {
/**
* tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws
* exception if we do). Comparison is based on the timestamp present in the wal name.
- * @throws Exception
*/
@Test
public void testWALComparator() throws Exception {
@@ -261,7 +259,6 @@ public abstract class AbstractTestFSWAL {
* This method tests this behavior by inserting edits and rolling the wal enough times to reach
* the max number of logs threshold. It checks whether we get the "right regions and stores" for
* flush on rolling the wal.
- * @throws Exception
*/
@Test
public void testFindMemStoresEligibleForFlush() throws Exception {
@@ -406,7 +403,6 @@ public abstract class AbstractTestFSWAL {
* slowing appends in the background ring buffer thread while in foreground we call flush. The
* addition of the sync over HRegion in flush should fix an issue where flush was returning before
* all of its appends had made it out to the WAL (HBASE-11109).
- * @throws IOException
* @see <a href="https://issues.apache.org/jira/browse/HBASE-11109">HBASE-11109</a>
*/
@Test
@@ -470,7 +466,7 @@ public abstract class AbstractTestFSWAL {
final RegionInfo info = region.getRegionInfo();
final WALKeyImpl logkey = new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC(), scopes);
- wal.append(info, logkey, edits, true);
+ wal.append(logkey, edits, true);
region.getMVCC().completeAndWait(logkey.getWriteEntry());
}
region.flush(true);
@@ -520,7 +516,7 @@ public abstract class AbstractTestFSWAL {
new WALKeyImpl(ri.getEncodedNameAsBytes(), td.getTableName(), SequenceId.NO_SEQUENCE_ID,
timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE, mvcc, scopes);
try {
- wal.append(ri, key, cols, true);
+ wal.append(key, cols, true);
fail("Should fail since the wal has already been closed");
} catch (IOException e) {
// expected
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index 9c069bd..5d58b3c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -28,7 +28,6 @@ import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
-
import java.io.FilterInputStream;
import java.io.IOException;
import java.lang.reflect.Field;
@@ -182,10 +181,6 @@ public abstract class AbstractTestWALReplay {
}
}
- /**
- *
- * @throws Exception
- */
@Test
public void testReplayEditsAfterRegionMovedWithMultiCF() throws Exception {
final TableName tableName =
@@ -263,7 +258,6 @@ public abstract class AbstractTestWALReplay {
/**
* Tests for hbase-2727.
- * @throws Exception
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2727">HBASE-2727</a>
*/
@Test
@@ -328,16 +322,11 @@ public abstract class AbstractTestWALReplay {
/**
* Test case of HRegion that is only made out of bulk loaded files. Assert
* that we don't 'crash'.
- * @throws IOException
- * @throws IllegalAccessException
- * @throws NoSuchFieldException
- * @throws IllegalArgumentException
- * @throws SecurityException
*/
@Test
- public void testRegionMadeOfBulkLoadedFilesOnly()
- throws IOException, SecurityException, IllegalArgumentException,
- NoSuchFieldException, IllegalAccessException, InterruptedException {
+ public void testRegionMadeOfBulkLoadedFilesOnly() throws
+ IOException, SecurityException, IllegalArgumentException,
+ NoSuchFieldException, IllegalAccessException, InterruptedException {
final TableName tableName =
TableName.valueOf("testRegionMadeOfBulkLoadedFilesOnly");
final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
@@ -394,11 +383,6 @@ public abstract class AbstractTestWALReplay {
* files) and an edit in the memstore.
* This is for HBASE-10958 "[dataloss] Bulk loading with seqids can prevent some log entries
* from being replayed"
- * @throws IOException
- * @throws IllegalAccessException
- * @throws NoSuchFieldException
- * @throws IllegalArgumentException
- * @throws SecurityException
*/
@Test
public void testCompactedBulkLoadedFiles()
@@ -464,11 +448,6 @@ public abstract class AbstractTestWALReplay {
/**
* Test writing edits into an HRegion, closing it, splitting logs, opening
* Region again. Verify seqids.
- * @throws IOException
- * @throws IllegalAccessException
- * @throws NoSuchFieldException
- * @throws IllegalArgumentException
- * @throws SecurityException
*/
@Test
public void testReplayEditsWrittenViaHRegion()
@@ -571,17 +550,11 @@ public abstract class AbstractTestWALReplay {
* died.
*
* We restart Region again, and verify that the edits were replayed.
- *
- * @throws IOException
- * @throws IllegalAccessException
- * @throws NoSuchFieldException
- * @throws IllegalArgumentException
- * @throws SecurityException
*/
@Test
- public void testReplayEditsAfterPartialFlush()
- throws IOException, SecurityException, IllegalArgumentException,
- NoSuchFieldException, IllegalAccessException, InterruptedException {
+ public void testReplayEditsAfterPartialFlush() throws
+ IOException, SecurityException, IllegalArgumentException,
+ NoSuchFieldException, IllegalAccessException, InterruptedException {
final TableName tableName =
TableName.valueOf("testReplayEditsWrittenViaHRegion");
final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
@@ -663,7 +636,6 @@ public abstract class AbstractTestWALReplay {
* Test that we could recover the data correctly after aborting flush. In the
* test, first we abort flush after writing some data, then writing more data
* and flush again, at last verify the data.
- * @throws IOException
*/
@Test
public void testReplayEditsAfterAbortingFlush() throws IOException {
@@ -749,10 +721,12 @@ public abstract class AbstractTestWALReplay {
List<Cell> results = new ArrayList<>();
while (true) {
boolean existMore = scanner.next(results);
- if (!results.isEmpty())
+ if (!results.isEmpty()) {
scannedCount++;
- if (!existMore)
+ }
+ if (!existMore) {
break;
+ }
results.clear();
}
return scannedCount;
@@ -761,7 +735,6 @@ public abstract class AbstractTestWALReplay {
/**
* Create an HRegion with the result of a WAL split and test we only see the
* good edits
- * @throws Exception
*/
@Test
public void testReplayEditsWrittenIntoWAL() throws Exception {
@@ -801,15 +774,13 @@ public abstract class AbstractTestWALReplay {
long now = ee.currentTime();
edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
now, rowName));
- wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
- edit);
+ wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
// Delete the c family to verify deletes make it over.
edit = new WALEdit();
now = ee.currentTime();
edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily));
- wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
- edit);
+ wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
// Sync.
wal.sync();
@@ -1163,7 +1134,7 @@ public abstract class AbstractTestWALReplay {
byte[] rowName, byte[] family, EnvironmentEdge ee, MultiVersionConcurrencyControl mvcc,
int index, NavigableMap<byte[], Integer> scopes) throws IOException {
FSWALEntry entry = new FSWALEntry(sequence, createWALKey(htd.getTableName(), hri, mvcc, scopes),
- createWALEdit(rowName, family, ee, index), hri, true, null);
+ createWALEdit(rowName, family, ee, index),true, null);
entry.stampRegionSequenceId(mvcc.begin());
return entry;
}
@@ -1173,7 +1144,7 @@ public abstract class AbstractTestWALReplay {
final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc,
NavigableMap<byte[], Integer> scopes) throws IOException {
for (int j = 0; j < count; j++) {
- wal.appendData(hri, createWALKey(tableName, hri, mvcc, scopes),
+ wal.appendData(createWALKey(tableName, hri, mvcc, scopes),
createWALEdit(rowName, family, ee, j));
}
wal.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
index f413baf..2e9729a1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
-
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
@@ -202,7 +201,7 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
SequenceId.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
HConstants.NO_NONCE, mvcc, scopes);
try {
- wal.append(ri, key, cols, true);
+ wal.append(key, cols, true);
} catch (IOException e) {
// should not happen
throw new UncheckedIOException(e);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index d0274de..5616ffb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -207,7 +207,7 @@ public class TestLogRollAbort {
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
scopes.put(Bytes.toBytes("column"), 0);
- log.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+ log.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
// Send the data to HDFS datanodes and close the HDFS writer
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 0712b59..4a12486 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -419,7 +419,6 @@ public class TestLogRolling extends AbstractTestLogRolling {
/**
* Test that WAL is rolled when all data nodes in the pipeline have been restarted.
- * @throws Exception
*/
@Test
public void testLogRollOnPipelineRestart() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index a8b6490..4119e49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -1,4 +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
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertFalse;
-
import java.io.IOException;
import java.util.NavigableMap;
import java.util.TreeMap;
@@ -78,8 +77,6 @@ public class TestLogRollingNoCluster {
/**
* Spin up a bunch of threads and have them all append to a WAL. Roll the
* WAL frequently to try and trigger NPE.
- * @throws IOException
- * @throws InterruptedException
*/
@Test
public void testContendedLogRolling() throws Exception {
@@ -171,7 +168,7 @@ public class TestLogRollingNoCluster {
for(byte[] fam: this.metaTableDescriptor.getColumnFamilyNames()) {
scopes.put(fam, 0);
}
- final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+ final long txid = wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
TableName.META_TABLE_NAME, now, mvcc, scopes), edit);
Threads.sleep(ThreadLocalRandom.current().nextInt(5));
wal.sync(txid);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index e40d7e1..7ea79a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.regionserver.wal;
import static org.junit.Assert.assertEquals;
-
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
@@ -111,8 +110,8 @@ public class TestWALActionsListener {
edit.add(kv);
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
scopes.put(b, 0);
- long txid = wal.appendData(hri,
- new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b), 0, mvcc, scopes), edit);
+ long txid = wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b),
+ 0, mvcc, scopes), edit);
wal.sync(txid);
if (i == 10) {
wal.registerWALActionsListener(laterobserver);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
index 2d72618..d6dd0f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
@@ -1,4 +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
@@ -362,7 +362,7 @@ import org.junit.experimental.categories.Category;
KeyValue kv = new KeyValue(b, famName, b);
WALEdit edit = new WALEdit();
edit.add(kv);
- txId = wal.appendData(info, getWalKeyImpl(), edit);
+ txId = wal.appendData(getWalKeyImpl(), edit);
}
wal.sync(txId);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 7c8f97b..72ff165 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableMap;
@@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
@@ -62,7 +62,6 @@ import org.junit.runners.Parameterized.Parameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
@RunWith(Parameterized.class)
@Category({ ReplicationTests.class, LargeTests.class })
@@ -424,7 +423,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
long now = EnvironmentEdgeManager.currentTime();
edit.add(new KeyValue(rowName, famName, qualifier, now, value));
WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes);
- wal.appendData(hri, walKey, edit);
+ wal.appendData(walKey, edit);
wal.sync();
Get get = new Get(rowName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 1c369e2..facfd69 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -1,4 +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
@@ -39,14 +39,11 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.WALCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.WALObserver;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -59,9 +56,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -87,7 +81,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
HBaseClassTestRule.forClass(TestRegionReplicaReplicationEndpointNoMaster.class);
private static final int NB_SERVERS = 2;
- private static TableName tableName = TableName.valueOf(
+ private static final TableName tableName = TableName.valueOf(
TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName());
private static Table table;
private static final byte[] row = "TestRegionReplicaReplicator".getBytes();
@@ -164,16 +158,6 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
public Optional<WALObserver> getWALObserver() {
return Optional.of(this);
}
-
- @Override
- public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
- // only keep primary region's edits
- if (logKey.getTableName().equals(tableName) && info.getReplicaId() == 0) {
- // Presume type is a WALKeyImpl
- entries.add(new Entry((WALKeyImpl)logKey, logEdit));
- }
- }
}
@Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 57900cd..7d9edfc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.replication.regionserver;
import static org.junit.Assert.assertEquals;
-
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Collections;
@@ -54,11 +53,17 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HFileTestUtil;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@@ -69,14 +74,6 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
-
@Category({ReplicationTests.class, LargeTests.class})
public class TestReplicationSink {
@@ -119,9 +116,6 @@ public class TestReplicationSink {
protected static String hfileArchiveDir;
protected static String replicationClusterId;
- /**
- * @throws java.lang.Exception
- */
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().set("hbase.replication.source.fs.conf.provider",
@@ -136,18 +130,12 @@ public class TestReplicationSink {
replicationClusterId = "12345";
}
- /**
- * @throws java.lang.Exception
- */
@AfterClass
public static void tearDownAfterClass() throws Exception {
STOPPABLE.stop("Shutting down");
TEST_UTIL.shutdownMiniCluster();
}
- /**
- * @throws java.lang.Exception
- */
@Before
public void setUp() throws Exception {
table1 = TEST_UTIL.deleteTableData(TABLE_NAME1);
@@ -156,13 +144,12 @@ public class TestReplicationSink {
/**
* Insert a whole batch of entries
- * @throws Exception
*/
@Test
public void testBatchSink() throws Exception {
List<WALEntry> entries = new ArrayList<>(BATCH_SIZE);
List<Cell> cells = new ArrayList<>();
- for(int i = 0; i < BATCH_SIZE; i++) {
+ for (int i = 0; i < BATCH_SIZE; i++) {
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
@@ -174,7 +161,6 @@ public class TestReplicationSink {
/**
* Insert a mix of puts and deletes
- * @throws Exception
*/
@Test
public void testMixedPutDelete() throws Exception {
@@ -236,15 +222,13 @@ public class TestReplicationSink {
/**
* Insert to 2 different tables
- * @throws Exception
*/
@Test
public void testMixedPutTables() throws Exception {
List<WALEntry> entries = new ArrayList<>(BATCH_SIZE/2);
List<Cell> cells = new ArrayList<>();
- for(int i = 0; i < BATCH_SIZE; i++) {
- entries.add(createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1,
- i, KeyValue.Type.Put, cells));
+ for (int i = 0; i < BATCH_SIZE; i++) {
+ entries.add(createEntry(i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
@@ -262,7 +246,6 @@ public class TestReplicationSink {
/**
* Insert then do different types of deletes
- * @throws Exception
*/
@Test
public void testMixedDeletes() throws Exception {
@@ -290,7 +273,6 @@ public class TestReplicationSink {
/**
* Puts are buffered, but this tests when a delete (not-buffered) is applied
* before the actual Put that creates it.
- * @throws Exception
*/
@Test
public void testApplyDeleteBeforePut() throws Exception {
@@ -390,8 +372,8 @@ public class TestReplicationSink {
ProtobufUtil.toBulkLoadDescriptor(TABLE_NAME1,
UnsafeByteOperations.unsafeWrap(regionInfo.getEncodedNameAsBytes()),
storeFiles, storeFilesSize, 1);
- edit = org.apache.hadoop.hbase.wal.WALEdit.createBulkLoadEvent(regionInfo,
- loadDescriptor);
+ edit = org.apache.hadoop.hbase.wal.WALEdit.createBulkLoadEvent(
+ regionInfo.getNonEmptyStartKey(), loadDescriptor);
}
List<WALEntry> entries = new ArrayList<>(1);
@@ -400,12 +382,12 @@ public class TestReplicationSink {
// 5. Copy the hfile to the path as it is in reality
for (int i = 0; i < 25; i++) {
- String pathToHfileFromNS =
- new StringBuilder(100).append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR)
- .append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR)
- .append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray()))
- .append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR)
- .append(hfilePrefix + i).toString();
+ String pathToHfileFromNS = new StringBuilder(100).
+ append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR).
+ append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR).
+ append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray())).
+ append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR).
+ append(hfilePrefix + i).toString();
String dst = baseNamespaceDir + Path.SEPARATOR + pathToHfileFromNS;
Path dstPath = new Path(dst);
FileUtil.copy(fs, p.get(0), fs, dstPath, false, conf);
@@ -439,14 +421,11 @@ public class TestReplicationSink {
final long now = System.currentTimeMillis();
KeyValue kv = null;
if(type.getCode() == KeyValue.Type.Put.getCode()) {
- kv = new KeyValue(rowBytes, fam, fam, now,
- KeyValue.Type.Put, Bytes.toBytes(row));
+ kv = new KeyValue(rowBytes, fam, fam, now, KeyValue.Type.Put, Bytes.toBytes(row));
} else if (type.getCode() == KeyValue.Type.DeleteColumn.getCode()) {
- kv = new KeyValue(rowBytes, fam, fam,
- now, KeyValue.Type.DeleteColumn);
+ kv = new KeyValue(rowBytes, fam, fam, now, KeyValue.Type.DeleteColumn);
} else if (type.getCode() == KeyValue.Type.DeleteFamily.getCode()) {
- kv = new KeyValue(rowBytes, fam, null,
- now, KeyValue.Type.DeleteFamily);
+ kv = new KeyValue(rowBytes, fam, null, now, KeyValue.Type.DeleteFamily);
}
WALEntry.Builder builder = createWALEntryBuilder(table);
cells.add(kv);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 2745ad6..5e642e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -1,4 +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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
-
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.URLEncoder;
@@ -96,10 +95,8 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
@@ -291,7 +288,7 @@ public abstract class TestReplicationSourceManager {
wal.rollWriter();
}
LOG.info(Long.toString(i));
- final long txid = wal.appendData(hri,
+ final long txid = wal.appendData(
new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
edit);
wal.sync(txid);
@@ -305,7 +302,7 @@ public abstract class TestReplicationSourceManager {
LOG.info(baseline + " and " + time);
for (int i = 0; i < 3; i++) {
- wal.appendData(hri,
+ wal.appendData(
new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
edit);
}
@@ -323,7 +320,7 @@ public abstract class TestReplicationSourceManager {
manager.logPositionAndCleanOldLogs(manager.getSources().get(0),
new WALEntryBatch(0, manager.getSources().get(0).getCurrentPath()));
- wal.appendData(hri,
+ wal.appendData(
new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
edit);
wal.sync();
@@ -431,7 +428,7 @@ public abstract class TestReplicationSourceManager {
WALProtos.CompactionDescriptor.getDefaultInstance();
RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(HConstants.EMPTY_START_ROW)
.setEndKey(HConstants.EMPTY_END_ROW).build();
- WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
+ WALEdit edit = WALEdit.createCompaction(hri.getNonEmptyStartKey(), compactionDescriptor);
ReplicationSourceWALActionListener.scopeWALEdits(new WALKeyImpl(), edit, conf);
}
@@ -479,7 +476,6 @@ public abstract class TestReplicationSourceManager {
* corresponding ReplicationSourceInterface correctly cleans up the corresponding
* replication queue and ReplicationPeer.
* See HBASE-16096.
- * @throws Exception
*/
@Test
public void testPeerRemovalCleanup() throws Exception{
@@ -575,10 +571,7 @@ public abstract class TestReplicationSourceManager {
/**
* Add a peer and wait for it to initialize
- * @param peerId
- * @param peerConfig
* @param waitForSource Whether to wait for replication source to initialize
- * @throws Exception
*/
private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
final boolean waitForSource) throws Exception {
@@ -617,8 +610,6 @@ public abstract class TestReplicationSourceManager {
/**
* Remove a peer and wait for it to get cleaned up
- * @param peerId
- * @throws Exception
*/
private void removePeerAndWait(final String peerId) throws Exception {
final ReplicationPeers rp = manager.getReplicationPeers();
@@ -671,7 +662,7 @@ public abstract class TestReplicationSourceManager {
UnsafeByteOperations.unsafeWrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
// 3. create bulk load wal edit event
- WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
+ WALEdit logEdit = WALEdit.createBulkLoadEvent(hri.getNonEmptyStartKey(), desc);
return logEdit;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index ae9bb67..cbc04e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -1,4 +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
@@ -582,7 +582,7 @@ public class TestWALEntryStream {
}
private void appendToLog(String key) throws IOException {
- final long txid = log.appendData(info,
+ final long txid = log.appendData(
new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(),
mvcc, scopes), getWALEdit(key));
log.sync(txid);
@@ -606,7 +606,7 @@ public class TestWALEntryStream {
}
private long appendToLog(int count) throws IOException {
- return log.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+ return log.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), getWALEdits(count));
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index f58d364..549abaa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -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
@@ -20,11 +19,9 @@
package org.apache.hadoop.hbase.wal;
import java.io.IOException;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
import org.apache.yetus.audience.InterfaceAudience;
@@ -63,12 +60,12 @@ public class FaultyFSLog extends FSHLog {
}
@Override
- protected long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+ protected long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
throws IOException {
if (this.ft == FailureType.APPEND) {
throw new IOException("append");
}
- return super.append(info, key, edits, inMemstore);
+ return super.append(key, edits, inMemstore);
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
index 72e4998..a3da7fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
-
import java.io.IOException;
import java.util.HashSet;
import java.util.NavigableMap;
@@ -157,15 +156,14 @@ public class TestFSHLogProvider {
long timestamp = System.currentTimeMillis();
WALEdit cols = new WALEdit();
cols.add(new KeyValue(row, row, row, timestamp, row));
- log.appendData(hri,
- getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes), cols);
+ log.appendData(getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes),
+ cols);
}
log.sync();
}
/**
* used by TestDefaultWALProviderWithHLogKey
- * @param scopes
*/
WALKeyImpl getWalKey(final byte[] info, final TableName tableName, final long timestamp,
NavigableMap<byte[], Integer> scopes) {
@@ -174,8 +172,6 @@ public class TestFSHLogProvider {
/**
* helper method to simulate region flush for a WAL.
- * @param wal
- * @param regionEncodedName
*/
protected void flushRegion(WAL wal, byte[] regionEncodedName, Set<byte[]> flushedFamilyNames) {
wal.startCacheFlush(regionEncodedName, flushedFamilyNames);
@@ -261,7 +257,6 @@ public class TestFSHLogProvider {
* archiving if for all the regions which have entries in that wal file, have flushed - past
* their maximum sequence id in that wal file.
* <p>
- * @throws IOException
*/
@Test
public void testWALArchiving() throws IOException {
@@ -337,7 +332,6 @@ public class TestFSHLogProvider {
/**
* Write to a log file with three concurrent threads and verifying all data is written.
- * @throws Exception
*/
@Test
public void testConcurrentWrites() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
index d8f82d8..aee2a1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
@@ -129,7 +129,7 @@ public class TestSecureWAL {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
- wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
wal.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index f1ac464..d053f8b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -1,4 +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
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Method;
@@ -35,7 +34,6 @@ import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -180,7 +178,6 @@ public class TestWALFactory {
/**
* Just write multiple logs then split. Before fix for HADOOP-2283, this
* would fail.
- * @throws IOException
*/
@Test
public void testSplit() throws IOException {
@@ -219,7 +216,7 @@ public class TestWALFactory {
LOG.info("Region " + i + ": " + edit);
WALKeyImpl walKey = new WALKeyImpl(infos[i].getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes);
- log.appendData(infos[i], walKey, edit);
+ log.appendData(walKey, edit);
walKey.getWriteEntry();
}
log.sync();
@@ -227,11 +224,11 @@ public class TestWALFactory {
}
}
wals.shutdown();
- // The below calculation of logDir relies on insider information... WALSplitter should be connected better
- // with the WAL system.... not requiring explicit path. The oldLogDir is just made up not used.
- Path logDir =
- new Path(new Path(hbaseWALDir, HConstants.HREGION_LOGDIR_NAME),
- this.currentServername.toString());
+ // The below calculation of logDir relies on insider information... WALSplitter should be
+ // connected better with the WAL system.... not requiring explicit path. The oldLogDir is just
+ // made up not used.
+ Path logDir = new Path(new Path(hbaseWALDir, HConstants.HREGION_LOGDIR_NAME),
+ this.currentServername.toString());
Path oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
List<Path> splits = WALSplitter.split(hbaseWALDir, logDir, oldLogDir, fs, conf, wals);
verifySplits(splits, howmany);
@@ -239,7 +236,6 @@ public class TestWALFactory {
/**
* Test new HDFS-265 sync.
- * @throws Exception
*/
@Test
public void Broken_testSync() throws Exception {
@@ -281,7 +277,7 @@ public class TestWALFactory {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
- wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
// Now call sync and try reading. Opening a Reader before you sync just
@@ -292,7 +288,9 @@ public class TestWALFactory {
reader = wals.createReader(fs, walPath);
int count = 0;
WAL.Entry entry = new WAL.Entry();
- while ((entry = reader.next(entry)) != null) count++;
+ while ((entry = reader.next(entry)) != null) {
+ count++;
+ }
assertEquals(total, count);
reader.close();
// Add test that checks to see that an open of a Reader works on a file
@@ -300,20 +298,24 @@ public class TestWALFactory {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
- wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
wal.sync();
reader = wals.createReader(fs, walPath);
count = 0;
- while((entry = reader.next(entry)) != null) count++;
+ while((entry = reader.next(entry)) != null) {
+ count++;
+ }
assertTrue(count >= total);
reader.close();
// If I sync, should see double the edits.
wal.sync();
reader = wals.createReader(fs, walPath);
count = 0;
- while((entry = reader.next(entry)) != null) count++;
+ while((entry = reader.next(entry)) != null) {
+ count++;
+ }
assertEquals(total * 2, count);
reader.close();
// Now do a test that ensures stuff works when we go over block boundary,
@@ -322,14 +324,16 @@ public class TestWALFactory {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value));
- wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
// Now I should have written out lots of blocks. Sync then read.
wal.sync();
reader = wals.createReader(fs, walPath);
count = 0;
- while((entry = reader.next(entry)) != null) count++;
+ while((entry = reader.next(entry)) != null) {
+ count++;
+ }
assertEquals(total * 3, count);
reader.close();
// shutdown and ensure that Reader gets right length also.
@@ -340,12 +344,13 @@ public class TestWALFactory {
assertEquals(total * 3, count);
reader.close();
} finally {
- if (reader != null) reader.close();
+ if (reader != null) {
+ reader.close();
+ }
}
}
- private void verifySplits(final List<Path> splits, final int howmany)
- throws IOException {
+ private void verifySplits(final List<Path> splits, final int howmany) throws IOException {
assertEquals(howmany * howmany, splits.size());
for (int i = 0; i < splits.size(); i++) {
LOG.info("Verifying=" + splits.get(i));
@@ -399,12 +404,12 @@ public class TestWALFactory {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
- wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
// Now call sync to send the data to HDFS datanodes
wal.sync();
- int namenodePort = cluster.getNameNodePort();
+ int namenodePort = cluster.getNameNodePort();
final Path walPath = AbstractFSWALProvider.getCurrentFileName(wal);
@@ -482,8 +487,9 @@ public class TestWALFactory {
throw new Exception("Timed out waiting for WAL.recoverLog()");
}
- if (t.exception != null)
+ if (t.exception != null) {
throw t.exception;
+ }
// Make sure you can read all the content
WAL.Reader reader = wals.createReader(fs, walPath);
@@ -532,7 +538,7 @@ public class TestWALFactory {
.setEndKey(Bytes.toBytes(Bytes.toString(row) + "1")).build();
final WAL log = wals.getWAL(info);
- final long txid = log.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(),
+ final long txid = log.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(),
htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
log.sync(txid);
log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
@@ -545,7 +551,9 @@ public class TestWALFactory {
// entry in the below... thats why we have '1'.
for (int i = 0; i < 1; i++) {
WAL.Entry entry = reader.next(null);
- if (entry == null) break;
+ if (entry == null) {
+ break;
+ }
WALKey key = entry.getKey();
WALEdit val = entry.getEdit();
assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
@@ -588,7 +596,7 @@ public class TestWALFactory {
}
RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
final WAL log = wals.getWAL(hri);
- final long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+ final long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
log.sync(txid);
log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
@@ -619,7 +627,6 @@ public class TestWALFactory {
/**
* Test that we can visit entries before they are appended
- * @throws Exception
*/
@Test
public void testVisitors() throws Exception {
@@ -640,7 +647,7 @@ public class TestWALFactory {
cols.add(new KeyValue(row, Bytes.toBytes("column"),
Bytes.toBytes(Integer.toString(i)),
timestamp, new byte[]{(byte) (i + '0')}));
- log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
+ log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), cols);
}
log.sync();
@@ -650,7 +657,7 @@ public class TestWALFactory {
cols.add(new KeyValue(row, Bytes.toBytes("column"),
Bytes.toBytes(Integer.toString(11)),
timestamp, new byte[]{(byte) (11 + '0')}));
- log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
+ log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), cols);
log.sync();
assertEquals(COL_COUNT, visitor.increments);
@@ -669,12 +676,6 @@ public class TestWALFactory {
static class DumbWALActionsListener implements WALActionsListener {
int increments = 0;
-
- @Override
- public void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {
- increments++;
- }
-
@Override
public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
// To change body of implemented methods use File | Settings | File
@@ -762,7 +763,8 @@ public class TestWALFactory {
public void testReaderClosedOnBadCodec() throws IOException {
// Create our own Configuration and WALFactory to avoid breaking other test methods
Configuration confWithCodec = new Configuration(conf);
- confWithCodec.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, BrokenWALCellCodec.class, Codec.class);
+ confWithCodec.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, BrokenWALCellCodec.class,
+ Codec.class);
WALFactory customFactory = new WALFactory(confWithCodec, this.currentServername.toString());
// Hack a Proxy over the FileSystem so that we can track the InputStreams opened by
@@ -802,7 +804,7 @@ public class TestWALFactory {
cols.add(new KeyValue(row, Bytes.toBytes("column"),
Bytes.toBytes("0"), System.currentTimeMillis(), new byte[] { 0 }));
final WAL log = customFactory.getWAL(hri);
- final long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+ final long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
// Sync the edit to the WAL
log.sync(txid);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
index f36c5b9..02a800b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
@@ -117,7 +117,7 @@ public class TestWALReaderOnSecureWAL {
} else {
kvs.add(kv);
}
- wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+ wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
System.currentTimeMillis(), mvcc, scopes), kvs);
}
wal.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
index 017b540..e109d9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.wal;
import static org.junit.Assert.assertEquals;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@@ -98,8 +97,7 @@ public class TestWALRootDir {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
System.currentTimeMillis(), value));
- long txid =
- log.appendData(regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 0), edit);
+ long txid = log.appendData(getWalKey(System.currentTimeMillis(), regionInfo, 0), edit);
log.sync(txid);
assertEquals("Expect 1 log have been created", 1,
getWALFiles(walFs, walRootDir).size());
@@ -109,7 +107,7 @@ public class TestWALRootDir {
HConstants.HREGION_LOGDIR_NAME)).size());
edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
System.currentTimeMillis(), value));
- txid = log.appendData(regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 1), edit);
+ txid = log.appendData(getWalKey(System.currentTimeMillis(), regionInfo, 1), edit);
log.sync(txid);
log.rollWriter();
log.shutdown();
@@ -129,7 +127,9 @@ public class TestWALRootDir {
LOG.debug("Scanning " + dir.toString() + " for WAL files");
FileStatus[] files = fs.listStatus(dir);
- if (files == null) return Collections.emptyList();
+ if (files == null) {
+ return Collections.emptyList();
+ }
for (FileStatus file : files) {
if (file.isDirectory()) {
// recurse into sub directories
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 5f22b45..62e7cbf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.FaultyProtobufLogReader;
import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -224,7 +225,9 @@ public class TestWALSplit {
long startCount = counter.get();
zombie.start();
// Wait till writer starts going.
- while (startCount == counter.get()) Threads.sleep(1);
+ while (startCount == counter.get()) {
+ Threads.sleep(1);
+ }
// Give it a second to write a few appends.
Threads.sleep(1000);
final Configuration conf2 = HBaseConfiguration.create(conf);
@@ -679,7 +682,9 @@ public class TestWALSplit {
Reader in = wals.createReader(fs, splitLog[0]);
@SuppressWarnings("unused")
Entry entry;
- while ((entry = in.next()) != null) ++actualCount;
+ while ((entry = in.next()) != null) {
+ ++actualCount;
+ }
assertEquals(expectedCount, actualCount);
in.close();
@@ -815,7 +820,9 @@ public class TestWALSplit {
final Thread someOldThread = new Thread("Some-old-thread") {
@Override
public void run() {
- while(!stop.get()) Threads.sleep(10);
+ while(!stop.get()) {
+ Threads.sleep(10);
+ }
}
};
someOldThread.setDaemon(true);
@@ -909,8 +916,7 @@ public class TestWALSplit {
final AtomicInteger count = new AtomicInteger();
- CancelableProgressable localReporter
- = new CancelableProgressable() {
+ CancelableProgressable localReporter = new CancelableProgressable() {
@Override
public boolean progress() {
count.getAndIncrement();
@@ -1031,11 +1037,13 @@ public class TestWALSplit {
@Override
public Entry answer(InvocationOnMock invocation) throws Throwable {
- if (index >= numFakeEdits) return null;
+ if (index >= numFakeEdits) {
+ return null;
+ }
// Generate r0 through r4 in round robin fashion
int regionIdx = index % regions.size();
- byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
+ byte [] region = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
Entry ret = createTestEntry(TABLE_NAME, region,
Bytes.toBytes(index / regions.size()),
@@ -1193,7 +1201,8 @@ public class TestWALSplit {
* @param leaveOpen index to leave un-closed. -1 to close all.
* @return the writer that's still open, or null if all were closed.
*/
- private Writer generateWALs(int writers, int entries, int leaveOpen, int regionEvents) throws IOException {
+ private Writer generateWALs(int writers, int entries, int leaveOpen, int regionEvents)
+ throws IOException {
makeRegionDirs(REGIONS);
fs.mkdirs(WALDIR);
Writer [] ws = new Writer[writers];
@@ -1348,7 +1357,7 @@ public class TestWALSplit {
.addAllCompactionInput(Arrays.asList(inputs))
.addCompactionOutput(output);
- WALEdit edit = WALEdit.createCompaction(hri, desc.build());
+ WALEdit edit = WALEdit.createCompaction(WALUtil.getRowForRegion(hri), desc.build());
WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
w.append(new Entry(key, edit));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index 602367c..e3cf7b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.wal;
import static com.codahale.metrics.MetricRegistry.name;
-
import com.codahale.metrics.ConsoleReporter;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
@@ -183,7 +182,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
RegionInfo hri = region.getRegionInfo();
final WALKeyImpl logkey =
new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes);
- wal.appendData(hri, logkey, walEdit);
+ wal.appendData(logkey, walEdit);
if (!this.noSync) {
if (++lastSync >= this.syncInterval) {
wal.sync();
@@ -408,9 +407,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
* Verify the content of the WAL file.
* Verify that the file has expected number of edits.
* @param wals may not be null
- * @param wal
* @return Count of edits.
- * @throws IOException
*/
private long verify(final WALFactory wals, final Path wal, final boolean verbose)
throws IOException {
@@ -574,9 +571,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
/**
* The guts of the {@link #main} method.
* Call this method to avoid the {@link #main(String[])} System.exit.
- * @param args
* @return errCode
- * @throws Exception
*/
static int innerMain(final Configuration c, final String [] args) throws Exception {
return ToolRunner.run(c, new WALPerformanceEvaluation(), args);
diff --git a/hbase-coprocessor/pom.xml b/hbase-wal/pom.xml
similarity index 85%
copy from hbase-coprocessor/pom.xml
copy to hbase-wal/pom.xml
index 0bcfcef..9e1e3bb 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-wal/pom.xml
@@ -29,9 +29,9 @@
<relativePath>../hbase-build-configuration</relativePath>
</parent>
- <artifactId>hbase-coprocessor</artifactId>
- <name>Apache HBase - Coprocessor</name>
- <description>Base Coprocessor Types</description>
+ <artifactId>hbase-wal</artifactId>
+ <name>Apache HBase - WAL</name>
+ <description>HBase Write-ahead Log</description>
<build>
<testResources>
@@ -98,5 +98,17 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-protocol-shaded</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-coprocessor</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
index 1deddf9..eaecee7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
@@ -1,4 +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
@@ -18,13 +18,12 @@
package org.apache.hadoop.hbase.coprocessor;
+import java.util.Optional;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import java.util.Optional;
-
/**
* WALCoprocessor don't support loading services using {@link #getServices()}.
*/
@@ -32,5 +31,4 @@ import java.util.Optional;
@InterfaceStability.Evolving
public interface WALCoprocessor extends Coprocessor {
Optional<WALObserver> getWALObserver();
-
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
index 71c72a2..51a3e96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
@@ -19,12 +19,12 @@
package org.apache.hadoop.hbase.coprocessor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
similarity index 73%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
index b2fa7ca..99bb20e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
@@ -20,12 +20,9 @@
package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
-
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@@ -36,7 +33,7 @@ import org.apache.yetus.audience.InterfaceStability;
* Note that implementers of WALObserver will not see WALEdits that report themselves
* as empty via {@link WALEdit#isEmpty()}.
*
- * {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} provides
+ * #org.apache.hadoop.hbase.coprocessor.RegionObserver provides
* hooks for adding logic for WALEdits in the region context during reconstruction.
*
* Defines coprocessor hooks for interacting with operations on the
@@ -67,30 +64,6 @@ import org.apache.yetus.audience.InterfaceStability;
@InterfaceStability.Evolving
public interface WALObserver {
/**
- * Called before a {@link WALEdit}
- * is writen to WAL.
- * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
- * damage.
- * @deprecated Since hbase-2.0.0. To be replaced with an alternative that does not expose
- * InterfaceAudience classes such as WALKey and WALEdit. Will be removed in hbase-3.0.0.
- */
- @Deprecated
- default void preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
-
- /**
- * Called after a {@link WALEdit}
- * is writen to WAL.
- * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
- * damage.
- * @deprecated Since hbase-2.0.0. To be replaced with an alternative that does not expose
- * InterfaceAudience classes such as WALKey and WALEdit. Will be removed in hbase-3.0.0.
- */
- @Deprecated
- default void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
- RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
-
- /**
* Called before rolling the current WAL
* @param oldPath the path of the current wal that we are replacing
* @param newPath the path of the wal we are going to create
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
index 16866e1..f0db427 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
@@ -1,4 +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
@@ -15,18 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase.regionserver.wal;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.EnumMap;
import java.util.Map;
-
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.TagCompressionContext;
import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Context that holds the various dictionaries for compression in WAL.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
similarity index 94%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
index a595436..e88cfe2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
@@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
/**
* Thrown when we fail close of the write-ahead-log file.
- * Package private. Only used inside this package.
+ * Package private.
*/
@InterfaceAudience.Public
public class FailedLogCloseException extends IOException {
@@ -33,18 +32,12 @@ public class FailedLogCloseException extends IOException {
public FailedLogCloseException() {
super();
}
-
- /**
- * @param msg
- */
public FailedLogCloseException(String msg) {
super(msg);
}
-
public FailedLogCloseException(final String msg, final Throwable t) {
super(msg, t);
}
-
public FailedLogCloseException(final Throwable t) {
super(t);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
similarity index 91%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
index 7fba7df..b98dc09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.yetus.audience.InterfaceAudience;
@@ -33,7 +32,7 @@ import org.apache.yetus.audience.InterfaceAudience;
public interface WALActionsListener {
/** The reason for the log roll request. */
- static enum RollRequestReason {
+ enum RollRequestReason {
/** The length of the log exceeds the roll size threshold. */
SIZE,
/** Too few replicas in the writer pipeline. */
@@ -42,7 +41,7 @@ public interface WALActionsListener {
SLOW_SYNC,
/** I/O or other error. */
ERROR
- };
+ }
/**
* The WAL is going to be rolled. The oldPath can be null if this is
@@ -84,15 +83,14 @@ public interface WALActionsListener {
*/
default void logCloseRequested() {}
- /**
- * Called before each write.
- */
- default void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {}
+ // /**
+ // * Called before each write.
+ // */
+ // default void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {}
/**
- * @param logKey
* @param logEdit TODO: Retire this in favor of
- * {@link #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit)} It only exists to get
+ * #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit) It only exists to get
* scope when replicating. Scope should be in the WALKey and not need us passing in a
* <code>htd</code>.
* @throws IOException If failed to parse the WALEdit
@@ -114,8 +112,7 @@ public interface WALActionsListener {
/**
* For notification post writer sync. Used by metrics system at least.
* @param timeInNanos How long the filesystem sync took in nanoseconds.
- * @param handlerSyncs How many sync handler calls were released by this call to filesystem
- * sync.
+ * @param handlerSyncs How many sync handler calls were released by this call to filesystem sync.
*/
default void postSync(final long timeInNanos, final int handlerSyncs) {}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
similarity index 83%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
index 12e9448..83d8074 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
@@ -1,6 +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
@@ -22,10 +20,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
@@ -45,18 +41,16 @@ import org.slf4j.LoggerFactory;
* loaded within a {@link WAL}.
*/
@InterfaceAudience.Private
-public class WALCoprocessorHost
- extends CoprocessorHost<WALCoprocessor, WALCoprocessorEnvironment> {
+public class WALCoprocessorHost extends
+ CoprocessorHost<WALCoprocessor, WALCoprocessorEnvironment> {
private static final Logger LOG = LoggerFactory.getLogger(WALCoprocessorHost.class);
/**
* Encapsulation of the environment of each coprocessor
*/
- static class WALEnvironment extends BaseEnvironment<WALCoprocessor>
- implements WALCoprocessorEnvironment {
-
+ static class WALEnvironment extends BaseEnvironment<WALCoprocessor> implements
+ WALCoprocessorEnvironment {
private final WAL wal;
-
private final MetricRegistry metricRegistry;
@Override
@@ -113,7 +107,7 @@ public class WALCoprocessorHost
super(null);
this.wal = log;
// load system default cp's from configuration.
- loadSystemCoprocessors(conf, WAL_COPROCESSOR_CONF_KEY);
+ loadSystemCoprocessors(conf, CoprocessorHost.WAL_COPROCESSOR_CONF_KEY);
}
@Override
@@ -139,37 +133,24 @@ public class WALCoprocessorHost
}
private ObserverGetter<WALCoprocessor, WALObserver> walObserverGetter =
- WALCoprocessor::getWALObserver;
+ WALCoprocessor::getWALObserver;
- abstract class WALObserverOperation extends
- ObserverOperationWithoutResult<WALObserver> {
+ abstract class WALObserverOperation extends ObserverOperationWithoutResult<WALObserver> {
public WALObserverOperation() {
super(walObserverGetter);
}
}
- public void preWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
- throws IOException {
+ public void preWALWrite(final WALKey logKey, final WALEdit logEdit) throws IOException {
// Not bypassable.
if (this.coprocEnvironments.isEmpty()) {
return;
}
- execOperation(new WALObserverOperation() {
- @Override
- public void call(WALObserver oserver) throws IOException {
- oserver.preWALWrite(this, info, logKey, logEdit);
- }
- });
+ // TODO: We used to call WALObserver methods but they have been removed. Call CP instead?
}
- public void postWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
- throws IOException {
- execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {
- @Override
- protected void call(WALObserver observer) throws IOException {
- observer.postWALWrite(this, info, logKey, logEdit);
- }
- });
+ public void postWALWrite(final WALKey logKey, final WALEdit logEdit) throws IOException {
+ // TODO: We used to call WALObserver methods but they have been removed. Call CP instead?
}
/**
@@ -178,7 +159,7 @@ public class WALCoprocessorHost
* @param newPath the path of the wal we are going to create
*/
public void preWALRoll(Path oldPath, Path newPath) throws IOException {
- execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {
+ execOperation(coprocEnvironments.isEmpty()? null : new WALObserverOperation() {
@Override
protected void call(WALObserver observer) throws IOException {
observer.preWALRoll(this, oldPath, newPath);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
index c60faa9..da7c8ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.replication.regionserver;
import java.util.OptionalLong;
-
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
@@ -32,6 +31,5 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
@FunctionalInterface
public interface WALFileLengthProvider {
-
OptionalLong getLogFileSizeIfBeingWritten(Path path);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
similarity index 89%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 20e45de..5bff009 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -17,22 +17,18 @@
*/
package org.apache.hadoop.hbase.wal;
+import static org.apache.commons.lang3.StringUtils.isNumeric;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
-import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-
-import static org.apache.commons.lang3.StringUtils.isNumeric;
/**
* A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
@@ -42,7 +38,6 @@ import static org.apache.commons.lang3.StringUtils.isNumeric;
* WAL.equals to determine if they have already seen a given WAL.
*/
@InterfaceAudience.Private
-@InterfaceStability.Evolving
public interface WAL extends Closeable, WALFileLengthProvider {
/**
@@ -64,9 +59,9 @@ public interface WAL extends Closeable, WALFileLengthProvider {
*
* @return If lots of logs, flush the stores of returned regions so next time through we
* can clean logs. Returns null if nothing to flush. Names are actual
- * region names as returned by {@link RegionInfo#getEncodedName()}
+ * region names as returned by RegionInfo#getEncodedName()
*/
- Map<byte[], List<byte[]>> rollWriter() throws FailedLogCloseException, IOException;
+ Map<byte[], List<byte[]>> rollWriter() throws IOException;
/**
* Roll the log writer. That is, start writing log messages to a new file.
@@ -80,7 +75,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* been written to the current writer
* @return If lots of logs, flush the stores of returned regions so next time through we
* can clean logs. Returns null if nothing to flush. Names are actual
- * region names as returned by {@link RegionInfo#getEncodedName()}
+ * region names as returned by RegionInfo#getEncodedName()
*/
Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException;
@@ -105,35 +100,32 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* The WAL is not flushed/sync'd after this transaction completes BUT on return this edit must
* have its region edit/sequence id assigned else it messes up our unification of mvcc and
* sequenceid. On return <code>key</code> will have the region edit/sequence id filled in.
- * @param info the regioninfo associated with append
* @param key Modified by this call; we add to it this edits region edit/sequence id.
* @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
* sequence id that is after all currently appended edits.
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
* in it.
- * @see #appendMarker(RegionInfo, WALKeyImpl, WALEdit)
+ * @see #appendMarker(WALKeyImpl, WALEdit)
*/
- long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
+ long appendData(WALKeyImpl key, WALEdit edits) throws IOException;
/**
* Append an operational 'meta' event marker edit to the WAL. A marker meta edit could
* be a FlushDescriptor, a compaction marker, or a region event marker; e.g. region open
* or region close. The difference between a 'marker' append and a 'data' append as in
- * {@link #appendData(RegionInfo, WALKeyImpl, WALEdit)}is that a marker will not have
+ * {@link #appendData(WALKeyImpl, WALEdit)}is that a marker will not have
* transitioned through the memstore.
* <p/>
* The WAL is not flushed/sync'd after this transaction completes BUT on return this edit must
* have its region edit/sequence id assigned else it messes up our unification of mvcc and
* sequenceid. On return <code>key</code> will have the region edit/sequence id filled in.
- * @param info the regioninfo associated with append
* @param key Modified by this call; we add to it this edits region edit/sequence id.
* @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
* sequence id that is after all currently appended edits.
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
* in it.
- * @see #appendData(RegionInfo, WALKeyImpl, WALEdit)
*/
- long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
+ long appendMarker(WALKeyImpl key, WALEdit edits) throws IOException;
/**
* updates the seuence number of a specific store.
@@ -181,10 +173,10 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* concurrent appends while we set up cache flush.
* @param families Families to flush. May be a subset of all families in the region.
* @return Returns {@link HConstants#NO_SEQNUM} if we are flushing the whole region OR if
- * we are flushing a subset of all families but there are no edits in those families not
- * being flushed; in other words, this is effectively same as a flush of all of the region
- * though we were passed a subset of regions. Otherwise, it returns the sequence id of the
- * oldest/lowest outstanding edit.
+ * we are flushing a subset of all families but there are no edits in those families not
+ * being flushed; in other words, this is effectively same as a flush of all of the region
+ * though we were passed a subset of regions. Otherwise, it returns the sequence id of the
+ * oldest/lowest outstanding edit.
* @see #completeCacheFlush(byte[], long)
* @see #abortCacheFlush(byte[])
*/
@@ -220,7 +212,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* @param encodedRegionName The region to get the number for.
* @return The earliest/lowest/oldest sequence id if present, HConstants.NO_SEQNUM if absent.
* @deprecated Since version 1.2.0. Removing because not used and exposes subtle internal
- * workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])}
+ * workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])}
*/
@Deprecated
long getEarliestMemStoreSeqNum(byte[] encodedRegionName);
@@ -324,7 +316,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
* @param name Name of the WAL file.
* @return Timestamp or -1.
*/
- public static long getTimestamp(String name) {
+ static long getTimestamp(String name) {
String [] splits = name.split("\\.");
if (splits.length <= 1) {
return -1;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
similarity index 92%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index cefc3a9..56ddf03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -23,20 +23,17 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
-
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
-
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
@@ -61,9 +58,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
* normal processing so would make sense to do this. Current system is an awkward marking of Cell
* columnfamily as {@link #METAFAMILY} and then setting qualifier based off meta edit type. For
* replay-time where we read Cell-at-a-time, there are utility methods below for figuring
- * meta type. See also
- * {@link #createBulkLoadEvent(RegionInfo, WALProtos.BulkLoadDescriptor)}, etc., for where we
- * create meta WALEdit instances.</p>
+ * meta type. See also #createBulkLoadEvent(RegionInfo, WALProtos.BulkLoadDescriptor), etc., for
+ * where we create meta WALEdit instances.</p>
*
* <p>WALEdit will accumulate a Set of all column family names referenced by the Cells
* {@link #add(Cell)}'d. This is an optimization. Usually when loading a WALEdit, we have the
@@ -308,8 +304,10 @@ public class WALEdit implements HeapSize {
return sb.toString();
}
- public static WALEdit createFlushWALEdit(RegionInfo hri, FlushDescriptor f) {
- KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
+ // Changed first param from RegionInfo to byte [] hbase3. Should be fine in a LimitedPrivate class
+ // going to a major version where this method is only called from a private util method.
+ public static WALEdit createFlushWALEdit(byte [] regionRow, FlushDescriptor f) {
+ KeyValue kv = new KeyValue(regionRow, METAFAMILY, FLUSH,
EnvironmentEdgeManager.currentTime(), f.toByteArray());
return new WALEdit().add(kv, METAFAMILY);
}
@@ -326,11 +324,6 @@ public class WALEdit implements HeapSize {
* {@link #REGION_EVENT_PREFIX} + {@link RegionEventDescriptor#getEventType()};
* for example HBASE::REGION_EVENT::REGION_CLOSE.
*/
- public static WALEdit createRegionEventWALEdit(RegionInfo hri,
- RegionEventDescriptor regionEventDesc) {
- return createRegionEventWALEdit(getRowForRegion(hri), regionEventDesc);
- }
-
@InterfaceAudience.Private
public static WALEdit createRegionEventWALEdit(byte [] rowForRegion,
RegionEventDescriptor regionEventDesc) {
@@ -371,23 +364,13 @@ public class WALEdit implements HeapSize {
/**
* @return A Marker WALEdit that has <code>c</code> serialized as its value
*/
- public static WALEdit createCompaction(final RegionInfo hri, final CompactionDescriptor c) {
+ public static WALEdit createCompaction(byte [] regionRow, final CompactionDescriptor c) {
byte [] pbbytes = c.toByteArray();
- KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
+ KeyValue kv = new KeyValue(regionRow, METAFAMILY, COMPACTION,
EnvironmentEdgeManager.currentTime(), pbbytes);
return new WALEdit().add(kv, METAFAMILY); //replication scope null so this won't be replicated
}
- public static byte[] getRowForRegion(RegionInfo hri) {
- byte[] startKey = hri.getStartKey();
- if (startKey.length == 0) {
- // empty row key is not allowed in mutations because it is both the start key and the end key
- // we return the smallest byte[] that is bigger (in lex comparison) than byte[0].
- return new byte[] {0};
- }
- return startKey;
- }
-
/**
* Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
* @param kv the key value
@@ -409,13 +392,14 @@ public class WALEdit implements HeapSize {
/**
* Create a bulk loader WALEdit
*
- * @param hri The RegionInfo for the region in which we are bulk loading
* @param bulkLoadDescriptor The descriptor for the Bulk Loader
* @return The WALEdit for the BulkLoad
*/
- public static WALEdit createBulkLoadEvent(RegionInfo hri,
+ // Changed first param from RegionInfo to byte [] hbase3. Should be fine in a LimitedPrivate class
+ // going to a major version where this method is only called from a private util method.
+ public static WALEdit createBulkLoadEvent(byte [] regionRow,
WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
- KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, BULK_LOAD,
+ KeyValue kv = new KeyValue(regionRow, METAFAMILY, BULK_LOAD,
EnvironmentEdgeManager.currentTime(), bulkLoadDescriptor.toByteArray());
return new WALEdit().add(kv, METAFAMILY);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index fdbacbd..bbfeb8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -17,19 +17,18 @@
*/
package org.apache.hadoop.hbase.wal;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.SequenceId;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.yetus.audience.InterfaceAudience;
-
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.SequenceId;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index b2a9956..d0dcfd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
@@ -1,4 +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
@@ -34,14 +34,12 @@ import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.ScopeType;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Default implementation of Key for an Entry in the WAL.
@@ -278,8 +276,6 @@ public class WALKeyImpl implements WALKey {
* @param tablename the tablename
* @param now Time at which this edit was written.
* @param clusterIds the clusters that have consumed the change(used in Replication)
- * @param nonceGroup
- * @param nonce
* @param mvcc mvcc control used to generate sequence numbers and control read/write points
*/
public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename,
@@ -296,7 +292,6 @@ public class WALKeyImpl implements WALKey {
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
- * @param tablename
* @param now Time at which this edit was written.
* @param clusterIds the clusters that have consumed the change(used in Replication)
* @param nonceGroup the nonceGroup
@@ -319,10 +314,6 @@ public class WALKeyImpl implements WALKey {
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
- * @param tablename
- * @param logSeqNum
- * @param nonceGroup
- * @param nonce
*/
// TODO: Fix being able to pass in sequenceid.
public WALKeyImpl(final byte[] encodedRegionName,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java b/hbase-wal/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
similarity index 100%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
rename to hbase-wal/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
diff --git a/pom.xml b/pom.xml
index 465cdb2..5fbc411 100755
--- a/pom.xml
+++ b/pom.xml
@@ -93,6 +93,7 @@
<module>hbase-asyncfs</module>
<module>hbase-logging</module>
<module>hbase-coprocessor</module>
+ <module>hbase-wal</module>
</modules>
<scm>
<connection>scm:git:git://gitbox.apache.org/repos/asf/hbase.git</connection>
@@ -1642,6 +1643,18 @@
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-wal</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-wal</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
<artifactId>hbase-protocol-shaded</artifactId>
<version>${project.version}</version>
</dependency>