HBASE-10378 Refactor write-ahead-log implementation
Incompatible changes called out in release notes on jira. * Cleaned up references to HLog * Deprecates HLogKey but maintains it for compatibility - Moves all Writeable from WALKey to HLogKey * Adds utility code to CoprocessorHost to help with evolving Coprocessor APIs * RSRpcServices roll WAL call now requests the non-meta LogRoller roll all logs - rolls actually happen asynchronously - deprecated old api (and noted incompatible behavior change) - modified api in new Admin interface to reflect lack of return values. * Moved WAL user facing API to "WAL" - only 1 sync offered - WALTrailer removed from API * make provider used by the WALFactory configurable. * Move all WAL requests to use opaque ids instead of paths * WALProvider provides API details for implementers and handles creation of WALs. * Refactor WALActionsListener to have a basic implementation. * turn MetricsWAL into a WALActionsListener. * tests that needs FSHLog implementation details use them directly, others just reference provider + factory - Some tests moved from Large to Medium based on run time. * pull out wal disabling into its own no-op class * update region open to delegate to WALFactory * update performance test tool to allow for multiple regions * Removed references to meta-specific wals within wal code - replaced with generic suffixes - WALFactory maintains a dedicated WALProvider for meta (and so knows about the distinction) * maintain backwards compat on HLogPrettyPrinter and mark it deprecated. - made WALPrettyPrinter IA.Private in favor of `bin/hbase wal` * move WALUtil stuff that's implementation specific to said implementation - WALUtil now acts as an integration point between the RegionServer and hte WAL code. Incorporates contributions from v.himanshu. Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
b97c3daf90
commit
f5e05eb836
@ -2617,37 +2617,60 @@ public class HBaseAdmin implements Admin {
|
||||
return getTableDescriptorsByTableName(tableNames);
|
||||
}
|
||||
|
||||
private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
|
||||
FailedLogCloseException {
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
|
||||
try {
|
||||
return admin.rollWALWriter(null, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
* Roll the log writer. I.e. when using a file system based write ahead log,
|
||||
* start writing log messages to a new file.
|
||||
*
|
||||
* Note that when talking to a version 1.0+ HBase deployment, the rolling is asynchronous.
|
||||
* This method will return as soon as the roll is requested and the return value will
|
||||
* always be null. Additionally, the named region server may schedule store flushes at the
|
||||
* request of the wal handling the roll request.
|
||||
*
|
||||
* When talking to a 0.98 or older HBase deployment, the rolling is synchronous and the
|
||||
* return value may be either null or a list of encoded region names.
|
||||
*
|
||||
* @param serverName
|
||||
* The servername of the regionserver. A server name is made of host,
|
||||
* port and startcode. This is mandatory. Here is an example:
|
||||
* <code> host187.example.com,60020,1289493121758</code>
|
||||
* @return If lots of logs, flush the 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 HRegionInfo#getEncodedName()}
|
||||
* @return a set of {@link HRegionInfo#getEncodedName()} that would allow the wal to
|
||||
* clean up some underlying files. null if there's nothing to flush.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @throws FailedLogCloseException
|
||||
* @deprecated use {@link #rollWALWriter(ServerName)}
|
||||
*/
|
||||
@Override
|
||||
public synchronized byte[][] rollHLogWriter(String serverName)
|
||||
@Deprecated
|
||||
public synchronized byte[][] rollHLogWriter(String serverName)
|
||||
throws IOException, FailedLogCloseException {
|
||||
ServerName sn = ServerName.valueOf(serverName);
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
|
||||
try {
|
||||
RollWALWriterResponse response = admin.rollWALWriter(null, request);
|
||||
int regionCount = response.getRegionToFlushCount();
|
||||
byte[][] regionsToFlush = new byte[regionCount][];
|
||||
for (int i = 0; i < regionCount; i++) {
|
||||
ByteString region = response.getRegionToFlush(i);
|
||||
regionsToFlush[i] = region.toByteArray();
|
||||
}
|
||||
return regionsToFlush;
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
final RollWALWriterResponse response = rollWALWriterImpl(sn);
|
||||
int regionCount = response.getRegionToFlushCount();
|
||||
if (0 == regionCount) {
|
||||
return null;
|
||||
}
|
||||
byte[][] regionsToFlush = new byte[regionCount][];
|
||||
for (int i = 0; i < regionCount; i++) {
|
||||
ByteString region = response.getRegionToFlush(i);
|
||||
regionsToFlush[i] = region.toByteArray();
|
||||
}
|
||||
return regionsToFlush;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void rollWALWriter(ServerName serverName)
|
||||
throws IOException, FailedLogCloseException {
|
||||
rollWALWriterImpl(serverName);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMetricsWALSource {
|
||||
|
||||
@Test(expected=RuntimeException.class)
|
||||
public void testGetInstanceNoHadoopCompat() throws Exception {
|
||||
//This should throw an exception because there is no compat lib on the class path.
|
||||
CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
|
||||
|
||||
}
|
||||
}
|
@ -0,0 +1,69 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.io;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* WALLink describes a link to a WAL.
|
||||
*
|
||||
* An wal can be in /hbase/.logs/<server>/<wal>
|
||||
* or it can be in /hbase/.oldlogs/<wal>
|
||||
*
|
||||
* The link checks first in the original path,
|
||||
* if it is not present it fallbacks to the archived path.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALLink extends FileLink {
|
||||
/**
|
||||
* @param conf {@link Configuration} from which to extract specific archive locations
|
||||
* @param serverName Region Server owner of the log
|
||||
* @param logName WAL file name
|
||||
* @throws IOException on unexpected error.
|
||||
*/
|
||||
public WALLink(final Configuration conf,
|
||||
final String serverName, final String logName) throws IOException {
|
||||
this(FSUtils.getRootDir(conf), serverName, logName);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param rootDir Path to the root directory where hbase files are stored
|
||||
* @param serverName Region Server owner of the log
|
||||
* @param logName WAL file name
|
||||
*/
|
||||
public WALLink(final Path rootDir, final String serverName, final String logName) {
|
||||
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
final Path logDir = new Path(new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), serverName);
|
||||
setLocations(new Path(logDir, logName), new Path(oldLogDir, logName));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param originPath Path to the wal in the log directory
|
||||
* @param archivePath Path to the wal in the archived log directory
|
||||
*/
|
||||
public WALLink(final Path originPath, final Path archivePath) {
|
||||
setLocations(originPath, archivePath);
|
||||
}
|
||||
}
|
@ -0,0 +1,287 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Reader;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapreduce.InputFormat;
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.RecordReader;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
|
||||
/**
|
||||
* Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} files.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
|
||||
private static final Log LOG = LogFactory.getLog(WALInputFormat.class);
|
||||
|
||||
public static final String START_TIME_KEY = "wal.start.time";
|
||||
public static final String END_TIME_KEY = "wal.end.time";
|
||||
|
||||
/**
|
||||
* {@link InputSplit} for {@link WAL} files. Each split represent
|
||||
* exactly one log file.
|
||||
*/
|
||||
static class WALSplit extends InputSplit implements Writable {
|
||||
private String logFileName;
|
||||
private long fileSize;
|
||||
private long startTime;
|
||||
private long endTime;
|
||||
|
||||
/** for serialization */
|
||||
public WALSplit() {}
|
||||
|
||||
/**
|
||||
* Represent an WALSplit, i.e. a single WAL file.
|
||||
* Start- and EndTime are managed by the split, so that WAL files can be
|
||||
* filtered before WALEdits are passed to the mapper(s).
|
||||
* @param logFileName
|
||||
* @param fileSize
|
||||
* @param startTime
|
||||
* @param endTime
|
||||
*/
|
||||
public WALSplit(String logFileName, long fileSize, long startTime, long endTime) {
|
||||
this.logFileName = logFileName;
|
||||
this.fileSize = fileSize;
|
||||
this.startTime = startTime;
|
||||
this.endTime = endTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength() throws IOException, InterruptedException {
|
||||
return fileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getLocations() throws IOException, InterruptedException {
|
||||
// TODO: Find the data node with the most blocks for this WAL?
|
||||
return new String[] {};
|
||||
}
|
||||
|
||||
public String getLogFileName() {
|
||||
return logFileName;
|
||||
}
|
||||
|
||||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
|
||||
public long getEndTime() {
|
||||
return endTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
logFileName = in.readUTF();
|
||||
fileSize = in.readLong();
|
||||
startTime = in.readLong();
|
||||
endTime = in.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeUTF(logFileName);
|
||||
out.writeLong(fileSize);
|
||||
out.writeLong(startTime);
|
||||
out.writeLong(endTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return logFileName + " (" + startTime + ":" + endTime + ") length:" + fileSize;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link RecordReader} for an {@link WAL} file.
|
||||
* Implementation shared with deprecated HLogInputFormat.
|
||||
*/
|
||||
static abstract class WALRecordReader<K extends WALKey> extends RecordReader<K, WALEdit> {
|
||||
private Reader reader = null;
|
||||
// visible until we can remove the deprecated HLogInputFormat
|
||||
Entry currentEntry = new Entry();
|
||||
private long startTime;
|
||||
private long endTime;
|
||||
|
||||
@Override
|
||||
public void initialize(InputSplit split, TaskAttemptContext context)
|
||||
throws IOException, InterruptedException {
|
||||
WALSplit hsplit = (WALSplit)split;
|
||||
Path logFile = new Path(hsplit.getLogFileName());
|
||||
Configuration conf = context.getConfiguration();
|
||||
LOG.info("Opening reader for "+split);
|
||||
try {
|
||||
this.reader = WALFactory.createReader(logFile.getFileSystem(conf), logFile, conf);
|
||||
} catch (EOFException x) {
|
||||
LOG.info("Ignoring corrupted WAL file: " + logFile
|
||||
+ " (This is normal when a RegionServer crashed.)");
|
||||
this.reader = null;
|
||||
}
|
||||
this.startTime = hsplit.getStartTime();
|
||||
this.endTime = hsplit.getEndTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nextKeyValue() throws IOException, InterruptedException {
|
||||
if (reader == null) return false;
|
||||
|
||||
Entry temp;
|
||||
long i = -1;
|
||||
do {
|
||||
// skip older entries
|
||||
try {
|
||||
temp = reader.next(currentEntry);
|
||||
i++;
|
||||
} catch (EOFException x) {
|
||||
LOG.info("Corrupted entry detected. Ignoring the rest of the file."
|
||||
+ " (This is normal when a RegionServer crashed.)");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
while(temp != null && temp.getKey().getWriteTime() < startTime);
|
||||
|
||||
if (temp == null) {
|
||||
if (i > 0) LOG.info("Skipped " + i + " entries.");
|
||||
LOG.info("Reached end of file.");
|
||||
return false;
|
||||
} else if (i > 0) {
|
||||
LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + ".");
|
||||
}
|
||||
boolean res = temp.getKey().getWriteTime() <= endTime;
|
||||
if (!res) {
|
||||
LOG.info("Reached ts: " + temp.getKey().getWriteTime() + " ignoring the rest of the file.");
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WALEdit getCurrentValue() throws IOException, InterruptedException {
|
||||
return currentEntry.getEdit();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException, InterruptedException {
|
||||
// N/A depends on total number of entries, which is unknown
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
LOG.info("Closing reader");
|
||||
if (reader != null) this.reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* handler for non-deprecated WALKey version. fold into WALRecordReader once we no longer
|
||||
* need to support HLogInputFormat.
|
||||
*/
|
||||
static class WALKeyRecordReader extends WALRecordReader<WALKey> {
|
||||
@Override
|
||||
public WALKey getCurrentKey() throws IOException, InterruptedException {
|
||||
return currentEntry.getKey();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<InputSplit> getSplits(JobContext context) throws IOException,
|
||||
InterruptedException {
|
||||
return getSplits(context, START_TIME_KEY, END_TIME_KEY);
|
||||
}
|
||||
|
||||
/**
|
||||
* implementation shared with deprecated HLogInputFormat
|
||||
*/
|
||||
List<InputSplit> getSplits(final JobContext context, final String startKey, final String endKey)
|
||||
throws IOException, InterruptedException {
|
||||
Configuration conf = context.getConfiguration();
|
||||
Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir"));
|
||||
|
||||
long startTime = conf.getLong(startKey, Long.MIN_VALUE);
|
||||
long endTime = conf.getLong(endKey, Long.MAX_VALUE);
|
||||
|
||||
FileSystem fs = inputDir.getFileSystem(conf);
|
||||
List<FileStatus> files = getFiles(fs, inputDir, startTime, endTime);
|
||||
|
||||
List<InputSplit> splits = new ArrayList<InputSplit>(files.size());
|
||||
for (FileStatus file : files) {
|
||||
splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime));
|
||||
}
|
||||
return splits;
|
||||
}
|
||||
|
||||
private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, long endTime)
|
||||
throws IOException {
|
||||
List<FileStatus> result = new ArrayList<FileStatus>();
|
||||
LOG.debug("Scanning " + dir.toString() + " for WAL files");
|
||||
|
||||
FileStatus[] files = fs.listStatus(dir);
|
||||
if (files == null) return Collections.emptyList();
|
||||
for (FileStatus file : files) {
|
||||
if (file.isDirectory()) {
|
||||
// recurse into sub directories
|
||||
result.addAll(getFiles(fs, file.getPath(), startTime, endTime));
|
||||
} else {
|
||||
String name = file.getPath().toString();
|
||||
int idx = name.lastIndexOf('.');
|
||||
if (idx > 0) {
|
||||
try {
|
||||
long fileStartTime = Long.parseLong(name.substring(idx+1));
|
||||
if (fileStartTime <= endTime) {
|
||||
LOG.info("Found: " + name);
|
||||
result.add(file);
|
||||
}
|
||||
} catch (NumberFormatException x) {
|
||||
idx = 0;
|
||||
}
|
||||
}
|
||||
if (idx == 0) {
|
||||
LOG.warn("File " + name + " does not appear to be an WAL file. Skipping...");
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<WALKey, WALEdit> createRecordReader(InputSplit split,
|
||||
TaskAttemptContext context) throws IOException, InterruptedException {
|
||||
return new WALKeyRecordReader();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,106 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
||||
/**
|
||||
* Handles log splitting a wal
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALSplitterHandler extends EventHandler {
|
||||
private static final Log LOG = LogFactory.getLog(WALSplitterHandler.class);
|
||||
private final ServerName serverName;
|
||||
private final CancelableProgressable reporter;
|
||||
private final AtomicInteger inProgressTasks;
|
||||
private final TaskExecutor splitTaskExecutor;
|
||||
private final RecoveryMode mode;
|
||||
private final SplitLogWorkerCoordination.SplitTaskDetails splitTaskDetails;
|
||||
private final SplitLogWorkerCoordination coordination;
|
||||
|
||||
|
||||
public WALSplitterHandler(final Server server, SplitLogWorkerCoordination coordination,
|
||||
SplitLogWorkerCoordination.SplitTaskDetails splitDetails, CancelableProgressable reporter,
|
||||
AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor, RecoveryMode mode) {
|
||||
super(server, EventType.RS_LOG_REPLAY);
|
||||
this.splitTaskDetails = splitDetails;
|
||||
this.coordination = coordination;
|
||||
this.reporter = reporter;
|
||||
this.inProgressTasks = inProgressTasks;
|
||||
this.inProgressTasks.incrementAndGet();
|
||||
this.serverName = server.getServerName();
|
||||
this.splitTaskExecutor = splitTaskExecutor;
|
||||
this.mode = mode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
long startTime = System.currentTimeMillis();
|
||||
try {
|
||||
Status status = this.splitTaskExecutor.exec(splitTaskDetails.getWALFile(), mode, reporter);
|
||||
switch (status) {
|
||||
case DONE:
|
||||
coordination.endTask(new SplitLogTask.Done(this.serverName,this.mode),
|
||||
SplitLogCounters.tot_wkr_task_done, splitTaskDetails);
|
||||
break;
|
||||
case PREEMPTED:
|
||||
SplitLogCounters.tot_wkr_preempt_task.incrementAndGet();
|
||||
LOG.warn("task execution prempted " + splitTaskDetails.getWALFile());
|
||||
break;
|
||||
case ERR:
|
||||
if (server != null && !server.isStopped()) {
|
||||
coordination.endTask(new SplitLogTask.Err(this.serverName, this.mode),
|
||||
SplitLogCounters.tot_wkr_task_err, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
// if the RS is exiting then there is probably a tons of stuff
|
||||
// that can go wrong. Resign instead of signaling error.
|
||||
//$FALL-THROUGH$
|
||||
case RESIGNED:
|
||||
if (server != null && server.isStopped()) {
|
||||
LOG.info("task execution interrupted because worker is exiting "
|
||||
+ splitTaskDetails.toString());
|
||||
}
|
||||
coordination.endTask(new SplitLogTask.Resigned(this.serverName, this.mode),
|
||||
SplitLogCounters.tot_wkr_task_resigned, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
} finally {
|
||||
LOG.info("worker " + serverName + " done with task " + splitTaskDetails.toString() + " in "
|
||||
+ (System.currentTimeMillis() - startTime) + "ms");
|
||||
this.inProgressTasks.decrementAndGet();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,101 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
/**
|
||||
* Helper methods to ease Region Server integration with the write ahead log.
|
||||
* Note that methods in this class specifically should not require access to anything
|
||||
* other than the API found in {@link WAL}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALUtil {
|
||||
static final Log LOG = LogFactory.getLog(WALUtil.class);
|
||||
|
||||
/**
|
||||
* Write the marker that a compaction has succeeded and is about to be committed.
|
||||
* This provides info to the HMaster to allow it to recover the compaction if
|
||||
* this regionserver dies in the middle (This part is not yet implemented). It also prevents
|
||||
* the compaction from finishing if this regionserver has already lost its lease on the log.
|
||||
* @param sequenceId Used by WAL to get sequence Id for the waledit.
|
||||
*/
|
||||
public static void writeCompactionMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
|
||||
final CompactionDescriptor c, AtomicLong sequenceId) throws IOException {
|
||||
TableName tn = TableName.valueOf(c.getTableName().toByteArray());
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
|
||||
WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
log.append(htd, info, key, WALEdit.createCompaction(info, c), sequenceId, false, null);
|
||||
log.sync();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a flush marker indicating a start / abort or a complete of a region flush
|
||||
*/
|
||||
public static long writeFlushMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
|
||||
final FlushDescriptor f, AtomicLong sequenceId, boolean sync) throws IOException {
|
||||
TableName tn = TableName.valueOf(f.getTableName().toByteArray());
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
|
||||
WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
long trx = log.append(htd, info, key, WALEdit.createFlushWALEdit(info, f), sequenceId, false,
|
||||
null);
|
||||
if (sync) log.sync(trx);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
|
||||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a region open marker indicating that the region is opened
|
||||
*/
|
||||
public static long writeRegionEventMarker(WAL log, HTableDescriptor htd, HRegionInfo info,
|
||||
final RegionEventDescriptor r, AtomicLong sequenceId) throws IOException {
|
||||
TableName tn = TableName.valueOf(r.getTableName().toByteArray());
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
|
||||
WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
long trx = log.append(htd, info, key, WALEdit.createRegionEventWALEdit(info, r),
|
||||
sequenceId, false, null);
|
||||
log.sync(trx);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
|
||||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,145 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Reader;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Wrapper class around WAL to help manage the implementation details
|
||||
* such as compression.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationWALReaderManager {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationWALReaderManager.class);
|
||||
private final FileSystem fs;
|
||||
private final Configuration conf;
|
||||
private long position = 0;
|
||||
private Reader reader;
|
||||
private Path lastPath;
|
||||
|
||||
/**
|
||||
* Creates the helper but doesn't open any file
|
||||
* Use setInitialPosition after using the constructor if some content needs to be skipped
|
||||
* @param fs
|
||||
* @param conf
|
||||
*/
|
||||
public ReplicationWALReaderManager(FileSystem fs, Configuration conf) {
|
||||
this.fs = fs;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens the file at the current position
|
||||
* @param path
|
||||
* @return an WAL reader.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Reader openReader(Path path) throws IOException {
|
||||
// Detect if this is a new file, if so get a new reader else
|
||||
// reset the current reader so that we see the new data
|
||||
if (this.reader == null || !this.lastPath.equals(path)) {
|
||||
this.closeReader();
|
||||
this.reader = WALFactory.createReader(this.fs, path, this.conf);
|
||||
this.lastPath = path;
|
||||
} else {
|
||||
try {
|
||||
this.reader.reset();
|
||||
} catch (NullPointerException npe) {
|
||||
throw new IOException("NPE resetting reader, likely HDFS-4380", npe);
|
||||
}
|
||||
}
|
||||
return this.reader;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next entry, returned and also added in the array
|
||||
* @return a new entry or null
|
||||
* @throws IOException
|
||||
*/
|
||||
public Entry readNextAndSetPosition() throws IOException {
|
||||
Entry entry = this.reader.next();
|
||||
// Store the position so that in the future the reader can start
|
||||
// reading from here. If the above call to next() throws an
|
||||
// exception, the position won't be changed and retry will happen
|
||||
// from the last known good position
|
||||
this.position = this.reader.getPosition();
|
||||
// We need to set the CC to null else it will be compressed when sent to the sink
|
||||
if (entry != null) {
|
||||
entry.setCompressionContext(null);
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
/**
|
||||
* Advance the reader to the current position
|
||||
* @throws IOException
|
||||
*/
|
||||
public void seek() throws IOException {
|
||||
if (this.position != 0) {
|
||||
this.reader.seek(this.position);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the position that we stopped reading at
|
||||
* @return current position, cannot be negative
|
||||
*/
|
||||
public long getPosition() {
|
||||
return this.position;
|
||||
}
|
||||
|
||||
public void setPosition(long pos) {
|
||||
this.position = pos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the current reader
|
||||
* @throws IOException
|
||||
*/
|
||||
public void closeReader() throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close();
|
||||
this.reader = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the helper to reset internal state
|
||||
*/
|
||||
void finishCurrentFile() {
|
||||
this.position = 0;
|
||||
try {
|
||||
this.closeReader();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to close reader", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,369 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
|
||||
/**
|
||||
* A WAL Provider that returns a single thread safe WAL that writes to HDFS.
|
||||
* By default, this implementation picks a directory in HDFS based on a combination of
|
||||
* <ul>
|
||||
* <li>the HBase root directory
|
||||
* <li>HConstants.HREGION_LOGDIR_NAME
|
||||
* <li>the given factory's factoryId (usually identifying the regionserver by host:port)
|
||||
* </ul>
|
||||
* It also uses the providerId to diffentiate among files.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class DefaultWALProvider implements WALProvider {
|
||||
private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
|
||||
|
||||
// Only public so classes back in regionserver.wal can access
|
||||
public interface Reader extends WAL.Reader {
|
||||
/**
|
||||
* @param fs File system.
|
||||
* @param path Path.
|
||||
* @param c Configuration.
|
||||
* @param s Input stream that may have been pre-opened by the caller; may be null.
|
||||
*/
|
||||
void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
|
||||
}
|
||||
|
||||
// Only public so classes back in regionserver.wal can access
|
||||
public interface Writer extends WALProvider.Writer {
|
||||
void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
|
||||
}
|
||||
|
||||
private FSHLog log = null;
|
||||
|
||||
/**
|
||||
* @param factory factory that made us, identity used for FS layout. may not be null
|
||||
* @param conf may not be null
|
||||
* @param listeners may be null
|
||||
* @param providerId differentiate between providers from one facotry, used for FS layout. may be
|
||||
* null
|
||||
*/
|
||||
@Override
|
||||
public void init(final WALFactory factory, final Configuration conf,
|
||||
final List<WALActionsListener> listeners, String providerId) throws IOException {
|
||||
if (null != log) {
|
||||
throw new IllegalStateException("WALProvider.init should only be called once.");
|
||||
}
|
||||
if (null == providerId) {
|
||||
providerId = DEFAULT_PROVIDER_ID;
|
||||
}
|
||||
final String logPrefix = factory.factoryId + WAL_FILE_NAME_DELIMITER + providerId;
|
||||
log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
|
||||
getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
|
||||
true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WAL getWAL(final byte[] identifier) throws IOException {
|
||||
return log;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
log.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() throws IOException {
|
||||
log.shutdown();
|
||||
}
|
||||
|
||||
// should be package private; more visible for use in FSHLog
|
||||
public static final String WAL_FILE_NAME_DELIMITER = ".";
|
||||
/** The hbase:meta region's WAL filename extension */
|
||||
@VisibleForTesting
|
||||
public static final String META_WAL_PROVIDER_ID = ".meta";
|
||||
static final String DEFAULT_PROVIDER_ID = "default";
|
||||
|
||||
// Implementation details that currently leak in tests or elsewhere follow
|
||||
/** File Extension used while splitting an WAL into regions (HBASE-2312) */
|
||||
public static final String SPLITTING_EXT = "-splitting";
|
||||
|
||||
/**
|
||||
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
|
||||
* count the number of files (rolled and active). if either of them aren't, count 0
|
||||
* for that provider.
|
||||
* @param walFactory may not be null.
|
||||
*/
|
||||
public static long getNumLogFiles(WALFactory walFactory) {
|
||||
long result = 0;
|
||||
if (walFactory.provider instanceof DefaultWALProvider) {
|
||||
result += ((FSHLog)((DefaultWALProvider)walFactory.provider).log).getNumLogFiles();
|
||||
}
|
||||
WALProvider meta = walFactory.metaProvider.get();
|
||||
if (meta instanceof DefaultWALProvider) {
|
||||
result += ((FSHLog)((DefaultWALProvider)meta).log).getNumLogFiles();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
|
||||
* count the size of files (rolled and active). if either of them aren't, count 0
|
||||
* for that provider.
|
||||
* @param walFactory may not be null.
|
||||
*/
|
||||
public static long getLogFileSize(WALFactory walFactory) {
|
||||
long result = 0;
|
||||
if (walFactory.provider instanceof DefaultWALProvider) {
|
||||
result += ((FSHLog)((DefaultWALProvider)walFactory.provider).log).getLogFileSize();
|
||||
}
|
||||
WALProvider meta = walFactory.metaProvider.get();
|
||||
if (meta instanceof DefaultWALProvider) {
|
||||
result += ((FSHLog)((DefaultWALProvider)meta).log).getLogFileSize();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* returns the number of rolled WAL files.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static int getNumRolledLogFiles(WAL wal) {
|
||||
return ((FSHLog)wal).getNumRolledLogFiles();
|
||||
}
|
||||
|
||||
/**
|
||||
* return the current filename from the current wal.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static Path getCurrentFileName(final WAL wal) {
|
||||
return ((FSHLog)wal).getCurrentFileName();
|
||||
}
|
||||
|
||||
/**
|
||||
* request a log roll, but don't actually do it.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static void requestLogRoll(final WAL wal) {
|
||||
((FSHLog)wal).requestLogRoll();
|
||||
}
|
||||
|
||||
/**
|
||||
* It returns the file create timestamp from the file name.
|
||||
* For name format see {@link #validateWALFilename(String)}
|
||||
* public until remaining tests move to o.a.h.h.wal
|
||||
* @param wal must not be null
|
||||
* @return the file number that is part of the WAL file name
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static long extractFileNumFromWAL(final WAL wal) {
|
||||
final Path walName = ((FSHLog)wal).getCurrentFileName();
|
||||
if (walName == null) {
|
||||
throw new IllegalArgumentException("The WAL path couldn't be null");
|
||||
}
|
||||
final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
|
||||
return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Pattern used to validate a WAL file name
|
||||
* see {@link #validateWALFilename(String)} for description.
|
||||
*/
|
||||
private static final Pattern pattern = Pattern.compile(".*\\.\\d*("+META_WAL_PROVIDER_ID+")*");
|
||||
|
||||
/**
|
||||
* A WAL file name is of the format:
|
||||
* <wal-name>{@link #WAL_FILE_NAME_DELIMITER}<file-creation-timestamp>[.meta].
|
||||
*
|
||||
* provider-name is usually made up of a server-name and a provider-id
|
||||
*
|
||||
* @param filename name of the file to validate
|
||||
* @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt>
|
||||
* otherwise
|
||||
*/
|
||||
public static boolean validateWALFilename(String filename) {
|
||||
return pattern.matcher(filename).matches();
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct the directory name for all WALs on a given server.
|
||||
*
|
||||
* @param serverName
|
||||
* Server name formatted as described in {@link ServerName}
|
||||
* @return the relative WAL directory name, e.g.
|
||||
* <code>.logs/1.example.org,60030,12345</code> if
|
||||
* <code>serverName</code> passed is
|
||||
* <code>1.example.org,60030,12345</code>
|
||||
*/
|
||||
public static String getWALDirectoryName(final String serverName) {
|
||||
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
|
||||
dirName.append("/");
|
||||
dirName.append(serverName);
|
||||
return dirName.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Pulls a ServerName out of a Path generated according to our layout rules.
|
||||
*
|
||||
* In the below layouts, this method ignores the format of the logfile component.
|
||||
*
|
||||
* Current format:
|
||||
*
|
||||
* [base directory for hbase]/hbase/.logs/ServerName/logfile
|
||||
* or
|
||||
* [base directory for hbase]/hbase/.logs/ServerName-splitting/logfile
|
||||
*
|
||||
* Expected to work for individual log files and server-specific directories.
|
||||
*
|
||||
* @return null if it's not a log file. Returns the ServerName of the region
|
||||
* server that created this log file otherwise.
|
||||
*/
|
||||
public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
|
||||
throws IOException {
|
||||
if (path == null
|
||||
|| path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (conf == null) {
|
||||
throw new IllegalArgumentException("parameter conf must be set");
|
||||
}
|
||||
|
||||
final String rootDir = conf.get(HConstants.HBASE_DIR);
|
||||
if (rootDir == null || rootDir.isEmpty()) {
|
||||
throw new IllegalArgumentException(HConstants.HBASE_DIR
|
||||
+ " key not found in conf.");
|
||||
}
|
||||
|
||||
final StringBuilder startPathSB = new StringBuilder(rootDir);
|
||||
if (!rootDir.endsWith("/"))
|
||||
startPathSB.append('/');
|
||||
startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
|
||||
if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))
|
||||
startPathSB.append('/');
|
||||
final String startPath = startPathSB.toString();
|
||||
|
||||
String fullPath;
|
||||
try {
|
||||
fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
|
||||
return null;
|
||||
}
|
||||
|
||||
if (!fullPath.startsWith(startPath)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final String serverNameAndFile = fullPath.substring(startPath.length());
|
||||
|
||||
if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
|
||||
// Either it's a file (not a directory) or it's not a ServerName format
|
||||
return null;
|
||||
}
|
||||
|
||||
Path p = new Path(path);
|
||||
return getServerNameFromWALDirectoryName(p);
|
||||
}
|
||||
|
||||
/**
|
||||
* This function returns region server name from a log file name which is in one of the following
|
||||
* formats:
|
||||
* <ul>
|
||||
* <li>hdfs://<name node>/hbase/.logs/<server name>-splitting/...
|
||||
* <li>hdfs://<name node>/hbase/.logs/<server name>/...
|
||||
* </ul>
|
||||
* @param logFile
|
||||
* @return null if the passed in logFile isn't a valid WAL file path
|
||||
*/
|
||||
public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
|
||||
String logDirName = logFile.getParent().getName();
|
||||
// We were passed the directory and not a file in it.
|
||||
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
|
||||
logDirName = logFile.getName();
|
||||
}
|
||||
ServerName serverName = null;
|
||||
if (logDirName.endsWith(SPLITTING_EXT)) {
|
||||
logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
|
||||
}
|
||||
try {
|
||||
serverName = ServerName.parseServerName(logDirName);
|
||||
} catch (IllegalArgumentException ex) {
|
||||
serverName = null;
|
||||
LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
|
||||
}
|
||||
if (serverName != null && serverName.getStartcode() < 0) {
|
||||
LOG.warn("Invalid log file path=" + logFile);
|
||||
serverName = null;
|
||||
}
|
||||
return serverName;
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(Path p) {
|
||||
return isMetaFile(p.getName());
|
||||
}
|
||||
|
||||
public static boolean isMetaFile(String p) {
|
||||
if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* public because of FSHLog. Should be package-private
|
||||
*/
|
||||
public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
|
||||
final boolean overwritable)
|
||||
throws IOException {
|
||||
// Configuration already does caching for the Class lookup.
|
||||
Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
|
||||
ProtobufLogWriter.class, Writer.class);
|
||||
try {
|
||||
Writer writer = logWriterClass.newInstance();
|
||||
writer.init(fs, path, conf, overwritable);
|
||||
return writer;
|
||||
} catch (Exception e) {
|
||||
LOG.debug("Error instantiating log writer.", e);
|
||||
throw new IOException("cannot get log writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,213 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* No-op implementation of {@link WALProvider} used when the WAL is disabled.
|
||||
*
|
||||
* Should only be used when severe data loss is acceptable.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class DisabledWALProvider implements WALProvider {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(DisabledWALProvider.class);
|
||||
|
||||
WAL disabled;
|
||||
|
||||
@Override
|
||||
public void init(final WALFactory factory, final Configuration conf,
|
||||
final List<WALActionsListener> listeners, final String providerId) throws IOException {
|
||||
if (null != disabled) {
|
||||
throw new IllegalStateException("WALProvider.init should only be called once.");
|
||||
}
|
||||
disabled = new DisabledWAL(new Path(FSUtils.getRootDir(conf), providerId), conf, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WAL getWAL(final byte[] identifier) throws IOException {
|
||||
return disabled;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
disabled.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() throws IOException {
|
||||
disabled.shutdown();
|
||||
}
|
||||
|
||||
private static class DisabledWAL implements WAL {
|
||||
protected final List<WALActionsListener> listeners =
|
||||
new CopyOnWriteArrayList<WALActionsListener>();
|
||||
protected final Path path;
|
||||
protected final WALCoprocessorHost coprocessorHost;
|
||||
protected final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
|
||||
public DisabledWAL(final Path path, final Configuration conf,
|
||||
final List<WALActionsListener> listeners) {
|
||||
this.coprocessorHost = new WALCoprocessorHost(this, conf);
|
||||
this.path = path;
|
||||
if (null != listeners) {
|
||||
for(WALActionsListener listener : listeners) {
|
||||
registerWALActionsListener(listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerWALActionsListener(final WALActionsListener listener) {
|
||||
listeners.add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean unregisterWALActionsListener(final WALActionsListener listener) {
|
||||
return listeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[][] rollWriter() {
|
||||
if (!listeners.isEmpty()) {
|
||||
for (WALActionsListener listener : listeners) {
|
||||
listener.logRollRequested();
|
||||
}
|
||||
for (WALActionsListener listener : listeners) {
|
||||
try {
|
||||
listener.preLogRoll(path, path);
|
||||
} catch (IOException exception) {
|
||||
LOG.debug("Ignoring exception from listener.", exception);
|
||||
}
|
||||
}
|
||||
for (WALActionsListener listener : listeners) {
|
||||
try {
|
||||
listener.postLogRoll(path, path);
|
||||
} catch (IOException exception) {
|
||||
LOG.debug("Ignoring exception from listener.", exception);
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[][] rollWriter(boolean force) {
|
||||
return rollWriter();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
if(closed.compareAndSet(false, true)) {
|
||||
if (!this.listeners.isEmpty()) {
|
||||
for (WALActionsListener listener : this.listeners) {
|
||||
listener.logCloseRequested();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
shutdown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
|
||||
AtomicLong sequenceId, boolean inMemstore, List<Cell> memstoreKVs) {
|
||||
if (!this.listeners.isEmpty()) {
|
||||
final long start = System.nanoTime();
|
||||
long len = 0;
|
||||
for (Cell cell : edits.getCells()) {
|
||||
len += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
final long elapsed = (System.nanoTime() - start)/1000000l;
|
||||
for (WALActionsListener listener : this.listeners) {
|
||||
listener.postAppend(len, elapsed);
|
||||
}
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync() {
|
||||
if (!this.listeners.isEmpty()) {
|
||||
for (WALActionsListener listener : this.listeners) {
|
||||
listener.postSync(0l, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync(long txid) {
|
||||
sync();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean startCacheFlush(final byte[] encodedRegionName) {
|
||||
return !(closed.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeCacheFlush(final byte[] encodedRegionName) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abortCacheFlush(byte[] encodedRegionName) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public WALCoprocessorHost getCoprocessorHost() {
|
||||
return coprocessorHost;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
|
||||
return HConstants.NO_SEQNUM;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "WAL disabled.";
|
||||
}
|
||||
}
|
||||
}
|
263
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
Normal file
263
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
Normal file
@ -0,0 +1,263 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
||||
// imports we use from yet-to-be-moved regionsever.wal
|
||||
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
|
||||
* APIs for WAL users (such as RegionServer) to use the WAL (do append, sync, etc).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public interface WAL {
|
||||
|
||||
/**
|
||||
* Registers WALActionsListener
|
||||
*/
|
||||
void registerWALActionsListener(final WALActionsListener listener);
|
||||
|
||||
/**
|
||||
* Unregisters WALActionsListener
|
||||
*/
|
||||
boolean unregisterWALActionsListener(final WALActionsListener listener);
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
*
|
||||
* <p>
|
||||
* The implementation is synchronized in order to make sure there's one rollWriter
|
||||
* running at any given time.
|
||||
*
|
||||
* @return If lots of logs, flush the 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 HRegionInfo#getEncodedName()}
|
||||
*/
|
||||
byte[][] rollWriter() throws FailedLogCloseException, IOException;
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
*
|
||||
* <p>
|
||||
* The implementation is synchronized in order to make sure there's one rollWriter
|
||||
* running at any given time.
|
||||
*
|
||||
* @param force
|
||||
* If true, force creation of a new writer even if no entries have
|
||||
* been written to the current writer
|
||||
* @return If lots of logs, flush the 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 HRegionInfo#getEncodedName()}
|
||||
*/
|
||||
byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException;
|
||||
|
||||
/**
|
||||
* Stop accepting new writes. If we have unsynced writes still in buffer, sync them.
|
||||
* Extant edits are left in place in backing storage to be replayed later.
|
||||
*/
|
||||
void shutdown() throws IOException;
|
||||
|
||||
/**
|
||||
* Caller no longer needs any edits from this WAL. Implementers are free to reclaim
|
||||
* underlying resources after this call; i.e. filesystem based WALs can archive or
|
||||
* delete files.
|
||||
*/
|
||||
void close() throws IOException;
|
||||
|
||||
/**
|
||||
* Append a set of edits to the WAL. 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
|
||||
* @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.
|
||||
* @param htd used to give scope for replication TODO refactor out in favor of table name and info
|
||||
* @param sequenceId A reference to the atomic long the <code>info</code> region is using as
|
||||
* source of its incrementing edits sequence id. Inside in this call we will increment it and
|
||||
* attach the sequence to the edit we apply the WAL.
|
||||
* @param inMemstore Always true except for case where we are writing a compaction completion
|
||||
* record into the WAL; in this case the entry is just so we can finish an unfinished compaction
|
||||
* -- it is not an edit for memstore.
|
||||
* @param memstoreKVs list of KVs added into memstore
|
||||
* @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
|
||||
* in it.
|
||||
*/
|
||||
long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
|
||||
AtomicLong sequenceId, boolean inMemstore, List<Cell> memstoreKVs)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Sync what we have in the WAL.
|
||||
* @throws IOException
|
||||
*/
|
||||
void sync() throws IOException;
|
||||
|
||||
/**
|
||||
* Sync the WAL if the txId was not already sync'd.
|
||||
* @param txid Transaction id to sync to.
|
||||
* @throws IOException
|
||||
*/
|
||||
void sync(long txid) throws IOException;
|
||||
|
||||
/**
|
||||
* WAL keeps track of the sequence numbers that were not yet flushed from memstores
|
||||
* in order to be able to do cleanup. This method tells WAL that some region is about
|
||||
* to flush memstore.
|
||||
*
|
||||
* <p>We stash the oldest seqNum for the region, and let the the next edit inserted in this
|
||||
* region be recorded in {@link #append(HTableDescriptor, HRegionInfo, WALKey, WALEdit,
|
||||
* AtomicLong, boolean, List)} as new oldest seqnum.
|
||||
* In case of flush being aborted, we put the stashed value back; in case of flush succeeding,
|
||||
* the seqNum of that first edit after start becomes the valid oldest seqNum for this region.
|
||||
*
|
||||
* @return true if the flush can proceed, false in case wal is closing (ususally, when server is
|
||||
* closing) and flush couldn't be started.
|
||||
*/
|
||||
boolean startCacheFlush(final byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* Complete the cache flush.
|
||||
* @param encodedRegionName Encoded region name.
|
||||
*/
|
||||
void completeCacheFlush(final byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* Abort a cache flush. Call if the flush fails. Note that the only recovery
|
||||
* for an aborted flush currently is a restart of the regionserver so the
|
||||
* snapshot content dropped by the failure gets restored to the memstore.v
|
||||
* @param encodedRegionName Encoded region name.
|
||||
*/
|
||||
void abortCacheFlush(byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* @return Coprocessor host.
|
||||
*/
|
||||
WALCoprocessorHost getCoprocessorHost();
|
||||
|
||||
|
||||
/** Gets the earliest sequence number in the memstore for this particular region.
|
||||
* This can serve as best-effort "recent" WAL number for this region.
|
||||
* @param encodedRegionName The region to get the number for.
|
||||
* @return The number if present, HConstants.NO_SEQNUM if absent.
|
||||
*/
|
||||
long getEarliestMemstoreSeqNum(byte[] encodedRegionName);
|
||||
|
||||
/**
|
||||
* Human readable identifying information about the state of this WAL.
|
||||
* Implementors are encouraged to include information appropriate for debugging.
|
||||
* Consumers are advised not to rely on the details of the returned String; it does
|
||||
* not have a defined structure.
|
||||
*/
|
||||
@Override
|
||||
String toString();
|
||||
|
||||
/**
|
||||
* When outside clients need to consume persisted WALs, they rely on a provided
|
||||
* Reader.
|
||||
*/
|
||||
interface Reader extends Closeable {
|
||||
Entry next() throws IOException;
|
||||
Entry next(Entry reuse) throws IOException;
|
||||
void seek(long pos) throws IOException;
|
||||
long getPosition() throws IOException;
|
||||
void reset() throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility class that lets us keep track of the edit with it's key.
|
||||
*/
|
||||
class Entry {
|
||||
private WALEdit edit;
|
||||
private WALKey key;
|
||||
|
||||
public Entry() {
|
||||
edit = new WALEdit();
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
|
||||
key = new HLogKey();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for both params
|
||||
*
|
||||
* @param edit log's edit
|
||||
* @param key log's key
|
||||
*/
|
||||
public Entry(WALKey key, WALEdit edit) {
|
||||
super();
|
||||
this.key = key;
|
||||
this.edit = edit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the edit
|
||||
*
|
||||
* @return edit
|
||||
*/
|
||||
public WALEdit getEdit() {
|
||||
return edit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the key
|
||||
*
|
||||
* @return key
|
||||
*/
|
||||
public WALKey getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set compression context for this entry.
|
||||
*
|
||||
* @param compressionContext
|
||||
* Compression context
|
||||
*/
|
||||
public void setCompressionContext(CompressionContext compressionContext) {
|
||||
edit.setCompressionContext(compressionContext);
|
||||
key.setCompressionContext(compressionContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.key + "=" + this.edit;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,426 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Reader;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
|
||||
/**
|
||||
* Entry point for users of the Write Ahead Log.
|
||||
* Acts as the shim between internal use and the particular WALProvider we use to handle wal
|
||||
* requests.
|
||||
*
|
||||
* Configure which provider gets used with the configuration setting "hbase.wal.provider". Available
|
||||
* implementations:
|
||||
* <ul>
|
||||
* <li><em>defaultProvider</em> : whatever provider is standard for the hbase version.</li>
|
||||
* </ul>
|
||||
*
|
||||
* Alternatively, you may provide a custome implementation of {@link WALProvider} by class name.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALFactory {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(WALFactory.class);
|
||||
|
||||
/**
|
||||
* Maps between configuration names for providers and implementation classes.
|
||||
*/
|
||||
static enum Providers {
|
||||
defaultProvider(DefaultWALProvider.class);
|
||||
|
||||
Class<? extends WALProvider> clazz;
|
||||
Providers(Class<? extends WALProvider> clazz) {
|
||||
this.clazz = clazz;
|
||||
}
|
||||
}
|
||||
|
||||
static final String WAL_PROVIDER = "hbase.wal.provider";
|
||||
static final String DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
|
||||
|
||||
static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";
|
||||
static final String DEFAULT_META_WAL_PROVIDER = Providers.defaultProvider.name();
|
||||
|
||||
final String factoryId;
|
||||
final WALProvider provider;
|
||||
// The meta updates are written to a different wal. If this
|
||||
// regionserver holds meta regions, then this ref will be non-null.
|
||||
// lazily intialized; most RegionServers don't deal with META
|
||||
final AtomicReference<WALProvider> metaProvider = new AtomicReference<WALProvider>();
|
||||
|
||||
/**
|
||||
* Configuration-specified WAL Reader used when a custom reader is requested
|
||||
*/
|
||||
private final Class<? extends DefaultWALProvider.Reader> logReaderClass;
|
||||
|
||||
/**
|
||||
* How long to attempt opening in-recovery wals
|
||||
*/
|
||||
private final int timeoutMillis;
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
// Used for the singleton WALFactory, see below.
|
||||
private WALFactory(Configuration conf) {
|
||||
// this code is duplicated here so we can keep our members final.
|
||||
// until we've moved reader/writer construction down into providers, this initialization must
|
||||
// happen prior to provider initialization, in case they need to instantiate a reader/writer.
|
||||
timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
|
||||
/* TODO Both of these are probably specific to the fs wal provider */
|
||||
logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
|
||||
DefaultWALProvider.Reader.class);
|
||||
this.conf = conf;
|
||||
// end required early initialization
|
||||
|
||||
// this instance can't create wals, just reader/writers.
|
||||
provider = null;
|
||||
factoryId = SINGLETON_ID;
|
||||
}
|
||||
|
||||
/**
|
||||
* instantiate a provider from a config property.
|
||||
* requires conf to have already been set (as well as anything the provider might need to read).
|
||||
*/
|
||||
WALProvider getProvider(final String key, final String defaultValue,
|
||||
final List<WALActionsListener> listeners, final String providerId) throws IOException {
|
||||
Class<? extends WALProvider> clazz;
|
||||
try {
|
||||
clazz = Providers.valueOf(conf.get(key, defaultValue)).clazz;
|
||||
} catch (IllegalArgumentException exception) {
|
||||
// Fall back to them specifying a class name
|
||||
// Note that the passed default class shouldn't actually be used, since the above only fails
|
||||
// when there is a config value present.
|
||||
clazz = conf.getClass(key, DefaultWALProvider.class, WALProvider.class);
|
||||
}
|
||||
try {
|
||||
final WALProvider result = clazz.newInstance();
|
||||
result.init(this, conf, listeners, providerId);
|
||||
return result;
|
||||
} catch (InstantiationException exception) {
|
||||
LOG.error("couldn't set up WALProvider, check config key " + key);
|
||||
LOG.debug("Exception details for failure to load WALProvider.", exception);
|
||||
throw new IOException("couldn't set up WALProvider", exception);
|
||||
} catch (IllegalAccessException exception) {
|
||||
LOG.error("couldn't set up WALProvider, check config key " + key);
|
||||
LOG.debug("Exception details for failure to load WALProvider.", exception);
|
||||
throw new IOException("couldn't set up WALProvider", exception);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param conf must not be null, will keep a reference to read params in later reader/writer
|
||||
* instances.
|
||||
* @param listeners may be null. will be given to all created wals (and not meta-wals)
|
||||
* @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations
|
||||
* to make a directory
|
||||
*/
|
||||
public WALFactory(final Configuration conf, final List<WALActionsListener> listeners,
|
||||
final String factoryId) throws IOException {
|
||||
// until we've moved reader/writer construction down into providers, this initialization must
|
||||
// happen prior to provider initialization, in case they need to instantiate a reader/writer.
|
||||
timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
|
||||
/* TODO Both of these are probably specific to the fs wal provider */
|
||||
logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
|
||||
DefaultWALProvider.Reader.class);
|
||||
this.conf = conf;
|
||||
this.factoryId = factoryId;
|
||||
// end required early initialization
|
||||
if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) {
|
||||
provider = getProvider(WAL_PROVIDER, DEFAULT_WAL_PROVIDER, listeners, null);
|
||||
} else {
|
||||
// special handling of existing configuration behavior.
|
||||
LOG.warn("Running with WAL disabled.");
|
||||
provider = new DisabledWALProvider();
|
||||
provider.init(this, conf, null, factoryId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown all WALs and clean up any underlying storage.
|
||||
* Use only when you will not need to replay and edits that have gone to any wals from this
|
||||
* factory.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
final WALProvider metaProvider = this.metaProvider.get();
|
||||
if (null != metaProvider) {
|
||||
metaProvider.close();
|
||||
}
|
||||
provider.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the underlying WAL providers to shut down, but do not clean up underlying storage.
|
||||
* If you are not ending cleanly and will need to replay edits from this factory's wals,
|
||||
* use this method if you can as it will try to leave things as tidy as possible.
|
||||
*/
|
||||
public void shutdown() throws IOException {
|
||||
IOException exception = null;
|
||||
final WALProvider metaProvider = this.metaProvider.get();
|
||||
if (null != metaProvider) {
|
||||
try {
|
||||
metaProvider.shutdown();
|
||||
} catch(IOException ioe) {
|
||||
exception = ioe;
|
||||
}
|
||||
}
|
||||
provider.shutdown();
|
||||
if (null != exception) {
|
||||
throw exception;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param identifier may not be null, contents will not be altered
|
||||
*/
|
||||
public WAL getWAL(final byte[] identifier) throws IOException {
|
||||
return provider.getWAL(identifier);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param identifier may not be null, contents will not be altered
|
||||
*/
|
||||
public WAL getMetaWAL(final byte[] identifier) throws IOException {
|
||||
WALProvider metaProvider = this.metaProvider.get();
|
||||
if (null == metaProvider) {
|
||||
final WALProvider temp = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER,
|
||||
Collections.<WALActionsListener>singletonList(new MetricsWAL()),
|
||||
DefaultWALProvider.META_WAL_PROVIDER_ID);
|
||||
if (this.metaProvider.compareAndSet(null, temp)) {
|
||||
metaProvider = temp;
|
||||
} else {
|
||||
// reference must now be to a provider created in another thread.
|
||||
temp.close();
|
||||
metaProvider = this.metaProvider.get();
|
||||
}
|
||||
}
|
||||
return metaProvider.getWAL(identifier);
|
||||
}
|
||||
|
||||
public Reader createReader(final FileSystem fs, final Path path) throws IOException {
|
||||
return createReader(fs, path, (CancelableProgressable)null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a reader for the WAL. If you are reading from a file that's being written to and need
|
||||
* to reopen it multiple times, use {@link WAL.Reader#reset()} instead of this method
|
||||
* then just seek back to the last known good position.
|
||||
* @return A WAL reader. Close when done with it.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Reader createReader(final FileSystem fs, final Path path,
|
||||
CancelableProgressable reporter) throws IOException {
|
||||
return createReader(fs, path, reporter, true);
|
||||
}
|
||||
|
||||
public Reader createReader(final FileSystem fs, final Path path,
|
||||
CancelableProgressable reporter, boolean allowCustom)
|
||||
throws IOException {
|
||||
Class<? extends DefaultWALProvider.Reader> lrClass =
|
||||
allowCustom ? logReaderClass : ProtobufLogReader.class;
|
||||
|
||||
try {
|
||||
// A wal file could be under recovery, so it may take several
|
||||
// tries to get it open. Instead of claiming it is corrupted, retry
|
||||
// to open it up to 5 minutes by default.
|
||||
long startWaiting = EnvironmentEdgeManager.currentTime();
|
||||
long openTimeout = timeoutMillis + startWaiting;
|
||||
int nbAttempt = 0;
|
||||
while (true) {
|
||||
try {
|
||||
if (lrClass != ProtobufLogReader.class) {
|
||||
// User is overriding the WAL reader, let them.
|
||||
DefaultWALProvider.Reader reader = lrClass.newInstance();
|
||||
reader.init(fs, path, conf, null);
|
||||
return reader;
|
||||
} else {
|
||||
FSDataInputStream stream = fs.open(path);
|
||||
// Note that zero-length file will fail to read PB magic, and attempt to create
|
||||
// a non-PB reader and fail the same way existing code expects it to. If we get
|
||||
// rid of the old reader entirely, we need to handle 0-size files differently from
|
||||
// merely non-PB files.
|
||||
byte[] magic = new byte[ProtobufLogReader.PB_WAL_MAGIC.length];
|
||||
boolean isPbWal = (stream.read(magic) == magic.length)
|
||||
&& Arrays.equals(magic, ProtobufLogReader.PB_WAL_MAGIC);
|
||||
DefaultWALProvider.Reader reader =
|
||||
isPbWal ? new ProtobufLogReader() : new SequenceFileLogReader();
|
||||
reader.init(fs, path, conf, stream);
|
||||
return reader;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
String msg = e.getMessage();
|
||||
if (msg != null && (msg.contains("Cannot obtain block length")
|
||||
|| msg.contains("Could not obtain the last block")
|
||||
|| msg.matches("Blocklist for [^ ]* has changed.*"))) {
|
||||
if (++nbAttempt == 1) {
|
||||
LOG.warn("Lease should have recovered. This is not expected. Will retry", e);
|
||||
}
|
||||
if (reporter != null && !reporter.progress()) {
|
||||
throw new InterruptedIOException("Operation is cancelled");
|
||||
}
|
||||
if (nbAttempt > 2 && openTimeout < EnvironmentEdgeManager.currentTime()) {
|
||||
LOG.error("Can't open after " + nbAttempt + " attempts and "
|
||||
+ (EnvironmentEdgeManager.currentTime() - startWaiting)
|
||||
+ "ms " + " for " + path);
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(nbAttempt < 3 ? 500 : 1000);
|
||||
continue; // retry
|
||||
} catch (InterruptedException ie) {
|
||||
InterruptedIOException iioe = new InterruptedIOException();
|
||||
iioe.initCause(ie);
|
||||
throw iioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
throw ie;
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Cannot get log reader", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a writer for the WAL.
|
||||
* should be package-private. public only for tests and
|
||||
* {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
|
||||
* @return A WAL writer. Close when done with it.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Writer createWALWriter(final FileSystem fs, final Path path) throws IOException {
|
||||
return DefaultWALProvider.createWriter(conf, fs, path, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* should be package-private, visible for recovery testing.
|
||||
* @return an overwritable writer for recovered edits. caller should close.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public Writer createRecoveredEditsWriter(final FileSystem fs, final Path path)
|
||||
throws IOException {
|
||||
return DefaultWALProvider.createWriter(conf, fs, path, true);
|
||||
}
|
||||
|
||||
// These static methods are currently used where it's impractical to
|
||||
// untangle the reliance on state in the filesystem. They rely on singleton
|
||||
// WALFactory that just provides Reader / Writers.
|
||||
// For now, first Configuration object wins. Practically this just impacts the reader/writer class
|
||||
private static final AtomicReference<WALFactory> singleton = new AtomicReference<WALFactory>();
|
||||
private static final String SINGLETON_ID = WALFactory.class.getName();
|
||||
|
||||
// public only for FSHLog
|
||||
public static WALFactory getInstance(Configuration configuration) {
|
||||
WALFactory factory = singleton.get();
|
||||
if (null == factory) {
|
||||
WALFactory temp = new WALFactory(configuration);
|
||||
if (singleton.compareAndSet(null, temp)) {
|
||||
factory = temp;
|
||||
} else {
|
||||
// someone else beat us to initializing
|
||||
try {
|
||||
temp.close();
|
||||
} catch (IOException exception) {
|
||||
LOG.debug("failed to close temporary singleton. ignoring.", exception);
|
||||
}
|
||||
factory = singleton.get();
|
||||
}
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a reader for the given path, accept custom reader classes from conf.
|
||||
* If you already have a WALFactory, you should favor the instance method.
|
||||
* @return a WAL Reader, caller must close.
|
||||
*/
|
||||
public static Reader createReader(final FileSystem fs, final Path path,
|
||||
final Configuration configuration) throws IOException {
|
||||
return getInstance(configuration).createReader(fs, path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a reader for the given path, accept custom reader classes from conf.
|
||||
* If you already have a WALFactory, you should favor the instance method.
|
||||
* @return a WAL Reader, caller must close.
|
||||
*/
|
||||
static Reader createReader(final FileSystem fs, final Path path,
|
||||
final Configuration configuration, final CancelableProgressable reporter) throws IOException {
|
||||
return getInstance(configuration).createReader(fs, path, reporter);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a reader for the given path, ignore custom reader classes from conf.
|
||||
* If you already have a WALFactory, you should favor the instance method.
|
||||
* only public pending move of {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
|
||||
* @return a WAL Reader, caller must close.
|
||||
*/
|
||||
public static Reader createReaderIgnoreCustomClass(final FileSystem fs, final Path path,
|
||||
final Configuration configuration) throws IOException {
|
||||
return getInstance(configuration).createReader(fs, path, null, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* If you already have a WALFactory, you should favor the instance method.
|
||||
* @return a Writer that will overwrite files. Caller must close.
|
||||
*/
|
||||
static Writer createRecoveredEditsWriter(final FileSystem fs, final Path path,
|
||||
final Configuration configuration)
|
||||
throws IOException {
|
||||
return DefaultWALProvider.createWriter(configuration, fs, path, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* If you already have a WALFactory, you should favor the instance method.
|
||||
* @return a writer that won't overwrite files. Caller must close.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static Writer createWALWriter(final FileSystem fs, final Path path,
|
||||
final Configuration configuration)
|
||||
throws IOException {
|
||||
return DefaultWALProvider.createWriter(configuration, fs, path, false);
|
||||
}
|
||||
}
|
@ -0,0 +1,553 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
|
||||
import org.apache.hadoop.hbase.regionserver.SequenceId;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
|
||||
/**
|
||||
* A Key for an entry in the change log.
|
||||
*
|
||||
* The log intermingles edits to many tables and rows, so each log entry
|
||||
* identifies the appropriate table and row. Within a table and row, they're
|
||||
* also sorted.
|
||||
*
|
||||
* <p>Some Transactional edits (START, COMMIT, ABORT) will not have an
|
||||
* associated row.
|
||||
*
|
||||
* Note that protected members marked @InterfaceAudience.Private are only protected
|
||||
* to support the legacy HLogKey class, which is in a different package.
|
||||
*/
|
||||
// TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
|
||||
// purposes. They need to be merged into WALEntry.
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
public class WALKey implements SequenceId, Comparable<WALKey> {
|
||||
public static final Log LOG = LogFactory.getLog(WALKey.class);
|
||||
|
||||
// should be < 0 (@see HLogKey#readFields(DataInput))
|
||||
// version 2 supports WAL compression
|
||||
// public members here are only public because of HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected enum Version {
|
||||
UNVERSIONED(0),
|
||||
// Initial number we put on WALKey when we introduced versioning.
|
||||
INITIAL(-1),
|
||||
// Version -2 introduced a dictionary compression facility. Only this
|
||||
// dictionary-based compression is available in version -2.
|
||||
COMPRESSED(-2);
|
||||
|
||||
public final int code;
|
||||
static final Version[] byCode;
|
||||
static {
|
||||
byCode = Version.values();
|
||||
for (int i = 0; i < byCode.length; i++) {
|
||||
if (byCode[i].code != -1 * i) {
|
||||
throw new AssertionError("Values in this enum should be descending by one");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Version(int code) {
|
||||
this.code = code;
|
||||
}
|
||||
|
||||
public boolean atLeast(Version other) {
|
||||
return code <= other.code;
|
||||
}
|
||||
|
||||
public static Version fromCode(int code) {
|
||||
return byCode[code * -1];
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* This is used for reading the log entries created by the previous releases
|
||||
* (0.94.11) which write the clusters information to the scopes of WALEdit.
|
||||
*/
|
||||
private static final String PREFIX_CLUSTER_KEY = ".";
|
||||
|
||||
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected static final Version VERSION = Version.COMPRESSED;
|
||||
|
||||
/** Used to represent when a particular wal key doesn't know/care about the sequence ordering. */
|
||||
public static final long NO_SEQUENCE_ID = -1;
|
||||
|
||||
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected byte [] encodedRegionName;
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected TableName tablename;
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected long logSeqNum;
|
||||
private long origLogSeqNum = 0;
|
||||
private CountDownLatch seqNumAssignedLatch = new CountDownLatch(1);
|
||||
// Time at which this edit was written.
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected long writeTime;
|
||||
|
||||
// The first element in the list is the cluster id on which the change has originated
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected List<UUID> clusterIds;
|
||||
|
||||
private NavigableMap<byte[], Integer> scopes;
|
||||
|
||||
private long nonceGroup = HConstants.NO_NONCE;
|
||||
private long nonce = HConstants.NO_NONCE;
|
||||
static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
|
||||
|
||||
// visible for deprecated HLogKey
|
||||
@InterfaceAudience.Private
|
||||
protected CompressionContext compressionContext;
|
||||
|
||||
public WALKey() {
|
||||
init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
|
||||
new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public WALKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
|
||||
final long now, UUID clusterId) {
|
||||
List<UUID> clusterIds = new ArrayList<UUID>();
|
||||
clusterIds.add(clusterId);
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
public WALKey(final byte[] encodedRegionName, final TableName tablename) {
|
||||
this(encodedRegionName, tablename, System.currentTimeMillis());
|
||||
}
|
||||
|
||||
public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
|
||||
init(encodedRegionName, tablename, NO_SEQUENCE_ID, now,
|
||||
EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the log key for writing to somewhere.
|
||||
* We maintain the tablename mainly for debugging purposes.
|
||||
* A regionName is always a sub-table object.
|
||||
* <p>Used by log splitting and snapshots.
|
||||
*
|
||||
* @param encodedRegionName Encoded name of the region as returned by
|
||||
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
|
||||
* @param tablename - name of table
|
||||
* @param logSeqNum - log sequence number
|
||||
* @param now Time at which this edit was written.
|
||||
* @param clusterIds the clusters that have consumed the change(used in Replication)
|
||||
*/
|
||||
public WALKey(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the log key for writing to somewhere.
|
||||
* We maintain the tablename mainly for debugging purposes.
|
||||
* A regionName is always a sub-table object.
|
||||
*
|
||||
* @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
|
||||
* @param nonce
|
||||
*/
|
||||
public WALKey(final byte [] encodedRegionName, final TableName tablename,
|
||||
final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds,
|
||||
nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the log key for writing to somewhere.
|
||||
* We maintain the tablename mainly for debugging purposes.
|
||||
* A regionName is always a sub-table object.
|
||||
*
|
||||
* @param encodedRegionName Encoded name of the region as returned by
|
||||
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
|
||||
* @param tablename
|
||||
* @param logSeqNum
|
||||
* @param nonceGroup
|
||||
* @param nonce
|
||||
*/
|
||||
public WALKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum,
|
||||
long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(),
|
||||
EMPTY_UUIDS, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
protected void init(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
this.logSeqNum = logSeqNum;
|
||||
this.writeTime = now;
|
||||
this.clusterIds = clusterIds;
|
||||
this.encodedRegionName = encodedRegionName;
|
||||
this.tablename = tablename;
|
||||
this.nonceGroup = nonceGroup;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param compressionContext Compression context to use
|
||||
*/
|
||||
public void setCompressionContext(CompressionContext compressionContext) {
|
||||
this.compressionContext = compressionContext;
|
||||
}
|
||||
|
||||
/** @return encoded region name */
|
||||
public byte [] getEncodedRegionName() {
|
||||
return encodedRegionName;
|
||||
}
|
||||
|
||||
/** @return table name */
|
||||
public TableName getTablename() {
|
||||
return tablename;
|
||||
}
|
||||
|
||||
/** @return log sequence number */
|
||||
public long getLogSeqNum() {
|
||||
return this.logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allow that the log sequence id to be set post-construction and release all waiters on assigned
|
||||
* sequence number.
|
||||
* Only public for {@link org.apache.hadoop.hbase.regionserver.wal.FSWALEntry}
|
||||
* @param sequence
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public void setLogSeqNum(final long sequence) {
|
||||
this.logSeqNum = sequence;
|
||||
this.seqNumAssignedLatch.countDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to set original seq Id for WALKey during wal replay
|
||||
* @param seqId
|
||||
*/
|
||||
public void setOrigLogSeqNum(final long seqId) {
|
||||
this.origLogSeqNum = seqId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a positive long if current WALKey is created from a replay edit
|
||||
* @return original sequence number of the WALEdit
|
||||
*/
|
||||
public long getOrigLogSeqNum() {
|
||||
return this.origLogSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for sequence number is assigned & return the assigned value
|
||||
* @return long the new assigned sequence number
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public long getSequenceId() throws IOException {
|
||||
try {
|
||||
this.seqNumAssignedLatch.await();
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Thread interrupted waiting for next log sequence number");
|
||||
InterruptedIOException iie = new InterruptedIOException();
|
||||
iie.initCause(ie);
|
||||
throw iie;
|
||||
}
|
||||
return this.logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the write time
|
||||
*/
|
||||
public long getWriteTime() {
|
||||
return this.writeTime;
|
||||
}
|
||||
|
||||
public NavigableMap<byte[], Integer> getScopes() {
|
||||
return scopes;
|
||||
}
|
||||
|
||||
/** @return The nonce group */
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup;
|
||||
}
|
||||
|
||||
/** @return The nonce */
|
||||
public long getNonce() {
|
||||
return nonce;
|
||||
}
|
||||
|
||||
public void setScopes(NavigableMap<byte[], Integer> scopes) {
|
||||
this.scopes = scopes;
|
||||
}
|
||||
|
||||
public void readOlderScopes(NavigableMap<byte[], Integer> scopes) {
|
||||
if (scopes != null) {
|
||||
Iterator<Map.Entry<byte[], Integer>> iterator = scopes.entrySet()
|
||||
.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Map.Entry<byte[], Integer> scope = iterator.next();
|
||||
String key = Bytes.toString(scope.getKey());
|
||||
if (key.startsWith(PREFIX_CLUSTER_KEY)) {
|
||||
addClusterId(UUID.fromString(key.substring(PREFIX_CLUSTER_KEY
|
||||
.length())));
|
||||
iterator.remove();
|
||||
}
|
||||
}
|
||||
if (scopes.size() > 0) {
|
||||
this.scopes = scopes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks that the cluster with the given clusterId has consumed the change
|
||||
*/
|
||||
public void addClusterId(UUID clusterId) {
|
||||
if (!clusterIds.contains(clusterId)) {
|
||||
clusterIds.add(clusterId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the set of cluster Ids that have consumed the change
|
||||
*/
|
||||
public List<UUID> getClusterIds() {
|
||||
return clusterIds;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the cluster id on which the change has originated. It there is no such cluster, it
|
||||
* returns DEFAULT_CLUSTER_ID (cases where replication is not enabled)
|
||||
*/
|
||||
public UUID getOriginatingClusterId(){
|
||||
return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
|
||||
logSeqNum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Produces a string map for this key. Useful for programmatic use and
|
||||
* manipulation of the data stored in an WALKey, for example, printing
|
||||
* as JSON.
|
||||
*
|
||||
* @return a Map containing data from this key
|
||||
*/
|
||||
public Map<String, Object> toStringMap() {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
stringMap.put("table", tablename);
|
||||
stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
|
||||
stringMap.put("sequence", logSeqNum);
|
||||
return stringMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((WALKey)obj) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = Bytes.hashCode(this.encodedRegionName);
|
||||
result ^= this.logSeqNum;
|
||||
result ^= this.writeTime;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(WALKey o) {
|
||||
int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName);
|
||||
if (result == 0) {
|
||||
if (this.logSeqNum < o.logSeqNum) {
|
||||
result = -1;
|
||||
} else if (this.logSeqNum > o.logSeqNum) {
|
||||
result = 1;
|
||||
}
|
||||
if (result == 0) {
|
||||
if (this.writeTime < o.writeTime) {
|
||||
result = -1;
|
||||
} else if (this.writeTime > o.writeTime) {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
// why isn't cluster id accounted for?
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Drop this instance's tablename byte array and instead
|
||||
* hold a reference to the provided tablename. This is not
|
||||
* meant to be a general purpose setter - it's only used
|
||||
* to collapse references to conserve memory.
|
||||
*/
|
||||
void internTableName(TableName tablename) {
|
||||
// We should not use this as a setter - only to swap
|
||||
// in a new reference to the same table name.
|
||||
assert tablename.equals(this.tablename);
|
||||
this.tablename = tablename;
|
||||
}
|
||||
|
||||
/**
|
||||
* Drop this instance's region name byte array and instead
|
||||
* hold a reference to the provided region name. This is not
|
||||
* meant to be a general purpose setter - it's only used
|
||||
* to collapse references to conserve memory.
|
||||
*/
|
||||
void internEncodedRegionName(byte []encodedRegionName) {
|
||||
// We should not use this as a setter - only to swap
|
||||
// in a new reference to the same table name.
|
||||
assert Bytes.equals(this.encodedRegionName, encodedRegionName);
|
||||
this.encodedRegionName = encodedRegionName;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor)
|
||||
throws IOException {
|
||||
org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder builder = org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.newBuilder();
|
||||
if (compressionContext == null) {
|
||||
builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName));
|
||||
builder.setTableName(ByteStringer.wrap(this.tablename.getName()));
|
||||
} else {
|
||||
builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
|
||||
compressionContext.regionDict));
|
||||
builder.setTableName(compressor.compress(this.tablename.getName(),
|
||||
compressionContext.tableDict));
|
||||
}
|
||||
builder.setLogSequenceNumber(this.logSeqNum);
|
||||
builder.setWriteTime(writeTime);
|
||||
if(this.origLogSeqNum > 0) {
|
||||
builder.setOrigSequenceNumber(this.origLogSeqNum);
|
||||
}
|
||||
if (this.nonce != HConstants.NO_NONCE) {
|
||||
builder.setNonce(nonce);
|
||||
}
|
||||
if (this.nonceGroup != HConstants.NO_NONCE) {
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
}
|
||||
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
|
||||
for (UUID clusterId : clusterIds) {
|
||||
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
|
||||
uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
|
||||
builder.addClusterIds(uuidBuilder.build());
|
||||
}
|
||||
if (scopes != null) {
|
||||
for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
|
||||
ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey())
|
||||
: compressor.compress(e.getKey(), compressionContext.familyDict);
|
||||
builder.addScopes(FamilyScope.newBuilder()
|
||||
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));
|
||||
}
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
public void readFieldsFromPb(
|
||||
org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKey, WALCellCodec.ByteStringUncompressor uncompressor) throws IOException {
|
||||
if (this.compressionContext != null) {
|
||||
this.encodedRegionName = uncompressor.uncompress(
|
||||
walKey.getEncodedRegionName(), compressionContext.regionDict);
|
||||
byte[] tablenameBytes = uncompressor.uncompress(
|
||||
walKey.getTableName(), compressionContext.tableDict);
|
||||
this.tablename = TableName.valueOf(tablenameBytes);
|
||||
} else {
|
||||
this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
|
||||
this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
|
||||
}
|
||||
clusterIds.clear();
|
||||
if (walKey.hasClusterId()) {
|
||||
//When we are reading the older log (0.95.1 release)
|
||||
//This is definitely the originating cluster
|
||||
clusterIds.add(new UUID(walKey.getClusterId().getMostSigBits(), walKey.getClusterId()
|
||||
.getLeastSigBits()));
|
||||
}
|
||||
for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
|
||||
clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
|
||||
}
|
||||
if (walKey.hasNonceGroup()) {
|
||||
this.nonceGroup = walKey.getNonceGroup();
|
||||
}
|
||||
if (walKey.hasNonce()) {
|
||||
this.nonce = walKey.getNonce();
|
||||
}
|
||||
this.scopes = null;
|
||||
if (walKey.getScopesCount() > 0) {
|
||||
this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
|
||||
for (FamilyScope scope : walKey.getScopesList()) {
|
||||
byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
|
||||
uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
|
||||
this.scopes.put(family, scope.getScopeType().getNumber());
|
||||
}
|
||||
}
|
||||
this.logSeqNum = walKey.getLogSequenceNumber();
|
||||
this.writeTime = walKey.getWriteTime();
|
||||
if(walKey.hasOrigSequenceNumber()) {
|
||||
this.origLogSeqNum = walKey.getOrigSequenceNumber();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,407 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.CommandLineParser;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.cli.PosixParser;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
// imports for things that haven't moved yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* WALPrettyPrinter prints the contents of a given WAL with a variety of
|
||||
* options affecting formatting and extent of content.
|
||||
*
|
||||
* It targets two usage cases: pretty printing for ease of debugging directly by
|
||||
* humans, and JSON output for consumption by monitoring and/or maintenance
|
||||
* scripts.
|
||||
*
|
||||
* It can filter by row, region, or sequence id.
|
||||
*
|
||||
* It can also toggle output of values.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALPrettyPrinter {
|
||||
private boolean outputValues;
|
||||
private boolean outputJSON;
|
||||
// The following enable filtering by sequence, region, and row, respectively
|
||||
private long sequence;
|
||||
private String region;
|
||||
private String row;
|
||||
// enable in order to output a single list of transactions from several files
|
||||
private boolean persistentOutput;
|
||||
private boolean firstTxn;
|
||||
// useful for programatic capture of JSON output
|
||||
private PrintStream out;
|
||||
// for JSON encoding
|
||||
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||
|
||||
/**
|
||||
* Basic constructor that simply initializes values to reasonable defaults.
|
||||
*/
|
||||
public WALPrettyPrinter() {
|
||||
outputValues = false;
|
||||
outputJSON = false;
|
||||
sequence = -1;
|
||||
region = null;
|
||||
row = null;
|
||||
persistentOutput = false;
|
||||
firstTxn = true;
|
||||
out = System.out;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fully specified constructor.
|
||||
*
|
||||
* @param outputValues
|
||||
* when true, enables output of values along with other log
|
||||
* information
|
||||
* @param outputJSON
|
||||
* when true, enables output in JSON format rather than a
|
||||
* "pretty string"
|
||||
* @param sequence
|
||||
* when nonnegative, serves as a filter; only log entries with this
|
||||
* sequence id will be printed
|
||||
* @param region
|
||||
* when not null, serves as a filter; only log entries from this
|
||||
* region will be printed
|
||||
* @param row
|
||||
* when not null, serves as a filter; only log entries from this row
|
||||
* will be printed
|
||||
* @param persistentOutput
|
||||
* keeps a single list running for multiple files. if enabled, the
|
||||
* endPersistentOutput() method must be used!
|
||||
* @param out
|
||||
* Specifies an alternative to stdout for the destination of this
|
||||
* PrettyPrinter's output.
|
||||
*/
|
||||
public WALPrettyPrinter(boolean outputValues, boolean outputJSON,
|
||||
long sequence, String region, String row, boolean persistentOutput,
|
||||
PrintStream out) {
|
||||
this.outputValues = outputValues;
|
||||
this.outputJSON = outputJSON;
|
||||
this.sequence = sequence;
|
||||
this.region = region;
|
||||
this.row = row;
|
||||
this.persistentOutput = persistentOutput;
|
||||
if (persistentOutput) {
|
||||
beginPersistentOutput();
|
||||
}
|
||||
this.out = out;
|
||||
this.firstTxn = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns value output on
|
||||
*/
|
||||
public void enableValues() {
|
||||
outputValues = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns value output off
|
||||
*/
|
||||
public void disableValues() {
|
||||
outputValues = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns JSON output on
|
||||
*/
|
||||
public void enableJSON() {
|
||||
outputJSON = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* turns JSON output off, and turns on "pretty strings" for human consumption
|
||||
*/
|
||||
public void disableJSON() {
|
||||
outputJSON = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param sequence
|
||||
* when nonnegative, serves as a filter; only log entries with this
|
||||
* sequence id will be printed
|
||||
*/
|
||||
public void setSequenceFilter(long sequence) {
|
||||
this.sequence = sequence;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param region
|
||||
* when not null, serves as a filter; only log entries from this
|
||||
* region will be printed
|
||||
*/
|
||||
public void setRegionFilter(String region) {
|
||||
this.region = region;
|
||||
}
|
||||
|
||||
/**
|
||||
* sets the region by which output will be filtered
|
||||
*
|
||||
* @param row
|
||||
* when not null, serves as a filter; only log entries from this row
|
||||
* will be printed
|
||||
*/
|
||||
public void setRowFilter(String row) {
|
||||
this.row = row;
|
||||
}
|
||||
|
||||
/**
|
||||
* enables output as a single, persistent list. at present, only relevant in
|
||||
* the case of JSON output.
|
||||
*/
|
||||
public void beginPersistentOutput() {
|
||||
if (persistentOutput)
|
||||
return;
|
||||
persistentOutput = true;
|
||||
firstTxn = true;
|
||||
if (outputJSON)
|
||||
out.print("[");
|
||||
}
|
||||
|
||||
/**
|
||||
* ends output of a single, persistent list. at present, only relevant in the
|
||||
* case of JSON output.
|
||||
*/
|
||||
public void endPersistentOutput() {
|
||||
if (!persistentOutput)
|
||||
return;
|
||||
persistentOutput = false;
|
||||
if (outputJSON)
|
||||
out.print("]");
|
||||
}
|
||||
|
||||
/**
|
||||
* reads a log file and outputs its contents, one transaction at a time, as
|
||||
* specified by the currently configured options
|
||||
*
|
||||
* @param conf
|
||||
* the HBase configuration relevant to this log file
|
||||
* @param p
|
||||
* the path of the log file to be read
|
||||
* @throws IOException
|
||||
* may be unable to access the configured filesystem or requested
|
||||
* file.
|
||||
*/
|
||||
public void processFile(final Configuration conf, final Path p)
|
||||
throws IOException {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
if (!fs.exists(p)) {
|
||||
throw new FileNotFoundException(p.toString());
|
||||
}
|
||||
if (!fs.isFile(p)) {
|
||||
throw new IOException(p + " is not a file");
|
||||
}
|
||||
if (outputJSON && !persistentOutput) {
|
||||
out.print("[");
|
||||
firstTxn = true;
|
||||
}
|
||||
WAL.Reader log = WALFactory.createReader(fs, p, conf);
|
||||
try {
|
||||
WAL.Entry entry;
|
||||
while ((entry = log.next()) != null) {
|
||||
WALKey key = entry.getKey();
|
||||
WALEdit edit = entry.getEdit();
|
||||
// begin building a transaction structure
|
||||
Map<String, Object> txn = key.toStringMap();
|
||||
long writeTime = key.getWriteTime();
|
||||
// check output filters
|
||||
if (sequence >= 0 && ((Long) txn.get("sequence")) != sequence)
|
||||
continue;
|
||||
if (region != null && !((String) txn.get("region")).equals(region))
|
||||
continue;
|
||||
// initialize list into which we will store atomic actions
|
||||
List<Map> actions = new ArrayList<Map>();
|
||||
for (Cell cell : edit.getCells()) {
|
||||
// add atomic operation to txn
|
||||
Map<String, Object> op = new HashMap<String, Object>(toStringMap(cell));
|
||||
if (outputValues) op.put("value", Bytes.toStringBinary(cell.getValue()));
|
||||
// check row output filter
|
||||
if (row == null || ((String) op.get("row")).equals(row))
|
||||
actions.add(op);
|
||||
}
|
||||
if (actions.size() == 0)
|
||||
continue;
|
||||
txn.put("actions", actions);
|
||||
if (outputJSON) {
|
||||
// JSON output is a straightforward "toString" on the txn object
|
||||
if (firstTxn)
|
||||
firstTxn = false;
|
||||
else
|
||||
out.print(",");
|
||||
// encode and print JSON
|
||||
out.print(MAPPER.writeValueAsString(txn));
|
||||
} else {
|
||||
// Pretty output, complete with indentation by atomic action
|
||||
out.println("Sequence " + txn.get("sequence") + " "
|
||||
+ "from region " + txn.get("region") + " " + "in table "
|
||||
+ txn.get("table") + " at write timestamp: " + new Date(writeTime));
|
||||
for (int i = 0; i < actions.size(); i++) {
|
||||
Map op = actions.get(i);
|
||||
out.println(" Action:");
|
||||
out.println(" row: " + op.get("row"));
|
||||
out.println(" column: " + op.get("family") + ":"
|
||||
+ op.get("qualifier"));
|
||||
out.println(" timestamp: "
|
||||
+ (new Date((Long) op.get("timestamp"))));
|
||||
if(op.get("tag") != null) {
|
||||
out.println(" tag: " + op.get("tag"));
|
||||
}
|
||||
if (outputValues)
|
||||
out.println(" value: " + op.get("value"));
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
log.close();
|
||||
}
|
||||
if (outputJSON && !persistentOutput) {
|
||||
out.print("]");
|
||||
}
|
||||
}
|
||||
|
||||
private static Map<String, Object> toStringMap(Cell cell) {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
stringMap.put("row",
|
||||
Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
|
||||
stringMap.put("family", Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength()));
|
||||
stringMap.put("qualifier",
|
||||
Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength()));
|
||||
stringMap.put("timestamp", cell.getTimestamp());
|
||||
stringMap.put("vlen", cell.getValueLength());
|
||||
if (cell.getTagsLength() > 0) {
|
||||
List<String> tagsString = new ArrayList<String>();
|
||||
Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
while (tagsIterator.hasNext()) {
|
||||
Tag tag = tagsIterator.next();
|
||||
tagsString.add((tag.getType()) + ":"
|
||||
+ Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
|
||||
}
|
||||
stringMap.put("tag", tagsString);
|
||||
}
|
||||
return stringMap;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
run(args);
|
||||
}
|
||||
|
||||
/**
|
||||
* Pass one or more log file names and formatting options and it will dump out
|
||||
* a text version of the contents on <code>stdout</code>.
|
||||
*
|
||||
* @param args
|
||||
* Command line arguments
|
||||
* @throws IOException
|
||||
* Thrown upon file system errors etc.
|
||||
* @throws ParseException
|
||||
* Thrown if command-line parsing fails.
|
||||
*/
|
||||
public static void run(String[] args) throws IOException {
|
||||
// create options
|
||||
Options options = new Options();
|
||||
options.addOption("h", "help", false, "Output help message");
|
||||
options.addOption("j", "json", false, "Output JSON");
|
||||
options.addOption("p", "printvals", false, "Print values");
|
||||
options.addOption("r", "region", true,
|
||||
"Region to filter by. Pass region name; e.g. 'hbase:meta,,1'");
|
||||
options.addOption("s", "sequence", true,
|
||||
"Sequence to filter by. Pass sequence number.");
|
||||
options.addOption("w", "row", true, "Row to filter by. Pass row name.");
|
||||
|
||||
WALPrettyPrinter printer = new WALPrettyPrinter();
|
||||
CommandLineParser parser = new PosixParser();
|
||||
List files = null;
|
||||
try {
|
||||
CommandLine cmd = parser.parse(options, args);
|
||||
files = cmd.getArgList();
|
||||
if (files.size() == 0 || cmd.hasOption("h")) {
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("WAL <filename...>", options, true);
|
||||
System.exit(-1);
|
||||
}
|
||||
// configure the pretty printer using command line options
|
||||
if (cmd.hasOption("p"))
|
||||
printer.enableValues();
|
||||
if (cmd.hasOption("j"))
|
||||
printer.enableJSON();
|
||||
if (cmd.hasOption("r"))
|
||||
printer.setRegionFilter(cmd.getOptionValue("r"));
|
||||
if (cmd.hasOption("s"))
|
||||
printer.setSequenceFilter(Long.parseLong(cmd.getOptionValue("s")));
|
||||
if (cmd.hasOption("w"))
|
||||
printer.setRowFilter(cmd.getOptionValue("w"));
|
||||
} catch (ParseException e) {
|
||||
e.printStackTrace();
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("HFile filename(s) ", options, true);
|
||||
System.exit(-1);
|
||||
}
|
||||
// get configuration, file system, and process the given files
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
|
||||
|
||||
// begin output
|
||||
printer.beginPersistentOutput();
|
||||
for (Object f : files) {
|
||||
Path file = new Path((String) f);
|
||||
FileSystem fs = file.getFileSystem(conf);
|
||||
if (!fs.exists(file)) {
|
||||
System.err.println("ERROR, file doesnt exist: " + file);
|
||||
return;
|
||||
}
|
||||
printer.processFile(conf, file);
|
||||
}
|
||||
printer.endPersistentOutput();
|
||||
}
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
|
||||
/**
|
||||
* The Write Ahead Log (WAL) stores all durable edits to the HRegion.
|
||||
* This interface provides the entry point for all WAL implementors.
|
||||
* <p>
|
||||
* See {@link DefaultWALProvider} for an example implementation.
|
||||
*
|
||||
* A single WALProvider will be used for retrieving multiple WALs in a particular region server
|
||||
* and must be threadsafe.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface WALProvider {
|
||||
|
||||
/**
|
||||
* Set up the provider to create wals.
|
||||
* will only be called once per instance.
|
||||
* @param factory factory that made us may not be null
|
||||
* @param conf may not be null
|
||||
* @param listeners may be null
|
||||
* @param providerId differentiate between providers from one factory. may be null
|
||||
*/
|
||||
void init(final WALFactory factory, final Configuration conf,
|
||||
final List<WALActionsListener> listeners, final String providerId) throws IOException;
|
||||
|
||||
/**
|
||||
* @param identifier may not be null. contents will not be altered.
|
||||
* @return a WAL for writing entries for the given region.
|
||||
*/
|
||||
WAL getWAL(final byte[] identifier) throws IOException;
|
||||
|
||||
/**
|
||||
* persist outstanding WALs to storage and stop accepting new appends.
|
||||
* This method serves as shorthand for sending a sync to every WAL provided by a given
|
||||
* implementation. Those WALs will also stop accepting new writes.
|
||||
*/
|
||||
void shutdown() throws IOException;
|
||||
|
||||
/**
|
||||
* shutdown utstanding WALs and clean up any persisted state.
|
||||
* Call this method only when you will not need to replay any of the edits to the WALs from
|
||||
* this provider. After this call completes, the underlying resources should have been reclaimed.
|
||||
*/
|
||||
void close() throws IOException;
|
||||
|
||||
// Writers are used internally. Users outside of the WAL should be relying on the
|
||||
// interface provided by WAL.
|
||||
interface Writer extends Closeable {
|
||||
void sync() throws IOException;
|
||||
void append(WAL.Entry entry) throws IOException;
|
||||
long getLength() throws IOException;
|
||||
}
|
||||
|
||||
}
|
@ -16,7 +16,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.FileNotFoundException;
|
||||
@ -29,6 +29,7 @@ import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
@ -45,11 +46,13 @@ import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
@ -57,6 +60,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
@ -71,11 +75,11 @@ import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagRewriteCell;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
|
||||
@ -83,6 +87,7 @@ import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
@ -94,15 +99,14 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.LastSequenceId;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Reader;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
@ -114,14 +118,24 @@ import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
|
||||
/**
|
||||
* This class is responsible for splitting up a bunch of regionserver commit log
|
||||
* files that are no longer being written to, into new files, one per region for
|
||||
* region to replay on startup. Delete the old log files when finished.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HLogSplitter {
|
||||
static final Log LOG = LogFactory.getLog(HLogSplitter.class);
|
||||
public class WALSplitter {
|
||||
static final Log LOG = LogFactory.getLog(WALSplitter.class);
|
||||
|
||||
/** By default we retry errors in splitting, rather than skipping. */
|
||||
public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
|
||||
|
||||
// Parameters for split process
|
||||
protected final Path rootDir;
|
||||
@ -137,6 +151,7 @@ public class HLogSplitter {
|
||||
private Set<TableName> disablingOrDisabledTables =
|
||||
new HashSet<TableName>();
|
||||
private BaseCoordinatedStateManager csm;
|
||||
private final WALFactory walFactory;
|
||||
|
||||
private MonitoredTask status;
|
||||
|
||||
@ -161,7 +176,7 @@ public class HLogSplitter {
|
||||
// Min batch size when replay WAL edits
|
||||
private final int minBatchSize;
|
||||
|
||||
HLogSplitter(Configuration conf, Path rootDir,
|
||||
WALSplitter(final WALFactory factory, Configuration conf, Path rootDir,
|
||||
FileSystem fs, LastSequenceId idChecker,
|
||||
CoordinatedStateManager csm, RecoveryMode mode) {
|
||||
this.conf = HBaseConfiguration.create(conf);
|
||||
@ -172,6 +187,7 @@ public class HLogSplitter {
|
||||
this.fs = fs;
|
||||
this.sequenceIdChecker = idChecker;
|
||||
this.csm = (BaseCoordinatedStateManager)csm;
|
||||
this.walFactory = factory;
|
||||
this.controller = new PipelineController();
|
||||
|
||||
entryBuffers = new EntryBuffers(controller,
|
||||
@ -197,7 +213,7 @@ public class HLogSplitter {
|
||||
}
|
||||
|
||||
/**
|
||||
* Splits a HLog file into region's recovered-edits directory.
|
||||
* Splits a WAL file into region's recovered-edits directory.
|
||||
* This is the main entry point for distributed log splitting from SplitLogWorker.
|
||||
* <p>
|
||||
* If the log file has N regions then N recovered.edits files will be produced.
|
||||
@ -214,23 +230,25 @@ public class HLogSplitter {
|
||||
*/
|
||||
public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
|
||||
Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
|
||||
CoordinatedStateManager cp, RecoveryMode mode) throws IOException {
|
||||
HLogSplitter s = new HLogSplitter(conf, rootDir, fs, idChecker, cp, mode);
|
||||
CoordinatedStateManager cp, RecoveryMode mode, final WALFactory factory) throws IOException {
|
||||
WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, idChecker, cp, mode);
|
||||
return s.splitLogFile(logfile, reporter);
|
||||
}
|
||||
|
||||
// A wrapper to split one log folder using the method used by distributed
|
||||
// log splitting. Used by tools and unit tests. It should be package private.
|
||||
// It is public only because TestWALObserver is in a different package,
|
||||
// which uses this method to to log splitting.
|
||||
// which uses this method to do log splitting.
|
||||
@VisibleForTesting
|
||||
public static List<Path> split(Path rootDir, Path logDir, Path oldLogDir,
|
||||
FileSystem fs, Configuration conf) throws IOException {
|
||||
FileStatus[] logfiles = fs.listStatus(logDir);
|
||||
FileSystem fs, Configuration conf, final WALFactory factory) throws IOException {
|
||||
final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
|
||||
Collections.singletonList(logDir), null);
|
||||
List<Path> splits = new ArrayList<Path>();
|
||||
if (logfiles != null && logfiles.length > 0) {
|
||||
for (FileStatus logfile: logfiles) {
|
||||
HLogSplitter s =
|
||||
new HLogSplitter(conf, rootDir, fs, null, null, RecoveryMode.LOG_SPLITTING);
|
||||
WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, null,
|
||||
RecoveryMode.LOG_SPLITTING);
|
||||
if (s.splitLogFile(logfile, null)) {
|
||||
finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
|
||||
if (s.outputSink.splits != null) {
|
||||
@ -245,13 +263,18 @@ public class HLogSplitter {
|
||||
return splits;
|
||||
}
|
||||
|
||||
// The real log splitter. It just splits one log file.
|
||||
/**
|
||||
* log splitting implementation, splits one log file.
|
||||
* @param logfile should be an actual log file.
|
||||
*/
|
||||
boolean splitLogFile(FileStatus logfile,
|
||||
CancelableProgressable reporter) throws IOException {
|
||||
boolean isCorrupted = false;
|
||||
Preconditions.checkState(status == null);
|
||||
Preconditions.checkArgument(logfile.isFile(),
|
||||
"passed in file status is for something other than a regular file.");
|
||||
boolean isCorrupted = false;
|
||||
boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
|
||||
HLog.SPLIT_SKIP_ERRORS_DEFAULT);
|
||||
SPLIT_SKIP_ERRORS_DEFAULT);
|
||||
int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
|
||||
Path logPath = logfile.getPath();
|
||||
boolean outputSinkStarted = false;
|
||||
@ -262,18 +285,18 @@ public class HLogSplitter {
|
||||
status =
|
||||
TaskMonitor.get().createStatus(
|
||||
"Splitting log file " + logfile.getPath() + "into a temporary staging area.");
|
||||
Reader in = null;
|
||||
try {
|
||||
long logLength = logfile.getLen();
|
||||
LOG.info("Splitting hlog: " + logPath + ", length=" + logLength);
|
||||
LOG.info("Splitting wal: " + logPath + ", length=" + logLength);
|
||||
LOG.info("DistributedLogReplay = " + this.distributedLogReplay);
|
||||
status.setStatus("Opening log file");
|
||||
if (reporter != null && !reporter.progress()) {
|
||||
progress_failed = true;
|
||||
return false;
|
||||
}
|
||||
Reader in = null;
|
||||
try {
|
||||
in = getReader(fs, logfile, conf, skipErrors, reporter);
|
||||
in = getReader(logfile, skipErrors, reporter);
|
||||
} catch (CorruptedLogFileException e) {
|
||||
LOG.warn("Could not get reader, corrupted log file " + logPath, e);
|
||||
ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
|
||||
@ -300,7 +323,7 @@ public class HLogSplitter {
|
||||
outputSinkStarted = true;
|
||||
Entry entry;
|
||||
Long lastFlushedSequenceId = -1L;
|
||||
ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logPath);
|
||||
ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logPath);
|
||||
failedServerName = (serverName == null) ? "" : serverName.getServerName();
|
||||
while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
|
||||
byte[] region = entry.getKey().getEncodedRegionName();
|
||||
@ -356,6 +379,14 @@ public class HLogSplitter {
|
||||
throw e;
|
||||
} finally {
|
||||
LOG.debug("Finishing writing output logs and closing down.");
|
||||
try {
|
||||
if (null != in) {
|
||||
in.close();
|
||||
}
|
||||
} catch (IOException exception) {
|
||||
LOG.warn("Could not close wal reader: " + exception.getMessage());
|
||||
LOG.debug("exception details", exception);
|
||||
}
|
||||
try {
|
||||
if (outputSinkStarted) {
|
||||
// Set progress_failed to true as the immediate following statement will reset its value
|
||||
@ -453,7 +484,7 @@ public class HLogSplitter {
|
||||
}
|
||||
|
||||
for (Path p : processedLogs) {
|
||||
Path newPath = FSHLog.getHLogArchivePath(oldLogDir, p);
|
||||
Path newPath = FSHLog.getWALArchivePath(oldLogDir, p);
|
||||
if (fs.exists(p)) {
|
||||
if (!FSUtils.renameAndSetModifyTime(fs, p, newPath)) {
|
||||
LOG.warn("Unable to move " + p + " to " + newPath);
|
||||
@ -483,7 +514,7 @@ public class HLogSplitter {
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
|
||||
String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
|
||||
Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
|
||||
Path dir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
|
||||
Path dir = getRegionDirRecoveredEditsDir(regiondir);
|
||||
|
||||
if (!fs.exists(regiondir)) {
|
||||
LOG.info("This region's directory doesn't exist: "
|
||||
@ -517,7 +548,7 @@ public class HLogSplitter {
|
||||
}
|
||||
|
||||
static String getTmpRecoveredEditsFileName(String fileName) {
|
||||
return fileName + HLog.RECOVERED_LOG_TMPFILE_SUFFIX;
|
||||
return fileName + RECOVERED_LOG_TMPFILE_SUFFIX;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -538,18 +569,158 @@ public class HLogSplitter {
|
||||
return String.format("%019d", seqid);
|
||||
}
|
||||
|
||||
private static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
|
||||
private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
|
||||
|
||||
/**
|
||||
* @param regiondir
|
||||
* This regions directory in the filesystem.
|
||||
* @return The directory that holds recovered edits files for the region
|
||||
* <code>regiondir</code>
|
||||
*/
|
||||
public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
|
||||
return new Path(regiondir, HConstants.RECOVERED_EDITS_DIR);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns sorted set of edit files made by splitter, excluding files
|
||||
* with '.temp' suffix.
|
||||
*
|
||||
* @param fs
|
||||
* @param regiondir
|
||||
* @return Files in passed <code>regiondir</code> as a sorted set.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
|
||||
final Path regiondir) throws IOException {
|
||||
NavigableSet<Path> filesSorted = new TreeSet<Path>();
|
||||
Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
|
||||
if (!fs.exists(editsdir))
|
||||
return filesSorted;
|
||||
FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
boolean result = false;
|
||||
try {
|
||||
// Return files and only files that match the editfile names pattern.
|
||||
// There can be other files in this directory other than edit files.
|
||||
// In particular, on error, we'll move aside the bad edit file giving
|
||||
// it a timestamp suffix. See moveAsideBadEditsFile.
|
||||
Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
|
||||
result = fs.isFile(p) && m.matches();
|
||||
// Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
|
||||
// because it means splitwal thread is writting this file.
|
||||
if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
|
||||
result = false;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed isFile check on " + p);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
});
|
||||
if (files == null) {
|
||||
return filesSorted;
|
||||
}
|
||||
for (FileStatus status : files) {
|
||||
filesSorted.add(status.getPath());
|
||||
}
|
||||
return filesSorted;
|
||||
}
|
||||
|
||||
/**
|
||||
* Move aside a bad edits file.
|
||||
*
|
||||
* @param fs
|
||||
* @param edits
|
||||
* Edits file to move aside.
|
||||
* @return The name of the moved aside file.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Path moveAsideBadEditsFile(final FileSystem fs, final Path edits)
|
||||
throws IOException {
|
||||
Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
|
||||
+ System.currentTimeMillis());
|
||||
if (!fs.rename(edits, moveAsideName)) {
|
||||
LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
|
||||
}
|
||||
return moveAsideName;
|
||||
}
|
||||
|
||||
private static final String SEQUENCE_ID_FILE_SUFFIX = "_seqid";
|
||||
|
||||
/**
|
||||
* Is the given file a region open sequence id file.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static boolean isSequenceIdFile(final Path file) {
|
||||
return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a file with name as region open sequence id
|
||||
*
|
||||
* @param fs
|
||||
* @param regiondir
|
||||
* @param newSeqId
|
||||
* @param saftyBumper
|
||||
* @return long new sequence Id value
|
||||
* @throws IOException
|
||||
*/
|
||||
public static long writeRegionOpenSequenceIdFile(final FileSystem fs, final Path regiondir,
|
||||
long newSeqId, long saftyBumper) throws IOException {
|
||||
|
||||
Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
|
||||
long maxSeqId = 0;
|
||||
FileStatus[] files = null;
|
||||
if (fs.exists(editsdir)) {
|
||||
files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path p) {
|
||||
return isSequenceIdFile(p);
|
||||
}
|
||||
});
|
||||
if (files != null) {
|
||||
for (FileStatus status : files) {
|
||||
String fileName = status.getPath().getName();
|
||||
try {
|
||||
Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
|
||||
- SEQUENCE_ID_FILE_SUFFIX.length()));
|
||||
maxSeqId = Math.max(tmpSeqId, maxSeqId);
|
||||
} catch (NumberFormatException ex) {
|
||||
LOG.warn("Invalid SeqId File Name=" + fileName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (maxSeqId > newSeqId) {
|
||||
newSeqId = maxSeqId;
|
||||
}
|
||||
newSeqId += saftyBumper; // bump up SeqId
|
||||
|
||||
// write a new seqId file
|
||||
Path newSeqIdFile = new Path(editsdir, newSeqId + SEQUENCE_ID_FILE_SUFFIX);
|
||||
if (!fs.createNewFile(newSeqIdFile)) {
|
||||
throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
|
||||
}
|
||||
// remove old ones
|
||||
if(files != null) {
|
||||
for (FileStatus status : files) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
return newSeqId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new {@link Reader} for reading logs to split.
|
||||
*
|
||||
* @param fs
|
||||
* @param file
|
||||
* @param conf
|
||||
* @return A new Reader instance
|
||||
* @return A new Reader instance, caller should close
|
||||
* @throws IOException
|
||||
* @throws CorruptedLogFileException
|
||||
*/
|
||||
protected Reader getReader(FileSystem fs, FileStatus file, Configuration conf,
|
||||
boolean skipErrors, CancelableProgressable reporter)
|
||||
protected Reader getReader(FileStatus file, boolean skipErrors, CancelableProgressable reporter)
|
||||
throws IOException, CorruptedLogFileException {
|
||||
Path path = file.getPath();
|
||||
long length = file.getLen();
|
||||
@ -565,7 +736,7 @@ public class HLogSplitter {
|
||||
try {
|
||||
FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf, reporter);
|
||||
try {
|
||||
in = getReader(fs, path, conf, reporter);
|
||||
in = getReader(path, reporter);
|
||||
} catch (EOFException e) {
|
||||
if (length <= 0) {
|
||||
// TODO should we ignore an empty, not-last log file if skip.errors
|
||||
@ -590,7 +761,7 @@ public class HLogSplitter {
|
||||
throw e; // Don't mark the file corrupted if interrupted, or not skipErrors
|
||||
}
|
||||
CorruptedLogFileException t =
|
||||
new CorruptedLogFileException("skipErrors=true Could not open hlog " +
|
||||
new CorruptedLogFileException("skipErrors=true Could not open wal " +
|
||||
path + " ignoring");
|
||||
t.initCause(e);
|
||||
throw t;
|
||||
@ -604,7 +775,7 @@ public class HLogSplitter {
|
||||
return in.next();
|
||||
} catch (EOFException eof) {
|
||||
// truncated files are expected if a RS crashes (see HBASE-2643)
|
||||
LOG.info("EOF from hlog " + path + ". continuing");
|
||||
LOG.info("EOF from wal " + path + ". continuing");
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
// If the IOE resulted from bad file format,
|
||||
@ -612,7 +783,7 @@ public class HLogSplitter {
|
||||
if (e.getCause() != null &&
|
||||
(e.getCause() instanceof ParseException ||
|
||||
e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
|
||||
LOG.warn("Parse exception " + e.getCause().toString() + " from hlog "
|
||||
LOG.warn("Parse exception " + e.getCause().toString() + " from wal "
|
||||
+ path + ". continuing");
|
||||
return null;
|
||||
}
|
||||
@ -621,7 +792,7 @@ public class HLogSplitter {
|
||||
}
|
||||
CorruptedLogFileException t =
|
||||
new CorruptedLogFileException("skipErrors=true Ignoring exception" +
|
||||
" while parsing hlog " + path + ". Marking as corrupted");
|
||||
" while parsing wal " + path + ". Marking as corrupted");
|
||||
t.initCause(e);
|
||||
throw t;
|
||||
}
|
||||
@ -629,18 +800,19 @@ public class HLogSplitter {
|
||||
|
||||
/**
|
||||
* Create a new {@link Writer} for writing log splits.
|
||||
* @return a new Writer instance, caller should close
|
||||
*/
|
||||
protected Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
|
||||
protected Writer createWriter(Path logfile)
|
||||
throws IOException {
|
||||
return HLogFactory.createRecoveredEditsWriter(fs, logfile, conf);
|
||||
return walFactory.createRecoveredEditsWriter(fs, logfile);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new {@link Reader} for reading logs to split.
|
||||
* @return new Reader instance, caller should close
|
||||
*/
|
||||
protected Reader getReader(FileSystem fs, Path curLogFile,
|
||||
Configuration conf, CancelableProgressable reporter) throws IOException {
|
||||
return HLogFactory.createReader(fs, curLogFile, conf, reporter);
|
||||
protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
|
||||
return walFactory.createReader(fs, curLogFile, reporter);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -718,7 +890,7 @@ public class HLogSplitter {
|
||||
* @throws IOException
|
||||
*/
|
||||
public void appendEntry(Entry entry) throws InterruptedException, IOException {
|
||||
HLogKey key = entry.getKey();
|
||||
WALKey key = entry.getKey();
|
||||
|
||||
RegionEntryBuffer buffer;
|
||||
long incrHeap;
|
||||
@ -735,7 +907,8 @@ public class HLogSplitter {
|
||||
synchronized (controller.dataAvailable) {
|
||||
totalBuffered += incrHeap;
|
||||
while (totalBuffered > maxHeapUsage && controller.thrown.get() == null) {
|
||||
LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
|
||||
LOG.debug("Used " + totalBuffered +
|
||||
" bytes of buffered edits, waiting for IO threads...");
|
||||
controller.dataAvailable.wait(2000);
|
||||
}
|
||||
controller.dataAvailable.notifyAll();
|
||||
@ -821,14 +994,14 @@ public class HLogSplitter {
|
||||
internify(entry);
|
||||
entryBuffer.add(entry);
|
||||
long incrHeap = entry.getEdit().heapSize() +
|
||||
ClassSize.align(2 * ClassSize.REFERENCE) + // HLogKey pointers
|
||||
ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
|
||||
0; // TODO linkedlist entry
|
||||
heapInBuffer += incrHeap;
|
||||
return incrHeap;
|
||||
}
|
||||
|
||||
private void internify(Entry entry) {
|
||||
HLogKey k = entry.getKey();
|
||||
WALKey k = entry.getKey();
|
||||
k.internTableName(this.tableName);
|
||||
k.internEncodedRegionName(this.encodedRegionName);
|
||||
}
|
||||
@ -1262,7 +1435,7 @@ public class HLogSplitter {
|
||||
if (blacklistedRegions.contains(region)) {
|
||||
return null;
|
||||
}
|
||||
ret = createWAP(region, entry, rootDir, fs, conf);
|
||||
ret = createWAP(region, entry, rootDir);
|
||||
if (ret == null) {
|
||||
blacklistedRegions.add(region);
|
||||
return null;
|
||||
@ -1271,8 +1444,10 @@ public class HLogSplitter {
|
||||
return ret;
|
||||
}
|
||||
|
||||
private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir, FileSystem fs,
|
||||
Configuration conf) throws IOException {
|
||||
/**
|
||||
* @return a path with a write for that path. caller should close.
|
||||
*/
|
||||
private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
|
||||
Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true);
|
||||
if (regionedits == null) {
|
||||
return null;
|
||||
@ -1285,7 +1460,7 @@ public class HLogSplitter {
|
||||
LOG.warn("Failed delete of old " + regionedits);
|
||||
}
|
||||
}
|
||||
Writer w = createWriter(fs, regionedits, conf);
|
||||
Writer w = createWriter(regionedits);
|
||||
LOG.info("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region));
|
||||
return (new WriterAndPath(regionedits, w));
|
||||
}
|
||||
@ -1308,7 +1483,9 @@ public class HLogSplitter {
|
||||
if (wap == null) {
|
||||
wap = getWriterAndPath(logEntry);
|
||||
if (wap == null) {
|
||||
// getWriterAndPath decided we don't need to write these edits
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1400,8 +1577,8 @@ public class HLogSplitter {
|
||||
* Map key -> value layout
|
||||
* <servername>:<table name> -> Queue<Row>
|
||||
*/
|
||||
private Map<String, List<Pair<HRegionLocation, HLog.Entry>>> serverToBufferQueueMap =
|
||||
new ConcurrentHashMap<String, List<Pair<HRegionLocation, HLog.Entry>>>();
|
||||
private Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
|
||||
new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
|
||||
private List<Throwable> thrown = new ArrayList<Throwable>();
|
||||
|
||||
// The following sink is used in distrubitedLogReplay mode for entries of regions in a disabling
|
||||
@ -1446,10 +1623,10 @@ public class HLogSplitter {
|
||||
// process workitems
|
||||
String maxLocKey = null;
|
||||
int maxSize = 0;
|
||||
List<Pair<HRegionLocation, HLog.Entry>> maxQueue = null;
|
||||
List<Pair<HRegionLocation, Entry>> maxQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (String key : this.serverToBufferQueueMap.keySet()) {
|
||||
List<Pair<HRegionLocation, HLog.Entry>> curQueue = this.serverToBufferQueueMap.get(key);
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = this.serverToBufferQueueMap.get(key);
|
||||
if (curQueue.size() > maxSize) {
|
||||
maxSize = curQueue.size();
|
||||
maxQueue = curQueue;
|
||||
@ -1483,7 +1660,7 @@ public class HLogSplitter {
|
||||
private void groupEditsByServer(List<Entry> entries) throws IOException {
|
||||
Set<TableName> nonExistentTables = null;
|
||||
Long cachedLastFlushedSequenceId = -1l;
|
||||
for (HLog.Entry entry : entries) {
|
||||
for (Entry entry : entries) {
|
||||
WALEdit edit = entry.getEdit();
|
||||
TableName table = entry.getKey().getTablename();
|
||||
// clear scopes which isn't needed for recovery
|
||||
@ -1556,7 +1733,7 @@ public class HLogSplitter {
|
||||
lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
|
||||
if (cachedLastFlushedSequenceId != null
|
||||
&& cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
|
||||
// skip the whole HLog entry
|
||||
// skip the whole WAL entry
|
||||
this.skippedEdits.incrementAndGet();
|
||||
needSkip = true;
|
||||
break;
|
||||
@ -1585,13 +1762,13 @@ public class HLogSplitter {
|
||||
|
||||
synchronized (serverToBufferQueueMap) {
|
||||
locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
|
||||
List<Pair<HRegionLocation, HLog.Entry>> queue = serverToBufferQueueMap.get(locKey);
|
||||
List<Pair<HRegionLocation, Entry>> queue = serverToBufferQueueMap.get(locKey);
|
||||
if (queue == null) {
|
||||
queue =
|
||||
Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, HLog.Entry>>());
|
||||
Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, Entry>>());
|
||||
serverToBufferQueueMap.put(locKey, queue);
|
||||
}
|
||||
queue.add(new Pair<HRegionLocation, HLog.Entry>(loc, entry));
|
||||
queue.add(new Pair<HRegionLocation, Entry>(loc, entry));
|
||||
}
|
||||
// store regions we have recovered so far
|
||||
addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
|
||||
@ -1660,7 +1837,7 @@ public class HLogSplitter {
|
||||
return loc;
|
||||
}
|
||||
|
||||
private void processWorkItems(String key, List<Pair<HRegionLocation, HLog.Entry>> actions)
|
||||
private void processWorkItems(String key, List<Pair<HRegionLocation, Entry>> actions)
|
||||
throws IOException {
|
||||
RegionServerWriter rsw = null;
|
||||
|
||||
@ -1742,7 +1919,7 @@ public class HLogSplitter {
|
||||
public boolean flush() throws IOException {
|
||||
String curLoc = null;
|
||||
int curSize = 0;
|
||||
List<Pair<HRegionLocation, HLog.Entry>> curQueue = null;
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (String locationKey : this.serverToBufferQueueMap.keySet()) {
|
||||
curQueue = this.serverToBufferQueueMap.get(locationKey);
|
||||
@ -1759,7 +1936,10 @@ public class HLogSplitter {
|
||||
|
||||
if (curSize > 0) {
|
||||
this.processWorkItems(curLoc, curQueue);
|
||||
controller.dataAvailable.notifyAll();
|
||||
// We should already have control of the monitor; ensure this is the case.
|
||||
synchronized(controller.dataAvailable) {
|
||||
controller.dataAvailable.notifyAll();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -1893,8 +2073,7 @@ public class HLogSplitter {
|
||||
synchronized (this.tableNameToHConnectionMap) {
|
||||
hconn = this.tableNameToHConnectionMap.get(tableName);
|
||||
if (hconn == null) {
|
||||
// Gets closed over in closeRegionServerWriters
|
||||
hconn = (HConnection)ConnectionFactory.createConnection(conf);
|
||||
hconn = HConnectionManager.getConnection(conf);
|
||||
this.tableNameToHConnectionMap.put(tableName, hconn);
|
||||
}
|
||||
}
|
||||
@ -1985,18 +2164,18 @@ public class HLogSplitter {
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is used to construct mutations from a WALEntry. It also reconstructs HLogKey &
|
||||
* This function is used to construct mutations from a WALEntry. It also reconstructs WALKey &
|
||||
* WALEdit from the passed in WALEntry
|
||||
* @param entry
|
||||
* @param cells
|
||||
* @param logEntry pair of HLogKey and WALEdit instance stores HLogKey and WALEdit instances
|
||||
* @param logEntry pair of WALKey and WALEdit instance stores WALKey and WALEdit instances
|
||||
* extracted from the passed in WALEntry.
|
||||
* @param addLogReplayTag
|
||||
* @return list of Pair<MutationType, Mutation> to be replayed
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry, CellScanner cells,
|
||||
Pair<HLogKey, WALEdit> logEntry, boolean addLogReplayTag, Durability durability)
|
||||
Pair<WALKey, WALEdit> logEntry, boolean addLogReplayTag, Durability durability)
|
||||
throws IOException {
|
||||
|
||||
if (entry == null) {
|
||||
@ -2010,7 +2189,7 @@ public class HLogSplitter {
|
||||
List<MutationReplay> mutations = new ArrayList<MutationReplay>();
|
||||
Cell previousCell = null;
|
||||
Mutation m = null;
|
||||
HLogKey key = null;
|
||||
WALKey key = null;
|
||||
WALEdit val = null;
|
||||
if (logEntry != null) val = new WALEdit();
|
||||
|
||||
@ -2054,16 +2233,17 @@ public class HLogSplitter {
|
||||
previousCell = cell;
|
||||
}
|
||||
|
||||
// reconstruct HLogKey
|
||||
// reconstruct WALKey
|
||||
if (logEntry != null) {
|
||||
WALKey walKey = entry.getKey();
|
||||
List<UUID> clusterIds = new ArrayList<UUID>(walKey.getClusterIdsCount());
|
||||
org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKeyProto = entry.getKey();
|
||||
List<UUID> clusterIds = new ArrayList<UUID>(walKeyProto.getClusterIdsCount());
|
||||
for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
|
||||
clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
|
||||
}
|
||||
key = new HLogKey(walKey.getEncodedRegionName().toByteArray(), TableName.valueOf(walKey
|
||||
.getTableName().toByteArray()), replaySeqId, walKey.getWriteTime(), clusterIds,
|
||||
walKey.getNonceGroup(), walKey.getNonce());
|
||||
// we use HLogKey here instead of WALKey directly to support legacy coprocessors.
|
||||
key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
|
||||
walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
|
||||
clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce());
|
||||
logEntry.setFirst(key);
|
||||
logEntry.setSecond(val);
|
||||
}
|
@ -0,0 +1,269 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader;
|
||||
import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.MapReduceTestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* JUnit tests for the WALRecordReader
|
||||
*/
|
||||
@Category({MapReduceTests.class, MediumTests.class})
|
||||
public class TestWALRecordReader {
|
||||
private final Log LOG = LogFactory.getLog(getClass());
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static Configuration conf;
|
||||
private static FileSystem fs;
|
||||
private static Path hbaseDir;
|
||||
// visible for TestHLogRecordReader
|
||||
static final TableName tableName = TableName.valueOf(getName());
|
||||
private static final byte [] rowName = tableName.getName();
|
||||
// visible for TestHLogRecordReader
|
||||
static final HRegionInfo info = new HRegionInfo(tableName,
|
||||
Bytes.toBytes(""), Bytes.toBytes(""), false);
|
||||
private static final byte [] family = Bytes.toBytes("column");
|
||||
private static final byte [] value = Bytes.toBytes("value");
|
||||
private static HTableDescriptor htd;
|
||||
private static Path logDir;
|
||||
|
||||
private static String getName() {
|
||||
return "TestWALRecordReader";
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
FileStatus[] entries = fs.listStatus(hbaseDir);
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
|
||||
}
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt("dfs.blocksize", 1024 * 1024);
|
||||
conf.setInt("dfs.replication", 1);
|
||||
TEST_UTIL.startMiniDFSCluster(1);
|
||||
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
fs = TEST_UTIL.getDFSCluster().getFileSystem();
|
||||
|
||||
hbaseDir = TEST_UTIL.createRootDir();
|
||||
|
||||
logDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
|
||||
|
||||
htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test partial reads from the log based on passed time range
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testPartialRead() throws Exception {
|
||||
final WALFactory walfactory = new WALFactory(conf, null, getName());
|
||||
WAL log = walfactory.getWAL(info.getEncodedNameAsBytes());
|
||||
// This test depends on timestamp being millisecond based and the filename of the WAL also
|
||||
// being millisecond based.
|
||||
long ts = System.currentTimeMillis();
|
||||
WALEdit edit = new WALEdit();
|
||||
final AtomicLong sequenceId = new AtomicLong(0);
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
|
||||
log.append(htd, info, getWalKey(ts), edit, sequenceId, true, null);
|
||||
edit = new WALEdit();
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
|
||||
log.append(htd, info, getWalKey(ts+1), edit, sequenceId,
|
||||
true, null);
|
||||
log.sync();
|
||||
LOG.info("Before 1st WAL roll " + log.toString());
|
||||
log.rollWriter();
|
||||
LOG.info("Past 1st WAL roll " + log.toString());
|
||||
|
||||
Thread.sleep(1);
|
||||
long ts1 = System.currentTimeMillis();
|
||||
|
||||
edit = new WALEdit();
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
|
||||
log.append(htd, info, getWalKey(ts1+1), edit, sequenceId,
|
||||
true, null);
|
||||
edit = new WALEdit();
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
|
||||
log.append(htd, info, getWalKey(ts1+2), edit, sequenceId,
|
||||
true, null);
|
||||
log.sync();
|
||||
log.shutdown();
|
||||
walfactory.shutdown();
|
||||
LOG.info("Closed WAL " + log.toString());
|
||||
|
||||
|
||||
WALInputFormat input = new WALInputFormat();
|
||||
Configuration jobConf = new Configuration(conf);
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString());
|
||||
jobConf.setLong(WALInputFormat.END_TIME_KEY, ts);
|
||||
|
||||
// only 1st file is considered, and only its 1st entry is used
|
||||
List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
|
||||
|
||||
assertEquals(1, splits.size());
|
||||
testSplit(splits.get(0), Bytes.toBytes("1"));
|
||||
|
||||
jobConf.setLong(WALInputFormat.START_TIME_KEY, ts+1);
|
||||
jobConf.setLong(WALInputFormat.END_TIME_KEY, ts1+1);
|
||||
splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
|
||||
// both files need to be considered
|
||||
assertEquals(2, splits.size());
|
||||
// only the 2nd entry from the 1st file is used
|
||||
testSplit(splits.get(0), Bytes.toBytes("2"));
|
||||
// only the 1nd entry from the 2nd file is used
|
||||
testSplit(splits.get(1), Bytes.toBytes("3"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test basic functionality
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testWALRecordReader() throws Exception {
|
||||
final WALFactory walfactory = new WALFactory(conf, null, getName());
|
||||
WAL log = walfactory.getWAL(info.getEncodedNameAsBytes());
|
||||
byte [] value = Bytes.toBytes("value");
|
||||
final AtomicLong sequenceId = new AtomicLong(0);
|
||||
WALEdit edit = new WALEdit();
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
|
||||
System.currentTimeMillis(), value));
|
||||
long txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true,
|
||||
null);
|
||||
log.sync(txid);
|
||||
|
||||
Thread.sleep(1); // make sure 2nd log gets a later timestamp
|
||||
long secondTs = System.currentTimeMillis();
|
||||
log.rollWriter();
|
||||
|
||||
edit = new WALEdit();
|
||||
edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
|
||||
System.currentTimeMillis(), value));
|
||||
txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true,
|
||||
null);
|
||||
log.sync(txid);
|
||||
log.shutdown();
|
||||
walfactory.shutdown();
|
||||
long thirdTs = System.currentTimeMillis();
|
||||
|
||||
// should have 2 log files now
|
||||
WALInputFormat input = new WALInputFormat();
|
||||
Configuration jobConf = new Configuration(conf);
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", logDir.toString());
|
||||
|
||||
// make sure both logs are found
|
||||
List<InputSplit> splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
|
||||
assertEquals(2, splits.size());
|
||||
|
||||
// should return exactly one KV
|
||||
testSplit(splits.get(0), Bytes.toBytes("1"));
|
||||
// same for the 2nd split
|
||||
testSplit(splits.get(1), Bytes.toBytes("2"));
|
||||
|
||||
// now test basic time ranges:
|
||||
|
||||
// set an endtime, the 2nd log file can be ignored completely.
|
||||
jobConf.setLong(WALInputFormat.END_TIME_KEY, secondTs-1);
|
||||
splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
|
||||
assertEquals(1, splits.size());
|
||||
testSplit(splits.get(0), Bytes.toBytes("1"));
|
||||
|
||||
// now set a start time
|
||||
jobConf.setLong(WALInputFormat.END_TIME_KEY, Long.MAX_VALUE);
|
||||
jobConf.setLong(WALInputFormat.START_TIME_KEY, thirdTs);
|
||||
splits = input.getSplits(MapreduceTestingShim.createJobContext(jobConf));
|
||||
// both logs need to be considered
|
||||
assertEquals(2, splits.size());
|
||||
// but both readers skip all edits
|
||||
testSplit(splits.get(0));
|
||||
testSplit(splits.get(1));
|
||||
}
|
||||
|
||||
protected WALKey getWalKey(final long sequenceid) {
|
||||
return new WALKey(info.getEncodedNameAsBytes(), tableName, sequenceid);
|
||||
}
|
||||
|
||||
protected WALRecordReader getReader() {
|
||||
return new WALKeyRecordReader();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new reader from the split, and match the edits against the passed columns.
|
||||
*/
|
||||
private void testSplit(InputSplit split, byte[]... columns) throws Exception {
|
||||
final WALRecordReader reader = getReader();
|
||||
reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf));
|
||||
|
||||
for (byte[] column : columns) {
|
||||
assertTrue(reader.nextKeyValue());
|
||||
Cell cell = reader.getCurrentValue().getCells().get(0);
|
||||
if (!Bytes.equals(column, cell.getQualifier())) {
|
||||
assertTrue("expected [" + Bytes.toString(column) + "], actual ["
|
||||
+ Bytes.toString(cell.getQualifier()) + "]", false);
|
||||
}
|
||||
}
|
||||
assertFalse(reader.nextKeyValue());
|
||||
reader.close();
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
public class InstrumentedLogWriter extends ProtobufLogWriter {
|
||||
|
||||
public InstrumentedLogWriter() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static boolean activateFailure = false;
|
||||
@Override
|
||||
public void append(Entry entry) throws IOException {
|
||||
super.append(entry);
|
||||
if (activateFailure &&
|
||||
Bytes.equals(entry.getKey().getEncodedRegionName(), "break".getBytes())) {
|
||||
System.out.println(getClass().getName() + ": I will throw an exception now...");
|
||||
throw(new IOException("This exception is instrumented and should only be thrown for testing"
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,479 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
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 static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.lang.mutable.MutableBoolean;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
/**
|
||||
* Provides FSHLog test cases.
|
||||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestFSHLog {
|
||||
protected static final Log LOG = LogFactory.getLog(TestFSHLog.class);
|
||||
{
|
||||
((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
|
||||
((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
|
||||
((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem"))
|
||||
.getLogger().setLevel(Level.ALL);
|
||||
((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
|
||||
}
|
||||
|
||||
protected static Configuration conf;
|
||||
protected static FileSystem fs;
|
||||
protected static Path dir;
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Rule
|
||||
public final TestName currentTest = new TestName();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
FileStatus[] entries = fs.listStatus(new Path("/"));
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
final Path hbaseDir = TEST_UTIL.createRootDir();
|
||||
dir = new Path(hbaseDir, currentTest.getMethodName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
|
||||
// quicker heartbeat interval for faster DN death notification
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
|
||||
|
||||
// faster failover with cluster.shutdown();fs.close() idiom
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt("hbase.ipc.client.connect.max.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"dfs.client.block.recovery.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"hbase.ipc.client.connection.maxidletime", 500);
|
||||
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
|
||||
SampleRegionWALObserver.class.getName());
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
fs = TEST_UTIL.getDFSCluster().getFileSystem();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* A loaded WAL coprocessor won't break existing WAL test cases.
|
||||
*/
|
||||
@Test
|
||||
public void testWALCoprocessorLoaded() throws Exception {
|
||||
// test to see whether the coprocessor is loaded or not.
|
||||
FSHLog log = null;
|
||||
try {
|
||||
log = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
|
||||
HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
|
||||
WALCoprocessorHost host = log.getCoprocessorHost();
|
||||
Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
|
||||
assertNotNull(c);
|
||||
} finally {
|
||||
if (log != null) {
|
||||
log.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void addEdits(WAL log, HRegionInfo hri, TableName tableName,
|
||||
int times, AtomicLong sequenceId) throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor("row"));
|
||||
|
||||
final byte [] row = Bytes.toBytes("row");
|
||||
for (int i = 0; i < times; i++) {
|
||||
long timestamp = System.currentTimeMillis();
|
||||
WALEdit cols = new WALEdit();
|
||||
cols.add(new KeyValue(row, row, row, timestamp, row));
|
||||
log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, timestamp), cols,
|
||||
sequenceId, true, null);
|
||||
}
|
||||
log.sync();
|
||||
}
|
||||
|
||||
/**
|
||||
* helper method to simulate region flush for a WAL.
|
||||
* @param wal
|
||||
* @param regionEncodedName
|
||||
*/
|
||||
protected void flushRegion(WAL wal, byte[] regionEncodedName) {
|
||||
wal.startCacheFlush(regionEncodedName);
|
||||
wal.completeCacheFlush(regionEncodedName);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
FSHLog wal1 = null;
|
||||
FSHLog walMeta = null;
|
||||
try {
|
||||
wal1 = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
|
||||
HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
|
||||
LOG.debug("Log obtained is: " + wal1);
|
||||
Comparator<Path> comp = wal1.LOG_NAME_COMPARATOR;
|
||||
Path p1 = wal1.computeFilename(11);
|
||||
Path p2 = wal1.computeFilename(12);
|
||||
// comparing with itself returns 0
|
||||
assertTrue(comp.compare(p1, p1) == 0);
|
||||
// comparing with different filenum.
|
||||
assertTrue(comp.compare(p1, p2) < 0);
|
||||
walMeta = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
|
||||
HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,
|
||||
DefaultWALProvider.META_WAL_PROVIDER_ID);
|
||||
Comparator<Path> compMeta = walMeta.LOG_NAME_COMPARATOR;
|
||||
|
||||
Path p1WithMeta = walMeta.computeFilename(11);
|
||||
Path p2WithMeta = walMeta.computeFilename(12);
|
||||
assertTrue(compMeta.compare(p1WithMeta, p1WithMeta) == 0);
|
||||
assertTrue(compMeta.compare(p1WithMeta, p2WithMeta) < 0);
|
||||
// mixing meta and non-meta logs gives error
|
||||
boolean ex = false;
|
||||
try {
|
||||
comp.compare(p1WithMeta, p2);
|
||||
} catch (IllegalArgumentException e) {
|
||||
ex = true;
|
||||
}
|
||||
assertTrue("Comparator doesn't complain while checking meta log files", ex);
|
||||
boolean exMeta = false;
|
||||
try {
|
||||
compMeta.compare(p1WithMeta, p2);
|
||||
} catch (IllegalArgumentException e) {
|
||||
exMeta = true;
|
||||
}
|
||||
assertTrue("Meta comparator doesn't complain while checking log files", exMeta);
|
||||
} finally {
|
||||
if (wal1 != null) {
|
||||
wal1.close();
|
||||
}
|
||||
if (walMeta != null) {
|
||||
walMeta.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* On rolling a wal after reaching the threshold, {@link WAL#rollWriter()} returns the
|
||||
* list of regions which should be flushed in order to archive the oldest wal file.
|
||||
* <p>
|
||||
* 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" for flush on
|
||||
* rolling the wal.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testFindMemStoresEligibleForFlush() throws Exception {
|
||||
LOG.debug("testFindMemStoresEligibleForFlush");
|
||||
Configuration conf1 = HBaseConfiguration.create(conf);
|
||||
conf1.setInt("hbase.regionserver.maxlogs", 1);
|
||||
FSHLog wal = new FSHLog(fs, FSUtils.getRootDir(conf1), dir.toString(),
|
||||
HConstants.HREGION_OLDLOGDIR_NAME, conf1, null, true, null, null);
|
||||
TableName t1 = TableName.valueOf("t1");
|
||||
TableName t2 = TableName.valueOf("t2");
|
||||
HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HRegionInfo hri2 = new HRegionInfo(t2, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
// variables to mock region sequenceIds
|
||||
final AtomicLong sequenceId1 = new AtomicLong(1);
|
||||
final AtomicLong sequenceId2 = new AtomicLong(1);
|
||||
// add edits and roll the wal
|
||||
try {
|
||||
addEdits(wal, hri1, t1, 2, sequenceId1);
|
||||
wal.rollWriter();
|
||||
// add some more edits and roll the wal. This would reach the log number threshold
|
||||
addEdits(wal, hri1, t1, 2, sequenceId1);
|
||||
wal.rollWriter();
|
||||
// with above rollWriter call, the max logs limit is reached.
|
||||
assertTrue(wal.getNumRolledLogFiles() == 2);
|
||||
|
||||
// get the regions to flush; since there is only one region in the oldest wal, it should
|
||||
// return only one region.
|
||||
byte[][] regionsToFlush = wal.findRegionsToForceFlush();
|
||||
assertEquals(1, regionsToFlush.length);
|
||||
assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]);
|
||||
// insert edits in second region
|
||||
addEdits(wal, hri2, t2, 2, sequenceId2);
|
||||
// get the regions to flush, it should still read region1.
|
||||
regionsToFlush = wal.findRegionsToForceFlush();
|
||||
assertEquals(regionsToFlush.length, 1);
|
||||
assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]);
|
||||
// flush region 1, and roll the wal file. Only last wal which has entries for region1 should
|
||||
// remain.
|
||||
flushRegion(wal, hri1.getEncodedNameAsBytes());
|
||||
wal.rollWriter();
|
||||
// only one wal should remain now (that is for the second region).
|
||||
assertEquals(1, wal.getNumRolledLogFiles());
|
||||
// flush the second region
|
||||
flushRegion(wal, hri2.getEncodedNameAsBytes());
|
||||
wal.rollWriter(true);
|
||||
// no wal should remain now.
|
||||
assertEquals(0, wal.getNumRolledLogFiles());
|
||||
// add edits both to region 1 and region 2, and roll.
|
||||
addEdits(wal, hri1, t1, 2, sequenceId1);
|
||||
addEdits(wal, hri2, t2, 2, sequenceId2);
|
||||
wal.rollWriter();
|
||||
// add edits and roll the writer, to reach the max logs limit.
|
||||
assertEquals(1, wal.getNumRolledLogFiles());
|
||||
addEdits(wal, hri1, t1, 2, sequenceId1);
|
||||
wal.rollWriter();
|
||||
// it should return two regions to flush, as the oldest wal file has entries
|
||||
// for both regions.
|
||||
regionsToFlush = wal.findRegionsToForceFlush();
|
||||
assertEquals(2, regionsToFlush.length);
|
||||
// flush both regions
|
||||
flushRegion(wal, hri1.getEncodedNameAsBytes());
|
||||
flushRegion(wal, hri2.getEncodedNameAsBytes());
|
||||
wal.rollWriter(true);
|
||||
assertEquals(0, wal.getNumRolledLogFiles());
|
||||
// Add an edit to region1, and roll the wal.
|
||||
addEdits(wal, hri1, t1, 2, sequenceId1);
|
||||
// tests partial flush: roll on a partial flush, and ensure that wal is not archived.
|
||||
wal.startCacheFlush(hri1.getEncodedNameAsBytes());
|
||||
wal.rollWriter();
|
||||
wal.completeCacheFlush(hri1.getEncodedNameAsBytes());
|
||||
assertEquals(1, wal.getNumRolledLogFiles());
|
||||
} finally {
|
||||
if (wal != null) {
|
||||
wal.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates WAL append ops for a region and tests
|
||||
* {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} API.
|
||||
* It compares the region sequenceIds with oldestFlushing and oldestUnFlushed entries.
|
||||
* If a region's entries are larger than min of (oldestFlushing, oldestUnFlushed), then the
|
||||
* region should be flushed before archiving this WAL.
|
||||
*/
|
||||
@Test
|
||||
public void testAllRegionsFlushed() {
|
||||
LOG.debug("testAllRegionsFlushed");
|
||||
Map<byte[], Long> oldestFlushingSeqNo = new HashMap<byte[], Long>();
|
||||
Map<byte[], Long> oldestUnFlushedSeqNo = new HashMap<byte[], Long>();
|
||||
Map<byte[], Long> seqNo = new HashMap<byte[], Long>();
|
||||
// create a table
|
||||
TableName t1 = TableName.valueOf("t1");
|
||||
// create a region
|
||||
HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
// variables to mock region sequenceIds
|
||||
final AtomicLong sequenceId1 = new AtomicLong(1);
|
||||
// test empty map
|
||||
assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
// add entries in the region
|
||||
seqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.incrementAndGet());
|
||||
oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get());
|
||||
// should say region1 is not flushed.
|
||||
assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
// test with entries in oldestFlushing map.
|
||||
oldestUnFlushedSeqNo.clear();
|
||||
oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get());
|
||||
assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
// simulate region flush, i.e., clear oldestFlushing and oldestUnflushed maps
|
||||
oldestFlushingSeqNo.clear();
|
||||
oldestUnFlushedSeqNo.clear();
|
||||
assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
// insert some large values for region1
|
||||
oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), 1000l);
|
||||
seqNo.put(hri1.getEncodedNameAsBytes(), 1500l);
|
||||
assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
|
||||
// tests when oldestUnFlushed/oldestFlushing contains larger value.
|
||||
// It means region is flushed.
|
||||
oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), 1200l);
|
||||
oldestUnFlushedSeqNo.clear();
|
||||
seqNo.put(hri1.getEncodedNameAsBytes(), 1199l);
|
||||
assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
|
||||
}
|
||||
|
||||
@Test(expected=IOException.class)
|
||||
public void testFailedToCreateWALIfParentRenamed() throws IOException {
|
||||
final String name = "testFailedToCreateWALIfParentRenamed";
|
||||
FSHLog log = new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME,
|
||||
conf, null, true, null, null);
|
||||
long filenum = System.currentTimeMillis();
|
||||
Path path = log.computeFilename(filenum);
|
||||
log.createWriterInstance(path);
|
||||
Path parent = path.getParent();
|
||||
path = log.computeFilename(filenum + 1);
|
||||
Path newPath = new Path(parent.getParent(), parent.getName() + "-splitting");
|
||||
fs.rename(parent, newPath);
|
||||
log.createWriterInstance(path);
|
||||
fail("It should fail to create the new WAL");
|
||||
}
|
||||
|
||||
/**
|
||||
* Test flush for sure has a sequence id that is beyond the last edit appended. We do this
|
||||
* by 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 HBASE-11109
|
||||
*/
|
||||
@Test
|
||||
public void testFlushSequenceIdIsGreaterThanAllEditsInHFile() throws IOException {
|
||||
String testName = "testFlushSequenceIdIsGreaterThanAllEditsInHFile";
|
||||
final TableName tableName = TableName.valueOf(testName);
|
||||
final HRegionInfo hri = new HRegionInfo(tableName);
|
||||
final byte[] rowName = tableName.getName();
|
||||
final HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("f"));
|
||||
HRegion r = HRegion.createHRegion(hri, TEST_UTIL.getDefaultRootDirPath(),
|
||||
TEST_UTIL.getConfiguration(), htd);
|
||||
HRegion.closeHRegion(r);
|
||||
final int countPerFamily = 10;
|
||||
final MutableBoolean goslow = new MutableBoolean(false);
|
||||
// subclass and doctor a method.
|
||||
FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDefaultRootDirPath(),
|
||||
testName, conf) {
|
||||
@Override
|
||||
void atHeadOfRingBufferEventHandlerAppend() {
|
||||
if (goslow.isTrue()) {
|
||||
Threads.sleep(100);
|
||||
LOG.debug("Sleeping before appending 100ms");
|
||||
}
|
||||
super.atHeadOfRingBufferEventHandlerAppend();
|
||||
}
|
||||
};
|
||||
HRegion region = HRegion.openHRegion(TEST_UTIL.getConfiguration(),
|
||||
TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(), hri, htd, wal);
|
||||
EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
|
||||
try {
|
||||
List<Put> puts = null;
|
||||
for (HColumnDescriptor hcd: htd.getFamilies()) {
|
||||
puts =
|
||||
TestWALReplay.addRegionEdits(rowName, hcd.getName(), countPerFamily, ee, region, "x");
|
||||
}
|
||||
|
||||
// Now assert edits made it in.
|
||||
final Get g = new Get(rowName);
|
||||
Result result = region.get(g);
|
||||
assertEquals(countPerFamily * htd.getFamilies().size(), result.size());
|
||||
|
||||
// Construct a WALEdit and add it a few times to the WAL.
|
||||
WALEdit edits = new WALEdit();
|
||||
for (Put p: puts) {
|
||||
CellScanner cs = p.cellScanner();
|
||||
while (cs.advance()) {
|
||||
edits.add(cs.current());
|
||||
}
|
||||
}
|
||||
// Add any old cluster id.
|
||||
List<UUID> clusterIds = new ArrayList<UUID>();
|
||||
clusterIds.add(UUID.randomUUID());
|
||||
// Now make appends run slow.
|
||||
goslow.setValue(true);
|
||||
for (int i = 0; i < countPerFamily; i++) {
|
||||
final HRegionInfo info = region.getRegionInfo();
|
||||
final WALKey logkey = new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis(), clusterIds, -1, -1);
|
||||
wal.append(htd, info, logkey, edits, region.getSequenceId(), true, null);
|
||||
}
|
||||
region.flushcache();
|
||||
// FlushResult.flushSequenceId is not visible here so go get the current sequence id.
|
||||
long currentSequenceId = region.getSequenceId().get();
|
||||
// Now release the appends
|
||||
goslow.setValue(false);
|
||||
synchronized (goslow) {
|
||||
goslow.notifyAll();
|
||||
}
|
||||
assertTrue(currentSequenceId >= region.getSequenceId().get());
|
||||
} finally {
|
||||
region.close(true);
|
||||
wal.close();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,209 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
/**
|
||||
* WAL tests that can be reused across providers.
|
||||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestProtobufLog {
|
||||
protected static final Log LOG = LogFactory.getLog(TestProtobufLog.class);
|
||||
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
protected FileSystem fs;
|
||||
protected Path dir;
|
||||
protected WALFactory wals;
|
||||
|
||||
@Rule
|
||||
public final TestName currentTest = new TestName();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
fs = TEST_UTIL.getDFSCluster().getFileSystem();
|
||||
dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
|
||||
wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
wals.close();
|
||||
FileStatus[] entries = fs.listStatus(new Path("/"));
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
|
||||
// needed for testAppendClose()
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
|
||||
// quicker heartbeat interval for faster DN death notification
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
|
||||
|
||||
// faster failover with cluster.shutdown();fs.close() idiom
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt("hbase.ipc.client.connect.max.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"dfs.client.block.recovery.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"hbase.ipc.client.connection.maxidletime", 500);
|
||||
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
|
||||
SampleRegionWALObserver.class.getName());
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the WAL with and without WALTrailer.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testWALTrailer() throws IOException {
|
||||
// read With trailer.
|
||||
doRead(true);
|
||||
// read without trailer
|
||||
doRead(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends entries in the WAL and reads it.
|
||||
* @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
|
||||
* so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
|
||||
* call. This means that reader is not aware of the trailer. In this scenario, if the
|
||||
* reader tries to read the trailer in its next() call, it returns false from
|
||||
* ProtoBufLogReader.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void doRead(boolean withTrailer) throws IOException {
|
||||
final int columnCount = 5;
|
||||
final int recordCount = 5;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
long timestamp = System.currentTimeMillis();
|
||||
Path path = new Path(dir, "tempwal");
|
||||
// delete the log if already exists, for test only
|
||||
fs.delete(path, true);
|
||||
WALProvider.Writer writer = null;
|
||||
ProtobufLogReader reader = null;
|
||||
try {
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
fs.mkdirs(dir);
|
||||
// Write log in pb format.
|
||||
writer = wals.createWALWriter(fs, path);
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WALKey key = new WALKey(
|
||||
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
|
||||
WALEdit edit = new WALEdit();
|
||||
for (int j = 0; j < columnCount; ++j) {
|
||||
if (i == 0) {
|
||||
htd.addFamily(new HColumnDescriptor("column" + j));
|
||||
}
|
||||
String value = i + "" + j;
|
||||
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
|
||||
}
|
||||
writer.append(new WAL.Entry(key, edit));
|
||||
}
|
||||
writer.sync();
|
||||
if (withTrailer) writer.close();
|
||||
|
||||
// Now read the log using standard means.
|
||||
reader = (ProtobufLogReader) wals.createReader(fs, path);
|
||||
if (withTrailer) {
|
||||
assertNotNull(reader.trailer);
|
||||
} else {
|
||||
assertNull(reader.trailer);
|
||||
}
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNotNull(entry);
|
||||
assertEquals(columnCount, entry.getEdit().size());
|
||||
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
|
||||
assertEquals(tableName, entry.getKey().getTablename());
|
||||
int idx = 0;
|
||||
for (Cell val : entry.getEdit().getCells()) {
|
||||
assertTrue(Bytes.equals(row, val.getRow()));
|
||||
String value = i + "" + idx;
|
||||
assertArrayEquals(Bytes.toBytes(value), val.getValue());
|
||||
idx++;
|
||||
}
|
||||
}
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNull(entry);
|
||||
} finally {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,221 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.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.hdfs.MiniDFSCluster;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
@Category({ReplicationTests.class, LargeTests.class})
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestReplicationWALReaderManager {
|
||||
|
||||
private static HBaseTestingUtility TEST_UTIL;
|
||||
private static Configuration conf;
|
||||
private static Path hbaseDir;
|
||||
private static FileSystem fs;
|
||||
private static MiniDFSCluster cluster;
|
||||
private static final TableName tableName = TableName.valueOf("tablename");
|
||||
private static final byte [] family = Bytes.toBytes("column");
|
||||
private static final byte [] qualifier = Bytes.toBytes("qualifier");
|
||||
private static final HRegionInfo info = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false);
|
||||
private static final HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
|
||||
private WAL log;
|
||||
private ReplicationWALReaderManager logManager;
|
||||
private PathWatcher pathWatcher;
|
||||
private int nbRows;
|
||||
private int walEditKVs;
|
||||
private final AtomicLong sequenceId = new AtomicLong(1);
|
||||
|
||||
@Parameters
|
||||
public static Collection<Object[]> parameters() {
|
||||
// Try out different combinations of row count and KeyValue count
|
||||
int[] NB_ROWS = { 1500, 60000 };
|
||||
int[] NB_KVS = { 1, 100 };
|
||||
// whether compression is used
|
||||
Boolean[] BOOL_VALS = { false, true };
|
||||
List<Object[]> parameters = new ArrayList<Object[]>();
|
||||
for (int nbRows : NB_ROWS) {
|
||||
for (int walEditKVs : NB_KVS) {
|
||||
for (boolean b : BOOL_VALS) {
|
||||
Object[] arr = new Object[3];
|
||||
arr[0] = nbRows;
|
||||
arr[1] = walEditKVs;
|
||||
arr[2] = b;
|
||||
parameters.add(arr);
|
||||
}
|
||||
}
|
||||
}
|
||||
return parameters;
|
||||
}
|
||||
|
||||
public TestReplicationWALReaderManager(int nbRows, int walEditKVs, boolean enableCompression) {
|
||||
this.nbRows = nbRows;
|
||||
this.walEditKVs = walEditKVs;
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION,
|
||||
enableCompression);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL = new HBaseTestingUtility();
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
|
||||
hbaseDir = TEST_UTIL.createRootDir();
|
||||
cluster = TEST_UTIL.getDFSCluster();
|
||||
fs = cluster.getFileSystem();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
logManager = new ReplicationWALReaderManager(fs, conf);
|
||||
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
|
||||
pathWatcher = new PathWatcher();
|
||||
listeners.add(pathWatcher);
|
||||
final WALFactory wals = new WALFactory(conf, listeners, "some server");
|
||||
log = wals.getWAL(info.getEncodedNameAsBytes());
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
log.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
// Grab the path that was generated when the log rolled as part of its creation
|
||||
Path path = pathWatcher.currentPath;
|
||||
|
||||
assertEquals(0, logManager.getPosition());
|
||||
|
||||
appendToLog();
|
||||
|
||||
// There's one edit in the log, read it. Reading past it needs to return nulls
|
||||
assertNotNull(logManager.openReader(path));
|
||||
logManager.seek();
|
||||
WAL.Entry entry = logManager.readNextAndSetPosition();
|
||||
assertNotNull(entry);
|
||||
entry = logManager.readNextAndSetPosition();
|
||||
assertNull(entry);
|
||||
logManager.closeReader();
|
||||
long oldPos = logManager.getPosition();
|
||||
|
||||
appendToLog();
|
||||
|
||||
// Read the newly added entry, make sure we made progress
|
||||
assertNotNull(logManager.openReader(path));
|
||||
logManager.seek();
|
||||
entry = logManager.readNextAndSetPosition();
|
||||
assertNotEquals(oldPos, logManager.getPosition());
|
||||
assertNotNull(entry);
|
||||
logManager.closeReader();
|
||||
oldPos = logManager.getPosition();
|
||||
|
||||
log.rollWriter();
|
||||
|
||||
// We rolled but we still should see the end of the first log and not get data
|
||||
assertNotNull(logManager.openReader(path));
|
||||
logManager.seek();
|
||||
entry = logManager.readNextAndSetPosition();
|
||||
assertEquals(oldPos, logManager.getPosition());
|
||||
assertNull(entry);
|
||||
logManager.finishCurrentFile();
|
||||
|
||||
path = pathWatcher.currentPath;
|
||||
|
||||
for (int i = 0; i < nbRows; i++) { appendToLogPlus(walEditKVs); }
|
||||
log.rollWriter();
|
||||
logManager.openReader(path);
|
||||
logManager.seek();
|
||||
for (int i = 0; i < nbRows; i++) {
|
||||
WAL.Entry e = logManager.readNextAndSetPosition();
|
||||
if (e == null) {
|
||||
fail("Should have enough entries");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void appendToLog() throws IOException {
|
||||
appendToLogPlus(1);
|
||||
}
|
||||
|
||||
private void appendToLogPlus(int count) throws IOException {
|
||||
final long txid = log.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), getWALEdits(count), sequenceId, true, null);
|
||||
log.sync(txid);
|
||||
}
|
||||
|
||||
private WALEdit getWALEdits(int count) {
|
||||
WALEdit edit = new WALEdit();
|
||||
for (int i = 0; i < count; i++) {
|
||||
edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), family, qualifier,
|
||||
System.currentTimeMillis(), qualifier));
|
||||
}
|
||||
return edit;
|
||||
}
|
||||
|
||||
class PathWatcher extends WALActionsListener.Base {
|
||||
|
||||
Path currentPath;
|
||||
|
||||
@Override
|
||||
public void preLogRoll(Path oldPath, Path newPath) throws IOException {
|
||||
currentPath = newPath;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,76 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
||||
// imports for things that haven't moved yet
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* This is a utility class, used by tests, which fails operation specified by FailureType enum
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class FaultyFSLog extends FSHLog {
|
||||
public enum FailureType {
|
||||
NONE, APPEND, SYNC
|
||||
}
|
||||
FailureType ft = FailureType.NONE;
|
||||
|
||||
public FaultyFSLog(FileSystem fs, Path rootDir, String logName, Configuration conf)
|
||||
throws IOException {
|
||||
super(fs, rootDir, logName, conf);
|
||||
}
|
||||
|
||||
public void setFailureType(FailureType fType) {
|
||||
this.ft = fType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync(long txid) throws IOException {
|
||||
if (this.ft == FailureType.SYNC) {
|
||||
throw new IOException("sync");
|
||||
}
|
||||
super.sync(txid);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
|
||||
AtomicLong sequenceId, boolean isInMemstore, List<Cell> cells) throws IOException {
|
||||
if (this.ft == FailureType.APPEND) {
|
||||
throw new IOException("append");
|
||||
}
|
||||
return super.append(htd, info, key, edits, sequenceId, isInMemstore, cells);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,333 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestDefaultWALProvider {
|
||||
protected static final Log LOG = LogFactory.getLog(TestDefaultWALProvider.class);
|
||||
|
||||
protected static Configuration conf;
|
||||
protected static FileSystem fs;
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Rule
|
||||
public final TestName currentTest = new TestName();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
FileStatus[] entries = fs.listStatus(new Path("/"));
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
|
||||
// quicker heartbeat interval for faster DN death notification
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
|
||||
|
||||
// faster failover with cluster.shutdown();fs.close() idiom
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt("hbase.ipc.client.connect.max.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"dfs.client.block.recovery.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"hbase.ipc.client.connection.maxidletime", 500);
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
|
||||
// Set up a working space for our tests.
|
||||
TEST_UTIL.createRootDir();
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
fs = TEST_UTIL.getDFSCluster().getFileSystem();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
static String getName() {
|
||||
return "TestDefaultWALProvider";
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetServerNameFromWALDirectoryName() throws IOException {
|
||||
ServerName sn = ServerName.valueOf("hn", 450, 1398);
|
||||
String hl = FSUtils.getRootDir(conf) + "/" +
|
||||
DefaultWALProvider.getWALDirectoryName(sn.toString());
|
||||
|
||||
// Must not throw exception
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, null));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
|
||||
FSUtils.getRootDir(conf).toUri().toString()));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, ""));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, " "));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, hl));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, hl + "qdf"));
|
||||
assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, "sfqf" + hl + "qdf"));
|
||||
|
||||
final String wals = "/WALs/";
|
||||
ServerName parsed = DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
|
||||
FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
|
||||
"/localhost%2C32984%2C1343316388997.1343316390417");
|
||||
assertEquals("standard", sn, parsed);
|
||||
|
||||
parsed = DefaultWALProvider.getServerNameFromWALDirectoryName(conf, hl + "/qdf");
|
||||
assertEquals("subdir", sn, parsed);
|
||||
|
||||
parsed = DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
|
||||
FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
|
||||
"-splitting/localhost%3A57020.1340474893931");
|
||||
assertEquals("split", sn, parsed);
|
||||
}
|
||||
|
||||
|
||||
protected void addEdits(WAL log, HRegionInfo hri, TableName tableName,
|
||||
int times, AtomicLong sequenceId) throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor("row"));
|
||||
|
||||
final byte [] row = Bytes.toBytes("row");
|
||||
for (int i = 0; i < times; i++) {
|
||||
long timestamp = System.currentTimeMillis();
|
||||
WALEdit cols = new WALEdit();
|
||||
cols.add(new KeyValue(row, row, row, timestamp, row));
|
||||
log.append(htd, hri, getWalKey(hri.getEncodedNameAsBytes(), tableName, timestamp), cols,
|
||||
sequenceId, true, null);
|
||||
}
|
||||
log.sync();
|
||||
}
|
||||
|
||||
/**
|
||||
* used by TestDefaultWALProviderWithHLogKey
|
||||
*/
|
||||
WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) {
|
||||
return new WALKey(info, tableName, timestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* helper method to simulate region flush for a WAL.
|
||||
* @param wal
|
||||
* @param regionEncodedName
|
||||
*/
|
||||
protected void flushRegion(WAL wal, byte[] regionEncodedName) {
|
||||
wal.startCacheFlush(regionEncodedName);
|
||||
wal.completeCacheFlush(regionEncodedName);
|
||||
}
|
||||
|
||||
private static final byte[] UNSPECIFIED_REGION = new byte[]{};
|
||||
|
||||
@Test
|
||||
public void testLogCleaning() throws Exception {
|
||||
LOG.info("testLogCleaning");
|
||||
final TableName tableName =
|
||||
TableName.valueOf("testLogCleaning");
|
||||
final TableName tableName2 =
|
||||
TableName.valueOf("testLogCleaning2");
|
||||
final Configuration localConf = new Configuration(conf);
|
||||
localConf.set(WALFactory.WAL_PROVIDER, DefaultWALProvider.class.getName());
|
||||
final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
try {
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HRegionInfo hri2 = new HRegionInfo(tableName2,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
// we want to mix edits from regions, so pick our own identifier.
|
||||
final WAL log = wals.getWAL(UNSPECIFIED_REGION);
|
||||
|
||||
// Add a single edit and make sure that rolling won't remove the file
|
||||
// Before HBASE-3198 it used to delete it
|
||||
addEdits(log, hri, tableName, 1, sequenceId);
|
||||
log.rollWriter();
|
||||
assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
|
||||
// See if there's anything wrong with more than 1 edit
|
||||
addEdits(log, hri, tableName, 2, sequenceId);
|
||||
log.rollWriter();
|
||||
assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
|
||||
// Now mix edits from 2 regions, still no flushing
|
||||
addEdits(log, hri, tableName, 1, sequenceId);
|
||||
addEdits(log, hri2, tableName2, 1, sequenceId);
|
||||
addEdits(log, hri, tableName, 1, sequenceId);
|
||||
addEdits(log, hri2, tableName2, 1, sequenceId);
|
||||
log.rollWriter();
|
||||
assertEquals(3, DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
|
||||
// Flush the first region, we expect to see the first two files getting
|
||||
// archived. We need to append something or writer won't be rolled.
|
||||
addEdits(log, hri2, tableName2, 1, sequenceId);
|
||||
log.startCacheFlush(hri.getEncodedNameAsBytes());
|
||||
log.completeCacheFlush(hri.getEncodedNameAsBytes());
|
||||
log.rollWriter();
|
||||
assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
|
||||
// Flush the second region, which removes all the remaining output files
|
||||
// since the oldest was completely flushed and the two others only contain
|
||||
// flush information
|
||||
addEdits(log, hri2, tableName2, 1, sequenceId);
|
||||
log.startCacheFlush(hri2.getEncodedNameAsBytes());
|
||||
log.completeCacheFlush(hri2.getEncodedNameAsBytes());
|
||||
log.rollWriter();
|
||||
assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(log));
|
||||
} finally {
|
||||
if (wals != null) {
|
||||
wals.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests wal archiving by adding data, doing flushing/rolling and checking we archive old logs
|
||||
* and also don't archive "live logs" (that is, a log with un-flushed entries).
|
||||
* <p>
|
||||
* This is what it does:
|
||||
* It creates two regions, and does a series of inserts along with log rolling.
|
||||
* Whenever a WAL is rolled, HLogBase checks previous wals for archiving. A wal is eligible for
|
||||
* 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 {
|
||||
LOG.debug("testWALArchiving");
|
||||
TableName table1 = TableName.valueOf("t1");
|
||||
TableName table2 = TableName.valueOf("t2");
|
||||
final Configuration localConf = new Configuration(conf);
|
||||
localConf.set(WALFactory.WAL_PROVIDER, DefaultWALProvider.class.getName());
|
||||
final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
|
||||
try {
|
||||
final WAL wal = wals.getWAL(UNSPECIFIED_REGION);
|
||||
assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
HRegionInfo hri1 = new HRegionInfo(table1, HConstants.EMPTY_START_ROW,
|
||||
HConstants.EMPTY_END_ROW);
|
||||
HRegionInfo hri2 = new HRegionInfo(table2, HConstants.EMPTY_START_ROW,
|
||||
HConstants.EMPTY_END_ROW);
|
||||
// ensure that we don't split the regions.
|
||||
hri1.setSplit(false);
|
||||
hri2.setSplit(false);
|
||||
// variables to mock region sequenceIds.
|
||||
final AtomicLong sequenceId1 = new AtomicLong(1);
|
||||
final AtomicLong sequenceId2 = new AtomicLong(1);
|
||||
// start with the testing logic: insert a waledit, and roll writer
|
||||
addEdits(wal, hri1, table1, 1, sequenceId1);
|
||||
wal.rollWriter();
|
||||
// assert that the wal is rolled
|
||||
assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// add edits in the second wal file, and roll writer.
|
||||
addEdits(wal, hri1, table1, 1, sequenceId1);
|
||||
wal.rollWriter();
|
||||
// assert that the wal is rolled
|
||||
assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// add a waledit to table1, and flush the region.
|
||||
addEdits(wal, hri1, table1, 3, sequenceId1);
|
||||
flushRegion(wal, hri1.getEncodedNameAsBytes());
|
||||
// roll log; all old logs should be archived.
|
||||
wal.rollWriter();
|
||||
assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// add an edit to table2, and roll writer
|
||||
addEdits(wal, hri2, table2, 1, sequenceId2);
|
||||
wal.rollWriter();
|
||||
assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// add edits for table1, and roll writer
|
||||
addEdits(wal, hri1, table1, 2, sequenceId1);
|
||||
wal.rollWriter();
|
||||
assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// add edits for table2, and flush hri1.
|
||||
addEdits(wal, hri2, table2, 2, sequenceId2);
|
||||
flushRegion(wal, hri1.getEncodedNameAsBytes());
|
||||
// the log : region-sequenceId map is
|
||||
// log1: region2 (unflushed)
|
||||
// log2: region1 (flushed)
|
||||
// log3: region2 (unflushed)
|
||||
// roll the writer; log2 should be archived.
|
||||
wal.rollWriter();
|
||||
assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
// flush region2, and all logs should be archived.
|
||||
addEdits(wal, hri2, table2, 2, sequenceId2);
|
||||
flushRegion(wal, hri2.getEncodedNameAsBytes());
|
||||
wal.rollWriter();
|
||||
assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal));
|
||||
} finally {
|
||||
if (wals != null) {
|
||||
wals.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write to a log file with three concurrent threads and verifying all data is written.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testConcurrentWrites() throws Exception {
|
||||
// Run the WPE tool with three threads writing 3000 edits each concurrently.
|
||||
// When done, verify that all edits were written.
|
||||
int errCode = WALPerformanceEvaluation.
|
||||
innerMain(new Configuration(TEST_UTIL.getConfiguration()),
|
||||
new String [] {"-threads", "3", "-verify", "-noclosefs", "-iterations", "3000"});
|
||||
assertEquals(0, errCode);
|
||||
}
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
|
||||
@Category({RegionServerTests.class, LargeTests.class})
|
||||
public class TestDefaultWALProviderWithHLogKey extends TestDefaultWALProvider {
|
||||
@Override
|
||||
WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) {
|
||||
return new HLogKey(info, tableName, timestamp);
|
||||
}
|
||||
}
|
@ -0,0 +1,138 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.log4j.Level;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestSecureWAL {
|
||||
static final Log LOG = LogFactory.getLog(TestSecureWAL.class);
|
||||
static {
|
||||
((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal"))
|
||||
.getLogger().setLevel(Level.ALL);
|
||||
};
|
||||
static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
|
||||
conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
|
||||
conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
|
||||
WAL.Reader.class);
|
||||
conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
|
||||
WALProvider.Writer.class);
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
|
||||
FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureWAL() throws Exception {
|
||||
TableName tableName = TableName.valueOf("TestSecureWAL");
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(tableName.getName()));
|
||||
HRegionInfo regioninfo = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
|
||||
final int total = 10;
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
final byte[] family = Bytes.toBytes("family");
|
||||
final byte[] value = Bytes.toBytes("Test value");
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "TestSecureWAL");
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
|
||||
// Write the WAL
|
||||
final WAL wal = wals.getWAL(regioninfo.getEncodedNameAsBytes());
|
||||
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
|
||||
wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
wal.sync();
|
||||
final Path walPath = DefaultWALProvider.getCurrentFileName(wal);
|
||||
wals.shutdown();
|
||||
|
||||
// Insure edits are not plaintext
|
||||
long length = fs.getFileStatus(walPath).getLen();
|
||||
FSDataInputStream in = fs.open(walPath);
|
||||
byte[] fileData = new byte[(int)length];
|
||||
IOUtils.readFully(in, fileData);
|
||||
in.close();
|
||||
assertFalse("Cells appear to be plaintext", Bytes.contains(fileData, value));
|
||||
|
||||
// Confirm the WAL can be read back
|
||||
WAL.Reader reader = wals.createReader(TEST_UTIL.getTestFileSystem(), walPath);
|
||||
int count = 0;
|
||||
WAL.Entry entry = new WAL.Entry();
|
||||
while (reader.next(entry) != null) {
|
||||
count++;
|
||||
List<Cell> cells = entry.getEdit().getCells();
|
||||
assertTrue("Should be one KV per WALEdit", cells.size() == 1);
|
||||
for (Cell cell: cells) {
|
||||
byte[] thisRow = cell.getRow();
|
||||
assertTrue("Incorrect row", Bytes.equals(thisRow, row));
|
||||
byte[] thisFamily = cell.getFamily();
|
||||
assertTrue("Incorrect family", Bytes.equals(thisFamily, family));
|
||||
byte[] thisValue = cell.getValue();
|
||||
assertTrue("Incorrect value", Bytes.equals(thisValue, value));
|
||||
}
|
||||
}
|
||||
assertEquals("Should have read back as many KVs as written", total, count);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,721 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
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 static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.BindException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* WAL tests that can be reused across providers.
|
||||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestWALFactory {
|
||||
protected static final Log LOG = LogFactory.getLog(TestWALFactory.class);
|
||||
|
||||
protected static Configuration conf;
|
||||
private static MiniDFSCluster cluster;
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
protected static Path hbaseDir;
|
||||
|
||||
protected FileSystem fs;
|
||||
protected Path dir;
|
||||
protected WALFactory wals;
|
||||
|
||||
@Rule
|
||||
public final TestName currentTest = new TestName();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
fs = cluster.getFileSystem();
|
||||
dir = new Path(hbaseDir, currentTest.getMethodName());
|
||||
wals = new WALFactory(conf, null, currentTest.getMethodName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
// testAppendClose closes the FileSystem, which will prevent us from closing cleanly here.
|
||||
try {
|
||||
wals.close();
|
||||
} catch (IOException exception) {
|
||||
LOG.warn("Encountered exception while closing wal factory. If you have other errors, this" +
|
||||
" may be the cause. Message: " + exception);
|
||||
LOG.debug("Exception details for failure to close wal factory.", exception);
|
||||
}
|
||||
FileStatus[] entries = fs.listStatus(new Path("/"));
|
||||
for (FileStatus dir : entries) {
|
||||
fs.delete(dir.getPath(), true);
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
// Make block sizes small.
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
|
||||
// needed for testAppendClose()
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
|
||||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
|
||||
// quicker heartbeat interval for faster DN death notification
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
|
||||
TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
|
||||
|
||||
// faster failover with cluster.shutdown();fs.close() idiom
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt("hbase.ipc.client.connect.max.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"dfs.client.block.recovery.retries", 1);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
"hbase.ipc.client.connection.maxidletime", 500);
|
||||
TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
|
||||
SampleRegionWALObserver.class.getName());
|
||||
TEST_UTIL.startMiniDFSCluster(3);
|
||||
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
cluster = TEST_UTIL.getDFSCluster();
|
||||
|
||||
hbaseDir = TEST_UTIL.createRootDir();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Just write multiple logs then split. Before fix for HADOOP-2283, this
|
||||
* would fail.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testSplit() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(currentTest.getMethodName());
|
||||
final byte [] rowName = tableName.getName();
|
||||
final Path logdir = new Path(hbaseDir,
|
||||
DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName()));
|
||||
Path oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
final int howmany = 3;
|
||||
HRegionInfo[] infos = new HRegionInfo[3];
|
||||
Path tabledir = FSUtils.getTableDir(hbaseDir, tableName);
|
||||
fs.mkdirs(tabledir);
|
||||
for(int i = 0; i < howmany; i++) {
|
||||
infos[i] = new HRegionInfo(tableName,
|
||||
Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
|
||||
fs.mkdirs(new Path(tabledir, infos[i].getEncodedName()));
|
||||
LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString());
|
||||
}
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor("column"));
|
||||
|
||||
// Add edits for three regions.
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
for (int ii = 0; ii < howmany; ii++) {
|
||||
for (int i = 0; i < howmany; i++) {
|
||||
final WAL log = wals.getWAL(infos[i].getEncodedNameAsBytes());
|
||||
for (int j = 0; j < howmany; j++) {
|
||||
WALEdit edit = new WALEdit();
|
||||
byte [] family = Bytes.toBytes("column");
|
||||
byte [] qualifier = Bytes.toBytes(Integer.toString(j));
|
||||
byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
|
||||
edit.add(new KeyValue(rowName, family, qualifier,
|
||||
System.currentTimeMillis(), column));
|
||||
LOG.info("Region " + i + ": " + edit);
|
||||
log.append(htd, infos[i], new WALKey(infos[i].getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), edit, sequenceId, true, null);
|
||||
}
|
||||
log.sync();
|
||||
log.rollWriter();
|
||||
}
|
||||
}
|
||||
wals.shutdown();
|
||||
List<Path> splits = WALSplitter.split(hbaseDir, logdir, oldLogDir, fs, conf, wals);
|
||||
verifySplits(splits, howmany);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test new HDFS-265 sync.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void Broken_testSync() throws Exception {
|
||||
TableName tableName = TableName.valueOf(currentTest.getMethodName());
|
||||
// First verify that using streams all works.
|
||||
Path p = new Path(dir, currentTest.getMethodName() + ".fsdos");
|
||||
FSDataOutputStream out = fs.create(p);
|
||||
out.write(tableName.getName());
|
||||
Method syncMethod = null;
|
||||
try {
|
||||
syncMethod = out.getClass().getMethod("hflush", new Class<?> []{});
|
||||
} catch (NoSuchMethodException e) {
|
||||
try {
|
||||
syncMethod = out.getClass().getMethod("sync", new Class<?> []{});
|
||||
} catch (NoSuchMethodException ex) {
|
||||
fail("This version of Hadoop supports neither Syncable.sync() " +
|
||||
"nor Syncable.hflush().");
|
||||
}
|
||||
}
|
||||
syncMethod.invoke(out, new Object[]{});
|
||||
FSDataInputStream in = fs.open(p);
|
||||
assertTrue(in.available() > 0);
|
||||
byte [] buffer = new byte [1024];
|
||||
int read = in.read(buffer);
|
||||
assertEquals(tableName.getName().length, read);
|
||||
out.close();
|
||||
in.close();
|
||||
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
final int total = 20;
|
||||
WAL.Reader reader = null;
|
||||
|
||||
try {
|
||||
HRegionInfo info = new HRegionInfo(tableName,
|
||||
null,null, false);
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor(tableName.getName()));
|
||||
final WAL wal = wals.getWAL(info.getEncodedNameAsBytes());
|
||||
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
|
||||
wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
// Now call sync and try reading. Opening a Reader before you sync just
|
||||
// gives you EOFE.
|
||||
wal.sync();
|
||||
// Open a Reader.
|
||||
Path walPath = DefaultWALProvider.getCurrentFileName(wal);
|
||||
reader = wals.createReader(fs, walPath);
|
||||
int count = 0;
|
||||
WAL.Entry entry = new WAL.Entry();
|
||||
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
|
||||
// that has had a sync done on it.
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
|
||||
wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
wal.sync();
|
||||
reader = wals.createReader(fs, walPath);
|
||||
count = 0;
|
||||
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++;
|
||||
assertEquals(total * 2, count);
|
||||
reader.close();
|
||||
// Now do a test that ensures stuff works when we go over block boundary,
|
||||
// especially that we return good length on file.
|
||||
final byte [] value = new byte[1025 * 1024]; // Make a 1M value.
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value));
|
||||
wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
// 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++;
|
||||
assertEquals(total * 3, count);
|
||||
reader.close();
|
||||
// shutdown and ensure that Reader gets right length also.
|
||||
wal.shutdown();
|
||||
reader = wals.createReader(fs, walPath);
|
||||
count = 0;
|
||||
while((entry = reader.next(entry)) != null) count++;
|
||||
assertEquals(total * 3, count);
|
||||
reader.close();
|
||||
} finally {
|
||||
if (reader != null) reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
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));
|
||||
WAL.Reader reader = wals.createReader(fs, splits.get(i));
|
||||
try {
|
||||
int count = 0;
|
||||
String previousRegion = null;
|
||||
long seqno = -1;
|
||||
WAL.Entry entry = new WAL.Entry();
|
||||
while((entry = reader.next(entry)) != null) {
|
||||
WALKey key = entry.getKey();
|
||||
String region = Bytes.toString(key.getEncodedRegionName());
|
||||
// Assert that all edits are for same region.
|
||||
if (previousRegion != null) {
|
||||
assertEquals(previousRegion, region);
|
||||
}
|
||||
LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getLogSeqNum());
|
||||
assertTrue(seqno < key.getLogSeqNum());
|
||||
seqno = key.getLogSeqNum();
|
||||
previousRegion = region;
|
||||
count++;
|
||||
}
|
||||
assertEquals(howmany, count);
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* We pass different values to recoverFileLease() so that different code paths are covered
|
||||
*
|
||||
* For this test to pass, requires:
|
||||
* 1. HDFS-200 (append support)
|
||||
* 2. HDFS-988 (SafeMode should freeze file operations
|
||||
* [FSNamesystem.nextGenerationStampForBlock])
|
||||
* 3. HDFS-142 (on restart, maintain pendingCreates)
|
||||
*/
|
||||
@Test (timeout=300000)
|
||||
public void testAppendClose() throws Exception {
|
||||
TableName tableName =
|
||||
TableName.valueOf(currentTest.getMethodName());
|
||||
HRegionInfo regioninfo = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
|
||||
|
||||
final WAL wal = wals.getWAL(regioninfo.getEncodedNameAsBytes());
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
final int total = 20;
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor(tableName.getName()));
|
||||
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
|
||||
wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
// Now call sync to send the data to HDFS datanodes
|
||||
wal.sync();
|
||||
int namenodePort = cluster.getNameNodePort();
|
||||
final Path walPath = DefaultWALProvider.getCurrentFileName(wal);
|
||||
|
||||
|
||||
// Stop the cluster. (ensure restart since we're sharing MiniDFSCluster)
|
||||
try {
|
||||
DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
|
||||
dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
|
||||
TEST_UTIL.shutdownMiniDFSCluster();
|
||||
try {
|
||||
// wal.writer.close() will throw an exception,
|
||||
// but still call this since it closes the LogSyncer thread first
|
||||
wal.shutdown();
|
||||
} catch (IOException e) {
|
||||
LOG.info(e);
|
||||
}
|
||||
fs.close(); // closing FS last so DFSOutputStream can't call close
|
||||
LOG.info("STOPPED first instance of the cluster");
|
||||
} finally {
|
||||
// Restart the cluster
|
||||
while (cluster.isClusterUp()){
|
||||
LOG.error("Waiting for cluster to go down");
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
assertFalse(cluster.isClusterUp());
|
||||
cluster = null;
|
||||
for (int i = 0; i < 100; i++) {
|
||||
try {
|
||||
cluster = TEST_UTIL.startMiniDFSClusterForTestWAL(namenodePort);
|
||||
break;
|
||||
} catch (BindException e) {
|
||||
LOG.info("Sleeping. BindException bringing up new cluster");
|
||||
Threads.sleep(1000);
|
||||
}
|
||||
}
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
LOG.info("STARTED second instance.");
|
||||
}
|
||||
|
||||
// set the lease period to be 1 second so that the
|
||||
// namenode triggers lease recovery upon append request
|
||||
Method setLeasePeriod = cluster.getClass()
|
||||
.getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE});
|
||||
setLeasePeriod.setAccessible(true);
|
||||
setLeasePeriod.invoke(cluster, 1000L, 1000L);
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info(e);
|
||||
}
|
||||
|
||||
// Now try recovering the log, like the HMaster would do
|
||||
final FileSystem recoveredFs = fs;
|
||||
final Configuration rlConf = conf;
|
||||
|
||||
class RecoverLogThread extends Thread {
|
||||
public Exception exception = null;
|
||||
public void run() {
|
||||
try {
|
||||
FSUtils.getInstance(fs, rlConf)
|
||||
.recoverFileLease(recoveredFs, walPath, rlConf, null);
|
||||
} catch (IOException e) {
|
||||
exception = e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
RecoverLogThread t = new RecoverLogThread();
|
||||
t.start();
|
||||
// Timeout after 60 sec. Without correct patches, would be an infinite loop
|
||||
t.join(60 * 1000);
|
||||
if(t.isAlive()) {
|
||||
t.interrupt();
|
||||
throw new Exception("Timed out waiting for WAL.recoverLog()");
|
||||
}
|
||||
|
||||
if (t.exception != null)
|
||||
throw t.exception;
|
||||
|
||||
// Make sure you can read all the content
|
||||
WAL.Reader reader = wals.createReader(fs, walPath);
|
||||
int count = 0;
|
||||
WAL.Entry entry = new WAL.Entry();
|
||||
while (reader.next(entry) != null) {
|
||||
count++;
|
||||
assertTrue("Should be one KeyValue per WALEdit",
|
||||
entry.getEdit().getCells().size() == 1);
|
||||
}
|
||||
assertEquals(total, count);
|
||||
reader.close();
|
||||
|
||||
// Reset the lease period
|
||||
setLeasePeriod.invoke(cluster, new Object[]{new Long(60000), new Long(3600000)});
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we can write out an edit, close, and then read it back in again.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testEditAdd() throws IOException {
|
||||
final int COL_COUNT = 10;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte [] row = Bytes.toBytes("row");
|
||||
WAL.Reader reader = null;
|
||||
try {
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
|
||||
// Write columns named 1, 2, 3, etc. and then values of single byte
|
||||
// 1, 2, 3...
|
||||
long timestamp = System.currentTimeMillis();
|
||||
WALEdit cols = new WALEdit();
|
||||
for (int i = 0; i < COL_COUNT; i++) {
|
||||
cols.add(new KeyValue(row, Bytes.toBytes("column"),
|
||||
Bytes.toBytes(Integer.toString(i)),
|
||||
timestamp, new byte[] { (byte)(i + '0') }));
|
||||
}
|
||||
HRegionInfo info = new HRegionInfo(tableName,
|
||||
row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor("column"));
|
||||
final WAL log = wals.getWAL(info.getEncodedNameAsBytes());
|
||||
|
||||
final long txid = log.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), cols, sequenceId, true, null);
|
||||
log.sync(txid);
|
||||
log.startCacheFlush(info.getEncodedNameAsBytes());
|
||||
log.completeCacheFlush(info.getEncodedNameAsBytes());
|
||||
log.shutdown();
|
||||
Path filename = DefaultWALProvider.getCurrentFileName(log);
|
||||
// Now open a reader on the log and assert append worked.
|
||||
reader = wals.createReader(fs, filename);
|
||||
// Above we added all columns on a single row so we only read one
|
||||
// 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;
|
||||
WALKey key = entry.getKey();
|
||||
WALEdit val = entry.getEdit();
|
||||
assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
|
||||
assertTrue(tableName.equals(key.getTablename()));
|
||||
Cell cell = val.getCells().get(0);
|
||||
assertTrue(Bytes.equals(row, cell.getRow()));
|
||||
assertEquals((byte)(i + '0'), cell.getValue()[0]);
|
||||
System.out.println(key + " " + val);
|
||||
}
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testAppend() throws IOException {
|
||||
final int COL_COUNT = 10;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte [] row = Bytes.toBytes("row");
|
||||
WAL.Reader reader = null;
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
try {
|
||||
// Write columns named 1, 2, 3, etc. and then values of single byte
|
||||
// 1, 2, 3...
|
||||
long timestamp = System.currentTimeMillis();
|
||||
WALEdit cols = new WALEdit();
|
||||
for (int i = 0; i < COL_COUNT; i++) {
|
||||
cols.add(new KeyValue(row, Bytes.toBytes("column"),
|
||||
Bytes.toBytes(Integer.toString(i)),
|
||||
timestamp, new byte[] { (byte)(i + '0') }));
|
||||
}
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor("column"));
|
||||
final WAL log = wals.getWAL(hri.getEncodedNameAsBytes());
|
||||
final long txid = log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), cols, sequenceId, true, null);
|
||||
log.sync(txid);
|
||||
log.startCacheFlush(hri.getEncodedNameAsBytes());
|
||||
log.completeCacheFlush(hri.getEncodedNameAsBytes());
|
||||
log.shutdown();
|
||||
Path filename = DefaultWALProvider.getCurrentFileName(log);
|
||||
// Now open a reader on the log and assert append worked.
|
||||
reader = wals.createReader(fs, filename);
|
||||
WAL.Entry entry = reader.next();
|
||||
assertEquals(COL_COUNT, entry.getEdit().size());
|
||||
int idx = 0;
|
||||
for (Cell val : entry.getEdit().getCells()) {
|
||||
assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(),
|
||||
entry.getKey().getEncodedRegionName()));
|
||||
assertTrue(tableName.equals(entry.getKey().getTablename()));
|
||||
assertTrue(Bytes.equals(row, val.getRow()));
|
||||
assertEquals((byte)(idx + '0'), val.getValue()[0]);
|
||||
System.out.println(entry.getKey() + " " + val);
|
||||
idx++;
|
||||
}
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we can visit entries before they are appended
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testVisitors() throws Exception {
|
||||
final int COL_COUNT = 10;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte [] row = Bytes.toBytes("row");
|
||||
final DumbWALActionsListener visitor = new DumbWALActionsListener();
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
long timestamp = System.currentTimeMillis();
|
||||
HTableDescriptor htd = new HTableDescriptor();
|
||||
htd.addFamily(new HColumnDescriptor("column"));
|
||||
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
final WAL log = wals.getWAL(hri.getEncodedNameAsBytes());
|
||||
log.registerWALActionsListener(visitor);
|
||||
for (int i = 0; i < COL_COUNT; i++) {
|
||||
WALEdit cols = new WALEdit();
|
||||
cols.add(new KeyValue(row, Bytes.toBytes("column"),
|
||||
Bytes.toBytes(Integer.toString(i)),
|
||||
timestamp, new byte[]{(byte) (i + '0')}));
|
||||
log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), cols, sequenceId, true, null);
|
||||
}
|
||||
log.sync();
|
||||
assertEquals(COL_COUNT, visitor.increments);
|
||||
log.unregisterWALActionsListener(visitor);
|
||||
WALEdit cols = new WALEdit();
|
||||
cols.add(new KeyValue(row, Bytes.toBytes("column"),
|
||||
Bytes.toBytes(Integer.toString(11)),
|
||||
timestamp, new byte[]{(byte) (11 + '0')}));
|
||||
log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), cols, sequenceId, true, null);
|
||||
log.sync();
|
||||
assertEquals(COL_COUNT, visitor.increments);
|
||||
}
|
||||
|
||||
/**
|
||||
* A loaded WAL coprocessor won't break existing WAL test cases.
|
||||
*/
|
||||
@Test
|
||||
public void testWALCoprocessorLoaded() throws Exception {
|
||||
// test to see whether the coprocessor is loaded or not.
|
||||
WALCoprocessorHost host = wals.getWAL(UNSPECIFIED_REGION).getCoprocessorHost();
|
||||
Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
|
||||
assertNotNull(c);
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testReadLegacyLog() throws IOException {
|
||||
final int columnCount = 5;
|
||||
final int recordCount = 5;
|
||||
final TableName tableName =
|
||||
TableName.valueOf("tablename");
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
long timestamp = System.currentTimeMillis();
|
||||
Path path = new Path(dir, "tempwal");
|
||||
SequenceFileLogWriter sflw = null;
|
||||
WAL.Reader reader = null;
|
||||
try {
|
||||
HRegionInfo hri = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
fs.mkdirs(dir);
|
||||
// Write log in pre-PB format.
|
||||
sflw = new SequenceFileLogWriter();
|
||||
sflw.init(fs, path, conf, false);
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WALKey key = new HLogKey(
|
||||
hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
|
||||
WALEdit edit = new WALEdit();
|
||||
for (int j = 0; j < columnCount; ++j) {
|
||||
if (i == 0) {
|
||||
htd.addFamily(new HColumnDescriptor("column" + j));
|
||||
}
|
||||
String value = i + "" + j;
|
||||
edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
|
||||
}
|
||||
sflw.append(new WAL.Entry(key, edit));
|
||||
}
|
||||
sflw.sync();
|
||||
sflw.close();
|
||||
|
||||
// Now read the log using standard means.
|
||||
reader = wals.createReader(fs, path);
|
||||
assertTrue(reader instanceof SequenceFileLogReader);
|
||||
for (int i = 0; i < recordCount; ++i) {
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNotNull(entry);
|
||||
assertEquals(columnCount, entry.getEdit().size());
|
||||
assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
|
||||
assertEquals(tableName, entry.getKey().getTablename());
|
||||
int idx = 0;
|
||||
for (Cell val : entry.getEdit().getCells()) {
|
||||
assertTrue(Bytes.equals(row, val.getRow()));
|
||||
String value = i + "" + idx;
|
||||
assertArrayEquals(Bytes.toBytes(value), val.getValue());
|
||||
idx++;
|
||||
}
|
||||
}
|
||||
WAL.Entry entry = reader.next();
|
||||
assertNull(entry);
|
||||
} finally {
|
||||
if (sflw != null) {
|
||||
sflw.close();
|
||||
}
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class DumbWALActionsListener extends WALActionsListener.Base {
|
||||
int increments = 0;
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey,
|
||||
WALEdit logEdit) {
|
||||
increments++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
|
||||
//To change body of implemented methods use File | Settings | File Templates.
|
||||
increments++;
|
||||
}
|
||||
}
|
||||
|
||||
private static final byte[] UNSPECIFIED_REGION = new byte[]{};
|
||||
|
||||
}
|
@ -0,0 +1,154 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestWALFiltering {
|
||||
private static final int NUM_MASTERS = 1;
|
||||
private static final int NUM_RS = 4;
|
||||
|
||||
private static final TableName TABLE_NAME =
|
||||
TableName.valueOf("TestWALFiltering");
|
||||
private static final byte[] CF1 = Bytes.toBytes("MyCF1");
|
||||
private static final byte[] CF2 = Bytes.toBytes("MyCF2");
|
||||
private static final byte[][] FAMILIES = { CF1, CF2 };
|
||||
|
||||
private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
fillTable();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private void fillTable() throws IOException, InterruptedException {
|
||||
Table table = TEST_UTIL.createTable(TABLE_NAME, FAMILIES, 3,
|
||||
Bytes.toBytes("row0"), Bytes.toBytes("row99"), NUM_RS);
|
||||
Random rand = new Random(19387129L);
|
||||
for (int iStoreFile = 0; iStoreFile < 4; ++iStoreFile) {
|
||||
for (int iRow = 0; iRow < 100; ++iRow) {
|
||||
final byte[] row = Bytes.toBytes("row" + iRow);
|
||||
Put put = new Put(row);
|
||||
Delete del = new Delete(row);
|
||||
for (int iCol = 0; iCol < 10; ++iCol) {
|
||||
final byte[] cf = rand.nextBoolean() ? CF1 : CF2;
|
||||
final long ts = Math.abs(rand.nextInt());
|
||||
final byte[] qual = Bytes.toBytes("col" + iCol);
|
||||
if (rand.nextBoolean()) {
|
||||
final byte[] value = Bytes.toBytes("value_for_row_" + iRow +
|
||||
"_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" +
|
||||
ts + "_random_" + rand.nextLong());
|
||||
put.add(cf, qual, ts, value);
|
||||
} else if (rand.nextDouble() < 0.8) {
|
||||
del.deleteColumn(cf, qual, ts);
|
||||
} else {
|
||||
del.deleteColumns(cf, qual, ts);
|
||||
}
|
||||
}
|
||||
table.put(put);
|
||||
table.delete(del);
|
||||
table.flushCommits();
|
||||
}
|
||||
}
|
||||
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFlushedSequenceIdsSentToHMaster()
|
||||
throws IOException, InterruptedException, ServiceException {
|
||||
SortedMap<byte[], Long> allFlushedSequenceIds =
|
||||
new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
|
||||
for (int i = 0; i < NUM_RS; ++i) {
|
||||
flushAllRegions(i);
|
||||
}
|
||||
Thread.sleep(10000);
|
||||
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
for (int i = 0; i < NUM_RS; ++i) {
|
||||
for (byte[] regionName : getRegionsByServer(i)) {
|
||||
if (allFlushedSequenceIds.containsKey(regionName)) {
|
||||
GetLastFlushedSequenceIdRequest req =
|
||||
RequestConverter.buildGetLastFlushedSequenceIdRequest(regionName);
|
||||
|
||||
assertEquals((long)allFlushedSequenceIds.get(regionName),
|
||||
master.getMasterRpcServices().getLastFlushedSequenceId(
|
||||
null, req).getLastFlushedSequenceId());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private List<byte[]> getRegionsByServer(int rsId) throws IOException {
|
||||
List<byte[]> regionNames = Lists.newArrayList();
|
||||
HRegionServer hrs = getRegionServer(rsId);
|
||||
for (HRegion r : hrs.getOnlineRegions(TABLE_NAME)) {
|
||||
regionNames.add(r.getRegionName());
|
||||
}
|
||||
return regionNames;
|
||||
}
|
||||
|
||||
private HRegionServer getRegionServer(int rsId) {
|
||||
return TEST_UTIL.getMiniHBaseCluster().getRegionServer(rsId);
|
||||
}
|
||||
|
||||
private void flushAllRegions(int rsId)
|
||||
throws ServiceException, IOException {
|
||||
HRegionServer hrs = getRegionServer(rsId);
|
||||
for (byte[] regionName : getRegionsByServer(rsId)) {
|
||||
FlushRegionRequest request =
|
||||
RequestConverter.buildFlushRegionRequest(regionName);
|
||||
hrs.getRSRpcServices().flushRegion(null, request);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,178 @@
|
||||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* Simple testing of a few WAL methods.
|
||||
*/
|
||||
@Category({RegionServerTests.class, SmallTests.class})
|
||||
public class TestWALMethods {
|
||||
private static final byte[] TEST_REGION = Bytes.toBytes("test_region");;
|
||||
private static final TableName TEST_TABLE =
|
||||
TableName.valueOf("test_table");
|
||||
|
||||
private final HBaseTestingUtility util = new HBaseTestingUtility();
|
||||
|
||||
/**
|
||||
* Assert that getSplitEditFilesSorted returns files in expected order and
|
||||
* that it skips moved-aside files.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test public void testGetSplitEditFilesSorted() throws IOException {
|
||||
FileSystem fs = FileSystem.get(util.getConfiguration());
|
||||
Path regiondir = util.getDataTestDir("regiondir");
|
||||
fs.delete(regiondir, true);
|
||||
fs.mkdirs(regiondir);
|
||||
Path recoverededits = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
|
||||
String first = WALSplitter.formatRecoveredEditsFileName(-1);
|
||||
createFile(fs, recoverededits, first);
|
||||
createFile(fs, recoverededits, WALSplitter.formatRecoveredEditsFileName(0));
|
||||
createFile(fs, recoverededits, WALSplitter.formatRecoveredEditsFileName(1));
|
||||
createFile(fs, recoverededits, WALSplitter
|
||||
.formatRecoveredEditsFileName(11));
|
||||
createFile(fs, recoverededits, WALSplitter.formatRecoveredEditsFileName(2));
|
||||
createFile(fs, recoverededits, WALSplitter
|
||||
.formatRecoveredEditsFileName(50));
|
||||
String last = WALSplitter.formatRecoveredEditsFileName(Long.MAX_VALUE);
|
||||
createFile(fs, recoverededits, last);
|
||||
createFile(fs, recoverededits,
|
||||
Long.toString(Long.MAX_VALUE) + "." + System.currentTimeMillis());
|
||||
|
||||
final Configuration walConf = new Configuration(util.getConfiguration());
|
||||
FSUtils.setRootDir(walConf, regiondir);
|
||||
(new WALFactory(walConf, null, "dummyLogName")).getWAL(new byte[]{});
|
||||
|
||||
NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
|
||||
assertEquals(7, files.size());
|
||||
assertEquals(files.pollFirst().getName(), first);
|
||||
assertEquals(files.pollLast().getName(), last);
|
||||
assertEquals(files.pollFirst().getName(),
|
||||
WALSplitter
|
||||
.formatRecoveredEditsFileName(0));
|
||||
assertEquals(files.pollFirst().getName(),
|
||||
WALSplitter
|
||||
.formatRecoveredEditsFileName(1));
|
||||
assertEquals(files.pollFirst().getName(),
|
||||
WALSplitter
|
||||
.formatRecoveredEditsFileName(2));
|
||||
assertEquals(files.pollFirst().getName(),
|
||||
WALSplitter
|
||||
.formatRecoveredEditsFileName(11));
|
||||
}
|
||||
|
||||
private void createFile(final FileSystem fs, final Path testdir,
|
||||
final String name)
|
||||
throws IOException {
|
||||
FSDataOutputStream fdos = fs.create(new Path(testdir, name), true);
|
||||
fdos.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRegionEntryBuffer() throws Exception {
|
||||
WALSplitter.RegionEntryBuffer reb = new WALSplitter.RegionEntryBuffer(
|
||||
TEST_TABLE, TEST_REGION);
|
||||
assertEquals(0, reb.heapSize());
|
||||
|
||||
reb.appendEntry(createTestLogEntry(1));
|
||||
assertTrue(reb.heapSize() > 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEntrySink() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
|
||||
EntryBuffers sink = new EntryBuffers(new PipelineController(), 1*1024*1024);
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
WAL.Entry entry = createTestLogEntry(i);
|
||||
sink.appendEntry(entry);
|
||||
}
|
||||
|
||||
assertTrue(sink.totalBuffered > 0);
|
||||
long amountInChunk = sink.totalBuffered;
|
||||
// Get a chunk
|
||||
RegionEntryBuffer chunk = sink.getChunkToWrite();
|
||||
assertEquals(chunk.heapSize(), amountInChunk);
|
||||
|
||||
// Make sure it got marked that a thread is "working on this"
|
||||
assertTrue(sink.isRegionCurrentlyWriting(TEST_REGION));
|
||||
|
||||
// Insert some more entries
|
||||
for (int i = 0; i < 500; i++) {
|
||||
WAL.Entry entry = createTestLogEntry(i);
|
||||
sink.appendEntry(entry);
|
||||
}
|
||||
// Asking for another chunk shouldn't work since the first one
|
||||
// is still writing
|
||||
assertNull(sink.getChunkToWrite());
|
||||
|
||||
// If we say we're done writing the first chunk, then we should be able
|
||||
// to get the second
|
||||
sink.doneWriting(chunk);
|
||||
|
||||
RegionEntryBuffer chunk2 = sink.getChunkToWrite();
|
||||
assertNotNull(chunk2);
|
||||
assertNotSame(chunk, chunk2);
|
||||
long amountInChunk2 = sink.totalBuffered;
|
||||
// The second chunk had fewer rows than the first
|
||||
assertTrue(amountInChunk2 < amountInChunk);
|
||||
|
||||
sink.doneWriting(chunk2);
|
||||
assertEquals(0, sink.totalBuffered);
|
||||
}
|
||||
|
||||
private WAL.Entry createTestLogEntry(int i) {
|
||||
long seq = i;
|
||||
long now = i * 1000;
|
||||
|
||||
WALEdit edit = new WALEdit();
|
||||
edit.add(KeyValueTestUtil.create("row", "fam", "qual", 1234, "val"));
|
||||
WALKey key = new WALKey(TEST_REGION, TEST_TABLE, seq, now,
|
||||
HConstants.DEFAULT_CLUSTER_ID);
|
||||
WAL.Entry entry = new WAL.Entry(key, edit);
|
||||
return entry;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,218 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureWALCellCodec;
|
||||
|
||||
/*
|
||||
* Test that verifies WAL written by SecureProtobufLogWriter is not readable by ProtobufLogReader
|
||||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestWALReaderOnSecureWAL {
|
||||
static final Log LOG = LogFactory.getLog(TestWALReaderOnSecureWAL.class);
|
||||
static {
|
||||
((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal"))
|
||||
.getLogger().setLevel(Level.ALL);
|
||||
};
|
||||
static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
final byte[] value = Bytes.toBytes("Test value");
|
||||
|
||||
private static final String WAL_ENCRYPTION = "hbase.regionserver.wal.encryption";
|
||||
|
||||
@Rule
|
||||
public TestName currentTest = new TestName();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
|
||||
conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
|
||||
conf.setBoolean("hbase.hlog.split.skip.errors", true);
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
|
||||
FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
|
||||
}
|
||||
|
||||
private Path writeWAL(final WALFactory wals, final String tblName) throws IOException {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
String clsName = conf.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
|
||||
conf.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, SecureWALCellCodec.class,
|
||||
WALCellCodec.class);
|
||||
try {
|
||||
TableName tableName = TableName.valueOf(tblName);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(tableName.getName()));
|
||||
HRegionInfo regioninfo = new HRegionInfo(tableName,
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
|
||||
final int total = 10;
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
final byte[] family = Bytes.toBytes("family");
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
Path logDir = TEST_UTIL.getDataTestDir(tblName);
|
||||
final AtomicLong sequenceId = new AtomicLong(1);
|
||||
|
||||
// Write the WAL
|
||||
WAL wal = wals.getWAL(regioninfo.getEncodedNameAsBytes());
|
||||
for (int i = 0; i < total; i++) {
|
||||
WALEdit kvs = new WALEdit();
|
||||
kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
|
||||
wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
|
||||
System.currentTimeMillis()), kvs, sequenceId, true, null);
|
||||
}
|
||||
wal.sync();
|
||||
final Path walPath = DefaultWALProvider.getCurrentFileName(wal);
|
||||
wal.shutdown();
|
||||
|
||||
return walPath;
|
||||
} finally {
|
||||
// restore the cell codec class
|
||||
conf.set(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, clsName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test()
|
||||
public void testWALReaderOnSecureWAL() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
|
||||
WAL.Reader.class);
|
||||
conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
|
||||
WALProvider.Writer.class);
|
||||
conf.setBoolean(WAL_ENCRYPTION, true);
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final WALFactory wals = new WALFactory(conf, null, currentTest.getMethodName());
|
||||
Path walPath = writeWAL(wals, currentTest.getMethodName());
|
||||
|
||||
// Insure edits are not plaintext
|
||||
long length = fs.getFileStatus(walPath).getLen();
|
||||
FSDataInputStream in = fs.open(walPath);
|
||||
byte[] fileData = new byte[(int)length];
|
||||
IOUtils.readFully(in, fileData);
|
||||
in.close();
|
||||
assertFalse("Cells appear to be plaintext", Bytes.contains(fileData, value));
|
||||
|
||||
// Confirm the WAL cannot be read back by ProtobufLogReader
|
||||
try {
|
||||
WAL.Reader reader = wals.createReader(TEST_UTIL.getTestFileSystem(), walPath);
|
||||
assertFalse(true);
|
||||
} catch (IOException ioe) {
|
||||
// expected IOE
|
||||
}
|
||||
|
||||
FileStatus[] listStatus = fs.listStatus(walPath.getParent());
|
||||
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
Path rootdir = FSUtils.getRootDir(conf);
|
||||
try {
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null, mode);
|
||||
s.splitLogFile(listStatus[0], null);
|
||||
Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()),
|
||||
"corrupt");
|
||||
assertTrue(fs.exists(file));
|
||||
// assertFalse("log splitting should have failed", true);
|
||||
} catch (IOException ioe) {
|
||||
assertTrue("WAL should have been sidelined", false);
|
||||
}
|
||||
wals.close();
|
||||
}
|
||||
|
||||
@Test()
|
||||
public void testSecureWALReaderOnWAL() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
|
||||
WAL.Reader.class);
|
||||
conf.setClass("hbase.regionserver.hlog.writer.impl", ProtobufLogWriter.class,
|
||||
WALProvider.Writer.class);
|
||||
conf.setBoolean(WAL_ENCRYPTION, false);
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final WALFactory wals = new WALFactory(conf, null, currentTest.getMethodName());
|
||||
Path walPath = writeWAL(wals, currentTest.getMethodName());
|
||||
|
||||
// Ensure edits are plaintext
|
||||
long length = fs.getFileStatus(walPath).getLen();
|
||||
FSDataInputStream in = fs.open(walPath);
|
||||
byte[] fileData = new byte[(int)length];
|
||||
IOUtils.readFully(in, fileData);
|
||||
in.close();
|
||||
assertTrue("Cells should be plaintext", Bytes.contains(fileData, value));
|
||||
|
||||
// Confirm the WAL can be read back by SecureProtobufLogReader
|
||||
try {
|
||||
WAL.Reader reader = wals.createReader(TEST_UTIL.getTestFileSystem(), walPath);
|
||||
reader.close();
|
||||
} catch (IOException ioe) {
|
||||
assertFalse(true);
|
||||
}
|
||||
|
||||
FileStatus[] listStatus = fs.listStatus(walPath.getParent());
|
||||
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
Path rootdir = FSUtils.getRootDir(conf);
|
||||
try {
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null, mode);
|
||||
s.splitLogFile(listStatus[0], null);
|
||||
Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()),
|
||||
"corrupt");
|
||||
assertTrue(!fs.exists(file));
|
||||
} catch (IOException ioe) {
|
||||
assertTrue("WAL should have been processed", false);
|
||||
}
|
||||
wals.close();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,36 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({RegionServerTests.class, LargeTests.class})
|
||||
public class TestWALSplitCompressed extends TestWALSplit {
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TestWALSplit.setUpBeforeClass();
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
|
||||
}
|
||||
}
|
@ -0,0 +1,567 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.htrace.Sampler;
|
||||
import org.htrace.Trace;
|
||||
import org.htrace.TraceScope;
|
||||
import org.htrace.impl.ProbabilitySampler;
|
||||
|
||||
import com.yammer.metrics.core.Histogram;
|
||||
import com.yammer.metrics.core.Meter;
|
||||
import com.yammer.metrics.core.MetricsRegistry;
|
||||
import com.yammer.metrics.reporting.ConsoleReporter;
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* This class runs performance benchmarks for {@link WAL}.
|
||||
* See usage for this tool by running:
|
||||
* <code>$ hbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation -h</code>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class WALPerformanceEvaluation extends Configured implements Tool {
|
||||
static final Log LOG = LogFactory.getLog(WALPerformanceEvaluation.class.getName());
|
||||
private final MetricsRegistry metrics = new MetricsRegistry();
|
||||
private final Meter syncMeter =
|
||||
metrics.newMeter(WALPerformanceEvaluation.class, "syncMeter", "syncs", TimeUnit.MILLISECONDS);
|
||||
private final Histogram syncHistogram =
|
||||
metrics.newHistogram(WALPerformanceEvaluation.class, "syncHistogram", "nanos-between-syncs",
|
||||
true);
|
||||
private final Histogram syncCountHistogram =
|
||||
metrics.newHistogram(WALPerformanceEvaluation.class, "syncCountHistogram", "countPerSync",
|
||||
true);
|
||||
private final Meter appendMeter =
|
||||
metrics.newMeter(WALPerformanceEvaluation.class, "appendMeter", "bytes",
|
||||
TimeUnit.MILLISECONDS);
|
||||
private final Histogram latencyHistogram =
|
||||
metrics.newHistogram(WALPerformanceEvaluation.class, "latencyHistogram", "nanos", true);
|
||||
|
||||
private HBaseTestingUtility TEST_UTIL;
|
||||
|
||||
static final String TABLE_NAME = "WALPerformanceEvaluation";
|
||||
static final String QUALIFIER_PREFIX = "q";
|
||||
static final String FAMILY_PREFIX = "cf";
|
||||
|
||||
private int numQualifiers = 1;
|
||||
private int valueSize = 512;
|
||||
private int keySize = 16;
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
super.setConf(conf);
|
||||
TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform WAL.append() of Put object, for the number of iterations requested.
|
||||
* Keys and Vaues are generated randomly, the number of column families,
|
||||
* qualifiers and key/value size is tunable by the user.
|
||||
*/
|
||||
class WALPutBenchmark implements Runnable {
|
||||
private final long numIterations;
|
||||
private final int numFamilies;
|
||||
private final boolean noSync;
|
||||
private final HRegion region;
|
||||
private final int syncInterval;
|
||||
private final HTableDescriptor htd;
|
||||
private final Sampler loopSampler;
|
||||
|
||||
WALPutBenchmark(final HRegion region, final HTableDescriptor htd,
|
||||
final long numIterations, final boolean noSync, final int syncInterval,
|
||||
final double traceFreq) {
|
||||
this.numIterations = numIterations;
|
||||
this.noSync = noSync;
|
||||
this.syncInterval = syncInterval;
|
||||
this.numFamilies = htd.getColumnFamilies().length;
|
||||
this.region = region;
|
||||
this.htd = htd;
|
||||
String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
|
||||
if (spanReceivers == null || spanReceivers.isEmpty()) {
|
||||
loopSampler = Sampler.NEVER;
|
||||
} else {
|
||||
if (traceFreq <= 0.0) {
|
||||
LOG.warn("Tracing enabled but traceFreq=0.");
|
||||
loopSampler = Sampler.NEVER;
|
||||
} else if (traceFreq >= 1.0) {
|
||||
loopSampler = Sampler.ALWAYS;
|
||||
if (numIterations > 1000) {
|
||||
LOG.warn("Full tracing of all iterations will produce a lot of data. Be sure your"
|
||||
+ " SpanReciever can keep up.");
|
||||
}
|
||||
} else {
|
||||
loopSampler = new ProbabilitySampler(traceFreq);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
byte[] key = new byte[keySize];
|
||||
byte[] value = new byte[valueSize];
|
||||
Random rand = new Random(Thread.currentThread().getId());
|
||||
WAL wal = region.getWAL();
|
||||
|
||||
TraceScope threadScope =
|
||||
Trace.startSpan("WALPerfEval." + Thread.currentThread().getName());
|
||||
try {
|
||||
long startTime = System.currentTimeMillis();
|
||||
int lastSync = 0;
|
||||
for (int i = 0; i < numIterations; ++i) {
|
||||
assert Trace.currentSpan() == threadScope.getSpan() : "Span leak detected.";
|
||||
TraceScope loopScope = Trace.startSpan("runLoopIter" + i, loopSampler);
|
||||
try {
|
||||
long now = System.nanoTime();
|
||||
Put put = setupPut(rand, key, value, numFamilies);
|
||||
WALEdit walEdit = new WALEdit();
|
||||
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
|
||||
HRegionInfo hri = region.getRegionInfo();
|
||||
final WALKey logkey = new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now);
|
||||
wal.append(htd, hri, logkey, walEdit, region.getSequenceId(), true, null);
|
||||
if (!this.noSync) {
|
||||
if (++lastSync >= this.syncInterval) {
|
||||
wal.sync();
|
||||
lastSync = 0;
|
||||
}
|
||||
}
|
||||
latencyHistogram.update(System.nanoTime() - now);
|
||||
} finally {
|
||||
loopScope.close();
|
||||
}
|
||||
}
|
||||
long totalTime = (System.currentTimeMillis() - startTime);
|
||||
logBenchmarkResult(Thread.currentThread().getName(), numIterations, totalTime);
|
||||
} catch (Exception e) {
|
||||
LOG.error(getClass().getSimpleName() + " Thread failed", e);
|
||||
} finally {
|
||||
threadScope.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(String[] args) throws Exception {
|
||||
Path rootRegionDir = null;
|
||||
int numThreads = 1;
|
||||
long numIterations = 1000000;
|
||||
int numFamilies = 1;
|
||||
int syncInterval = 0;
|
||||
boolean noSync = false;
|
||||
boolean verify = false;
|
||||
boolean verbose = false;
|
||||
boolean cleanup = true;
|
||||
boolean noclosefs = false;
|
||||
long roll = Long.MAX_VALUE;
|
||||
boolean compress = false;
|
||||
String cipher = null;
|
||||
int numRegions = 1;
|
||||
String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
|
||||
boolean trace = spanReceivers != null && !spanReceivers.isEmpty();
|
||||
double traceFreq = 1.0;
|
||||
// Process command line args
|
||||
for (int i = 0; i < args.length; i++) {
|
||||
String cmd = args[i];
|
||||
try {
|
||||
if (cmd.equals("-threads")) {
|
||||
numThreads = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-iterations")) {
|
||||
numIterations = Long.parseLong(args[++i]);
|
||||
} else if (cmd.equals("-path")) {
|
||||
rootRegionDir = new Path(args[++i]);
|
||||
} else if (cmd.equals("-families")) {
|
||||
numFamilies = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-qualifiers")) {
|
||||
numQualifiers = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-keySize")) {
|
||||
keySize = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-valueSize")) {
|
||||
valueSize = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-syncInterval")) {
|
||||
syncInterval = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-nosync")) {
|
||||
noSync = true;
|
||||
} else if (cmd.equals("-verify")) {
|
||||
verify = true;
|
||||
} else if (cmd.equals("-verbose")) {
|
||||
verbose = true;
|
||||
} else if (cmd.equals("-nocleanup")) {
|
||||
cleanup = false;
|
||||
} else if (cmd.equals("-noclosefs")) {
|
||||
noclosefs = true;
|
||||
} else if (cmd.equals("-roll")) {
|
||||
roll = Long.parseLong(args[++i]);
|
||||
} else if (cmd.equals("-compress")) {
|
||||
compress = true;
|
||||
} else if (cmd.equals("-encryption")) {
|
||||
cipher = args[++i];
|
||||
} else if (cmd.equals("-regions")) {
|
||||
numRegions = Integer.parseInt(args[++i]);
|
||||
} else if (cmd.equals("-traceFreq")) {
|
||||
traceFreq = Double.parseDouble(args[++i]);
|
||||
} else if (cmd.equals("-h")) {
|
||||
printUsageAndExit();
|
||||
} else if (cmd.equals("--help")) {
|
||||
printUsageAndExit();
|
||||
} else {
|
||||
System.err.println("UNEXPECTED: " + cmd);
|
||||
printUsageAndExit();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
printUsageAndExit();
|
||||
}
|
||||
}
|
||||
|
||||
if (compress) {
|
||||
Configuration conf = getConf();
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
|
||||
}
|
||||
|
||||
if (cipher != null) {
|
||||
// Set up WAL for encryption
|
||||
Configuration conf = getConf();
|
||||
conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
|
||||
conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
|
||||
conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
|
||||
WAL.Reader.class);
|
||||
conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
|
||||
Writer.class);
|
||||
conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
|
||||
conf.set(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, cipher);
|
||||
}
|
||||
|
||||
if (numThreads < numRegions) {
|
||||
LOG.warn("Number of threads is less than the number of regions; some regions will sit idle.");
|
||||
}
|
||||
|
||||
// Internal config. goes off number of threads; if more threads than handlers, stuff breaks.
|
||||
// In regionserver, number of handlers == number of threads.
|
||||
getConf().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, numThreads);
|
||||
|
||||
// Run WAL Performance Evaluation
|
||||
// First set the fs from configs. In case we are on hadoop1
|
||||
FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf()));
|
||||
FileSystem fs = FileSystem.get(getConf());
|
||||
LOG.info("FileSystem: " + fs);
|
||||
|
||||
SpanReceiverHost receiverHost = trace ? SpanReceiverHost.getInstance(getConf()) : null;
|
||||
TraceScope scope = Trace.startSpan("WALPerfEval", trace ? Sampler.ALWAYS : Sampler.NEVER);
|
||||
|
||||
try {
|
||||
if (rootRegionDir == null) {
|
||||
rootRegionDir = TEST_UTIL.getDataTestDirOnTestFS("WALPerformanceEvaluation");
|
||||
}
|
||||
rootRegionDir = rootRegionDir.makeQualified(fs);
|
||||
cleanRegionRootDir(fs, rootRegionDir);
|
||||
FSUtils.setRootDir(getConf(), rootRegionDir);
|
||||
final WALFactory wals = new WALFactory(getConf(), null, "wals");
|
||||
final HRegion[] regions = new HRegion[numRegions];
|
||||
final Runnable[] benchmarks = new Runnable[numRegions];
|
||||
|
||||
try {
|
||||
for(int i = 0; i < numRegions; i++) {
|
||||
// Initialize Table Descriptor
|
||||
// a table per desired region means we can avoid carving up the key space
|
||||
final HTableDescriptor htd = createHTableDescriptor(i, numFamilies);
|
||||
regions[i] = openRegion(fs, rootRegionDir, htd, wals, roll);
|
||||
benchmarks[i] = Trace.wrap(new WALPutBenchmark(regions[i], htd, numIterations, noSync,
|
||||
syncInterval, traceFreq));
|
||||
}
|
||||
ConsoleReporter.enable(this.metrics, 30, TimeUnit.SECONDS);
|
||||
long putTime = runBenchmark(benchmarks, numThreads);
|
||||
logBenchmarkResult("Summary: threads=" + numThreads + ", iterations=" + numIterations +
|
||||
", syncInterval=" + syncInterval, numIterations * numThreads, putTime);
|
||||
|
||||
for (int i = 0; i < numRegions; i++) {
|
||||
if (regions[i] != null) {
|
||||
closeRegion(regions[i]);
|
||||
regions[i] = null;
|
||||
}
|
||||
}
|
||||
if (verify) {
|
||||
Path dir = new Path(FSUtils.getRootDir(getConf()),
|
||||
DefaultWALProvider.getWALDirectoryName("wals"));
|
||||
long editCount = 0;
|
||||
FileStatus [] fsss = fs.listStatus(dir);
|
||||
if (fsss.length == 0) throw new IllegalStateException("No WAL found");
|
||||
for (FileStatus fss: fsss) {
|
||||
Path p = fss.getPath();
|
||||
if (!fs.exists(p)) throw new IllegalStateException(p.toString());
|
||||
editCount += verify(wals, p, verbose);
|
||||
}
|
||||
long expected = numIterations * numThreads;
|
||||
if (editCount != expected) {
|
||||
throw new IllegalStateException("Counted=" + editCount + ", expected=" + expected);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
for (int i = 0; i < numRegions; i++) {
|
||||
if (regions[i] != null) {
|
||||
closeRegion(regions[i]);
|
||||
}
|
||||
}
|
||||
wals.shutdown();
|
||||
// Remove the root dir for this test region
|
||||
if (cleanup) cleanRegionRootDir(fs, rootRegionDir);
|
||||
}
|
||||
} finally {
|
||||
// We may be called inside a test that wants to keep on using the fs.
|
||||
if (!noclosefs) fs.close();
|
||||
scope.close();
|
||||
if (receiverHost != null) receiverHost.closeReceivers();
|
||||
}
|
||||
|
||||
return(0);
|
||||
}
|
||||
|
||||
private static HTableDescriptor createHTableDescriptor(final int regionNum,
|
||||
final int numFamilies) {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE_NAME + ":" + regionNum));
|
||||
for (int i = 0; i < numFamilies; ++i) {
|
||||
HColumnDescriptor colDef = new HColumnDescriptor(FAMILY_PREFIX + i);
|
||||
htd.addFamily(colDef);
|
||||
}
|
||||
return htd;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
WAL.Reader reader = wals.createReader(wal.getFileSystem(getConf()), wal);
|
||||
long count = 0;
|
||||
Map<String, Long> sequenceIds = new HashMap<String, Long>();
|
||||
try {
|
||||
while (true) {
|
||||
WAL.Entry e = reader.next();
|
||||
if (e == null) {
|
||||
LOG.debug("Read count=" + count + " from " + wal);
|
||||
break;
|
||||
}
|
||||
count++;
|
||||
long seqid = e.getKey().getLogSeqNum();
|
||||
if (sequenceIds.containsKey(Bytes.toString(e.getKey().getEncodedRegionName()))) {
|
||||
// sequenceIds should be increasing for every regions
|
||||
if (sequenceIds.get(Bytes.toString(e.getKey().getEncodedRegionName())) >= seqid) {
|
||||
throw new IllegalStateException("wal = " + wal.getName() + ", " + "previous seqid = "
|
||||
+ sequenceIds.get(Bytes.toString(e.getKey().getEncodedRegionName()))
|
||||
+ ", current seqid = " + seqid);
|
||||
}
|
||||
}
|
||||
// update the sequence Id.
|
||||
sequenceIds.put(Bytes.toString(e.getKey().getEncodedRegionName()), seqid);
|
||||
if (verbose) LOG.info("seqid=" + seqid);
|
||||
}
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
private static void logBenchmarkResult(String testName, long numTests, long totalTime) {
|
||||
float tsec = totalTime / 1000.0f;
|
||||
LOG.info(String.format("%s took %.3fs %.3fops/s", testName, tsec, numTests / tsec));
|
||||
|
||||
}
|
||||
|
||||
private void printUsageAndExit() {
|
||||
System.err.printf("Usage: bin/hbase %s [options]\n", getClass().getName());
|
||||
System.err.println(" where [options] are:");
|
||||
System.err.println(" -h|-help Show this help and exit.");
|
||||
System.err.println(" -threads <N> Number of threads writing on the WAL.");
|
||||
System.err.println(" -regions <N> Number of regions to open in the WAL. Default: 1");
|
||||
System.err.println(" -iterations <N> Number of iterations per thread.");
|
||||
System.err.println(" -path <PATH> Path where region's root directory is created.");
|
||||
System.err.println(" -families <N> Number of column families to write.");
|
||||
System.err.println(" -qualifiers <N> Number of qualifiers to write.");
|
||||
System.err.println(" -keySize <N> Row key size in byte.");
|
||||
System.err.println(" -valueSize <N> Row/Col value size in byte.");
|
||||
System.err.println(" -nocleanup Do NOT remove test data when done.");
|
||||
System.err.println(" -noclosefs Do NOT close the filesystem when done.");
|
||||
System.err.println(" -nosync Append without syncing");
|
||||
System.err.println(" -syncInterval <N> Append N edits and then sync. " +
|
||||
"Default=0, i.e. sync every edit.");
|
||||
System.err.println(" -verify Verify edits written in sequence");
|
||||
System.err.println(" -verbose Output extra info; " +
|
||||
"e.g. all edit seq ids when verifying");
|
||||
System.err.println(" -roll <N> Roll the way every N appends");
|
||||
System.err.println(" -encryption <A> Encrypt the WAL with algorithm A, e.g. AES");
|
||||
System.err.println(" -traceFreq <N> Rate of trace sampling. Default: 1.0, " +
|
||||
"only respected when tracing is enabled, ie -Dhbase.trace.spanreceiver.classes=...");
|
||||
System.err.println("");
|
||||
System.err.println("Examples:");
|
||||
System.err.println("");
|
||||
System.err.println(" To run 100 threads on hdfs with log rolling every 10k edits and " +
|
||||
"verification afterward do:");
|
||||
System.err.println(" $ ./bin/hbase org.apache.hadoop.hbase.wal." +
|
||||
"WALPerformanceEvaluation \\");
|
||||
System.err.println(" -conf ./core-site.xml -path hdfs://example.org:7000/tmp " +
|
||||
"-threads 100 -roll 10000 -verify");
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd,
|
||||
final WALFactory wals, final long whenToRoll) throws IOException {
|
||||
// Initialize HRegion
|
||||
HRegionInfo regionInfo = new HRegionInfo(htd.getTableName());
|
||||
// Initialize WAL
|
||||
final WAL wal = wals.getWAL(regionInfo.getEncodedNameAsBytes());
|
||||
wal.registerWALActionsListener(new WALActionsListener.Base() {
|
||||
private int appends = 0;
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey,
|
||||
WALEdit logEdit) {
|
||||
this.appends++;
|
||||
if (this.appends % whenToRoll == 0) {
|
||||
LOG.info("Rolling after " + appends + " edits");
|
||||
// We used to do explicit call to rollWriter but changed it to a request
|
||||
// to avoid dead lock (there are less threads going on in this class than
|
||||
// in the regionserver -- regionserver does not have the issue).
|
||||
// TODO I think this means no rolling actually happens; the request relies on there
|
||||
// being a LogRoller.
|
||||
DefaultWALProvider.requestLogRoll(wal);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postSync(final long timeInNanos, final int handlerSyncs) {
|
||||
syncMeter.mark();
|
||||
syncHistogram.update(timeInNanos);
|
||||
syncCountHistogram.update(handlerSyncs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postAppend(final long size, final long elapsedTime) {
|
||||
appendMeter.mark(size);
|
||||
}
|
||||
});
|
||||
wal.rollWriter();
|
||||
|
||||
return HRegion.createHRegion(regionInfo, dir, getConf(), htd, wal);
|
||||
}
|
||||
|
||||
private void closeRegion(final HRegion region) throws IOException {
|
||||
if (region != null) {
|
||||
region.close();
|
||||
WAL wal = region.getWAL();
|
||||
if (wal != null) {
|
||||
wal.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanRegionRootDir(final FileSystem fs, final Path dir) throws IOException {
|
||||
if (fs.exists(dir)) {
|
||||
fs.delete(dir, true);
|
||||
}
|
||||
}
|
||||
|
||||
private Put setupPut(Random rand, byte[] key, byte[] value, final int numFamilies) {
|
||||
rand.nextBytes(key);
|
||||
Put put = new Put(key);
|
||||
for (int cf = 0; cf < numFamilies; ++cf) {
|
||||
for (int q = 0; q < numQualifiers; ++q) {
|
||||
rand.nextBytes(value);
|
||||
put.add(Bytes.toBytes(FAMILY_PREFIX + cf), Bytes.toBytes(QUALIFIER_PREFIX + q), value);
|
||||
}
|
||||
}
|
||||
return put;
|
||||
}
|
||||
|
||||
private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
|
||||
WALEdit walEdit) {
|
||||
for (List<Cell> edits : familyMap.values()) {
|
||||
for (Cell cell : edits) {
|
||||
walEdit.add(cell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private long runBenchmark(Runnable[] runnable, final int numThreads) throws InterruptedException {
|
||||
Thread[] threads = new Thread[numThreads];
|
||||
long startTime = System.currentTimeMillis();
|
||||
for (int i = 0; i < numThreads; ++i) {
|
||||
threads[i] = new Thread(runnable[i%runnable.length], "t" + i + ",r" + (i%runnable.length));
|
||||
threads[i].start();
|
||||
}
|
||||
for (Thread t : threads) t.join();
|
||||
long endTime = System.currentTimeMillis();
|
||||
return(endTime - startTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
System.exit(innerMain(HBaseConfiguration.create(), args));
|
||||
}
|
||||
}
|
42
hbase-shell/src/main/ruby/shell/commands/wal_roll.rb
Normal file
42
hbase-shell/src/main/ruby/shell/commands/wal_roll.rb
Normal file
@ -0,0 +1,42 @@
|
||||
#
|
||||
#
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
#
|
||||
module Shell
|
||||
module Commands
|
||||
class WalRoll < Command
|
||||
def help
|
||||
return <<-EOF
|
||||
Roll the log writer. That is, start writing log messages to a new file.
|
||||
The name of the regionserver should be given as the parameter. A
|
||||
'server_name' is the host, port plus startcode of a regionserver. For
|
||||
example: host187.example.com,60020,1289493121758 (find servername in
|
||||
master ui or when you do detailed status in shell)
|
||||
EOF
|
||||
end
|
||||
|
||||
def command(server_name)
|
||||
format_simple_command do
|
||||
admin.wal_roll(server_name)
|
||||
end
|
||||
end
|
||||
end
|
||||
#TODO remove old HLog version
|
||||
class HlogRoll < WalRoll
|
||||
end
|
||||
end
|
||||
end
|
Loading…
x
Reference in New Issue
Block a user