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:
Sean Busbey 2014-08-02 13:27:36 -05:00 committed by stack
parent b97c3daf90
commit f5e05eb836
33 changed files with 8586 additions and 288 deletions

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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();
}
}
}

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -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:
* &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}&lt;file-creation-timestamp&gt;[.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);
}
}
}

View File

@ -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.";
}
}
}

View 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;
}
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -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();
}
}

View File

@ -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"
));
}
}
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}
}
}

View File

@ -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;
}
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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[]{};
}

View File

@ -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);
}
}
}

View File

@ -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;
}
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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));
}
}

View 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