Skip to content

Commit

Permalink
HBASE-24616 Remove BoundedRecoveredHFilesOutputSink dependency on a T…
Browse files Browse the repository at this point in the history
…ableDescriptor (#1955)

Purge query Master for table descriptors; make do w/ generic options.

Logging cleanup.

hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
 Undo fetching Table Descriptor. Not reliably available at recovery time.

Signed-off-by: Duo Zhang <zhangduo@apache.org>
  • Loading branch information
saintstack committed Jun 25, 2020
1 parent 1378776 commit e54e3af
Show file tree
Hide file tree
Showing 8 changed files with 63 additions and 92 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@ private void cleanupSplitDir(MasterProcedureEnv env) {
try {
splitWALManager.deleteWALDir(serverName);
} catch (IOException e) {
LOG.warn("remove WAL directory of server {} failed, ignore...", serverName, e);
LOG.warn("Remove WAL directory of server {} failed, ignore...", serverName, e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2056,8 +2056,7 @@ private void startServices() throws IOException {
// SplitLogWorker needs csm. If none, don't start this.
this.splitLogWorker = new SplitLogWorker(sinkConf, this, this, walFactory);
splitLogWorker.start();
} else {
LOG.warn("SplitLogWorker Service NOT started; CoordinatedStateManager is null");
LOG.debug("SplitLogWorker started");
}

// Memstore services.
Expand Down Expand Up @@ -2281,7 +2280,7 @@ public void postOpenDeployTasks(final PostOpenDeployContext context) throws IOEx
long openProcId = context.getOpenProcId();
long masterSystemTime = context.getMasterSystemTime();
rpcServices.checkOpen();
LOG.info("Post open deploy tasks for {}, openProcId={}, masterSystemTime={}",
LOG.info("Post open deploy tasks for {}, pid={}, masterSystemTime={}",
r.getRegionInfo().getRegionNameAsString(), openProcId, masterSystemTime);
// Do checks to see if we need to compact (references or too many files)
for (HStore s : r.stores.values()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -435,7 +435,7 @@ private void open() throws IOException {
if (cfBloomType != BloomType.NONE) {
initialReader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
if (hfileBloomType != cfBloomType) {
LOG.info("HFile Bloom filter type for "
LOG.debug("HFile Bloom filter type for "
+ initialReader.getHFileReader().getName() + ": " + hfileBloomType
+ ", but " + cfBloomType + " specified in column family "
+ "configuration");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,8 @@ public class SplitLogWorker implements Runnable {

Thread worker;
// thread pool which executes recovery work
private SplitLogWorkerCoordination coordination;
private RegionServerServices server;
private final SplitLogWorkerCoordination coordination;
private final RegionServerServices server;

public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
TaskExecutor splitTaskExecutor) {
Expand Down Expand Up @@ -152,7 +152,10 @@ private static boolean processSyncReplicationWAL(String name, Configuration conf
return true;
}

static Status splitLog(String name, CancelableProgressable p, Configuration conf,
/**
* @return Result either DONE, RESIGNED, or ERR.
*/
static Status splitLog(String filename, CancelableProgressable p, Configuration conf,
RegionServerServices server, LastSequenceId sequenceIdChecker, WALFactory factory) {
Path walDir;
FileSystem fs;
Expand All @@ -164,11 +167,11 @@ static Status splitLog(String name, CancelableProgressable p, Configuration conf
return Status.RESIGNED;
}
try {
if (!processSyncReplicationWAL(name, conf, server, fs, walDir)) {
if (!processSyncReplicationWAL(filename, conf, server, fs, walDir)) {
return Status.DONE;
}
} catch (IOException e) {
LOG.warn("failed to process sync replication wal {}", name, e);
LOG.warn("failed to process sync replication wal {}", filename, e);
return Status.RESIGNED;
}
// TODO have to correctly figure out when log splitting has been
Expand All @@ -178,31 +181,32 @@ static Status splitLog(String name, CancelableProgressable p, Configuration conf
SplitLogWorkerCoordination splitLogWorkerCoordination =
server.getCoordinatedStateManager() == null ? null
: server.getCoordinatedStateManager().getSplitLogWorkerCoordination();
if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, name)), fs, conf, p,
if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)), fs, conf, p,
sequenceIdChecker, splitLogWorkerCoordination, factory, server)) {
return Status.PREEMPTED;
}
} catch (InterruptedIOException iioe) {
LOG.warn("Resigning, interrupted splitting WAL {}", name, iioe);
LOG.warn("Resigning, interrupted splitting WAL {}", filename, iioe);
return Status.RESIGNED;
} catch (IOException e) {
if (e instanceof FileNotFoundException) {
// A wal file may not exist anymore. Nothing can be recovered so move on
LOG.warn("Done, WAL {} does not exist anymore", name, e);
LOG.warn("Done, WAL {} does not exist anymore", filename, e);
return Status.DONE;
}
Throwable cause = e.getCause();
if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
|| cause instanceof ConnectException || cause instanceof SocketTimeoutException)) {
LOG.warn("Resigning, can't connect to target regionserver splitting WAL {}", name, e);
LOG.warn("Resigning, can't connect to target regionserver splitting WAL {}", filename, e);
return Status.RESIGNED;
} else if (cause instanceof InterruptedException) {
LOG.warn("Resigning, interrupted splitting WAL {}", name, e);
LOG.warn("Resigning, interrupted splitting WAL {}", filename, e);
return Status.RESIGNED;
}
LOG.warn("Error splitting WAL {}", name, e);
LOG.warn("Error splitting WAL {}", filename, e);
return Status.ERR;
}
LOG.debug("Done splitting WAL {}", filename);
return Status.DONE;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
/**
*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
Expand All @@ -17,17 +16,14 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;

import java.io.IOException;
import java.util.concurrent.locks.Lock;

import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
import org.apache.hadoop.hbase.util.KeyLocker;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;

Expand Down Expand Up @@ -75,21 +71,43 @@ public EventType getEventType() {
return EventType.RS_LOG_REPLAY;
}

public static class PreemptedWALSplitException extends HBaseIOException {
PreemptedWALSplitException(String wal) {
super(wal);
}
}

public static class ResignedWALSplitException extends HBaseIOException {
ResignedWALSplitException(String wal) {
super(wal);
}
}

public static class ErrorWALSplitException extends HBaseIOException {
ErrorWALSplitException(String wal) {
super(wal);
}
}

@Override
public Void call() throws Exception {
if (initError != null) {
throw initError;
}
//grab a lock
splitWALLock = splitWALLocks.acquireLock(walPath);
try{
splitWal();
LOG.info("Successful split of {}", walPath);
} catch (IOException e){
LOG.warn("Failed split of {}.", walPath, e);
throw e;
}
finally {
try {
switch (SplitLogWorker.splitLog(walPath, null, rs.getConfiguration(), rs, rs, rs.getWalFactory())) {
case DONE:
break;
case PREEMPTED:
throw new PreemptedWALSplitException(this.walPath);
case RESIGNED:
throw new ResignedWALSplitException(this.walPath);
default:
throw new ErrorWALSplitException(this.walPath);
}
} finally {
splitWALLock.unlock();
}
return null;
Expand All @@ -98,12 +116,4 @@ public Void call() throws Exception {
public String getWalPath() {
return this.walPath;
}

private void splitWal() throws IOException {
SplitLogWorker.TaskExecutor.Status status =
SplitLogWorker.splitLog(walPath, null, rs.getConfiguration(), rs, rs, rs.getWalFactory());
if (status != SplitLogWorker.TaskExecutor.Status.DONE) {
throw new IOException("Failed WAL split, status=" + status + ", wal=" + walPath);
}
}
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
Expand Down Expand Up @@ -48,7 +48,7 @@ public void process() {
try {
callable.call();
} catch (Throwable t) {
LOG.error("Error when call RSProcedureCallable: ", t);
LOG.error("pid=" + this.procId, t);
error = t;
} finally {
((HRegionServer) server).remoteProcedureComplete(procId, error);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,6 @@
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
Expand Down Expand Up @@ -67,13 +65,10 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
// Need a counter to track the opening writers.
private final AtomicInteger openingWritersNum = new AtomicInteger(0);

private final ConcurrentMap<TableName, TableDescriptor> tableDescCache;

public BoundedRecoveredHFilesOutputSink(WALSplitter walSplitter,
WALSplitter.PipelineController controller, EntryBuffers entryBuffers, int numWriters) {
super(controller, entryBuffers, numWriters);
this.walSplitter = walSplitter;
this.tableDescCache = new ConcurrentHashMap<>();
}

@Override
Expand Down Expand Up @@ -191,50 +186,22 @@ boolean keepRegionEvent(Entry entry) {
return false;
}

/**
* @return Returns a base HFile without compressions or encodings; good enough for recovery
* given hfile has metadata on how it was written.
*/
private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
long seqId, String familyName, boolean isMetaTable) throws IOException {
Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
tableName, regionName, familyName);
StoreFileWriter.Builder writerBuilder =
new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
.withOutputDir(outputDir);

TableDescriptor tableDesc =
tableDescCache.computeIfAbsent(tableName, t -> getTableDescriptor(t));
if (tableDesc == null) {
throw new IOException("Failed to get table descriptor for table " + tableName);
}
ColumnFamilyDescriptor cfd = tableDesc.getColumnFamily(Bytes.toBytesBinary(familyName));
HFileContext hFileContext = createFileContext(cfd, isMetaTable);
return writerBuilder.withFileContext(hFileContext).withBloomType(cfd.getBloomFilterType())
.build();
}

private HFileContext createFileContext(ColumnFamilyDescriptor cfd, boolean isMetaTable)
throws IOException {
return new HFileContextBuilder().withCompression(cfd.getCompressionType())
.withChecksumType(HStore.getChecksumType(walSplitter.conf))
.withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf))
.withBlockSize(cfd.getBlocksize()).withCompressTags(cfd.isCompressTags())
.withDataBlockEncoding(cfd.getDataBlockEncoding()).withCellComparator(
isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR)
.build();
}

private TableDescriptor getTableDescriptor(TableName tableName) {
if (walSplitter.rsServices != null) {
try {
return walSplitter.rsServices.getConnection().getAdmin().getDescriptor(tableName);
} catch (IOException e) {
LOG.warn("Failed to get table descriptor for {}", tableName, e);
}
}
LOG.info("Failed getting {} table descriptor from master; trying local", tableName);
try {
return walSplitter.tableDescriptors.get(tableName);
} catch (IOException e) {
LOG.warn("Failed to get table descriptor for {}", tableName, e);
return null;
}
HFileContext hFileContext = new HFileContextBuilder().
withChecksumType(HStore.getChecksumType(walSplitter.conf)).
withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf)).
withCellComparator(isMetaTable?
CellComparatorImpl.META_COMPARATOR: CellComparatorImpl.COMPARATOR).build();
return writerBuilder.withFileContext(hFileContext).build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
import org.apache.hadoop.hbase.master.SplitLogManager;
Expand All @@ -51,7 +50,6 @@
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.wal.WAL.Reader;
Expand Down Expand Up @@ -88,7 +86,6 @@ public class WALSplitter {
final Path rootDir;
final FileSystem rootFS;
final RegionServerServices rsServices;
final TableDescriptors tableDescriptors;

// Major subcomponents of the split process.
// These are separated into inner classes to make testing easier.
Expand Down Expand Up @@ -152,12 +149,6 @@ public class WALSplitter {
this.sequenceIdChecker = idChecker;
this.splitLogWorkerCoordination = splitLogWorkerCoordination;
this.rsServices = rsServices;
if (rsServices != null) {
this.tableDescriptors = rsServices.getTableDescriptors();
} else {
this.tableDescriptors = new FSTableDescriptors(rootFS, rootDir, true, true);
}

this.walFactory = factory;
PipelineController controller = new PipelineController();
this.tmpDirName =
Expand Down

0 comments on commit e54e3af

Please sign in to comment.