From 6f1f7bc76290dc52021f072b240f3a8565e3a158 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Thu, 26 Sep 2019 19:57:41 +0900 Subject: [PATCH 01/51] HBASE-23205 Correctly update the position of WALs currently being replicated --- .../regionserver/ReplicationSource.java | 64 ++-- .../ReplicationSourceManager.java | 44 ++- .../ReplicationSourceWALReaderThread.java | 356 +++++++++--------- .../TestReplicationSmallTests.java | 6 +- .../replication/TestReplicationSource.java | 235 +++++++++++- .../TestGlobalReplicationThrottler.java | 3 +- .../regionserver/TestWALEntryStream.java | 129 +++++-- 7 files changed, 548 insertions(+), 289 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 124da63b9fcb..25f986060542 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.Service; @@ -145,8 +146,6 @@ public enum WorkerState { FINISHED // The worker is done processing a recovered queue } - private AtomicLong totalBufferUsed; - /** * Instantiation method used by region servers * @@ -192,7 +191,6 @@ public void init(final Configuration conf, final FileSystem fs, defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0); currentBandwidth = getCurrentBandwidth(); this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0); - this.totalBufferUsed = manager.getTotalBufferUsed(); LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId + ", currentBandwidth=" + this.currentBandwidth); } @@ -439,14 +437,22 @@ public String getPeerClusterId() { } @Override + @VisibleForTesting public Path getCurrentPath() { - // only for testing for (ReplicationSourceShipperThread worker : workerThreads.values()) { if (worker.getCurrentPath() != null) return worker.getCurrentPath(); } return null; } + @VisibleForTesting + public long getLastLoggedPosition() { + for (ReplicationSourceShipperThread worker : workerThreads.values()) { + return worker.getLastLoggedPosition(); + } + return 0; + } + private boolean isSourceActive() { return !this.stopper.isStopped() && this.sourceRunning; } @@ -481,7 +487,7 @@ public String getStats() { for (Map.Entry entry : workerThreads.entrySet()) { String walGroupId = entry.getKey(); ReplicationSourceShipperThread worker = entry.getValue(); - long position = worker.getCurrentPosition(); + long position = worker.getLastLoggedPosition(); Path currentPath = worker.getCurrentPath(); sb.append("walGroup [").append(walGroupId).append("]: "); if (currentPath != null) { @@ -535,7 +541,7 @@ public Map getWalGroupStatus() { .withQueueSize(queueSize) .withWalGroup(walGroupId) .withCurrentPath(currentPath) - .withCurrentPosition(worker.getCurrentPosition()) + .withCurrentPosition(worker.getLastLoggedPosition()) .withFileSize(fileSize) .withAgeOfLastShippedOp(ageOfLastShippedOp) .withReplicationDelay(replicationDelay); @@ -599,14 +605,11 @@ public void run() { try { WALEntryBatch entryBatch = entryReader.take(); shipEdits(entryBatch); - releaseBufferQuota((int) entryBatch.getHeapSize()); - if (replicationQueueInfo.isQueueRecovered() && entryBatch.getWalEntries().isEmpty() - && entryBatch.getLastSeqIds().isEmpty()) { - LOG.debug("Finished recovering queue for group " + walGroupId + " of peer " - + peerClusterZnode); + manager.releaseBufferQuota(entryBatch.getHeapSizeExcludeBulkLoad()); + if (!entryBatch.hasMoreEntries()) { + LOG.debug("Finished recovering queue for group " + walGroupId + " of peer " + peerClusterZnode); metrics.incrCompletedRecoveryQueue(); setWorkerState(WorkerState.FINISHED); - continue; } } catch (InterruptedException e) { LOG.trace("Interrupted while waiting for next replication entry batch", e); @@ -614,7 +617,7 @@ public void run() { } } - if (replicationQueueInfo.isQueueRecovered() && getWorkerState() == WorkerState.FINISHED) { + if (getWorkerState() == WorkerState.FINISHED) { // use synchronize to make sure one last thread will clean the queue synchronized (this) { Threads.sleep(100);// wait a short while for other worker thread to fully exit @@ -676,18 +679,6 @@ private void checkBandwidthChangeAndResetThrottler() { } } - /** - * get batchEntry size excludes bulk load file sizes. - * Uses ReplicationSourceWALReader's static method. - */ - private int getBatchEntrySizeExcludeBulkLoad(WALEntryBatch entryBatch) { - int totalSize = 0; - for(Entry entry : entryBatch.getWalEntries()) { - totalSize += entryReader.getEntrySizeExcludeBulkLoad(entry); - } - return totalSize; - } - /** * Do the shipping logic */ @@ -697,16 +688,14 @@ protected void shipEdits(WALEntryBatch entryBatch) { currentPath = entryBatch.getLastWalPath(); int sleepMultiplier = 0; if (entries.isEmpty()) { - if (lastLoggedPosition != lastReadPosition) { - updateLogPosition(lastReadPosition); - // if there was nothing to ship and it's not an error - // set "ageOfLastShippedOp" to to indicate that we're current - metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId); - } + updateLogPosition(lastReadPosition); + // if there was nothing to ship and it's not an error + // set "ageOfLastShippedOp" to to indicate that we're current + metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId); return; } int currentSize = (int) entryBatch.getHeapSize(); - int sizeExcludeBulkLoad = getBatchEntrySizeExcludeBulkLoad(entryBatch); + int sizeExcludeBulkLoad = (int) entryBatch.getHeapSizeExcludeBulkLoad(); while (isWorkerActive()) { try { checkBandwidthChangeAndResetThrottler(); @@ -787,7 +776,6 @@ protected void shipEdits(WALEntryBatch entryBatch) { } private void updateLogPosition(long lastReadPosition) { - manager.setPendingShipment(false); manager.logPositionAndCleanOldLogs(currentPath, peerClusterZnode, lastReadPosition, this.replicationQueueInfo.isQueueRecovered(), false); lastLoggedPosition = lastReadPosition; @@ -938,11 +926,11 @@ private Path getReplSyncUpPath(Path path) throws IOException { } public Path getCurrentPath() { - return this.entryReader.getCurrentPath(); + return currentPath; } - public long getCurrentPosition() { - return this.lastLoggedPosition; + public long getLastLoggedPosition() { + return lastLoggedPosition; } private boolean isWorkerActive() { @@ -983,9 +971,5 @@ public void setWorkerState(WorkerState state) { public WorkerState getWorkerState() { return state; } - - private void releaseBufferQuota(int size) { - totalBufferUsed.addAndGet(-size); - } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index 6b8b6e273c47..58c5ce97b3bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -121,10 +121,9 @@ public class ReplicationSourceManager implements ReplicationListener { private final Random rand; private final boolean replicationForBulkLoadDataEnabled; + private long totalBufferQuota; private AtomicLong totalBufferUsed = new AtomicLong(); - private boolean pendingShipment; - /** * Creates a replication manager and sets the watch on all the other registered region servers * @param replicationQueues the interface for manipulating replication queues @@ -177,6 +176,8 @@ public ReplicationSourceManager(final ReplicationQueues replicationQueues, replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); + totalBufferQuota = conf.getLong(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY, + HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_DFAULT); } /** @@ -192,18 +193,12 @@ public ReplicationSourceManager(final ReplicationQueues replicationQueues, */ public synchronized void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered, boolean holdLogInZK) { - if (!this.pendingShipment) { - String fileName = log.getName(); - this.replicationQueues.setLogPosition(id, fileName, position); - if (holdLogInZK) { - return; - } - cleanOldLogs(fileName, id, queueRecovered); + String fileName = log.getName(); + this.replicationQueues.setLogPosition(id, fileName, position); + if (holdLogInZK) { + return; } - } - - public synchronized void setPendingShipment(boolean pendingShipment) { - this.pendingShipment = pendingShipment; + cleanOldLogs(fileName, id, queueRecovered); } /** @@ -466,8 +461,27 @@ void postLogRoll(Path newLog) throws IOException { } @VisibleForTesting - public AtomicLong getTotalBufferUsed() { - return totalBufferUsed; + public long getTotalBufferUsed() { + return totalBufferUsed.get(); + } + + /** + * @param size delta size for grown buffer + * @return true if total buffer size limit reached, after adding size + */ + public boolean acquireBufferQuota(long size) { + return totalBufferUsed.addAndGet(size) >= totalBufferQuota; + } + + public void releaseBufferQuota(long size) { + totalBufferUsed.addAndGet(-size); + } + + /** + * @return true if total buffer size limit reached + */ + public boolean isBufferQuotaReached() { + return totalBufferUsed.get() >= totalBufferQuota; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 1d94a7a2c815..5b25186cc977 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -21,14 +21,11 @@ import java.io.EOFException; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -37,7 +34,6 @@ 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor; @@ -70,6 +66,7 @@ public class ReplicationSourceWALReaderThread extends Thread { private int replicationBatchCountCapacity; // position in the WAL to start reading at private long currentPosition; + private Path currentPath; private WALEntryFilter filter; private long sleepForRetries; //Indicates whether this particular worker is running @@ -78,10 +75,7 @@ public class ReplicationSourceWALReaderThread extends Thread { private int maxRetriesMultiplier; private MetricsSource metrics; - private AtomicLong totalBufferUsed; - private long totalBufferQuota; - - private ReplicationSourceManager replicationSourceManager; + private ReplicationSourceManager manager; /** * Creates a reader worker for a given WAL queue. Reads WAL entries off a given queue, batches the @@ -101,6 +95,7 @@ public ReplicationSourceWALReaderThread(ReplicationSourceManager manager, FileSystem fs, Configuration conf, WALEntryFilter filter, MetricsSource metrics) { this.replicationQueueInfo = replicationQueueInfo; this.logQueue = logQueue; + this.currentPath = logQueue.peek(); this.currentPosition = startPosition; this.fs = fs; this.conf = conf; @@ -111,10 +106,7 @@ public ReplicationSourceWALReaderThread(ReplicationSourceManager manager, // memory used will be batchSizeCapacity * (nb.batches + 1) // the +1 is for the current thread reading before placing onto the queue int batchCount = conf.getInt("replication.source.nb.batches", 1); - this.replicationSourceManager = manager; - this.totalBufferUsed = manager.getTotalBufferUsed(); - this.totalBufferQuota = conf.getLong(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY, - HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_DFAULT); + this.manager = manager; this.sleepForRetries = this.conf.getLong("replication.source.sleepforretries", 1000); // 1 second this.maxRetriesMultiplier = @@ -135,59 +127,44 @@ public void run() { try (WALEntryStream entryStream = new WALEntryStream(logQueue, fs, conf, currentPosition, metrics)) { while (isReaderRunning()) { // loop here to keep reusing stream while we can - if (!checkQuota()) { + if (manager.isBufferQuotaReached()) { + Threads.sleep(sleepForRetries); continue; } - WALEntryBatch batch = null; - while (entryStream.hasNext()) { - if (batch == null) { - batch = new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath()); - } + WALEntryBatch batch = new WALEntryBatch(replicationBatchCountCapacity, replicationBatchSizeCapacity); + boolean hasNext; + while (hasNext = entryStream.hasNext()) { Entry entry = entryStream.next(); entry = filterEntry(entry); if (entry != null) { WALEdit edit = entry.getEdit(); if (edit != null && !edit.isEmpty()) { - long entrySize = getEntrySizeIncludeBulkLoad(entry); - long entrySizeExlucdeBulkLoad = getEntrySizeExcludeBulkLoad(entry); - batch.addEntry(entry); - replicationSourceManager.setPendingShipment(true); - updateBatchStats(batch, entry, entryStream.getPosition(), entrySize); - boolean totalBufferTooLarge = acquireBufferQuota(entrySizeExlucdeBulkLoad); + long entrySizeExcludeBulkLoad = batch.addEntry(entry); + boolean totalBufferTooLarge = manager.acquireBufferQuota(entrySizeExcludeBulkLoad); // Stop if too many entries or too big - if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity - || batch.getNbEntries() >= replicationBatchCountCapacity) { + if (totalBufferTooLarge || batch.isLimitReached()) { break; } } - } else { - replicationSourceManager.logPositionAndCleanOldLogs(entryStream.getCurrentPath(), - this.replicationQueueInfo.getPeerClusterZnode(), - entryStream.getPosition(), - this.replicationQueueInfo.isQueueRecovered(), false); } } - if (batch != null && (!batch.getLastSeqIds().isEmpty() || batch.getNbEntries() > 0)) { - if (LOG.isTraceEnabled()) { - LOG.trace(String.format("Read %s WAL entries eligible for replication", - batch.getNbEntries())); - } - entryBatchQueue.put(batch); + + if (LOG.isTraceEnabled()) { + LOG.trace(String.format("Read %s WAL entries eligible for replication", batch.getNbEntries())); + } + + updateBatch(entryStream, batch, hasNext); + if (isShippable(batch)) { sleepMultiplier = 1; - } else { // got no entries and didn't advance position in WAL - LOG.trace("Didn't read any new entries from WAL"); - if (replicationQueueInfo.isQueueRecovered()) { - // we're done with queue recovery, shut ourself down + entryBatchQueue.put(batch); + if (!batch.hasMoreEntries()) { + // we're done with queue recovery, shut ourselves down setReaderRunning(false); - // shuts down shipper thread immediately - entryBatchQueue.put(batch != null ? batch - : new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath())); - } else { - Thread.sleep(sleepForRetries); } + } else { + Thread.sleep(sleepForRetries); } - currentPosition = entryStream.getPosition(); - entryStream.reset(); // reuse stream + resetStream(entryStream); } } catch (IOException | WALEntryStreamRuntimeException e) { // stream related if (sleepMultiplier < maxRetriesMultiplier) { @@ -205,6 +182,26 @@ public void run() { } } + private void updateBatch(WALEntryStream entryStream, WALEntryBatch batch, boolean moreData) { + batch.updatePosition(entryStream); + batch.setMoreEntries(!replicationQueueInfo.isQueueRecovered() || moreData); + } + + private boolean isShippable(WALEntryBatch batch) { + return !batch.isEmpty() || checkIfWALRolled(batch) || !batch.hasMoreEntries(); + } + + private boolean checkIfWALRolled(WALEntryBatch batch) { + return currentPath == null && batch.lastWalPath != null + || currentPath != null && !currentPath.equals(batch.lastWalPath); + } + + private void resetStream(WALEntryStream stream) throws IOException { + currentPosition = stream.getPosition(); + currentPath = stream.getCurrentPath(); + stream.reset(); // reuse stream + } + // if we get an EOF due to a zero-length log, and there are other logs in queue // (highly likely we've closed the current log), we've hit the max retries, and autorecovery is // enabled, then dump the log @@ -223,26 +220,6 @@ private void handleEofException(Exception e) { } } - public Path getCurrentPath() { - // if we've read some WAL entries, get the Path we read from - WALEntryBatch batchQueueHead = entryBatchQueue.peek(); - if (batchQueueHead != null) { - return batchQueueHead.lastWalPath; - } - // otherwise, we must be currently reading from the head of the log queue - return logQueue.peek(); - } - - //returns false if we've already exceeded the global quota - private boolean checkQuota() { - // try not to go over total quota - if (totalBufferUsed.get() > totalBufferQuota) { - Threads.sleep(sleepForRetries); - return false; - } - return true; - } - private Entry filterEntry(Entry entry) { Entry filtered = filter.filter(entry); if (entry != null && filtered == null) { @@ -265,107 +242,6 @@ public WALEntryBatch poll(long timeout) throws InterruptedException { return entryBatchQueue.poll(timeout, TimeUnit.MILLISECONDS); } - private long getEntrySizeIncludeBulkLoad(Entry entry) { - WALEdit edit = entry.getEdit(); - WALKey key = entry.getKey(); - return edit.heapSize() + sizeOfStoreFilesIncludeBulkLoad(edit) + - key.estimatedSerializedSizeOf(); - } - - public long getEntrySizeExcludeBulkLoad(Entry entry) { - WALEdit edit = entry.getEdit(); - WALKey key = entry.getKey(); - return edit.heapSize() + key.estimatedSerializedSizeOf(); - } - - private void updateBatchStats(WALEntryBatch batch, Entry entry, long entryPosition, long entrySize) { - WALEdit edit = entry.getEdit(); - if (edit != null && !edit.isEmpty()) { - batch.incrementHeapSize(entrySize); - Pair nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit); - batch.incrementNbRowKeys(nbRowsAndHFiles.getFirst()); - batch.incrementNbHFiles(nbRowsAndHFiles.getSecond()); - } - batch.lastWalPosition = entryPosition; - } - - /** - * Count the number of different row keys in the given edit because of mini-batching. We assume - * that there's at least one Cell in the WALEdit. - * @param edit edit to count row keys from - * @return number of different row keys and HFiles - */ - private Pair countDistinctRowKeysAndHFiles(WALEdit edit) { - List cells = edit.getCells(); - int distinctRowKeys = 1; - int totalHFileEntries = 0; - Cell lastCell = cells.get(0); - - int totalCells = edit.size(); - for (int i = 0; i < totalCells; i++) { - // Count HFiles to be replicated - if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { - try { - BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); - List stores = bld.getStoresList(); - int totalStores = stores.size(); - for (int j = 0; j < totalStores; j++) { - totalHFileEntries += stores.get(j).getStoreFileList().size(); - } - } catch (IOException e) { - LOG.error("Failed to deserialize bulk load entry from wal edit. " - + "Then its hfiles count will not be added into metric."); - } - } - - if (!CellUtil.matchingRow(cells.get(i), lastCell)) { - distinctRowKeys++; - } - lastCell = cells.get(i); - } - - Pair result = new Pair<>(distinctRowKeys, totalHFileEntries); - return result; - } - - /** - * Calculate the total size of all the store files - * @param edit edit to count row keys from - * @return the total size of the store files - */ - private int sizeOfStoreFilesIncludeBulkLoad(WALEdit edit) { - List cells = edit.getCells(); - int totalStoreFilesSize = 0; - - int totalCells = edit.size(); - for (int i = 0; i < totalCells; i++) { - if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { - try { - BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); - List stores = bld.getStoresList(); - int totalStores = stores.size(); - for (int j = 0; j < totalStores; j++) { - totalStoreFilesSize += stores.get(j).getStoreFileSizeBytes(); - } - } catch (IOException e) { - LOG.error("Failed to deserialize bulk load entry from wal edit. " - + "Size of HFiles part of cell will not be considered in replication " - + "request size calculation.", - e); - } - } - } - return totalStoreFilesSize; - } - - /** - * @param size delta size for grown buffer - * @return true if we should clear buffer and push all - */ - private boolean acquireBufferQuota(long size) { - return totalBufferUsed.addAndGet(size) >= totalBufferQuota; - } - /** * @return whether the reader thread is running */ @@ -385,6 +261,8 @@ public void setReaderRunning(boolean readerRunning) { * */ static class WALEntryBatch { + private final int maxNbEntries; + private final long maxSizeBytes; private List walEntries; // last WAL that was read private Path lastWalPath; @@ -396,21 +274,23 @@ static class WALEntryBatch { private int nbHFiles = 0; // heap size of data we need to replicate private long heapSize = 0; - // save the last sequenceid for each region if the table has serial-replication scope - private Map lastSeqIds = new HashMap<>(); + // whether more entries to read exist in WALs or not + private boolean moreEntries = true; + // heap size of data we need to replicate, the size of bulk loaded hfiles is not included. + private long heapSizeExcludeBulkLoad; /** - * @param walEntries - * @param lastWalPath Path of the WAL the last entry in this batch was read from - * @param lastWalPosition Position in the WAL the last entry in this batch was read from + * @param maxNbEntries */ - private WALEntryBatch(int maxNbEntries, Path lastWalPath) { + private WALEntryBatch(int maxNbEntries, long maxSizeBytes) { this.walEntries = new ArrayList<>(maxNbEntries); - this.lastWalPath = lastWalPath; + this.maxNbEntries = maxNbEntries; + this.maxSizeBytes = maxSizeBytes; } - public void addEntry(Entry entry) { + public long addEntry(Entry entry) { walEntries.add(entry); + return updateBatchStats(entry); } /** @@ -466,11 +346,8 @@ public long getHeapSize() { return heapSize; } - /** - * @return the last sequenceid for each region if the table has serial-replication scope - */ - public Map getLastSeqIds() { - return lastSeqIds; + public long getHeapSizeExcludeBulkLoad() { + return heapSizeExcludeBulkLoad; } private void incrementNbRowKeys(int increment) { @@ -484,5 +361,120 @@ private void incrementNbHFiles(int increment) { private void incrementHeapSize(long increment) { heapSize += increment; } + + private void incrementHeapSizeExcludeBulkLoad(long increment) { + heapSizeExcludeBulkLoad += increment; + } + + public boolean isEmpty() { + return walEntries.isEmpty(); + } + + public void updatePosition(WALEntryStream entryStream) { + lastWalPath = entryStream.getCurrentPath(); + lastWalPosition = entryStream.getPosition(); + } + + public boolean hasMoreEntries() { + return moreEntries; + } + + public void setMoreEntries(boolean moreEntries) { + this.moreEntries = moreEntries; + } + + public boolean isLimitReached() { + return getHeapSize() >= maxSizeBytes || getNbEntries() >= maxNbEntries; + } + + private long getEntrySizeExcludeBulkLoad(Entry entry) { + WALEdit edit = entry.getEdit(); + WALKey key = entry.getKey(); + return edit.heapSize() + key.estimatedSerializedSizeOf(); + } + + private long updateBatchStats(Entry entry) { + WALEdit edit = entry.getEdit(); + + long entrySizeExcludeBulkLoad = getEntrySizeExcludeBulkLoad(entry); + long entrySize = entrySizeExcludeBulkLoad + sizeOfStoreFilesIncludeBulkLoad(entry.getEdit()); + incrementHeapSize(entrySize); + incrementHeapSizeExcludeBulkLoad(entrySizeExcludeBulkLoad); + + Pair nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit); + incrementNbRowKeys(nbRowsAndHFiles.getFirst()); + incrementNbHFiles(nbRowsAndHFiles.getSecond()); + + return entrySizeExcludeBulkLoad; + } + + /** + * Count the number of different row keys in the given edit because of mini-batching. We assume + * that there's at least one Cell in the WALEdit. + * @param edit edit to count row keys from + * @return number of different row keys and HFiles + */ + private Pair countDistinctRowKeysAndHFiles(WALEdit edit) { + List cells = edit.getCells(); + int distinctRowKeys = 1; + int totalHFileEntries = 0; + Cell lastCell = cells.get(0); + + int totalCells = edit.size(); + for (int i = 0; i < totalCells; i++) { + // Count HFiles to be replicated + if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { + try { + BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); + List stores = bld.getStoresList(); + int totalStores = stores.size(); + for (int j = 0; j < totalStores; j++) { + totalHFileEntries += stores.get(j).getStoreFileList().size(); + } + } catch (IOException e) { + LOG.error("Failed to deserialize bulk load entry from wal edit. " + + "Then its hfiles count will not be added into metric."); + } + } + + if (!CellUtil.matchingRow(cells.get(i), lastCell)) { + distinctRowKeys++; + } + lastCell = cells.get(i); + } + + Pair result = new Pair<>(distinctRowKeys, totalHFileEntries); + return result; + } + + /** + * Calculate the total size of all the store files + * @param edit edit to count row keys from + * @return the total size of the store files + */ + private int sizeOfStoreFilesIncludeBulkLoad(WALEdit edit) { + List cells = edit.getCells(); + int totalStoreFilesSize = 0; + + int totalCells = edit.size(); + for (int i = 0; i < totalCells; i++) { + if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { + try { + BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); + List stores = bld.getStoresList(); + int totalStores = stores.size(); + for (int j = 0; j < totalStores; j++) { + totalStoreFilesSize += stores.get(j).getStoreFileSizeBytes(); + } + } catch (IOException e) { + LOG.error("Failed to deserialize bulk load entry from wal edit. " + + "Size of HFiles part of cell will not be considered in replication " + + "request size calculation.", + e); + } + } + } + return totalStoreFilesSize; + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index f94ad5a1b468..87e34e7d7bd9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -66,7 +66,9 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.wal.DefaultWALProvider; 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.apache.hadoop.mapreduce.Job; import org.junit.Before; import org.junit.Test; @@ -829,8 +831,8 @@ public void testEmptyWALRecovery() throws Exception { WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo); Path currentWalPath = DefaultWALProvider.getCurrentFileName(wal); String walGroupId = DefaultWALProvider.getWALPrefixFromWALName(currentWalPath.getName()); - Path emptyWalPath = new Path(utility1.getDataTestDir(), walGroupId + "." + ts); - utility1.getTestFileSystem().create(emptyWalPath).close(); + Path emptyWalPath = new Path(currentWalPath.getParent(), walGroupId + "." + ts); + WALFactory.createWALWriter(utility1.getTestFileSystem(), emptyWalPath, utility1.getConfiguration()).close(); emptyWalPaths.add(emptyWalPath); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 5d9059a7eb5a..b9288289ff90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -34,35 +34,55 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CoordinatedStateManager; -import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint; +import org.apache.hadoop.hbase.replication.regionserver.MetricsSource; import org.apache.hadoop.hbase.replication.regionserver.Replication; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKey; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint; -import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; +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.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; +import org.mockito.ArgumentCaptor; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.UUID; + +import static org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.internal.verification.VerificationModeFactory.times; @Category(MediumTests.class) public class TestReplicationSource { @@ -92,6 +112,24 @@ public static void setUpBeforeClass() throws Exception { if (FS.exists(logDir)) FS.delete(logDir, true); } + @Before + public void setup() throws IOException { + if (!FS.exists(logDir)) FS.mkdirs(logDir); + if (!FS.exists(oldLogDir)) FS.mkdirs(oldLogDir); + + ReplicationEndpointForTest.contructedCount.set(0); + ReplicationEndpointForTest.startedCount.set(0); + ReplicationEndpointForTest.replicateCount.set(0); + ReplicationEndpointForTest.stoppedCount.set(0); + ReplicationEndpointForTest.lastEntries = null; + } + + @After + public void tearDown() throws IOException { + if (FS.exists(oldLogDir)) FS.delete(oldLogDir, true); + if (FS.exists(logDir)) FS.delete(logDir, true); + } + @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL_PEER.shutdownMiniHBaseCluster(); @@ -108,8 +146,6 @@ public static void tearDownAfterClass() throws Exception { @Test public void testLogMoving() throws Exception{ Path logPath = new Path(logDir, "log"); - if (!FS.exists(logDir)) FS.mkdirs(logDir); - if (!FS.exists(oldLogDir)) FS.mkdirs(oldLogDir); WALProvider.Writer writer = WALFactory.createWALWriter(FS, logPath, TEST_UTIL.getConfiguration()); for(int i = 0; i < 3; i++) { @@ -166,7 +202,6 @@ protected void doStop() { Configuration testConf = HBaseConfiguration.create(); testConf.setInt("replication.source.maxretriesmultiplier", 1); ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class); - Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong()); source.init(testConf, null, manager, null, mockPeers, null, "testPeer", null, replicationEndpoint, null); ExecutorService executor = Executors.newSingleThreadExecutor(); @@ -189,6 +224,180 @@ public boolean evaluate() throws Exception { } + private void appendEntries(WALProvider.Writer writer, int numEntries, boolean closeAfterAppends) throws IOException { + for(int i = 0; i < numEntries; i++) { + byte[] b = Bytes.toBytes(Integer.toString(i)); + KeyValue kv = new KeyValue(b,b,b); + WALEdit edit = new WALEdit(); + edit.add(kv); + WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0, + HConstants.DEFAULT_CLUSTER_ID); + NavigableMap scopes = new TreeMap(Bytes.BYTES_COMPARATOR); + scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL); + key.setScopes(scopes); + writer.append(new WAL.Entry(key, edit)); + writer.sync(false); + } + if (closeAfterAppends) { + writer.close(); + } + } + + private void appendEntries(WALProvider.Writer writer, int numEntries) throws IOException { + appendEntries(writer, numEntries, true); + } + + private long getPosition(WALFactory wals, Path log2, int numEntries) throws IOException { + WAL.Reader reader = wals.createReader(FS, log2); + for (int i = 0; i < numEntries; i++) { + reader.next(); + } + return reader.getPosition(); + } + + private static class Mocks { + private final ReplicationSourceManager manager = mock(ReplicationSourceManager.class); + private final ReplicationQueues queues = mock(ReplicationQueues.class); + private final ReplicationPeers peers = mock(ReplicationPeers.class); + private final MetricsSource metrics = mock(MetricsSource.class); + private final ReplicationPeer peer = mock(ReplicationPeer.class); + private final ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class); + + private Mocks() { + when(peers.getStatusOfPeer(anyString())).thenReturn(true); + when(context.getReplicationPeer()).thenReturn(peer); + } + + ReplicationSource createReplicationSourceWithMocks(ReplicationEndpoint endpoint) throws IOException { + final ReplicationSource source = new ReplicationSource(); + endpoint.init(context); + source.init(conf, FS, manager, queues, peers, mock(Stoppable.class), + "testPeerClusterZnode", UUID.randomUUID(), endpoint, metrics); + return source; + } + } + + @Test + public void testSetLogPositionForWALCurrentlyReadingWhenLogsRolled() throws Exception { + final int numWALEntries = 5; + conf.setInt("replication.source.nb.capacity", numWALEntries); + + Mocks mocks = new Mocks(); + final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() { + @Override + public WALEntryFilter getWALEntryfilter() { + return null; + } + }; + WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test"); + final Path log1 = new Path(logDir, "log.1"); + final Path log2 = new Path(logDir, "log.2"); + + WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration()); + WALProvider.Writer writer2 = WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration()); + + appendEntries(writer1, 3); + appendEntries(writer2, 2); + + long pos = getPosition(wals, log2, 2); + + final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint); + source.run(); + + source.enqueueLog(log1); + // log rolled + source.enqueueLog(log2); + + Waiter.waitFor(conf, 20000, new Waiter.Predicate() { + @Override public boolean evaluate() throws Exception { + return endpoint.replicateCount.get() > 0; + } + }); + + ArgumentCaptor pathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); + verify(mocks.manager, times(1)) + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + assertTrue(endpoint.lastEntries.size() == 5); + assertThat(pathCaptor.getValue(), is(log2)); + assertThat(positionCaptor.getValue(), is(pos)); + } + + @Test + public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exception { + Mocks mocks = new Mocks(); + + final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest(); + final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint); + WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test"); + + final Path log1 = new Path(logDir, "log.1"); + final Path log2 = new Path(logDir, "log.2"); + + WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration()).close(); + WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration()).close(); + final long startPos = getPosition(wals, log2, 0); + + source.run(); + source.enqueueLog(log1); + source.enqueueLog(log2); + + Waiter.waitFor(conf, 20000, new Waiter.Predicate() { + @Override public boolean evaluate() throws Exception { + return log2.equals(source.getCurrentPath()) && source.getLastLoggedPosition() >= startPos; + } + }); + + ArgumentCaptor pathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); + + verify(mocks.manager, times(1)) + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + assertThat(pathCaptor.getValue(), is(log2)); + assertThat(positionCaptor.getValue(), is(startPos)); + } + + @Test + public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exception { + Mocks mocks = new Mocks(); + // set table cfs to filter all cells out + final TableName replicatedTable = TableName.valueOf("replicated_table"); + final Map> cfs = Collections.singletonMap(replicatedTable, Collections.emptyList()); + when(mocks.peer.getTableCFs()).thenReturn(cfs); + + WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test"); + final Path log1 = new Path(logDir, "log.1"); + final Path log2 = new Path(logDir, "log.2"); + + WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration()); + WALProvider.Writer writer2 = WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration()); + + appendEntries(writer1, 3); + appendEntries(writer2, 2, false); + final long pos = getPosition(wals, log2, 2); + + final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest(); + final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint); + source.enqueueLog(log1); + source.enqueueLog(log2); + source.run(); + Waiter.waitFor(conf, 20000, new Waiter.Predicate() { + @Override public boolean evaluate() throws Exception { + // wait until reader read all cells + return log2.equals(source.getCurrentPath()) && source.getLastLoggedPosition() >= pos; + } + }); + + ArgumentCaptor pathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); + + // all old wals should be removed by updating wal position, even if no cfs replicated doesn't exist + verify(mocks.manager, times(1)) + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + assertThat(pathCaptor.getValue(), is(log2)); + assertThat(positionCaptor.getValue(), is(pos)); + } + /** * Tests that recovered queues are preserved on a regionserver shutdown. * See HBASE-18192 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java index 641ea50b38d6..d779c125bb17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java @@ -124,10 +124,9 @@ public void testQuota() throws IOException { public void run() { Replication replication = (Replication) utility1.getMiniHBaseCluster() .getRegionServer(0).getReplicationSourceService(); - AtomicLong bufferUsed = replication.getReplicationManager().getTotalBufferUsed(); testQuotaPass = true; while (!Thread.interrupted()) { - long size = bufferUsed.get(); + long size = replication.getReplicationManager().getTotalBufferUsed(); //the reason here doing "numOfPeer + 1" is because by using method addEntryToBatch(), even the // batch size (after added last entry) exceeds quota, it still keeps the last one in the batch // so total used buffer size can be one "replication.total.buffer.quota" larger than expected diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index 9f077da57a97..dd2a3a65ba3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -25,21 +25,22 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.NoSuchElementException; import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.atomic.AtomicLong; @@ -57,7 +58,10 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.ChainWALEntryFilter; +import org.apache.hadoop.hbase.replication.ReplicationPeer; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; +import org.apache.hadoop.hbase.replication.TableCfWALEntryFilter; import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReaderThread.WALEntryBatch; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -77,7 +81,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.mockito.runners.MockitoJUnitRunner; @@ -357,7 +360,6 @@ public void testReplicationSourceWALReaderThread() throws Exception { // start up a batcher ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class); - when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); ReplicationSourceWALReaderThread batcher = new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); Path walPath = walQueue.peek(); @@ -378,37 +380,34 @@ public void testReplicationSourceWALReaderThread() throws Exception { } @Test - public void testReplicationSourceUpdatesLogPositionOnFilteredEntries() throws Exception { + public void testReplicationSourceWALReaderThreadRecoveredQueue() throws Exception { appendEntriesToLog(3); - // get ending position + log.rollWriter(); + appendEntriesToLog(2); + long position; try (WALEntryStream entryStream = - new WALEntryStream(walQueue, fs, conf, new MetricsSource("1"))) { + new WALEntryStream(new PriorityBlockingQueue<>(walQueue), fs, conf, new MetricsSource("1"))) { + entryStream.next(); + entryStream.next(); entryStream.next(); entryStream.next(); entryStream.next(); position = entryStream.getPosition(); } - // start up a readerThread with a WALEntryFilter that always filter the entries - ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class); - when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); - ReplicationSourceWALReaderThread readerThread = new ReplicationSourceWALReaderThread( - mockSourceManager, getQueueInfo(), walQueue, 0, fs, conf, new WALEntryFilter() { - @Override - public Entry filter(Entry entry) { - return null; - } - }, new MetricsSource("1")); - readerThread.start(); - Thread.sleep(100); - ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); - verify(mockSourceManager, times(3)) - .logPositionAndCleanOldLogs(any(Path.class), - anyString(), - positionCaptor.capture(), - anyBoolean(), - anyBoolean()); - assertEquals(position, positionCaptor.getValue().longValue()); + + ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); + ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, + getQueueInfo("1-1"), walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); + Path walPath = walQueue.toArray(new Path[2])[1]; + reader.start(); + WALEntryBatch entryBatch = reader.take(); + + assertNotNull(entryBatch); + assertEquals(5, entryBatch.getWalEntries().size()); + assertEquals(position, entryBatch.getLastWalPosition()); + assertEquals(walPath, entryBatch.getLastWalPath()); + assertFalse(entryBatch.hasMoreEntries()); } @Test @@ -434,6 +433,54 @@ public void testWALKeySerialization() throws Exception { for (Map.Entry entry : deserializedKey.getExtendedAttributes().entrySet()) { assertArrayEquals(key.getExtendedAttribute(entry.getKey()), entry.getValue()); } + } + + @Test + public void testReplicationSourceWALReaderThreadWithFilter() throws Exception { + final byte[] notReplicatedCf = Bytes.toBytes("notReplicated"); + final Map> tableCfs = new HashMap<>(); + tableCfs.put(tableName, Collections.singletonList(Bytes.toString(family))); + ReplicationPeer peer = mock(ReplicationPeer.class); + when(peer.getTableCFs()).thenReturn(tableCfs); + WALEntryFilter filter = new ChainWALEntryFilter(new TableCfWALEntryFilter(peer)); + + appendToLogPlus(3, notReplicatedCf); + + Path firstWAL = walQueue.peek(); + ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); + final ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, + getQueueInfo(), walQueue, 0, fs, conf, filter, new MetricsSource("1")); + reader.start(); + + // reader won't put any batch, even if EOF reached. + ExecutorService executor = Executors.newSingleThreadExecutor(); + Future future = executor.submit(new Callable() { + @Override + public WALEntryBatch call() throws Exception { + return reader.take(); + } + }); + Thread.sleep(2000); + assertFalse(future.isDone()); + future.cancel(true); + + log.rollWriter(); + + // should get empty batch with current wal position, after wal rolled + WALEntryBatch entryBatch = reader.take(); + + Path lastWAL= walQueue.peek(); + WALEntryStream entryStream = + new WALEntryStream(new PriorityBlockingQueue<>(walQueue), fs, conf, new MetricsSource("1")); + entryStream.hasNext(); + long positionToBeLogged = entryStream.getPosition(); + + assertNotNull(entryBatch); + assertTrue(entryBatch.isEmpty()); + assertTrue(walQueue.size() == 1); + assertNotEquals(firstWAL, entryBatch.getLastWalPath()); + assertEquals(lastWAL, entryBatch.getLastWalPath()); + assertEquals(positionToBeLogged, entryBatch.getLastWalPosition()); } private String getRow(WAL.Entry entry) { @@ -459,17 +506,25 @@ private void appendToLog() throws IOException { } private void appendToLogPlus(int count) throws IOException { + appendToLogPlus(count, family, qualifier); + } + + private void appendToLogPlus(int count, byte[] cf) throws IOException { + appendToLogPlus(count, cf, qualifier); + } + + private void appendToLogPlus(int count, byte[] cf, byte[] cq) throws IOException { final long txid = log.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc), - getWALEdits(count), true); + getWALEdits(count, cf, cq), true); log.sync(txid); } - private WALEdit getWALEdits(int count) { + private WALEdit getWALEdits(int count, byte[] cf, byte[] cq) { WALEdit edit = new WALEdit(); for (int i = 0; i < count; i++) { - edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), family, qualifier, - System.currentTimeMillis(), qualifier)); + edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), cf, cq, + System.currentTimeMillis(), cq)); } return edit; } @@ -492,7 +547,11 @@ public Entry filter(Entry entry) { } private ReplicationQueueInfo getQueueInfo() { - return new ReplicationQueueInfo("1"); + return getQueueInfo("1"); + } + + private ReplicationQueueInfo getQueueInfo(String znode) { + return new ReplicationQueueInfo(znode); } class PathWatcher extends WALActionsListener.Base { From affc75aa2dab883a8b43b1746cca2467b6fbb08d Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Thu, 24 Oct 2019 15:27:43 +0900 Subject: [PATCH 02/51] fix checkstyle warnings --- .../regionserver/ReplicationSource.java | 3 +- .../ReplicationSourceWALReaderThread.java | 13 +-- .../TestReplicationSmallTests.java | 4 +- .../replication/TestReplicationSource.java | 79 +++++++++++-------- .../TestGlobalReplicationThrottler.java | 1 - .../regionserver/TestWALEntryStream.java | 27 ++++--- 6 files changed, 71 insertions(+), 56 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 25f986060542..93e62a98b9b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -607,7 +607,8 @@ public void run() { shipEdits(entryBatch); manager.releaseBufferQuota(entryBatch.getHeapSizeExcludeBulkLoad()); if (!entryBatch.hasMoreEntries()) { - LOG.debug("Finished recovering queue for group " + walGroupId + " of peer " + peerClusterZnode); + LOG.debug("Finished recovering queue for group " + + walGroupId + " of peer " + peerClusterZnode); metrics.incrCompletedRecoveryQueue(); setWorkerState(WorkerState.FINISHED); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 5b25186cc977..2061cae7983b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -131,9 +131,10 @@ public void run() { Threads.sleep(sleepForRetries); continue; } - WALEntryBatch batch = new WALEntryBatch(replicationBatchCountCapacity, replicationBatchSizeCapacity); + WALEntryBatch batch = + new WALEntryBatch(replicationBatchCountCapacity, replicationBatchSizeCapacity); boolean hasNext; - while (hasNext = entryStream.hasNext()) { + while ((hasNext = entryStream.hasNext()) != true) { Entry entry = entryStream.next(); entry = filterEntry(entry); if (entry != null) { @@ -150,7 +151,8 @@ public void run() { } if (LOG.isTraceEnabled()) { - LOG.trace(String.format("Read %s WAL entries eligible for replication", batch.getNbEntries())); + LOG.trace(String.format("Read %s WAL entries eligible for replication", + batch.getNbEntries())); } updateBatch(entryStream, batch, hasNext); @@ -193,7 +195,7 @@ private boolean isShippable(WALEntryBatch batch) { private boolean checkIfWALRolled(WALEntryBatch batch) { return currentPath == null && batch.lastWalPath != null - || currentPath != null && !currentPath.equals(batch.lastWalPath); + || currentPath != null && !currentPath.equals(batch.lastWalPath); } private void resetStream(WALEntryStream stream) throws IOException { @@ -280,7 +282,8 @@ static class WALEntryBatch { private long heapSizeExcludeBulkLoad; /** - * @param maxNbEntries + * @param maxNbEntries the number of entries a batch can have + * @param maxSizeBytes max (heap) size of each batch */ private WALEntryBatch(int maxNbEntries, long maxSizeBytes) { this.walEntries = new ArrayList<>(maxNbEntries); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 87e34e7d7bd9..5af1ada74f5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -68,7 +68,6 @@ 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.apache.hadoop.mapreduce.Job; import org.junit.Before; import org.junit.Test; @@ -832,7 +831,8 @@ public void testEmptyWALRecovery() throws Exception { Path currentWalPath = DefaultWALProvider.getCurrentFileName(wal); String walGroupId = DefaultWALProvider.getWALPrefixFromWALName(currentWalPath.getName()); Path emptyWalPath = new Path(currentWalPath.getParent(), walGroupId + "." + ts); - WALFactory.createWALWriter(utility1.getTestFileSystem(), emptyWalPath, utility1.getConfiguration()).close(); + WALFactory.createWALWriter(utility1.getTestFileSystem(), + emptyWalPath, utility1.getConfiguration()).close(); emptyWalPaths.add(emptyWalPath); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index b9288289ff90..f3d337f609af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -16,17 +16,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hadoop.hbase.replication; +import static org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; 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 static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.internal.verification.VerificationModeFactory.times; import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -63,26 +79,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Mockito; import org.mockito.ArgumentCaptor; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.UUID; - -import static org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -import static org.mockito.internal.verification.VerificationModeFactory.times; +import org.mockito.Mockito; @Category(MediumTests.class) public class TestReplicationSource { @@ -114,8 +112,12 @@ public static void setUpBeforeClass() throws Exception { @Before public void setup() throws IOException { - if (!FS.exists(logDir)) FS.mkdirs(logDir); - if (!FS.exists(oldLogDir)) FS.mkdirs(oldLogDir); + if (!FS.exists(logDir)) { + FS.mkdirs(logDir); + } + if (!FS.exists(oldLogDir)) { + FS.mkdirs(oldLogDir); + } ReplicationEndpointForTest.contructedCount.set(0); ReplicationEndpointForTest.startedCount.set(0); @@ -126,8 +128,12 @@ public void setup() throws IOException { @After public void tearDown() throws IOException { - if (FS.exists(oldLogDir)) FS.delete(oldLogDir, true); - if (FS.exists(logDir)) FS.delete(logDir, true); + if (FS.exists(oldLogDir)) { + FS.delete(oldLogDir, true); + } + if (FS.exists(logDir)) { + FS.delete(logDir, true); + } } @AfterClass @@ -224,8 +230,9 @@ public boolean evaluate() throws Exception { } - private void appendEntries(WALProvider.Writer writer, int numEntries, boolean closeAfterAppends) throws IOException { - for(int i = 0; i < numEntries; i++) { + private void appendEntries(WALProvider.Writer writer, int numEntries, boolean closeAfterAppends) + throws IOException { + for (int i = 0; i < numEntries; i++) { byte[] b = Bytes.toBytes(Integer.toString(i)); KeyValue kv = new KeyValue(b,b,b); WALEdit edit = new WALEdit(); @@ -255,7 +262,7 @@ private long getPosition(WALFactory wals, Path log2, int numEntries) throws IOEx return reader.getPosition(); } - private static class Mocks { + private static final class Mocks { private final ReplicationSourceManager manager = mock(ReplicationSourceManager.class); private final ReplicationQueues queues = mock(ReplicationQueues.class); private final ReplicationPeers peers = mock(ReplicationPeers.class); @@ -268,7 +275,8 @@ private Mocks() { when(context.getReplicationPeer()).thenReturn(peer); } - ReplicationSource createReplicationSourceWithMocks(ReplicationEndpoint endpoint) throws IOException { + ReplicationSource createReplicationSourceWithMocks(ReplicationEndpoint endpoint) + throws IOException { final ReplicationSource source = new ReplicationSource(); endpoint.init(context); source.init(conf, FS, manager, queues, peers, mock(Stoppable.class), @@ -317,7 +325,8 @@ public WALEntryFilter getWALEntryfilter() { ArgumentCaptor pathCaptor = ArgumentCaptor.forClass(Path.class); ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); verify(mocks.manager, times(1)) - .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), + anyBoolean(), anyBoolean()); assertTrue(endpoint.lastEntries.size() == 5); assertThat(pathCaptor.getValue(), is(log2)); assertThat(positionCaptor.getValue(), is(pos)); @@ -352,7 +361,8 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exc ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); verify(mocks.manager, times(1)) - .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), + anyBoolean(), anyBoolean()); assertThat(pathCaptor.getValue(), is(log2)); assertThat(positionCaptor.getValue(), is(startPos)); } @@ -362,7 +372,8 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc Mocks mocks = new Mocks(); // set table cfs to filter all cells out final TableName replicatedTable = TableName.valueOf("replicated_table"); - final Map> cfs = Collections.singletonMap(replicatedTable, Collections.emptyList()); + final Map> cfs = + Collections.singletonMap(replicatedTable, Collections.emptyList()); when(mocks.peer.getTableCFs()).thenReturn(cfs); WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test"); @@ -391,9 +402,10 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc ArgumentCaptor pathCaptor = ArgumentCaptor.forClass(Path.class); ArgumentCaptor positionCaptor = ArgumentCaptor.forClass(Long.class); - // all old wals should be removed by updating wal position, even if no cfs replicated doesn't exist + // all old wals should be removed by updating wal position, even if all cells are filtered out. verify(mocks.manager, times(1)) - .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), anyBoolean(), anyBoolean()); + .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(), + anyBoolean(), anyBoolean()); assertThat(pathCaptor.getValue(), is(log2)); assertThat(positionCaptor.getValue(), is(pos)); } @@ -401,7 +413,6 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc /** * Tests that recovered queues are preserved on a regionserver shutdown. * See HBASE-18192 - * @throws Exception */ @Test public void testServerShutdownRecoveredQueue() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java index d779c125bb17..7d017e5e404a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index dd2a3a65ba3d..71706a9a3957 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -42,7 +42,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.PriorityBlockingQueue; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -360,8 +359,9 @@ public void testReplicationSourceWALReaderThread() throws Exception { // start up a batcher ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class); - ReplicationSourceWALReaderThread batcher = new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),walQueue, 0, - fs, conf, getDummyFilter(), new MetricsSource("1")); + ReplicationSourceWALReaderThread batcher = + new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),walQueue, 0, + fs, conf, getDummyFilter(), new MetricsSource("1")); Path walPath = walQueue.peek(); batcher.start(); WALEntryBatch entryBatch = batcher.take(); @@ -386,8 +386,8 @@ public void testReplicationSourceWALReaderThreadRecoveredQueue() throws Exceptio appendEntriesToLog(2); long position; - try (WALEntryStream entryStream = - new WALEntryStream(new PriorityBlockingQueue<>(walQueue), fs, conf, new MetricsSource("1"))) { + try (WALEntryStream entryStream = new WALEntryStream(new PriorityBlockingQueue<>(walQueue), + fs, conf, new MetricsSource("1"))) { entryStream.next(); entryStream.next(); entryStream.next(); @@ -397,8 +397,9 @@ public void testReplicationSourceWALReaderThreadRecoveredQueue() throws Exceptio } ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); - ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, - getQueueInfo("1-1"), walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); + ReplicationSourceWALReaderThread reader = + new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo("1-1"), + walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); Path walPath = walQueue.toArray(new Path[2])[1]; reader.start(); WALEntryBatch entryBatch = reader.take(); @@ -433,7 +434,7 @@ public void testWALKeySerialization() throws Exception { for (Map.Entry entry : deserializedKey.getExtendedAttributes().entrySet()) { assertArrayEquals(key.getExtendedAttribute(entry.getKey()), entry.getValue()); } - } + } @Test public void testReplicationSourceWALReaderThreadWithFilter() throws Exception { @@ -448,8 +449,9 @@ public void testReplicationSourceWALReaderThreadWithFilter() throws Exception { Path firstWAL = walQueue.peek(); ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); - final ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, - getQueueInfo(), walQueue, 0, fs, conf, filter, new MetricsSource("1")); + final ReplicationSourceWALReaderThread reader = + new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), walQueue, + 0, fs, conf, filter, new MetricsSource("1")); reader.start(); // reader won't put any batch, even if EOF reached. @@ -470,8 +472,8 @@ public WALEntryBatch call() throws Exception { WALEntryBatch entryBatch = reader.take(); Path lastWAL= walQueue.peek(); - WALEntryStream entryStream = - new WALEntryStream(new PriorityBlockingQueue<>(walQueue), fs, conf, new MetricsSource("1")); + WALEntryStream entryStream = new WALEntryStream(new PriorityBlockingQueue<>(walQueue), + fs, conf, new MetricsSource("1")); entryStream.hasNext(); long positionToBeLogged = entryStream.getPosition(); @@ -564,5 +566,4 @@ public void preLogRoll(Path oldPath, Path newPath) throws IOException { currentPath = newPath; } } - } From b6efe7e0475c6216b8984f7282a2ab96025a7980 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Thu, 24 Oct 2019 20:07:38 +0900 Subject: [PATCH 03/51] Fix typo --- .../regionserver/ReplicationSourceWALReaderThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 2061cae7983b..6f17667130e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -134,7 +134,7 @@ public void run() { WALEntryBatch batch = new WALEntryBatch(replicationBatchCountCapacity, replicationBatchSizeCapacity); boolean hasNext; - while ((hasNext = entryStream.hasNext()) != true) { + while ((hasNext = entryStream.hasNext()) == true) { Entry entry = entryStream.next(); entry = filterEntry(entry); if (entry != null) { From 114aa1b1a7b9919c5429fadcb74079cd08629513 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Wed, 30 Oct 2019 10:48:58 +0900 Subject: [PATCH 04/51] (fix) close writer --- .../hbase/replication/TestReplicationSource.java | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index f3d337f609af..c46b535bd0e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -230,8 +230,7 @@ public boolean evaluate() throws Exception { } - private void appendEntries(WALProvider.Writer writer, int numEntries, boolean closeAfterAppends) - throws IOException { + private void appendEntries(WALProvider.Writer writer, int numEntries) throws IOException { for (int i = 0; i < numEntries; i++) { byte[] b = Bytes.toBytes(Integer.toString(i)); KeyValue kv = new KeyValue(b,b,b); @@ -245,13 +244,7 @@ private void appendEntries(WALProvider.Writer writer, int numEntries, boolean cl writer.append(new WAL.Entry(key, edit)); writer.sync(false); } - if (closeAfterAppends) { - writer.close(); - } - } - - private void appendEntries(WALProvider.Writer writer, int numEntries) throws IOException { - appendEntries(writer, numEntries, true); + writer.close(); } private long getPosition(WALFactory wals, Path log2, int numEntries) throws IOException { @@ -384,7 +377,7 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc WALProvider.Writer writer2 = WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration()); appendEntries(writer1, 3); - appendEntries(writer2, 2, false); + appendEntries(writer2, 2); final long pos = getPosition(wals, log2, 2); final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest(); From 4bcc397f3eb321081e703f1385efbcbec7ba7936 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Wed, 30 Oct 2019 09:22:39 -0500 Subject: [PATCH 05/51] HBASE-23229 Update branch-1 to 1.6.0-SNAPSHOT (#772) Signed-off-by: Wellington Chevreuil --- hbase-annotations/pom.xml | 2 +- hbase-archetypes/hbase-archetype-builder/pom.xml | 2 +- hbase-archetypes/hbase-client-project/pom.xml | 2 +- hbase-archetypes/hbase-shaded-client-project/pom.xml | 2 +- hbase-archetypes/pom.xml | 2 +- hbase-assembly/pom.xml | 2 +- hbase-checkstyle/pom.xml | 4 ++-- hbase-client/pom.xml | 2 +- hbase-common/pom.xml | 2 +- hbase-error-prone/pom.xml | 4 ++-- hbase-examples/pom.xml | 2 +- hbase-external-blockcache/pom.xml | 2 +- hbase-hadoop-compat/pom.xml | 2 +- hbase-hadoop2-compat/pom.xml | 2 +- hbase-hbtop/pom.xml | 2 +- hbase-it/pom.xml | 2 +- hbase-metrics-api/pom.xml | 2 +- hbase-metrics/pom.xml | 2 +- hbase-prefix-tree/pom.xml | 2 +- hbase-procedure/pom.xml | 2 +- hbase-protocol/pom.xml | 2 +- hbase-resource-bundle/pom.xml | 2 +- hbase-rest/pom.xml | 2 +- hbase-rsgroup/pom.xml | 2 +- hbase-server/pom.xml | 2 +- hbase-shaded/hbase-shaded-check-invariants/pom.xml | 2 +- hbase-shaded/hbase-shaded-client/pom.xml | 2 +- hbase-shaded/hbase-shaded-server/pom.xml | 2 +- hbase-shaded/hbase-shaded-testing-util-tester/pom.xml | 2 +- hbase-shaded/hbase-shaded-testing-util/pom.xml | 2 +- hbase-shaded/pom.xml | 2 +- hbase-shell/pom.xml | 2 +- hbase-testing-util/pom.xml | 2 +- hbase-thrift/pom.xml | 2 +- pom.xml | 2 +- 35 files changed, 37 insertions(+), 37 deletions(-) diff --git a/hbase-annotations/pom.xml b/hbase-annotations/pom.xml index 02289d1a3e2b..e8767dfa99fc 100644 --- a/hbase-annotations/pom.xml +++ b/hbase-annotations/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-archetypes/hbase-archetype-builder/pom.xml b/hbase-archetypes/hbase-archetype-builder/pom.xml index d34f0816dfa7..9992a618b06c 100644 --- a/hbase-archetypes/hbase-archetype-builder/pom.xml +++ b/hbase-archetypes/hbase-archetype-builder/pom.xml @@ -25,7 +25,7 @@ hbase-archetypes org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-archetypes/hbase-client-project/pom.xml b/hbase-archetypes/hbase-client-project/pom.xml index 22d5d4f5d2f9..cbe3a1a11a89 100644 --- a/hbase-archetypes/hbase-client-project/pom.xml +++ b/hbase-archetypes/hbase-client-project/pom.xml @@ -26,7 +26,7 @@ hbase-archetypes org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-client-project diff --git a/hbase-archetypes/hbase-shaded-client-project/pom.xml b/hbase-archetypes/hbase-shaded-client-project/pom.xml index 91fe85a04c59..74d7204449b5 100644 --- a/hbase-archetypes/hbase-shaded-client-project/pom.xml +++ b/hbase-archetypes/hbase-shaded-client-project/pom.xml @@ -26,7 +26,7 @@ hbase-archetypes org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-shaded-client-project diff --git a/hbase-archetypes/pom.xml b/hbase-archetypes/pom.xml index 069a8be5ce87..2d6be10d6dca 100644 --- a/hbase-archetypes/pom.xml +++ b/hbase-archetypes/pom.xml @@ -24,7 +24,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml index d1863e063ab5..5fe3d6deefac 100644 --- a/hbase-assembly/pom.xml +++ b/hbase-assembly/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-assembly diff --git a/hbase-checkstyle/pom.xml b/hbase-checkstyle/pom.xml index 332572b7f961..764f559b76c3 100644 --- a/hbase-checkstyle/pom.xml +++ b/hbase-checkstyle/pom.xml @@ -24,14 +24,14 @@ 4.0.0 org.apache.hbase hbase-checkstyle -1.5.1-SNAPSHOT +1.6.0-SNAPSHOT Apache HBase - Checkstyle Module to hold Checkstyle properties for HBase. hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index 38e7ec438da3..455a5e76ad0b 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -24,7 +24,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index 1795399ad863..b76f8ceaa13e 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-error-prone/pom.xml b/hbase-error-prone/pom.xml index c3681dfd1ddf..3302cafcaa2c 100644 --- a/hbase-error-prone/pom.xml +++ b/hbase-error-prone/pom.xml @@ -23,11 +23,11 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-error-prone - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT Apache HBase - Error Prone Rules Module to hold error prone custom rules for HBase. diff --git a/hbase-examples/pom.xml b/hbase-examples/pom.xml index 7720ab584f28..02eef28b4390 100644 --- a/hbase-examples/pom.xml +++ b/hbase-examples/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-examples diff --git a/hbase-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml index 2ae2dbde7914..2fa9a20b32db 100644 --- a/hbase-external-blockcache/pom.xml +++ b/hbase-external-blockcache/pom.xml @@ -25,7 +25,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-external-blockcache diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml index d606ab981e3c..01055a77e79a 100644 --- a/hbase-hadoop-compat/pom.xml +++ b/hbase-hadoop-compat/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml index 0d01ff9a050a..0b5547ae6c1c 100644 --- a/hbase-hadoop2-compat/pom.xml +++ b/hbase-hadoop2-compat/pom.xml @@ -21,7 +21,7 @@ limitations under the License. hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-hbtop/pom.xml b/hbase-hbtop/pom.xml index a1cf67e16a2f..ad0aa370abd1 100644 --- a/hbase-hbtop/pom.xml +++ b/hbase-hbtop/pom.xml @@ -25,7 +25,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-hbtop diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml index fc515cf7c6db..2e2e25dc7581 100644 --- a/hbase-it/pom.xml +++ b/hbase-it/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml index 6fa44fec8701..86d53f9193ac 100644 --- a/hbase-metrics-api/pom.xml +++ b/hbase-metrics-api/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml index ff3836c9f190..0d100aa1bc5d 100644 --- a/hbase-metrics/pom.xml +++ b/hbase-metrics/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-prefix-tree/pom.xml b/hbase-prefix-tree/pom.xml index 1e8260e010ce..9bd7e2cb1f3c 100644 --- a/hbase-prefix-tree/pom.xml +++ b/hbase-prefix-tree/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-procedure/pom.xml b/hbase-procedure/pom.xml index 2dcb9d95526a..ed79224e2e8a 100644 --- a/hbase-procedure/pom.xml +++ b/hbase-procedure/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml index 6a5078e9be35..658dc4454445 100644 --- a/hbase-protocol/pom.xml +++ b/hbase-protocol/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-resource-bundle/pom.xml b/hbase-resource-bundle/pom.xml index 18b1220cc9ba..249bbb942d7c 100644 --- a/hbase-resource-bundle/pom.xml +++ b/hbase-resource-bundle/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml index 383472ee961d..ac0b9da978ef 100644 --- a/hbase-rest/pom.xml +++ b/hbase-rest/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-rest diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml index 796e0657ca55..1f7b6cd983ad 100644 --- a/hbase-rsgroup/pom.xml +++ b/hbase-rsgroup/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index ea0e16fcd999..34eb84b4882a 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-server diff --git a/hbase-shaded/hbase-shaded-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-check-invariants/pom.xml index 83015bbd1a8b..3e590a5472d4 100644 --- a/hbase-shaded/hbase-shaded-check-invariants/pom.xml +++ b/hbase-shaded/hbase-shaded-check-invariants/pom.xml @@ -16,7 +16,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT ../.. hbase-shaded-check-invariants diff --git a/hbase-shaded/hbase-shaded-client/pom.xml b/hbase-shaded/hbase-shaded-client/pom.xml index d86f6eacd566..aa69811c0782 100644 --- a/hbase-shaded/hbase-shaded-client/pom.xml +++ b/hbase-shaded/hbase-shaded-client/pom.xml @@ -24,7 +24,7 @@ hbase-shaded org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-shaded-client diff --git a/hbase-shaded/hbase-shaded-server/pom.xml b/hbase-shaded/hbase-shaded-server/pom.xml index 42dfe348132a..bf55de503108 100644 --- a/hbase-shaded/hbase-shaded-server/pom.xml +++ b/hbase-shaded/hbase-shaded-server/pom.xml @@ -24,7 +24,7 @@ hbase-shaded org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-shaded-server diff --git a/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml b/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml index 03281c5087be..12befb6b6543 100644 --- a/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml +++ b/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml @@ -25,7 +25,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT ../.. diff --git a/hbase-shaded/hbase-shaded-testing-util/pom.xml b/hbase-shaded/hbase-shaded-testing-util/pom.xml index 26182066bcc5..7e8d6cd5450a 100644 --- a/hbase-shaded/hbase-shaded-testing-util/pom.xml +++ b/hbase-shaded/hbase-shaded-testing-util/pom.xml @@ -25,7 +25,7 @@ hbase-shaded org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml index 529eb239f34e..1fce8f1fee1b 100644 --- a/hbase-shaded/pom.xml +++ b/hbase-shaded/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-shaded diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml index 939d55c9bbdf..c995882db654 100644 --- a/hbase-shell/pom.xml +++ b/hbase-shell/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-shell diff --git a/hbase-testing-util/pom.xml b/hbase-testing-util/pom.xml index 46fd2a383dc5..5a2fb3a0a4f9 100644 --- a/hbase-testing-util/pom.xml +++ b/hbase-testing-util/pom.xml @@ -23,7 +23,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-testing-util diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml index 91216ba80871..35b559a9c3e6 100644 --- a/hbase-thrift/pom.xml +++ b/hbase-thrift/pom.xml @@ -25,7 +25,7 @@ hbase org.apache.hbase - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT .. hbase-thrift diff --git a/pom.xml b/pom.xml index 376fe0270183..92c435d99b6d 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ org.apache.hbase hbase pom - 1.5.1-SNAPSHOT + 1.6.0-SNAPSHOT Apache HBase Apache HBase™ is the Hadoop database. Use it when you need From 1dbf6f7ad279a79cfe38f451f0ffa3188802f598 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Thu, 31 Oct 2019 19:53:38 +0900 Subject: [PATCH 06/51] (fix) revert test for HBASE-18137 --- .../regionserver/ReplicationSource.java | 30 +++++++++++++------ .../ReplicationSourceWALReaderThread.java | 26 +++++++++------- .../TestReplicationSmallTests.java | 10 +++---- .../replication/TestReplicationSource.java | 4 +-- 4 files changed, 42 insertions(+), 28 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 93e62a98b9b8..1a4574906001 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -446,9 +446,17 @@ public Path getCurrentPath() { } @VisibleForTesting - public long getLastLoggedPosition() { + public Path getCurrentReadPath() { for (ReplicationSourceShipperThread worker : workerThreads.values()) { - return worker.getLastLoggedPosition(); + return worker.getCurrentReadPath(); + } + return null; + } + + @VisibleForTesting + public long getCurrentPosition() { + for (ReplicationSourceShipperThread worker : workerThreads.values()) { + return worker.getCurrentPosition(); } return 0; } @@ -487,7 +495,7 @@ public String getStats() { for (Map.Entry entry : workerThreads.entrySet()) { String walGroupId = entry.getKey(); ReplicationSourceShipperThread worker = entry.getValue(); - long position = worker.getLastLoggedPosition(); + long position = worker.getCurrentPosition(); Path currentPath = worker.getCurrentPath(); sb.append("walGroup [").append(walGroupId).append("]: "); if (currentPath != null) { @@ -541,7 +549,7 @@ public Map getWalGroupStatus() { .withQueueSize(queueSize) .withWalGroup(walGroupId) .withCurrentPath(currentPath) - .withCurrentPosition(worker.getLastLoggedPosition()) + .withCurrentPosition(worker.getCurrentPosition()) .withFileSize(fileSize) .withAgeOfLastShippedOp(ageOfLastShippedOp) .withReplicationDelay(replicationDelay); @@ -561,7 +569,7 @@ public class ReplicationSourceShipperThread extends Thread { // Last position in the log that we sent to ZooKeeper private long lastLoggedPosition = -1; // Path of the current log - private volatile Path currentPath; + private volatile Path lastLoggedPath; // Current state of the worker thread private WorkerState state; ReplicationSourceWALReaderThread entryReader; @@ -686,7 +694,7 @@ private void checkBandwidthChangeAndResetThrottler() { protected void shipEdits(WALEntryBatch entryBatch) { List entries = entryBatch.getWalEntries(); long lastReadPosition = entryBatch.getLastWalPosition(); - currentPath = entryBatch.getLastWalPath(); + lastLoggedPath = entryBatch.getLastWalPath(); int sleepMultiplier = 0; if (entries.isEmpty()) { updateLogPosition(lastReadPosition); @@ -777,7 +785,7 @@ protected void shipEdits(WALEntryBatch entryBatch) { } private void updateLogPosition(long lastReadPosition) { - manager.logPositionAndCleanOldLogs(currentPath, peerClusterZnode, lastReadPosition, + manager.logPositionAndCleanOldLogs(lastLoggedPath, peerClusterZnode, lastReadPosition, this.replicationQueueInfo.isQueueRecovered(), false); lastLoggedPosition = lastReadPosition; } @@ -926,11 +934,15 @@ private Path getReplSyncUpPath(Path path) throws IOException { return path; } + public Path getCurrentReadPath() { + return entryReader.getCurrentPath(); + } + public Path getCurrentPath() { - return currentPath; + return lastLoggedPath; } - public long getLastLoggedPosition() { + public long getCurrentPosition() { return lastLoggedPosition; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 6f17667130e3..6661e240b422 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -65,8 +65,8 @@ public class ReplicationSourceWALReaderThread extends Thread { // max count of each batch - multiply by number of batches in queue to get total private int replicationBatchCountCapacity; // position in the WAL to start reading at - private long currentPosition; - private Path currentPath; + private long lastReadPosition; + private Path lastReadPath; private WALEntryFilter filter; private long sleepForRetries; //Indicates whether this particular worker is running @@ -95,8 +95,8 @@ public ReplicationSourceWALReaderThread(ReplicationSourceManager manager, FileSystem fs, Configuration conf, WALEntryFilter filter, MetricsSource metrics) { this.replicationQueueInfo = replicationQueueInfo; this.logQueue = logQueue; - this.currentPath = logQueue.peek(); - this.currentPosition = startPosition; + this.lastReadPath = logQueue.peek(); + this.lastReadPosition = startPosition; this.fs = fs; this.conf = conf; this.filter = filter; @@ -125,7 +125,7 @@ public void run() { int sleepMultiplier = 1; while (isReaderRunning()) { // we only loop back here if something fatal happened to our stream try (WALEntryStream entryStream = - new WALEntryStream(logQueue, fs, conf, currentPosition, metrics)) { + new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) { while (isReaderRunning()) { // loop here to keep reusing stream while we can if (manager.isBufferQuotaReached()) { Threads.sleep(sleepForRetries); @@ -194,13 +194,13 @@ private boolean isShippable(WALEntryBatch batch) { } private boolean checkIfWALRolled(WALEntryBatch batch) { - return currentPath == null && batch.lastWalPath != null - || currentPath != null && !currentPath.equals(batch.lastWalPath); + return lastReadPath == null && batch.lastWalPath != null + || lastReadPath != null && !lastReadPath.equals(batch.lastWalPath); } private void resetStream(WALEntryStream stream) throws IOException { - currentPosition = stream.getPosition(); - currentPath = stream.getCurrentPath(); + lastReadPosition = stream.getPosition(); + lastReadPath = stream.getCurrentPath(); stream.reset(); // reuse stream } @@ -213,8 +213,8 @@ private void handleEofException(Exception e) { try { if (fs.getFileStatus(logQueue.peek()).getLen() == 0) { LOG.warn("Forcing removal of 0 length log in queue: " + logQueue.peek()); - logQueue.remove(); - currentPosition = 0; + lastReadPath = logQueue.remove(); + lastReadPosition = 0; } } catch (IOException ioe) { LOG.warn("Couldn't get file length information about log " + logQueue.peek()); @@ -222,6 +222,10 @@ private void handleEofException(Exception e) { } } + public Path getCurrentPath() { + return logQueue.peek(); + } + private Entry filterEntry(Entry entry) { Entry filtered = filter.filter(entry); if (entry != null && filtered == null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 5af1ada74f5f..fe54dfc6dd55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.wal.DefaultWALProvider; 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.mapreduce.Job; import org.junit.Before; @@ -830,9 +829,8 @@ public void testEmptyWALRecovery() throws Exception { WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo); Path currentWalPath = DefaultWALProvider.getCurrentFileName(wal); String walGroupId = DefaultWALProvider.getWALPrefixFromWALName(currentWalPath.getName()); - Path emptyWalPath = new Path(currentWalPath.getParent(), walGroupId + "." + ts); - WALFactory.createWALWriter(utility1.getTestFileSystem(), - emptyWalPath, utility1.getConfiguration()).close(); + Path emptyWalPath = new Path(utility1.getDataTestDir(), walGroupId + "." + ts); + utility1.getTestFileSystem().create(emptyWalPath).close(); emptyWalPaths.add(emptyWalPath); } @@ -881,10 +879,10 @@ public boolean evaluate() throws Exception { for (ReplicationSourceInterface rsi : replicationService.getReplicationManager() .getSources()) { ReplicationSource source = (ReplicationSource) rsi; - if (!invert && !emptyWalPaths.get(i).equals(source.getCurrentPath())) { + if (!invert && !emptyWalPaths.get(i).equals(source.getCurrentReadPath())) { return false; } - if (invert && emptyWalPaths.get(i).equals(source.getCurrentPath())) { + if (invert && emptyWalPaths.get(i).equals(source.getCurrentReadPath())) { return false; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index c46b535bd0e1..a2e8c89c1ca1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -346,7 +346,7 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exc Waiter.waitFor(conf, 20000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - return log2.equals(source.getCurrentPath()) && source.getLastLoggedPosition() >= startPos; + return log2.equals(source.getCurrentPath()) && source.getCurrentPosition() >= startPos; } }); @@ -388,7 +388,7 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc Waiter.waitFor(conf, 20000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { // wait until reader read all cells - return log2.equals(source.getCurrentPath()) && source.getLastLoggedPosition() >= pos; + return log2.equals(source.getCurrentPath()) && source.getCurrentPosition() >= pos; } }); From c0b8f7b6957359c76e52277c08b31160556345fe Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Thu, 31 Oct 2019 20:33:41 +0900 Subject: [PATCH 07/51] Revert unnecessary codes --- .../regionserver/ReplicationSource.java | 23 +- .../ReplicationSourceManager.java | 28 +- .../ReplicationSourceWALReaderThread.java | 250 +++++++++--------- .../replication/TestReplicationSource.java | 2 + .../TestGlobalReplicationThrottler.java | 4 +- .../regionserver/TestWALEntryStream.java | 4 + 6 files changed, 162 insertions(+), 149 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 1a4574906001..9aa4c776fe9f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -146,6 +146,8 @@ public enum WorkerState { FINISHED // The worker is done processing a recovered queue } + private AtomicLong totalBufferUsed; + /** * Instantiation method used by region servers * @@ -191,6 +193,7 @@ public void init(final Configuration conf, final FileSystem fs, defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0); currentBandwidth = getCurrentBandwidth(); this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0); + this.totalBufferUsed = manager.getTotalBufferUsed(); LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId + ", currentBandwidth=" + this.currentBandwidth); } @@ -613,7 +616,7 @@ public void run() { try { WALEntryBatch entryBatch = entryReader.take(); shipEdits(entryBatch); - manager.releaseBufferQuota(entryBatch.getHeapSizeExcludeBulkLoad()); + releaseBufferQuota((int) entryBatch.getHeapSize()); if (!entryBatch.hasMoreEntries()) { LOG.debug("Finished recovering queue for group " + walGroupId + " of peer " + peerClusterZnode); @@ -688,6 +691,18 @@ private void checkBandwidthChangeAndResetThrottler() { } } + /** + * get batchEntry size excludes bulk load file sizes. + * Uses ReplicationSourceWALReader's static method. + */ + private int getBatchEntrySizeExcludeBulkLoad(WALEntryBatch entryBatch) { + int totalSize = 0; + for(Entry entry : entryBatch.getWalEntries()) { + totalSize += entryReader.getEntrySizeExcludeBulkLoad(entry); + } + return totalSize; + } + /** * Do the shipping logic */ @@ -704,7 +719,7 @@ protected void shipEdits(WALEntryBatch entryBatch) { return; } int currentSize = (int) entryBatch.getHeapSize(); - int sizeExcludeBulkLoad = (int) entryBatch.getHeapSizeExcludeBulkLoad(); + int sizeExcludeBulkLoad = getBatchEntrySizeExcludeBulkLoad(entryBatch); while (isWorkerActive()) { try { checkBandwidthChangeAndResetThrottler(); @@ -984,5 +999,9 @@ public void setWorkerState(WorkerState state) { public WorkerState getWorkerState() { return state; } + + private void releaseBufferQuota(int size) { + totalBufferUsed.addAndGet(-size); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index 58c5ce97b3bd..0e3724a5a1ef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -121,7 +121,6 @@ public class ReplicationSourceManager implements ReplicationListener { private final Random rand; private final boolean replicationForBulkLoadDataEnabled; - private long totalBufferQuota; private AtomicLong totalBufferUsed = new AtomicLong(); /** @@ -176,8 +175,6 @@ public ReplicationSourceManager(final ReplicationQueues replicationQueues, replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); - totalBufferQuota = conf.getLong(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY, - HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_DFAULT); } /** @@ -192,7 +189,7 @@ public ReplicationSourceManager(final ReplicationQueues replicationQueues, * @param holdLogInZK if true then the log is retained in ZK */ public synchronized void logPositionAndCleanOldLogs(Path log, String id, long position, - boolean queueRecovered, boolean holdLogInZK) { + boolean queueRecovered, boolean holdLogInZK) { String fileName = log.getName(); this.replicationQueues.setLogPosition(id, fileName, position); if (holdLogInZK) { @@ -461,27 +458,8 @@ void postLogRoll(Path newLog) throws IOException { } @VisibleForTesting - public long getTotalBufferUsed() { - return totalBufferUsed.get(); - } - - /** - * @param size delta size for grown buffer - * @return true if total buffer size limit reached, after adding size - */ - public boolean acquireBufferQuota(long size) { - return totalBufferUsed.addAndGet(size) >= totalBufferQuota; - } - - public void releaseBufferQuota(long size) { - totalBufferUsed.addAndGet(-size); - } - - /** - * @return true if total buffer size limit reached - */ - public boolean isBufferQuotaReached() { - return totalBufferUsed.get() >= totalBufferQuota; + public AtomicLong getTotalBufferUsed() { + return totalBufferUsed; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 6661e240b422..4cf2490be355 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -26,6 +26,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -34,6 +35,7 @@ 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor; @@ -75,7 +77,8 @@ public class ReplicationSourceWALReaderThread extends Thread { private int maxRetriesMultiplier; private MetricsSource metrics; - private ReplicationSourceManager manager; + private AtomicLong totalBufferUsed; + private long totalBufferQuota; /** * Creates a reader worker for a given WAL queue. Reads WAL entries off a given queue, batches the @@ -106,7 +109,9 @@ public ReplicationSourceWALReaderThread(ReplicationSourceManager manager, // memory used will be batchSizeCapacity * (nb.batches + 1) // the +1 is for the current thread reading before placing onto the queue int batchCount = conf.getInt("replication.source.nb.batches", 1); - this.manager = manager; + this.totalBufferUsed = manager.getTotalBufferUsed(); + this.totalBufferQuota = conf.getLong(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY, + HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_DFAULT); this.sleepForRetries = this.conf.getLong("replication.source.sleepforretries", 1000); // 1 second this.maxRetriesMultiplier = @@ -127,12 +132,10 @@ public void run() { try (WALEntryStream entryStream = new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) { while (isReaderRunning()) { // loop here to keep reusing stream while we can - if (manager.isBufferQuotaReached()) { - Threads.sleep(sleepForRetries); + if (!checkQuota()) { continue; } - WALEntryBatch batch = - new WALEntryBatch(replicationBatchCountCapacity, replicationBatchSizeCapacity); + WALEntryBatch batch = new WALEntryBatch(replicationBatchCountCapacity); boolean hasNext; while ((hasNext = entryStream.hasNext()) == true) { Entry entry = entryStream.next(); @@ -140,10 +143,14 @@ public void run() { if (entry != null) { WALEdit edit = entry.getEdit(); if (edit != null && !edit.isEmpty()) { - long entrySizeExcludeBulkLoad = batch.addEntry(entry); - boolean totalBufferTooLarge = manager.acquireBufferQuota(entrySizeExcludeBulkLoad); + long entrySize = getEntrySizeIncludeBulkLoad(entry); + long entrySizeExcludeBulkLoad = getEntrySizeExcludeBulkLoad(entry); + batch.addEntry(entry); + updateBatchStats(batch, entry, entryStream.getPosition(), entrySize); + boolean totalBufferTooLarge = acquireBufferQuota(entrySizeExcludeBulkLoad); // Stop if too many entries or too big - if (totalBufferTooLarge || batch.isLimitReached()) { + if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity + || batch.getNbEntries() >= replicationBatchCountCapacity) { break; } } @@ -226,6 +233,16 @@ public Path getCurrentPath() { return logQueue.peek(); } + //returns false if we've already exceeded the global quota + private boolean checkQuota() { + // try not to go over total quota + if (totalBufferUsed.get() > totalBufferQuota) { + Threads.sleep(sleepForRetries); + return false; + } + return true; + } + private Entry filterEntry(Entry entry) { Entry filtered = filter.filter(entry); if (entry != null && filtered == null) { @@ -248,6 +265,107 @@ public WALEntryBatch poll(long timeout) throws InterruptedException { return entryBatchQueue.poll(timeout, TimeUnit.MILLISECONDS); } + private long getEntrySizeIncludeBulkLoad(Entry entry) { + WALEdit edit = entry.getEdit(); + WALKey key = entry.getKey(); + return edit.heapSize() + sizeOfStoreFilesIncludeBulkLoad(edit) + + key.estimatedSerializedSizeOf(); + } + + public long getEntrySizeExcludeBulkLoad(Entry entry) { + WALEdit edit = entry.getEdit(); + WALKey key = entry.getKey(); + return edit.heapSize() + key.estimatedSerializedSizeOf(); + } + + private void updateBatchStats(WALEntryBatch batch, Entry entry, long entryPosition, long entrySize) { + WALEdit edit = entry.getEdit(); + if (edit != null && !edit.isEmpty()) { + batch.incrementHeapSize(entrySize); + Pair nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit); + batch.incrementNbRowKeys(nbRowsAndHFiles.getFirst()); + batch.incrementNbHFiles(nbRowsAndHFiles.getSecond()); + } + batch.lastWalPosition = entryPosition; + } + + /** + * Count the number of different row keys in the given edit because of mini-batching. We assume + * that there's at least one Cell in the WALEdit. + * @param edit edit to count row keys from + * @return number of different row keys and HFiles + */ + private Pair countDistinctRowKeysAndHFiles(WALEdit edit) { + List cells = edit.getCells(); + int distinctRowKeys = 1; + int totalHFileEntries = 0; + Cell lastCell = cells.get(0); + + int totalCells = edit.size(); + for (int i = 0; i < totalCells; i++) { + // Count HFiles to be replicated + if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { + try { + BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); + List stores = bld.getStoresList(); + int totalStores = stores.size(); + for (int j = 0; j < totalStores; j++) { + totalHFileEntries += stores.get(j).getStoreFileList().size(); + } + } catch (IOException e) { + LOG.error("Failed to deserialize bulk load entry from wal edit. " + + "Then its hfiles count will not be added into metric."); + } + } + + if (!CellUtil.matchingRow(cells.get(i), lastCell)) { + distinctRowKeys++; + } + lastCell = cells.get(i); + } + + Pair result = new Pair<>(distinctRowKeys, totalHFileEntries); + return result; + } + + /** + * Calculate the total size of all the store files + * @param edit edit to count row keys from + * @return the total size of the store files + */ + private int sizeOfStoreFilesIncludeBulkLoad(WALEdit edit) { + List cells = edit.getCells(); + int totalStoreFilesSize = 0; + + int totalCells = edit.size(); + for (int i = 0; i < totalCells; i++) { + if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { + try { + BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); + List stores = bld.getStoresList(); + int totalStores = stores.size(); + for (int j = 0; j < totalStores; j++) { + totalStoreFilesSize += stores.get(j).getStoreFileSizeBytes(); + } + } catch (IOException e) { + LOG.error("Failed to deserialize bulk load entry from wal edit. " + + "Size of HFiles part of cell will not be considered in replication " + + "request size calculation.", + e); + } + } + } + return totalStoreFilesSize; + } + + /** + * @param size delta size for grown buffer + * @return true if we should clear buffer and push all + */ + private boolean acquireBufferQuota(long size) { + return totalBufferUsed.addAndGet(size) >= totalBufferQuota; + } + /** * @return whether the reader thread is running */ @@ -267,8 +385,6 @@ public void setReaderRunning(boolean readerRunning) { * */ static class WALEntryBatch { - private final int maxNbEntries; - private final long maxSizeBytes; private List walEntries; // last WAL that was read private Path lastWalPath; @@ -282,22 +398,16 @@ static class WALEntryBatch { private long heapSize = 0; // whether more entries to read exist in WALs or not private boolean moreEntries = true; - // heap size of data we need to replicate, the size of bulk loaded hfiles is not included. - private long heapSizeExcludeBulkLoad; /** * @param maxNbEntries the number of entries a batch can have - * @param maxSizeBytes max (heap) size of each batch */ - private WALEntryBatch(int maxNbEntries, long maxSizeBytes) { + private WALEntryBatch(int maxNbEntries) { this.walEntries = new ArrayList<>(maxNbEntries); - this.maxNbEntries = maxNbEntries; - this.maxSizeBytes = maxSizeBytes; } - public long addEntry(Entry entry) { + public void addEntry(Entry entry) { walEntries.add(entry); - return updateBatchStats(entry); } /** @@ -353,10 +463,6 @@ public long getHeapSize() { return heapSize; } - public long getHeapSizeExcludeBulkLoad() { - return heapSizeExcludeBulkLoad; - } - private void incrementNbRowKeys(int increment) { nbRowKeys += increment; } @@ -369,10 +475,6 @@ private void incrementHeapSize(long increment) { heapSize += increment; } - private void incrementHeapSizeExcludeBulkLoad(long increment) { - heapSizeExcludeBulkLoad += increment; - } - public boolean isEmpty() { return walEntries.isEmpty(); } @@ -389,99 +491,5 @@ public boolean hasMoreEntries() { public void setMoreEntries(boolean moreEntries) { this.moreEntries = moreEntries; } - - public boolean isLimitReached() { - return getHeapSize() >= maxSizeBytes || getNbEntries() >= maxNbEntries; - } - - private long getEntrySizeExcludeBulkLoad(Entry entry) { - WALEdit edit = entry.getEdit(); - WALKey key = entry.getKey(); - return edit.heapSize() + key.estimatedSerializedSizeOf(); - } - - private long updateBatchStats(Entry entry) { - WALEdit edit = entry.getEdit(); - - long entrySizeExcludeBulkLoad = getEntrySizeExcludeBulkLoad(entry); - long entrySize = entrySizeExcludeBulkLoad + sizeOfStoreFilesIncludeBulkLoad(entry.getEdit()); - incrementHeapSize(entrySize); - incrementHeapSizeExcludeBulkLoad(entrySizeExcludeBulkLoad); - - Pair nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit); - incrementNbRowKeys(nbRowsAndHFiles.getFirst()); - incrementNbHFiles(nbRowsAndHFiles.getSecond()); - - return entrySizeExcludeBulkLoad; - } - - /** - * Count the number of different row keys in the given edit because of mini-batching. We assume - * that there's at least one Cell in the WALEdit. - * @param edit edit to count row keys from - * @return number of different row keys and HFiles - */ - private Pair countDistinctRowKeysAndHFiles(WALEdit edit) { - List cells = edit.getCells(); - int distinctRowKeys = 1; - int totalHFileEntries = 0; - Cell lastCell = cells.get(0); - - int totalCells = edit.size(); - for (int i = 0; i < totalCells; i++) { - // Count HFiles to be replicated - if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { - try { - BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); - List stores = bld.getStoresList(); - int totalStores = stores.size(); - for (int j = 0; j < totalStores; j++) { - totalHFileEntries += stores.get(j).getStoreFileList().size(); - } - } catch (IOException e) { - LOG.error("Failed to deserialize bulk load entry from wal edit. " - + "Then its hfiles count will not be added into metric."); - } - } - - if (!CellUtil.matchingRow(cells.get(i), lastCell)) { - distinctRowKeys++; - } - lastCell = cells.get(i); - } - - Pair result = new Pair<>(distinctRowKeys, totalHFileEntries); - return result; - } - - /** - * Calculate the total size of all the store files - * @param edit edit to count row keys from - * @return the total size of the store files - */ - private int sizeOfStoreFilesIncludeBulkLoad(WALEdit edit) { - List cells = edit.getCells(); - int totalStoreFilesSize = 0; - - int totalCells = edit.size(); - for (int i = 0; i < totalCells; i++) { - if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) { - try { - BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i)); - List stores = bld.getStoresList(); - int totalStores = stores.size(); - for (int j = 0; j < totalStores; j++) { - totalStoreFilesSize += stores.get(j).getStoreFileSizeBytes(); - } - } catch (IOException e) { - LOG.error("Failed to deserialize bulk load entry from wal edit. " - + "Size of HFiles part of cell will not be considered in replication " - + "request size calculation.", - e); - } - } - } - return totalStoreFilesSize; - } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index a2e8c89c1ca1..44f11afe532a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -43,6 +43,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -266,6 +267,7 @@ private static final class Mocks { private Mocks() { when(peers.getStatusOfPeer(anyString())).thenReturn(true); when(context.getReplicationPeer()).thenReturn(peer); + when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); } ReplicationSource createReplicationSourceWithMocks(ReplicationEndpoint endpoint) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java index 7d017e5e404a..641ea50b38d6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -123,9 +124,10 @@ public void testQuota() throws IOException { public void run() { Replication replication = (Replication) utility1.getMiniHBaseCluster() .getRegionServer(0).getReplicationSourceService(); + AtomicLong bufferUsed = replication.getReplicationManager().getTotalBufferUsed(); testQuotaPass = true; while (!Thread.interrupted()) { - long size = replication.getReplicationManager().getTotalBufferUsed(); + long size = bufferUsed.get(); //the reason here doing "numOfPeer + 1" is because by using method addEntryToBatch(), even the // batch size (after added last entry) exceeds quota, it still keeps the last one in the batch // so total used buffer size can be one "replication.total.buffer.quota" larger than expected diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index 71706a9a3957..fcddcafcd75f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -42,6 +42,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -359,6 +360,7 @@ public void testReplicationSourceWALReaderThread() throws Exception { // start up a batcher ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class); + when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); ReplicationSourceWALReaderThread batcher = new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); @@ -397,6 +399,7 @@ fs, conf, new MetricsSource("1"))) { } ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); + when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo("1-1"), walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); @@ -449,6 +452,7 @@ public void testReplicationSourceWALReaderThreadWithFilter() throws Exception { Path firstWAL = walQueue.peek(); ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); + when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); final ReplicationSourceWALReaderThread reader = new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), walQueue, 0, fs, conf, filter, new MetricsSource("1")); From 577db5d7e50c56b4773c9ce92b807aae80bf5706 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Thu, 31 Oct 2019 17:11:20 +0000 Subject: [PATCH 08/51] HBASE-23238 Additional test and checks for null references on ScannerCallableWithReplicas (#780) Signed-off-by: Sean Busbey --- .../client/ScannerCallableWithReplicas.java | 11 ++- .../client/TestScannersFromClientSide.java | 74 +++++++++++++++++++ 2 files changed, 84 insertions(+), 1 deletion(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java index 10c20d7aa24b..50cfe1cae877 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java @@ -94,7 +94,12 @@ public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConne } public void setClose() { - currentScannerCallable.setClose(); + if(currentScannerCallable != null) { + currentScannerCallable.setClose(); + } else { + LOG.warn("Calling close on ScannerCallable reference that is already null, " + + "which shouldn't happen."); + } } public void setRenew(boolean val) { @@ -136,6 +141,10 @@ public MoreResults moreResultsForScan() { Result[] r = currentScannerCallable.call(timeout); currentScannerCallable = null; return r; + } else if(currentScannerCallable == null) { + LOG.warn("Another call received, but our ScannerCallable is already null. " + + "This shouldn't happen, but there's not much to do, so logging and returning null."); + return null; } // We need to do the following: //1. When a scan goes out to a certain replica (default or not), we need to diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index 4bac32db5551..eb2863f6ce31 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -737,4 +739,76 @@ public void testScanWithSameStartRowStopRow() throws IOException { } } } + + @Test + public void testScannerWithPartialResults() throws Exception { + TableName tableName = TableName.valueOf("testScannerWithPartialResults"); + try (Table table = TEST_UTIL.createMultiRegionTable(tableName, + Bytes.toBytes("c"), 4)) { + List puts = new ArrayList<>(); + byte[] largeArray = new byte[10000]; + Put put = new Put(Bytes.toBytes("aaaa0")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), Bytes.toBytes("1")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("2"), Bytes.toBytes("2")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("3"), Bytes.toBytes("3")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("4"), Bytes.toBytes("4")); + puts.add(put); + put = new Put(Bytes.toBytes("aaaa1")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), Bytes.toBytes("1")); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("2"), largeArray); + put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("3"), largeArray); + puts.add(put); + table.put(puts); + Scan scan = new Scan(); + scan.addFamily(Bytes.toBytes("c")); + scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, tableName.getName()); + scan.setMaxResultSize(10001); + scan.setStopRow(Bytes.toBytes("bbbb")); + scan.setFilter(new LimitKVsReturnFilter()); + ResultScanner rs = table.getScanner(scan); + Result result; + int expectedKvNumber = 6; + int returnedKvNumber = 0; + while((result = rs.next()) != null){ + returnedKvNumber += result.listCells().size(); + } + rs.close(); + assertEquals(expectedKvNumber, returnedKvNumber); + } + } + + public static class LimitKVsReturnFilter extends FilterBase { + + private static int total = 0; + + @Override + public ReturnCode filterKeyValue(Cell v) throws IOException { + if(total>=6) { + total++; + return ReturnCode.SKIP; + } + total++; + return ReturnCode.INCLUDE; + } + + @Override + public boolean filterAllRemaining() throws IOException { + if(total<7) { + return false; + } + total++; + return true; + } + + @Override + public String toString() { + return this.getClass().getSimpleName(); + } + + public static LimitKVsReturnFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + return new LimitKVsReturnFilter(); + } + } + } From 3c7c1b5489edaa307c46c461468dcadc2f5c0b1a Mon Sep 17 00:00:00 2001 From: bitterfox Date: Fri, 1 Nov 2019 07:23:47 +0900 Subject: [PATCH 09/51] HBASE-23185 Fix high cpu usage because getTable()#put() gets config value every time [Take2] (#748) * HBASE-23185 Fix test failure by HBASE-23185 changes * HBASE-23185 Fix high cpu usage because getTable()#put() gets config value every time This reverts commit db2ce23a9332cdd89af5025d4a9f18f37a202350. --- .../hadoop/hbase/client/AsyncProcess.java | 90 ++++++++++++------- .../hbase/client/BufferedMutatorImpl.java | 25 +++--- .../hbase/client/ConnectionConfiguration.java | 48 +++++++++- .../hbase/client/ConnectionManager.java | 13 +-- .../apache/hadoop/hbase/client/HTable.java | 8 +- .../client/HConnectionTestingUtility.java | 9 +- 6 files changed, 128 insertions(+), 65 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 4c571e486599..0c2247349a24 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -323,35 +323,60 @@ public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService po this.id = COUNTER.incrementAndGet(); - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); - if (configuredPauseForCQTBE < pause) { - LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " - + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE - + ", will use " + pause + " instead."); - this.pauseForCQTBE = pause; - } else { - this.pauseForCQTBE = configuredPauseForCQTBE; - } - this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, - HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); + ConnectionConfiguration connConf = + hc.getConfiguration() == conf + ? hc.getConnectionConfiguration() + // Slow: parse conf in ConnectionConfiguration constructor + : new ConnectionConfiguration(conf); + if (connConf == null) { + // Slow: parse conf in ConnectionConfiguration constructor + connConf = new ConnectionConfiguration(conf); + } + + this.pause = connConf.getPause(); + this.pauseForCQTBE = connConf.getPauseForCQTBE(); + + this.numTries = connConf.getRetriesNumber(); this.rpcTimeout = rpcTimeout; - this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000); - - this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS); - this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS); - this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS); - this.maxHeapSizePerRequest = conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, - DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); - this.maxHeapSizeSubmit = conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE); + this.operationTimeout = connConf.getOperationTimeout(); + + // Parse config once and reuse config values of hc's AsyncProcess in AsyncProcess for put + // Can be null when constructing hc's AsyncProcess or it's not reusable + AsyncProcess globalAsyncProcess = hc.getConfiguration() == conf ? hc.getAsyncProcess() : null; + + this.primaryCallTimeoutMicroseconds = + globalAsyncProcess == null + ? conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000) + : globalAsyncProcess.primaryCallTimeoutMicroseconds; + + this.maxTotalConcurrentTasks = + globalAsyncProcess == null + ? conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS) + : globalAsyncProcess.maxTotalConcurrentTasks; + this.maxConcurrentTasksPerServer = + globalAsyncProcess == null + ? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS) + : globalAsyncProcess.maxConcurrentTasksPerServer; + this.maxConcurrentTasksPerRegion = + globalAsyncProcess == null + ? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS) + : globalAsyncProcess.maxConcurrentTasksPerRegion; + this.maxHeapSizePerRequest = + globalAsyncProcess == null + ? conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, + DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE) + : globalAsyncProcess.maxHeapSizePerRequest; + this.maxHeapSizeSubmit = + globalAsyncProcess == null + ? conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE) + : globalAsyncProcess.maxHeapSizeSubmit; this.startLogErrorsCnt = - conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT); + globalAsyncProcess == null + ? conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT) + : globalAsyncProcess.startLogErrorsCnt; if (this.maxTotalConcurrentTasks <= 0) { throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks); @@ -387,11 +412,16 @@ public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService po this.rpcCallerFactory = rpcCaller; this.rpcFactory = rpcFactory; - this.logBatchErrorDetails = conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false); + this.logBatchErrorDetails = + globalAsyncProcess == null + ? conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false) + : globalAsyncProcess.logBatchErrorDetails; this.thresholdToLogUndoneTaskDetails = - conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS, - DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS); + globalAsyncProcess == null + ? conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS, + DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS) + : globalAsyncProcess.thresholdToLogUndoneTaskDetails; } public void setRpcTimeout(int rpcTimeout) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index e33bd7ce369f..dafc66fd03a1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -34,7 +34,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; // Needed for write rpc timeout import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -112,32 +111,32 @@ public class BufferedMutatorImpl implements BufferedMutator { this.pool = params.getPool(); this.listener = params.getListener(); - ConnectionConfiguration tableConf = new ConnectionConfiguration(conf); + ConnectionConfiguration connConf = conn.getConnectionConfiguration(); + if (connConf == null) { + // Slow: parse conf in ConnectionConfiguration constructor + connConf = new ConnectionConfiguration(conf); + } this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ? - params.getWriteBufferSize() : tableConf.getWriteBufferSize(); + params.getWriteBufferSize() : connConf.getWriteBufferSize(); // Set via the setter because it does value validation and starts/stops the TimerTask long newWriteBufferPeriodicFlushTimeoutMs = params.getWriteBufferPeriodicFlushTimeoutMs() != UNSET ? params.getWriteBufferPeriodicFlushTimeoutMs() - : tableConf.getWriteBufferPeriodicFlushTimeoutMs(); + : connConf.getWriteBufferPeriodicFlushTimeoutMs(); long newWriteBufferPeriodicFlushTimerTickMs = params.getWriteBufferPeriodicFlushTimerTickMs() != UNSET ? params.getWriteBufferPeriodicFlushTimerTickMs() - : tableConf.getWriteBufferPeriodicFlushTimerTickMs(); + : connConf.getWriteBufferPeriodicFlushTimerTickMs(); this.setWriteBufferPeriodicFlush( newWriteBufferPeriodicFlushTimeoutMs, newWriteBufferPeriodicFlushTimerTickMs); this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ? - params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize(); - - this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.operationTimeout = conn.getConfiguration().getInt( - HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); + params.getMaxKeyValueSize() : connConf.getMaxKeyValueSize(); + + this.writeRpcTimeout = connConf.getWriteRpcTimeout(); + this.operationTimeout = connConf.getOperationTimeout(); // puts need to track errors globally due to how the APIs currently work. ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, writeRpcTimeout); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java index 0e5164447e28..1189802edf02 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java @@ -12,6 +12,8 @@ package org.apache.hadoop.hbase.client; import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -26,6 +28,7 @@ */ @InterfaceAudience.Private public class ConnectionConfiguration { + static final Log LOG = LogFactory.getLog(ConnectionConfiguration.class); public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer"; public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152; @@ -50,6 +53,10 @@ public class ConnectionConfiguration { private final int metaReplicaCallTimeoutMicroSecondScan; private final int retries; private final int maxKeyValueSize; + private final int readRpcTimeout; + private final int writeRpcTimeout; + private final long pause; + private final long pauseForCQTBE; /** * Constructor @@ -90,9 +97,28 @@ public class ConnectionConfiguration { HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT); this.retries = conf.getInt( - HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); + HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT); + + this.readRpcTimeout = conf.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, + conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + + this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, + conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + + this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); + if (configuredPauseForCQTBE < pause) { + LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " + + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE + + ", will use " + pause + " instead."); + this.pauseForCQTBE = pause; + } else { + this.pauseForCQTBE = configuredPauseForCQTBE; + } } /** @@ -115,6 +141,10 @@ protected ConnectionConfiguration() { HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT; this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER; this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT; + this.readRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; + this.writeRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; + this.pause = HConstants.DEFAULT_HBASE_CLIENT_PAUSE; + this.pauseForCQTBE = HConstants.DEFAULT_HBASE_CLIENT_PAUSE; } public long getWriteBufferSize() { @@ -164,4 +194,20 @@ public int getMaxKeyValueSize() { public long getScannerMaxResultSize() { return scannerMaxResultSize; } + + public int getReadRpcTimeout() { + return readRpcTimeout; + } + + public int getWriteRpcTimeout() { + return writeRpcTimeout; + } + + public long getPause() { + return pause; + } + + public long getPauseForCQTBE() { + return pauseForCQTBE; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index 7d9af647feb6..e8498c4c597b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -669,17 +669,8 @@ static class HConnectionImplementation implements ClusterConnection, Closeable { this.managed = managed; this.connectionConfig = new ConnectionConfiguration(conf); this.closed = false; - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); - if (configuredPauseForCQTBE < pause) { - LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " - + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE - + ", will use " + pause + " instead."); - this.pauseForCQTBE = pause; - } else { - this.pauseForCQTBE = configuredPauseForCQTBE; - } + this.pause = connectionConfig.getPause(); + this.pauseForCQTBE = connectionConfig.getPauseForCQTBE(); this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS, HConstants.DEFAULT_USE_META_REPLICAS); this.metaReplicaCallTimeoutScanInMicroSecond = diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 91a8f922bfb1..cbb7b01bcee7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -366,12 +366,8 @@ private void finishSetup() throws IOException { } this.operationTimeout = tableName.isSystemTable() ? connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout(); - this.readRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, - configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.writeRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + this.readRpcTimeout = connConfiguration.getReadRpcTimeout(); + this.writeRpcTimeout = connConfiguration.getWriteRpcTimeout(); this.scannerCaching = connConfiguration.getScannerCaching(); this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); if (this.rpcCallerFactory == null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index 2e3c7261f462..6c4464fde815 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -162,10 +162,11 @@ public static ClusterConnection getMockedConnectionAndDecorate(final Configurati } NonceGenerator ng = Mockito.mock(NonceGenerator.class); Mockito.when(c.getNonceGenerator()).thenReturn(ng); - Mockito.when(c.getAsyncProcess()).thenReturn( - new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false, - RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT))); + AsyncProcess asyncProcess = + new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false, + RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + Mockito.when(c.getAsyncProcess()).thenReturn(asyncProcess); Mockito.doNothing().when(c).incCount(); Mockito.doNothing().when(c).decCount(); Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn( From 245102332773db0b721aad72b3d8af9707ec8aa2 Mon Sep 17 00:00:00 2001 From: Thiruvel Thirumoolan Date: Fri, 25 Oct 2019 14:55:04 -0700 Subject: [PATCH 10/51] HBASE-23219 Re-enable ZKLess tests for branch-1 (Revert HBASE-14622) Signed-off-by: Andrew Purtell --- .../hbase/master/TestZKLessAMOnCluster.java | 42 +++++++++++++++++++ .../namespace/TestZKLessNamespaceAuditor.java | 33 +++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestZKLessNamespaceAuditor.java diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java new file mode 100644 index 000000000000..0e49f1cb2874 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKLessAMOnCluster.java @@ -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. + */ +package org.apache.hadoop.hbase.master; + +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +/** + * This tests AssignmentManager with a testing cluster. + */ +@Category(MediumTests.class) +public class TestZKLessAMOnCluster extends TestAssignmentManagerOnCluster { + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // Don't use ZK for region assignment + conf.setBoolean("hbase.assignment.usezk", false); + setupOnce(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TestAssignmentManagerOnCluster.tearDownAfterClass(); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestZKLessNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestZKLessNamespaceAuditor.java new file mode 100644 index 000000000000..ede19eca70c7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestZKLessNamespaceAuditor.java @@ -0,0 +1,33 @@ +/** + * 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.namespace; + +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +@Category(MediumTests.class) +public class TestZKLessNamespaceAuditor extends TestNamespaceAuditor { + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void before() throws Exception { + UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false); + setupOnce(); + } + + @AfterClass + public static void tearDown() throws Exception { + TestNamespaceAuditor.tearDown(); + } +} From 3f9ce864b554e33e11bd2e4be31fd6a6149491df Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Mon, 4 Nov 2019 10:08:11 -0800 Subject: [PATCH 11/51] HBASE-23246 Fix error prone warning in TestMetricsUserSourceImpl (#789) Signed-off-by: Sean Busbey Signed-off-by: Guangxu Cheng --- .../hadoop/hbase/regionserver/TestMetricsUserSourceImpl.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsUserSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsUserSourceImpl.java index 3155e66fc985..aa8c3d75ee68 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsUserSourceImpl.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsUserSourceImpl.java @@ -47,10 +47,8 @@ public void testCompareToHashCodeEquals() throws Exception { assertTrue(one.compareTo(two) != 0); assertTrue(two.compareTo(one) != 0); assertTrue(two.compareTo(one) != one.compareTo(two)); - assertTrue(two.compareTo(two) == 0); } - @Test (expected = RuntimeException.class) public void testNoGetRegionServerMetricsSourceImpl() throws Exception { // This should throw an exception because MetricsUserSourceImpl should only From 75620b0c833aee225641f743d494be855401950b Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Mon, 4 Nov 2019 23:26:53 +0900 Subject: [PATCH 12/51] (fix) Change newly added method name --- .../regionserver/ReplicationSource.java | 26 +++++++++---------- .../TestReplicationSmallTests.java | 4 +-- .../replication/TestReplicationSource.java | 4 +-- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 9aa4c776fe9f..5acb70922f83 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -449,17 +449,17 @@ public Path getCurrentPath() { } @VisibleForTesting - public Path getCurrentReadPath() { + public Path getLastLoggedPath() { for (ReplicationSourceShipperThread worker : workerThreads.values()) { - return worker.getCurrentReadPath(); + return worker.getLastLoggedPath(); } return null; } @VisibleForTesting - public long getCurrentPosition() { + public long getLastLoggedPosition() { for (ReplicationSourceShipperThread worker : workerThreads.values()) { - return worker.getCurrentPosition(); + return worker.getLastLoggedPosition(); } return 0; } @@ -498,8 +498,8 @@ public String getStats() { for (Map.Entry entry : workerThreads.entrySet()) { String walGroupId = entry.getKey(); ReplicationSourceShipperThread worker = entry.getValue(); - long position = worker.getCurrentPosition(); - Path currentPath = worker.getCurrentPath(); + long position = worker.getLastLoggedPosition(); + Path currentPath = worker.getLastLoggedPath(); sb.append("walGroup [").append(walGroupId).append("]: "); if (currentPath != null) { sb.append("currently replicating from: ").append(currentPath).append(" at position: ") @@ -534,7 +534,7 @@ public Map getWalGroupStatus() { int queueSize = queues.get(walGroupId).size(); replicationDelay = ReplicationLoad.calculateReplicationDelay(ageOfLastShippedOp, lastTimeStamp, queueSize); - Path currentPath = worker.getCurrentPath(); + Path currentPath = worker.getLastLoggedPath(); fileSize = -1; if (currentPath != null) { try { @@ -552,7 +552,7 @@ public Map getWalGroupStatus() { .withQueueSize(queueSize) .withWalGroup(walGroupId) .withCurrentPath(currentPath) - .withCurrentPosition(worker.getCurrentPosition()) + .withCurrentPosition(worker.getLastLoggedPosition()) .withFileSize(fileSize) .withAgeOfLastShippedOp(ageOfLastShippedOp) .withReplicationDelay(replicationDelay); @@ -812,7 +812,7 @@ public void startup() { public void uncaughtException(final Thread t, final Throwable e) { RSRpcServices.exitIfOOME(e); LOG.error("Unexpected exception in ReplicationSourceWorkerThread," + " currentPath=" - + getCurrentPath(), e); + + getLastLoggedPath(), e); stopper.stop("Unexpected exception in ReplicationSourceWorkerThread"); } }; @@ -949,15 +949,15 @@ private Path getReplSyncUpPath(Path path) throws IOException { return path; } - public Path getCurrentReadPath() { - return entryReader.getCurrentPath(); + public Path getCurrentPath() { + return this.entryReader.getCurrentPath(); } - public Path getCurrentPath() { + public Path getLastLoggedPath() { return lastLoggedPath; } - public long getCurrentPosition() { + public long getLastLoggedPosition() { return lastLoggedPosition; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index fe54dfc6dd55..f94ad5a1b468 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -879,10 +879,10 @@ public boolean evaluate() throws Exception { for (ReplicationSourceInterface rsi : replicationService.getReplicationManager() .getSources()) { ReplicationSource source = (ReplicationSource) rsi; - if (!invert && !emptyWalPaths.get(i).equals(source.getCurrentReadPath())) { + if (!invert && !emptyWalPaths.get(i).equals(source.getCurrentPath())) { return false; } - if (invert && emptyWalPaths.get(i).equals(source.getCurrentReadPath())) { + if (invert && emptyWalPaths.get(i).equals(source.getCurrentPath())) { return false; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 44f11afe532a..d3f7ddfc9b91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -348,7 +348,7 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exc Waiter.waitFor(conf, 20000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - return log2.equals(source.getCurrentPath()) && source.getCurrentPosition() >= startPos; + return log2.equals(source.getLastLoggedPath()) && source.getLastLoggedPosition() >= startPos; } }); @@ -390,7 +390,7 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exc Waiter.waitFor(conf, 20000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { // wait until reader read all cells - return log2.equals(source.getCurrentPath()) && source.getCurrentPosition() >= pos; + return log2.equals(source.getLastLoggedPath()) && source.getLastLoggedPosition() >= pos; } }); From c92d79e8ca50cd9a369d49d0d0d28eb43a795988 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Tue, 5 Nov 2019 17:50:01 +0900 Subject: [PATCH 13/51] (fix) add getRecoveredQueueInfo() to make a test more recognizable --- .../hbase/replication/regionserver/TestWALEntryStream.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index fcddcafcd75f..8ad35220893a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -401,7 +401,7 @@ fs, conf, new MetricsSource("1"))) { ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class); when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); ReplicationSourceWALReaderThread reader = - new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo("1-1"), + new ReplicationSourceWALReaderThread(mockSourceManager, getRecoveredQueueInfo(), walQueue, 0, fs, conf, getDummyFilter(), new MetricsSource("1")); Path walPath = walQueue.toArray(new Path[2])[1]; reader.start(); @@ -552,6 +552,10 @@ public Entry filter(Entry entry) { }; } + private ReplicationQueueInfo getRecoveredQueueInfo() { + return getQueueInfo("1-1"); + } + private ReplicationQueueInfo getQueueInfo() { return getQueueInfo("1"); } From d3ed53393fb6b7f03fc782896faf65fb12595b61 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Tue, 5 Nov 2019 23:23:42 +0900 Subject: [PATCH 14/51] (fix) a check style warning --- .../apache/hadoop/hbase/replication/TestReplicationSource.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index d3f7ddfc9b91..b4ac71bf5c8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -348,7 +348,8 @@ public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exc Waiter.waitFor(conf, 20000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - return log2.equals(source.getLastLoggedPath()) && source.getLastLoggedPosition() >= startPos; + return log2.equals(source.getLastLoggedPath()) + && source.getLastLoggedPosition() >= startPos; } }); From 13608167866dcaff02218952373ac02f252c94f5 Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Tue, 5 Nov 2019 20:21:16 +0100 Subject: [PATCH 15/51] HBASE-23250 Log message about CleanerChore delegate initialization should be at INFO Signed-off-by: Jan Hentschel --- .../apache/hadoop/hbase/master/cleaner/CleanerChore.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java index 78be50bd8ae9..50cd98040399 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java @@ -150,7 +150,7 @@ private void initCleanerChain(String confKey) { for (String className : logCleaners) { T logCleaner = newFileCleaner(className, conf); if (logCleaner != null) { - LOG.debug("initialize cleaner=" + className); + LOG.info("initialize cleaner=" + className); this.cleanersChain.add(logCleaner); } } @@ -254,7 +254,7 @@ private boolean checkAndDeleteFiles(List files) { } Iterable filteredFiles = cleaner.getDeletableFiles(deletableValidFiles); - + // trace which cleaner is holding on to each file if (LOG.isTraceEnabled()) { ImmutableSet filteredFileSet = ImmutableSet.copyOf(filteredFiles); @@ -264,10 +264,10 @@ private boolean checkAndDeleteFiles(List files) { } } } - + deletableValidFiles = filteredFiles; } - + Iterable filesToDelete = Iterables.concat(invalidFiles, deletableValidFiles); return deleteFiles(filesToDelete) == files.size(); } From a5f09cd7568561856ed84759ccf1b6f6bcfd9d3d Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 5 Nov 2019 11:13:44 +0530 Subject: [PATCH 16/51] HBASE-23212 Dynamically reload configs for Region Recovery chore (#803) Signed-off-by: Andrew Purtell --- .../org/apache/hadoop/hbase/HConstants.java | 5 + .../apache/hadoop/hbase/master/HMaster.java | 5 + .../hbase/master/RegionsRecoveryChore.java | 25 +++- .../master/RegionsRecoveryConfigManager.java | 100 ++++++++++++++ .../TestRegionsRecoveryConfigManager.java | 126 ++++++++++++++++++ 5 files changed, 254 insertions(+), 7 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryConfigManager.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 59d4fb3ae39c..16d96a2fc5e5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1348,6 +1348,11 @@ public static enum Modify { // default -1 indicates there is no threshold on high storeRefCount public static final int DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD = -1; + public static final String REGIONS_RECOVERY_INTERVAL = + "hbase.master.regions.recovery.check.interval"; + + public static final int DEFAULT_REGIONS_RECOVERY_INTERVAL = 1200 * 1000; // Default 20 min + /** * Configurations for master executor services. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index c6bbf248bccf..278d86b3555b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -305,6 +305,8 @@ public void run() { private RegionsRecoveryChore regionsRecoveryChore = null; + private RegionsRecoveryConfigManager regionsRecoveryConfigManager = null; + // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting // operations/debugging. @@ -912,6 +914,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) configurationManager.registerObserver(this.cleanerPool); configurationManager.registerObserver(this.hfileCleaner); configurationManager.registerObserver(this.logCleaner); + configurationManager.registerObserver(this.regionsRecoveryConfigManager); // Set master as 'initialized'. setInitialized(true); @@ -1289,6 +1292,8 @@ private void startServiceThreads() throws IOException { } } + this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this); + serviceStarted = true; if (LOG.isTraceEnabled()) { LOG.trace("Started service threads"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryChore.java index 78d4b785bd35..06ca30f4e956 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryChore.java @@ -52,11 +52,6 @@ public class RegionsRecoveryChore extends ScheduledChore { private static final String REGIONS_RECOVERY_CHORE_NAME = "RegionsRecoveryChore"; - private static final String REGIONS_RECOVERY_INTERVAL = - "hbase.master.regions.recovery.check.interval"; - - private static final int DEFAULT_REGIONS_RECOVERY_INTERVAL = 1200 * 1000; // Default 20 min ? - private static final String ERROR_REOPEN_REIONS_MSG = "Error reopening regions with high storeRefCount. "; @@ -76,8 +71,8 @@ public class RegionsRecoveryChore extends ScheduledChore { RegionsRecoveryChore(final Stoppable stopper, final Configuration configuration, final HMaster hMaster) { - super(REGIONS_RECOVERY_CHORE_NAME, stopper, configuration.getInt(REGIONS_RECOVERY_INTERVAL, - DEFAULT_REGIONS_RECOVERY_INTERVAL)); + super(REGIONS_RECOVERY_CHORE_NAME, stopper, configuration.getInt( + HConstants.REGIONS_RECOVERY_INTERVAL, HConstants.DEFAULT_REGIONS_RECOVERY_INTERVAL)); this.hMaster = hMaster; this.storeFileRefCountThreshold = configuration.getInt( HConstants.STORE_FILE_REF_COUNT_THRESHOLD, @@ -180,4 +175,20 @@ private void prepareTableToReopenRegionsMap( } + // hashcode/equals implementation to ensure at-most one object of RegionsRecoveryChore + // is scheduled at a time - RegionsRecoveryConfigManager + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + return o != null && getClass() == o.getClass(); + } + + @Override + public int hashCode() { + return 31; + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryConfigManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryConfigManager.java new file mode 100644 index 000000000000..7bf078f5fa48 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionsRecoveryConfigManager.java @@ -0,0 +1,100 @@ +/* + * 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.master; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.conf.ConfigurationObserver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Config manager for RegionsRecovery Chore - Dynamically reload config and update chore + * accordingly + */ +@InterfaceAudience.Private +public class RegionsRecoveryConfigManager implements ConfigurationObserver { + + private static final Logger LOG = LoggerFactory.getLogger(RegionsRecoveryConfigManager.class); + + private final HMaster hMaster; + private int prevMaxStoreFileRefCount; + private int prevRegionsRecoveryInterval; + + RegionsRecoveryConfigManager(final HMaster hMaster) { + this.hMaster = hMaster; + Configuration conf = hMaster.getConfiguration(); + this.prevMaxStoreFileRefCount = getMaxStoreFileRefCount(conf); + this.prevRegionsRecoveryInterval = getRegionsRecoveryChoreInterval(conf); + } + + @Override + public void onConfigurationChange(Configuration conf) { + final int newMaxStoreFileRefCount = getMaxStoreFileRefCount(conf); + final int newRegionsRecoveryInterval = getRegionsRecoveryChoreInterval(conf); + + if (prevMaxStoreFileRefCount == newMaxStoreFileRefCount + && prevRegionsRecoveryInterval == newRegionsRecoveryInterval) { + // no need to re-schedule the chore with updated config + // as there is no change in desired configs + return; + } + + LOG.info("Config Reload for RegionsRecovery Chore. prevMaxStoreFileRefCount: {}," + + " newMaxStoreFileRefCount: {}, prevRegionsRecoveryInterval: {}, " + + "newRegionsRecoveryInterval: {}", prevMaxStoreFileRefCount, newMaxStoreFileRefCount, + prevRegionsRecoveryInterval, newRegionsRecoveryInterval); + + RegionsRecoveryChore regionsRecoveryChore = new RegionsRecoveryChore(this.hMaster, + conf, this.hMaster); + ChoreService choreService = this.hMaster.getChoreService(); + + // Regions Reopen based on very high storeFileRefCount is considered enabled + // only if hbase.regions.recovery.store.file.ref.count has value > 0 + + synchronized (this) { + if (newMaxStoreFileRefCount > 0) { + // reschedule the chore + // provide mayInterruptIfRunning - false to take care of completion + // of in progress task if any + choreService.cancelChore(regionsRecoveryChore, false); + choreService.scheduleChore(regionsRecoveryChore); + } else { + choreService.cancelChore(regionsRecoveryChore, false); + } + this.prevMaxStoreFileRefCount = newMaxStoreFileRefCount; + this.prevRegionsRecoveryInterval = newRegionsRecoveryInterval; + } + } + + private int getMaxStoreFileRefCount(Configuration configuration) { + return configuration.getInt( + HConstants.STORE_FILE_REF_COUNT_THRESHOLD, + HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD); + } + + private int getRegionsRecoveryChoreInterval(Configuration configuration) { + return configuration.getInt( + HConstants.REGIONS_RECOVERY_INTERVAL, + HConstants.DEFAULT_REGIONS_RECOVERY_INTERVAL); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java new file mode 100644 index 000000000000..2949cc2e5466 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java @@ -0,0 +1,126 @@ +/* + * 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.master; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test for Regions Recovery Config Manager + */ +@Category({MasterTests.class, MediumTests.class}) +public class TestRegionsRecoveryConfigManager { + + private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility(); + + private MiniHBaseCluster cluster; + + private HMaster hMaster; + + private RegionsRecoveryChore regionsRecoveryChore; + + private RegionsRecoveryConfigManager regionsRecoveryConfigManager; + + private Configuration conf; + + @Before + public void setup() throws Exception { + conf = HBASE_TESTING_UTILITY.getConfiguration(); + conf.unset("hbase.regions.recovery.store.file.ref.count"); + conf.unset("hbase.master.regions.recovery.check.interval"); + HBASE_TESTING_UTILITY.startMiniCluster(1, 1); + cluster = HBASE_TESTING_UTILITY.getMiniHBaseCluster(); + } + + @After + public void tearDown() throws Exception { + HBASE_TESTING_UTILITY.shutdownMiniCluster(); + } + + @Test + public void testChoreSchedule() throws Exception { + + this.hMaster = cluster.getMaster(); + + Stoppable stoppable = new StoppableImplementation(); + this.regionsRecoveryChore = new RegionsRecoveryChore(stoppable, conf, hMaster); + + this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this.hMaster); + // not yet scheduled + Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // not yet scheduled + Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + conf.setInt("hbase.master.regions.recovery.check.interval", 10); + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // not yet scheduled - missing config: hbase.regions.recovery.store.file.ref.count + Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + conf.setInt("hbase.regions.recovery.store.file.ref.count", 10); + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // chore scheduled + Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + conf.setInt("hbase.regions.recovery.store.file.ref.count", 20); + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // chore re-scheduled + Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + conf.setInt("hbase.regions.recovery.store.file.ref.count", 20); + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // chore scheduling untouched + Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + + conf.unset("hbase.regions.recovery.store.file.ref.count"); + this.regionsRecoveryConfigManager.onConfigurationChange(conf); + // chore un-scheduled + Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore)); + } + + /** + * Simple helper class that just keeps track of whether or not its stopped. + */ + private static class StoppableImplementation implements Stoppable { + + private boolean stop = false; + + @Override + public void stop(String why) { + this.stop = true; + } + + @Override + public boolean isStopped() { + return this.stop; + } + + } + +} \ No newline at end of file From cf02e6f59aedd748762a361578d9c9f38d998b09 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 6 Nov 2019 09:40:13 +0800 Subject: [PATCH 17/51] HBASE-23236 Upgrade to yetus 0.11.1 Signed-off-by: stack --- dev-support/Jenkinsfile | 2 +- dev-support/Jenkinsfile_GitHub | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index ea7d1c6c22d9..3be546763ac3 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -31,7 +31,7 @@ pipeline { disableConcurrentBuilds() } environment { - YETUS_RELEASE = '0.11.0' + YETUS_RELEASE = '0.11.1' // where we'll write everything from different steps. Need a copy here so the final step can check for success/failure. OUTPUT_DIR_RELATIVE_GENERAL = 'output-general' OUTPUT_DIR_RELATIVE_JDK7 = 'output-jdk7' diff --git a/dev-support/Jenkinsfile_GitHub b/dev-support/Jenkinsfile_GitHub index 9f8def562b71..95f3d28f12a2 100644 --- a/dev-support/Jenkinsfile_GitHub +++ b/dev-support/Jenkinsfile_GitHub @@ -37,7 +37,7 @@ pipeline { DOCKERFILE = "${SOURCEDIR}/dev-support/docker/Dockerfile" YETUS='yetus' // Branch or tag name. Yetus release tags are 'rel/X.Y.Z' - YETUS_VERSION='rel/0.11.0' + YETUS_VERSION='rel/0.11.1' } parameters { From 72d622b7bf035aa36b57af804141eaf104934da6 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 8 Nov 2019 10:07:24 -0600 Subject: [PATCH 18/51] HBASE-23228 Allow for jdk8 specific modules on branch-1 in precommit/nightly testing (#804) Signed-off-by: Josh Elser --- dev-support/hbase-personality.sh | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index aaec957a919c..806ac49ebcc9 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -132,6 +132,8 @@ function personality_modules local repostatus=$1 local testtype=$2 local extra="" + local branch1jdk8=() + local jdk8module="" local MODULES=("${CHANGED_MODULES[@]}") yetus_info "Personality: ${repostatus} ${testtype}" @@ -169,6 +171,21 @@ function personality_modules return fi + # This list should include any modules that require jdk8. Maven should be configured to only + # include them when a proper JDK is in use, but that doesn' work if we specifically ask for the + # module to build as yetus does if something changes in the module. Rather than try to + # figure out what jdk is in use so we can duplicate the module activation logic, just + # build at the top level if anything changes in one of these modules and let maven sort it out. + branch1jdk8=(hbase-error-prone hbase-tinylfu-blockcache) + if [[ "${PATCH_BRANCH}" = branch-1* ]]; then + for jdk8module in "${branch1jdk8[@]}"; do + if [[ "${MODULES[*]}" =~ ${jdk8module} ]]; then + MODULES=(.) + break + fi + done + fi + if [[ ${testtype} == findbugs ]]; then # Run findbugs on each module individually to diff pre-patch and post-patch results and # report new warnings for changed modules only. @@ -189,7 +206,8 @@ function personality_modules return fi - if [[ ${testtype} == compile ]] && [[ "${SKIP_ERRORPRONE}" != "true" ]]; then + if [[ ${testtype} == compile ]] && [[ "${SKIP_ERRORPRONE}" != "true" ]] && + [[ "${PATCH_BRANCH}" != branch-1* ]] ; then extra="${extra} -PerrorProne" fi From abf6ec0d735ecbaba1a99e6eaa6ca9c89275db61 Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Fri, 8 Nov 2019 18:56:21 +0100 Subject: [PATCH 19/51] HBASE-18439 Subclasses of o.a.h.h.chaos.actions.Action all use the same logger Signed-off-by: Jan Hentschel Signed-off-by: Guangxu Cheng --- .../hadoop/hbase/chaos/actions/Action.java | 6 +-- .../hbase/chaos/actions/AddColumnAction.java | 3 ++ .../chaos/actions/BatchRestartRsAction.java | 4 ++ .../actions/ChangeBloomFilterAction.java | 3 ++ .../actions/ChangeCompressionAction.java | 4 ++ .../chaos/actions/ChangeEncodingAction.java | 3 ++ .../actions/ChangeSplitPolicyAction.java | 7 ++- .../chaos/actions/ChangeVersionsAction.java | 3 ++ .../CompactRandomRegionOfTableAction.java | 4 ++ .../chaos/actions/CompactTableAction.java | 3 ++ .../actions/DumpClusterStatusAction.java | 5 ++ .../FlushRandomRegionOfTableAction.java | 6 ++- .../hbase/chaos/actions/FlushTableAction.java | 4 ++ .../chaos/actions/ForceBalancerAction.java | 6 +++ ...rgeRandomAdjacentRegionsOfTableAction.java | 4 ++ .../MoveRandomRegionOfTableAction.java | 4 ++ .../actions/MoveRegionsOfTableAction.java | 4 ++ .../chaos/actions/RemoveColumnAction.java | 4 ++ .../actions/RestartActionBaseAction.java | 4 ++ .../actions/RestartActiveMasterAction.java | 4 ++ .../actions/RestartActiveNameNodeAction.java | 4 ++ .../actions/RestartRandomDataNodeAction.java | 12 +++-- .../chaos/actions/RestartRandomRsAction.java | 4 ++ .../actions/RestartRandomZKNodeAction.java | 4 ++ .../actions/RestartRsHoldingMetaAction.java | 5 ++ .../actions/RestartRsHoldingTableAction.java | 4 ++ .../actions/RollingBatchRestartRsAction.java | 47 ++++++++++--------- .../chaos/actions/SnapshotTableAction.java | 4 ++ .../actions/SplitAllRegionOfTableAction.java | 10 ++-- .../SplitRandomRegionOfTableAction.java | 4 ++ .../chaos/actions/TruncateTableAction.java | 4 ++ .../UnbalanceKillAndRebalanceAction.java | 4 ++ .../chaos/actions/UnbalanceRegionsAction.java | 4 ++ 33 files changed, 159 insertions(+), 36 deletions(-) diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java index 4d34a6e0a631..a751b92b2f36 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java @@ -27,8 +27,6 @@ import java.util.Set; import org.apache.commons.lang.math.RandomUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HBaseCluster; @@ -39,6 +37,8 @@ import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A (possibly mischievous) action that the ChaosMonkey can perform. @@ -64,7 +64,7 @@ public class Action { public static final String START_NAMENODE_TIMEOUT_KEY = "hbase.chaosmonkey.action.startnamenodetimeout"; - protected static final Log LOG = LogFactory.getLog(Action.class); + private static final Logger LOG = LoggerFactory.getLogger(Action.class); protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT; protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java index 27268a4632a2..08eef68727e9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java @@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action the adds a column family to a table. @@ -32,6 +34,7 @@ public class AddColumnAction extends Action { private final TableName tableName; private Admin admin; + private static final Logger LOG = LoggerFactory.getLogger(AddColumnAction.class); public AddColumnAction(TableName tableName) { this.tableName = tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java index ce660006b273..129721be72fc 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java @@ -24,12 +24,16 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Restarts a ratio of the running regionservers at the same time */ public class BatchRestartRsAction extends RestartActionBaseAction { float ratio; //ratio of regionservers to restart + private static final Logger LOG = + LoggerFactory.getLogger(BatchRestartRsAction.class); public BatchRestartRsAction(long sleepTime, float ratio) { super(sleepTime); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java index 684cd629863c..907c3f963450 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java @@ -26,6 +26,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to adjust the bloom filter setting on all the columns of a @@ -34,6 +36,7 @@ public class ChangeBloomFilterAction extends Action { private final long sleepTime; private final TableName tableName; + private static final Logger LOG = LoggerFactory.getLogger(ChangeBloomFilterAction.class); public ChangeBloomFilterAction(TableName tableName) { this(-1, tableName); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java index 9c7bf45e3abe..9a2a5f01e451 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java @@ -28,6 +28,9 @@ import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.io.compress.Compressor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Action that changes the compression algorithm on a column family from a list of tables. */ @@ -36,6 +39,7 @@ public class ChangeCompressionAction extends Action { private Admin admin; private Random random; + private static final Logger LOG = LoggerFactory.getLogger(ChangeCompressionAction.class); public ChangeCompressionAction(TableName tableName) { this.tableName = tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java index c4553f155240..e678afe1d463 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java @@ -26,6 +26,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that changes the encoding on a column family from a list of tables. @@ -35,6 +37,7 @@ public class ChangeEncodingAction extends Action { private Admin admin; private Random random; + private static final Logger LOG = LoggerFactory.getLogger(ChangeEncodingAction.class); public ChangeEncodingAction(TableName tableName) { this.tableName = tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java index b5f759f3bcb7..dc1cfee5c559 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.chaos.actions; +import java.util.Random; + import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -24,10 +26,11 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy; - -import java.util.Random; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ChangeSplitPolicyAction extends Action { + private static final Logger LOG = LoggerFactory.getLogger(ChangeSplitPolicyAction.class); private final TableName tableName; private final String[] possiblePolicies; private final Random random; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java index 76e152f19f13..7e2332c89677 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java @@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that changes the number of versions on a column family from a list of tables. @@ -33,6 +35,7 @@ */ public class ChangeVersionsAction extends Action { private final TableName tableName; + private static final Logger LOG = LoggerFactory.getLogger(ChangeVersionsAction.class); private Admin admin; private Random random; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java index 114b511ee865..6003240c48f6 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java @@ -26,6 +26,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Region that queues a compaction of a random region from the table. @@ -34,6 +36,8 @@ public class CompactRandomRegionOfTableAction extends Action { private final int majorRatio; private final long sleepTime; private final TableName tableName; + private static final Logger LOG = + LoggerFactory.getLogger(CompactRandomRegionOfTableAction.class); public CompactRandomRegionOfTableAction( TableName tableName, float majorRatio) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java index 796cc43a0876..2f5436a4217c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java @@ -22,6 +22,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that queues a table compaction. @@ -30,6 +32,7 @@ public class CompactTableAction extends Action { private final TableName tableName; private final int majorRatio; private final long sleepTime; + private static final Logger LOG = LoggerFactory.getLogger(CompactTableAction.class); public CompactTableAction(TableName tableName, float majorRatio) { this(-1, tableName, majorRatio); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java index 0403fe047f92..11246ea4a279 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java @@ -20,10 +20,15 @@ import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Action to dump the cluster status. */ public class DumpClusterStatusAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(DumpClusterStatusAction.class); @Override public void init(ActionContext context) throws IOException { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java index c919789f9fb2..c4286dbf1f9e 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java @@ -25,16 +25,20 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to flush a random region of a table. */ public class FlushRandomRegionOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(FlushRandomRegionOfTableAction.class); private final long sleepTime; private final TableName tableName; public FlushRandomRegionOfTableAction(TableName tableName) { - this (-1, tableName); + this (-1, tableName); } public FlushRandomRegionOfTableAction(int sleepTime, TableName tableName) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java index ddce57e8b53b..994fd057a5f1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java @@ -21,11 +21,15 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to flush a table. */ public class FlushTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(FlushTableAction.class); private final long sleepTime; private final TableName tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java index 04f389a9519e..d75475432a12 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java @@ -18,10 +18,16 @@ package org.apache.hadoop.hbase.chaos.actions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Action that tries to force a balancer run. */ public class ForceBalancerAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(ForceBalancerAction.class); + @Override public void perform() throws Exception { // Don't try the flush if we're stopping diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java index 8645dc4c9407..eac7d30100a9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java @@ -25,11 +25,15 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action to merge regions of a table. */ public class MergeRandomAdjacentRegionsOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(MergeRandomAdjacentRegionsOfTableAction.class); private final TableName tableName; private final long sleepTime; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java index 96cd0086456b..52816f323d41 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java @@ -25,11 +25,15 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to move a random region of a table. */ public class MoveRandomRegionOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(MoveRandomRegionOfTableAction.class); private final long sleepTime; private final TableName tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java index d5f0e9652bf4..e38309b4d36c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java @@ -29,11 +29,15 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to move every region of a table. */ public class MoveRegionsOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(MoveRegionsOfTableAction.class); private final long sleepTime; private final TableName tableName; private final long maxTime; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java index 20bdaa3c1d4b..e5ca3e857d2a 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java @@ -27,11 +27,15 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that removes a column family. */ public class RemoveColumnAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(RemoveColumnAction.class); private final TableName tableName; private final Set protectedColumns; private Admin admin; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java index 22d7e2618e3d..8376f51d81a9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java @@ -22,11 +22,15 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.util.Threads; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Base class for restarting HBaseServer's */ public class RestartActionBaseAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(RestartActionBaseAction.class); long sleepTime; // how long should we sleep public RestartActionBaseAction(long sleepTime) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java index a9bc23ab7542..ab7decd3f1c8 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java @@ -19,11 +19,15 @@ package org.apache.hadoop.hbase.chaos.actions; import org.apache.hadoop.hbase.ServerName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to restart the active master. */ public class RestartActiveMasterAction extends RestartActionBaseAction { + private static final Logger LOG = LoggerFactory.getLogger(RestartActionBaseAction.class); + public RestartActiveMasterAction(long sleepTime) { super(sleepTime); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java index 710ac1477862..d9cbfbddc8ae 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java @@ -29,11 +29,15 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.server.namenode.ha.proto.HAZKInfoProtos.ActiveNodeInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to restart the active namenode. */ public class RestartActiveNameNodeAction extends RestartActionBaseAction { + private static final Logger LOG = + LoggerFactory.getLogger(RestartActiveNameNodeAction.class); // Value taken from org.apache.hadoop.ha.ActiveStandbyElector.java, variable :- LOCK_FILENAME private static final String ACTIVE_NN_LOCK_NAME = "ActiveStandbyElectorLock"; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java index 7299e79ed16b..09e2990db613 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java @@ -18,6 +18,10 @@ package org.apache.hadoop.hbase.chaos.actions; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.util.FSUtils; @@ -25,15 +29,15 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that restarts a random datanode. */ public class RestartRandomDataNodeAction extends RestartActionBaseAction { + private static final Logger LOG = LoggerFactory.getLogger(RestartRandomDataNodeAction.class); + public RestartRandomDataNodeAction(long sleepTime) { super(sleepTime); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java index 7b09dd310513..48458b68dcf4 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java @@ -20,11 +20,15 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that restarts a random HRegionServer */ public class RestartRandomRsAction extends RestartActionBaseAction { + private static final Logger LOG = LoggerFactory.getLogger(RestartRandomRsAction.class); + public RestartRandomRsAction(long sleepTime) { super(sleepTime); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java index 6043acde1bae..7984af7ba4a3 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java @@ -21,11 +21,15 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.zookeeper.ZKServerTool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that restarts a random zookeeper node. */ public class RestartRandomZKNodeAction extends RestartActionBaseAction { + private static final Logger LOG = LoggerFactory.getLogger(RestartRandomZKNodeAction.class); + public RestartRandomZKNodeAction(long sleepTime) { super(sleepTime); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java index a6b4fc766013..09b3db69c3d5 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java @@ -20,11 +20,16 @@ import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ServerName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to restart the HRegionServer holding Meta. */ public class RestartRsHoldingMetaAction extends RestartActionBaseAction { + private static final Logger LOG = + LoggerFactory.getLogger(RestartRsHoldingMetaAction.class); + public RestartRsHoldingMetaAction(long sleepTime) { super(sleepTime); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java index b1ea8e531262..79e91fe08202 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java @@ -26,11 +26,15 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that restarts an HRegionServer holding one of the regions of the table. */ public class RestartRsHoldingTableAction extends RestartActionBaseAction { + private static final Logger LOG = + LoggerFactory.getLogger(RestartRsHoldingTableAction.class); private final String tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java index e79ff5b5739d..347340799c1d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java @@ -32,8 +32,9 @@ /** * Restarts a ratio of the regionservers in a rolling fashion. At each step, either kills a - * server, or starts one, sleeping randomly (0-sleepTime) in between steps. The parameter maxDeadServers - * limits the maximum number of servers that can be down at the same time during rolling restarts. + * server, or starts one, sleeping randomly (0-sleepTime) in between steps. + * The parameter maxDeadServers limits the maximum number of servers that + * can be down at the same time during rolling restarts. */ public class RollingBatchRestartRsAction extends BatchRestartRsAction { private static final Log LOG = LogFactory.getLog(RollingBatchRestartRsAction.class); @@ -81,27 +82,27 @@ public void perform() throws Exception { ServerName server; switch (action) { - case KILL: - server = serversToBeKilled.remove(); - try { - killRs(server); - } catch (org.apache.hadoop.util.Shell.ExitCodeException e) { - // We've seen this in test runs where we timeout but the kill went through. HBASE-9743 - // So, add to deadServers even if exception so the start gets called. - LOG.info("Problem killing but presume successful; code=" + e.getExitCode(), e); - } - deadServers.add(server); - break; - case START: - try { - server = deadServers.remove(); - startRs(server); - } catch (org.apache.hadoop.util.Shell.ExitCodeException e) { - // The start may fail but better to just keep going though we may lose server. - // - LOG.info("Problem starting, will retry; code=" + e.getExitCode(), e); - } - break; + case KILL: + server = serversToBeKilled.remove(); + try { + killRs(server); + } catch (org.apache.hadoop.util.Shell.ExitCodeException e) { + // We've seen this in test runs where we timeout but the kill went through. HBASE-9743 + // So, add to deadServers even if exception so the start gets called. + LOG.info("Problem killing but presume successful; code=" + e.getExitCode(), e); + } + deadServers.add(server); + break; + case START: + try { + server = deadServers.remove(); + startRs(server); + } catch (org.apache.hadoop.util.Shell.ExitCodeException e) { + // The start may fail but better to just keep going though we may lose server. + // + LOG.info("Problem starting, will retry; code=" + e.getExitCode(), e); + } + break; } sleep(RandomUtils.nextInt((int)sleepTime)); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java index 15b8e86feec9..7e7dc8da24ce 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java @@ -21,11 +21,15 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to take a snapshot of a table. */ public class SnapshotTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(SnapshotTableAction.class); private final TableName tableName; private final long sleepTime; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java index 5b29d00c4030..5a24af9eeb62 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java @@ -17,15 +17,19 @@ */ package org.apache.hadoop.hbase.chaos.actions; +import java.io.IOException; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; - -import java.io.IOException; -import java.util.concurrent.ThreadLocalRandom; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SplitAllRegionOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(SplitAllRegionOfTableAction.class); private static final int DEFAULT_MAX_SPLITS = 3; private static final String MAX_SPLIT_KEY = "hbase.chaosmonkey.action.maxFullTableSplits"; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java index 25c80e918cfb..df424ec9e475 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java @@ -25,11 +25,15 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to split a random region of a table. */ public class SplitRandomRegionOfTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(SplitRandomRegionOfTableAction.class); private final long sleepTime; private final TableName tableName; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java index 2a4871d3e729..12bbd094ce67 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java @@ -24,11 +24,15 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.TableName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to truncate of a table. */ public class TruncateTableAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(TruncateTableAction.class); private final TableName tableName; private final Random random; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java index 0035c2c4c729..264a54f191cb 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java @@ -28,9 +28,13 @@ import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ServerName; import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** This action is too specific to put in ChaosMonkey; put it here */ public class UnbalanceKillAndRebalanceAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(UnbalanceKillAndRebalanceAction.class); /** Fractions of servers to get regions and live and die respectively; from all other * servers, HOARD_FRC_OF_REGIONS will be removed to the above randomly */ private static final double FRC_SERVERS_THAT_HOARD_AND_LIVE = 0.1; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java index 2779bd1ca1ad..54690bf3fff5 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java @@ -25,11 +25,15 @@ import org.apache.commons.lang.math.RandomUtils; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.ServerName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Action that tries to unbalance the regions of a cluster. */ public class UnbalanceRegionsAction extends Action { + private static final Logger LOG = + LoggerFactory.getLogger(UnbalanceRegionsAction.class); private double fractionOfRegions; private double fractionOfServers; From 9b30df5c63fbb422b67d29ab86387dc8aac7c792 Mon Sep 17 00:00:00 2001 From: Guangxu Cheng Date: Sun, 10 Nov 2019 20:37:48 +0800 Subject: [PATCH 20/51] HBASE-23273 Fix table header display is incorrect on table.jsp when viewing hbase:meta Signed-off-by: Guangxu Cheng --- .../src/main/resources/hbase-webapps/master/table.jsp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index 93fd3a81679a..b45d633d08e1 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -159,10 +159,10 @@ if ( fqtn != null ) { table = new HTable(conf, fqtn); if (table.getTableDescriptor().getRegionReplication() > 1) { - tableHeader = "

Table Regions

NameStart KeyEnd Key"; + tableHeader = "

Table Regions

Region ServerLocalityRequestsReplicaID
"; withReplica = true; } else { - tableHeader = "

Table Regions

NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd KeyReplicaID
NameStart KeyEnd Key"; + tableHeader = "

Table Regions

Region ServerLocalityRequests
"; } if ( !readOnly && action != null ) { %> From aa2e4871befd6fc48565f88e9cd4637cb15af6a3 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Mon, 11 Nov 2019 21:52:02 +0530 Subject: [PATCH 21/51] HBASE-23245 : MutableHistogram constructor changes and provide HistogramImpl maxExpected as long (#787) Signed-off-by: Andrew Purtell Signed-off-by: Xu Cang Signed-off-by: Guangxu Cheng --- .../hadoop/metrics2/lib/MutableHistogram.java | 4 --- .../metrics2/lib/MutableRangeHistogram.java | 6 +--- .../metrics2/lib/MutableSizeHistogram.java | 7 ++-- .../metrics2/lib/MutableTimeHistogram.java | 6 +--- .../hbase/metrics/impl/HistogramImpl.java | 2 +- .../hbase/metrics/impl/TestHistogramImpl.java | 36 ++++++++++++++----- 6 files changed, 32 insertions(+), 29 deletions(-) diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java index 6a2f2039fc7a..bfa24dc4ef42 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java @@ -42,10 +42,6 @@ public MutableHistogram(MetricsInfo info) { } public MutableHistogram(String name, String description) { - this(name, description, Integer.MAX_VALUE << 2); - } - - protected MutableHistogram(String name, String description, long maxExpected) { this.name = StringUtils.capitalize(name); this.desc = StringUtils.uncapitalize(description); this.histogram = new HistogramImpl(); diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java index e15d0a86fd03..bd04b02c761a 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java @@ -35,11 +35,7 @@ public MutableRangeHistogram(MetricsInfo info) { } public MutableRangeHistogram(String name, String description) { - this(name, description, Integer.MAX_VALUE << 2); - } - - public MutableRangeHistogram(String name, String description, long expectedMax) { - super(name, description, expectedMax); + super(name, description); } /** diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java index 38e78a2324e2..67e27c773e41 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java @@ -26,6 +26,7 @@ */ @InterfaceAudience.Private public class MutableSizeHistogram extends MutableRangeHistogram { + private final static String RANGE_TYPE = "SizeRangeCount"; private final static long[] RANGES = {10,100,1000,10000,100000,1000000,10000000,100000000}; @@ -34,11 +35,7 @@ public MutableSizeHistogram(MetricsInfo info) { } public MutableSizeHistogram(String name, String description) { - this(name, description, RANGES[RANGES.length-2]); - } - - public MutableSizeHistogram(String name, String description, long expectedMax) { - super(name, description, expectedMax); + super(name, description); } @Override diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java index aaf4359f18cb..7847202523a4 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java @@ -35,11 +35,7 @@ public MutableTimeHistogram(MetricsInfo info) { } public MutableTimeHistogram(String name, String description) { - this(name, description, RANGES[RANGES.length - 2]); - } - - public MutableTimeHistogram(String name, String description, long expectedMax) { - super(name, description, expectedMax); + super(name, description); } @Override diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java index 17a179dafa53..7312230e0ab1 100644 --- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java +++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java @@ -38,7 +38,7 @@ public class HistogramImpl implements Histogram { private final CounterImpl counter; public HistogramImpl() { - this(Integer.MAX_VALUE << 2); + this((long) Integer.MAX_VALUE << 2); } public HistogramImpl(long maxExpected) { diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java index 5d3b1faa2101..713b3dce24d2 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java @@ -62,19 +62,19 @@ public void testSnapshot() { Snapshot snapshot = histogram.snapshot(); assertEquals(100, snapshot.getCount()); - assertEquals(50, snapshot.getMedian()); + assertEquals(49, snapshot.getMedian()); assertEquals(49, snapshot.getMean()); assertEquals(0, snapshot.getMin()); assertEquals(99, snapshot.getMax()); - assertEquals(25, snapshot.get25thPercentile()); - assertEquals(75, snapshot.get75thPercentile()); - assertEquals(90, snapshot.get90thPercentile()); - assertEquals(95, snapshot.get95thPercentile()); - assertEquals(98, snapshot.get98thPercentile()); - assertEquals(99, snapshot.get99thPercentile()); - assertEquals(99, snapshot.get999thPercentile()); + assertEquals(24, snapshot.get25thPercentile()); + assertEquals(74, snapshot.get75thPercentile()); + assertEquals(89, snapshot.get90thPercentile()); + assertEquals(94, snapshot.get95thPercentile()); + assertEquals(97, snapshot.get98thPercentile()); + assertEquals(98, snapshot.get99thPercentile()); + assertEquals(98, snapshot.get999thPercentile()); - assertEquals(51, snapshot.getCountAtOrBelow(50)); + assertEquals(100, snapshot.getCountAtOrBelow(50)); // check that histogram is reset. assertEquals(100, histogram.getCount()); // count does not reset @@ -99,5 +99,23 @@ public void testSnapshot() { assertEquals(198, snapshot.get98thPercentile()); assertEquals(199, snapshot.get99thPercentile()); assertEquals(199, snapshot.get999thPercentile()); + + IntStream.range(500, 1000).forEach(histogram::update); + + snapshot = histogram.snapshot(); + + assertEquals(500, snapshot.getCount()); + assertEquals(749, snapshot.getMedian()); + assertEquals(749, snapshot.getMean()); + assertEquals(500, snapshot.getMin()); + assertEquals(999, snapshot.getMax()); + assertEquals(624, snapshot.get25thPercentile()); + assertEquals(874, snapshot.get75thPercentile()); + assertEquals(949, snapshot.get90thPercentile()); + assertEquals(974, snapshot.get95thPercentile()); + assertEquals(989, snapshot.get98thPercentile()); + assertEquals(994, snapshot.get99thPercentile()); + assertEquals(998, snapshot.get999thPercentile()); + } } From caef9f0cd560e403872ad73daef4f2dbabfa190b Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 13 Nov 2019 04:32:49 +0530 Subject: [PATCH 22/51] HBASE-23245 : Test Histogram Impl changes for histogram update (Addendum) (#818) Signed-off-by: Andrew Purtell --- .../apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java index 713b3dce24d2..3cd076740eb4 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java @@ -100,7 +100,9 @@ public void testSnapshot() { assertEquals(199, snapshot.get99thPercentile()); assertEquals(199, snapshot.get999thPercentile()); - IntStream.range(500, 1000).forEach(histogram::update); + for (int i = 500; i < 1000; i++) { + histogram.update(i); + } snapshot = histogram.snapshot(); From a154bd8a73a9d79f846f2e4bd4f78e980654fcac Mon Sep 17 00:00:00 2001 From: Mingliang Liu Date: Tue, 12 Nov 2019 23:59:08 -0800 Subject: [PATCH 23/51] HBASE-23283 Provide clear and consistent logging about the period of enabled chores Signed-off-by: Sean Busbey --- .../src/main/java/org/apache/hadoop/hbase/ChoreService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java index 49ac9070fd60..249a7a25b14d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java @@ -152,9 +152,10 @@ public synchronized boolean scheduleChore(ScheduledChore chore) { try { if (chore.getPeriod() <= 0) { - LOG.info("The period is " + chore.getPeriod() + " seconds, " + chore.getName() + " is disabled"); + LOG.info("Chore " + chore + " is disabled because its period is not positive."); return false; } + LOG.info("Chore " + chore + " is enabled."); chore.setChoreServicer(this); ScheduledFuture future = scheduler.scheduleAtFixedRate(chore, chore.getInitialDelay(), chore.getPeriod(), From 5130bc52d5bf0a1ccc32c3339b57c066b67e4e76 Mon Sep 17 00:00:00 2001 From: Baiqiang Zhao Date: Wed, 13 Nov 2019 19:32:18 +0800 Subject: [PATCH 24/51] HBASE-23287 LogCleaner is not added to choreService Signed-off-by: Sean Busbey --- .../src/main/java/org/apache/hadoop/hbase/master/HMaster.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 278d86b3555b..5eb60d620942 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1259,6 +1259,7 @@ private void startServiceThreads() throws IOException { this.logCleaner = new LogCleaner(cleanerInterval, this, conf, getMasterFileSystem().getOldLogDir().getFileSystem(conf), getMasterFileSystem().getOldLogDir(), cleanerPool); + getChoreService().scheduleChore(logCleaner); //start the hfile archive cleaner thread Path archiveDir = HFileArchiveUtil.getArchivePath(conf); Map params = new HashMap(); From b566a4f9bcbaca3948a8fcb20b19c18fcf4d5052 Mon Sep 17 00:00:00 2001 From: Josh Elser Date: Tue, 16 Jul 2019 17:10:27 -0400 Subject: [PATCH 25/51] HBASE-22701 Disable the DynamicClassLoader when it fails to initialize Signed-off-by: Ankit Singhal --- .../hadoop/hbase/util/DynamicClassLoader.java | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java index 07ca3485c254..7f9e08c3710e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java @@ -69,7 +69,10 @@ public class DynamicClassLoader extends ClassLoaderBase { private static final String DYNAMIC_JARS_OPTIONAL_CONF_KEY = "hbase.use.dynamic.jars"; private static final boolean DYNAMIC_JARS_OPTIONAL_DEFAULT = true; - private boolean useDynamicJars; + // The user-provided value for using the DynamicClassLoader + private final boolean userConfigUseDynamicJars; + // The current state of whether to use the DynamicClassLoader + private final boolean useDynamicJars; private File localDir; @@ -91,12 +94,23 @@ public DynamicClassLoader( final Configuration conf, final ClassLoader parent) { super(parent); - useDynamicJars = conf.getBoolean( + // Save off the user's original configuration value for the DynamicClassLoader + userConfigUseDynamicJars = conf.getBoolean( DYNAMIC_JARS_OPTIONAL_CONF_KEY, DYNAMIC_JARS_OPTIONAL_DEFAULT); - if (useDynamicJars) { - initTempDir(conf); + boolean dynamicJarsEnabled = userConfigUseDynamicJars; + if (dynamicJarsEnabled) { + try { + initTempDir(conf); + dynamicJarsEnabled = true; + } catch (Exception e) { + LOG.error("Disabling the DynamicClassLoader as it failed to initialize its temp directory." + + " Check your configuration and filesystem permissions. Custom coprocessor code may" + + " not be loaded as a result of this failure.", e); + dynamicJarsEnabled = false; + } } + useDynamicJars = dynamicJarsEnabled; } // FindBugs: Making synchronized to avoid IS2_INCONSISTENT_SYNC complaints about @@ -132,12 +146,13 @@ public Class loadClass(String name) try { return parent.loadClass(name); } catch (ClassNotFoundException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Class " + name + " not found - using dynamical class loader"); - } - if (useDynamicJars) { + LOG.debug("Class " + name + " not found - using dynamical class loader"); return tryRefreshClass(name); + } else if (userConfigUseDynamicJars) { + // If the user tried to enable the DCL, then warn again. + LOG.debug("Not checking DynamicClassLoader for missing class because it is disabled." + + " See the log for previous errors."); } throw e; } From 8e60b0c0c6b4b9a8e2a439b2d566e8135a8f1081 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 12 Nov 2019 01:04:12 +0530 Subject: [PATCH 26/51] HBASE-23261 Region stuck in transition while splitting Processing ZK BadVersionException during node transition Signed-off-by: Andrew Purtell --- .../hadoop/hbase/zookeeper/ZKAssign.java | 10 +++++++++- .../ZKSplitTransactionCoordination.java | 20 +++++++++++++++++-- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java index 297e96e74979..b2e1e1eeb15f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java @@ -868,7 +868,15 @@ public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region, try { rt = RegionTransition.createRegionTransition( endState, region.getRegionName(), serverName, payload); - if(!ZKUtil.setData(zkw, node, rt.toByteArray(), stat.getVersion())) { + boolean isDataSet; + try { + isDataSet = ZKUtil.setData(zkw, node, rt.toByteArray(), stat.getVersion()); + } catch (KeeperException.BadVersionException e) { + isDataSet = false; + LOG.error("Received BadVersionException from ZK for " + encoded + + ", version: " + stat.getVersion()); + } + if (!isDataSet) { LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for " + encoded + " from " + beginState + " to " + endState + " failed, " + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java index f6e96fa3073b..24164e51658e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitTransactionCoordination.java @@ -40,6 +40,13 @@ public class ZKSplitTransactionCoordination implements SplitTransactionCoordinat private CoordinatedStateManager coordinationManager; private final ZooKeeperWatcher watcher; + // max wait for split transaction - 100 times in a loop with 100 ms of thread sleep each time + // this accounts for ~24 s due to calls involved in loop. even for busy cluster, by this time, + // we should have been able to complete setData() In fact, ideally, 2nd retry after failed + // attempt should be sufficient to retrieve correct ZK node version and successfully updating + // RIT info in ZK node. + private static final int SPIN_WAIT_TIMEOUT = 100; + private static final Log LOG = LogFactory.getLog(ZKSplitTransactionCoordination.class); public ZKSplitTransactionCoordination(CoordinatedStateManager coordinationProvider, @@ -163,6 +170,10 @@ public void waitForSplitTransaction(final RegionServerServices services, Region } Thread.sleep(100); spins++; + if (spins > SPIN_WAIT_TIMEOUT) { + throw new IOException("Waiting time for Split Transaction exceeded for region: " + + parent.getRegionInfo().getRegionNameAsString()); + } byte[] data = ZKAssign.getDataNoWatch(watcher, node, stat); if (data == null) { throw new IOException("Data is null, splitting node " + node + " no longer exists"); @@ -222,9 +233,14 @@ public void completeSplitTransaction(final RegionServerServices services, Region // Tell master about split by updating zk. If we fail, abort. if (coordinationManager.getServer() != null) { try { - zstd.setZnodeVersion(transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(), + int newNodeVersion = transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo(), coordinationManager.getServer().getServerName(), zstd, - RS_ZK_REGION_SPLITTING, RS_ZK_REGION_SPLIT)); + RS_ZK_REGION_SPLITTING, RS_ZK_REGION_SPLIT); + if (newNodeVersion == -1) { + throw new IOException("Notifying master of RS split failed for region: " + + parent.getRegionInfo().getRegionNameAsString()); + } + zstd.setZnodeVersion(newNodeVersion); int spins = 0; // Now wait for the master to process the split. We know it's done From 0cae0043c74d6d7f5dc2ae4c6b48d2788c73d564 Mon Sep 17 00:00:00 2001 From: Jeongdae Kim Date: Mon, 18 Nov 2019 13:30:56 +0900 Subject: [PATCH 27/51] (fix) log a message even in empty batch case --- .../ReplicationSourceWALReaderThread.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java index 4cf2490be355..60f6bdab2528 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java @@ -157,11 +157,6 @@ public void run() { } } - if (LOG.isTraceEnabled()) { - LOG.trace(String.format("Read %s WAL entries eligible for replication", - batch.getNbEntries())); - } - updateBatch(entryStream, batch, hasNext); if (isShippable(batch)) { sleepMultiplier = 1; @@ -192,10 +187,22 @@ public void run() { } private void updateBatch(WALEntryStream entryStream, WALEntryBatch batch, boolean moreData) { + logMessage(batch); batch.updatePosition(entryStream); batch.setMoreEntries(!replicationQueueInfo.isQueueRecovered() || moreData); } + private void logMessage(WALEntryBatch batch) { + if (LOG.isTraceEnabled()) { + if (batch.isEmpty()) { + LOG.trace("Didn't read any new entries from WAL"); + } else { + LOG.trace(String.format("Read %s WAL entries eligible for replication", + batch.getNbEntries())); + } + } + } + private boolean isShippable(WALEntryBatch batch) { return !batch.isEmpty() || checkIfWALRolled(batch) || !batch.hasMoreEntries(); } From 5f3634394908c5e8f008e92e855d175233a8cd9a Mon Sep 17 00:00:00 2001 From: Geoffrey Jacoby Date: Mon, 18 Nov 2019 16:56:10 -0800 Subject: [PATCH 28/51] HBASE-23288 - Backport HBASE-23251 (Add Column Family and Table Names to HFileContext) to branch-1 (#822) Signed-off-by: Andrew Purtell --- .../hadoop/hbase/io/hfile/HFileContext.java | 41 +++++++++++++++--- .../hbase/io/hfile/HFileContextBuilder.java | 16 ++++++- .../hbase/io/hfile/AbstractHFileWriter.java | 16 ++++++- .../hadoop/hbase/io/hfile/HFileBlock.java | 2 + .../hbase/mapreduce/HFileOutputFormat2.java | 4 +- .../hadoop/hbase/regionserver/HStore.java | 3 ++ .../hadoop/hbase/io/hfile/TestHFile.java | 43 +++++++++++++++++++ .../hadoop/hbase/regionserver/TestStore.java | 12 ++++++ .../hadoop/hbase/util/HFileTestUtil.java | 1 + 9 files changed, 128 insertions(+), 10 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java index 716e1b097719..91644918859a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java @@ -57,6 +57,8 @@ public class HFileContext implements HeapSize, Cloneable { private Encryption.Context cryptoContext = Encryption.Context.NONE; private long fileCreateTime; private String hfileName; + private byte[] columnFamily; + private byte[] tableName; //Empty constructor. Go with setters public HFileContext() { @@ -79,12 +81,15 @@ public HFileContext(HFileContext context) { this.cryptoContext = context.cryptoContext; this.fileCreateTime = context.fileCreateTime; this.hfileName = context.hfileName; + this.columnFamily = context.columnFamily; + this.tableName = context.tableName; } public HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean includesTags, - Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType, - int bytesPerChecksum, int blockSize, DataBlockEncoding encoding, - Encryption.Context cryptoContext, long fileCreateTime, String hfileName) { + Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType, + int bytesPerChecksum, int blockSize, DataBlockEncoding encoding, + Encryption.Context cryptoContext, long fileCreateTime, String hfileName, + byte[] columnFamily, byte[] tableName) { this.usesHBaseChecksum = useHBaseChecksum; this.includesMvcc = includesMvcc; this.includesTags = includesTags; @@ -99,6 +104,8 @@ public HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean incl this.cryptoContext = cryptoContext; this.fileCreateTime = fileCreateTime; this.hfileName = hfileName; + this.columnFamily = columnFamily; + this.tableName = tableName; } /** @@ -194,6 +201,14 @@ public String getHFileName() { return this.hfileName; } + public byte[] getColumnFamily() { + return this.columnFamily; + } + + public byte[] getTableName() { + return this.tableName; + } + /** * HeapSize implementation * NOTE : The heapsize should be altered as and when new state variable are added @@ -207,7 +222,15 @@ public long heapSize() { 2 * Bytes.SIZEOF_INT + // usesHBaseChecksum, includesMvcc, includesTags and compressTags 4 * Bytes.SIZEOF_BOOLEAN + + //column family, table name byte arrays + 2 * ClassSize.ARRAY + 2 * ClassSize.REFERENCE + Bytes.SIZEOF_LONG); + if (this.columnFamily != null){ + size += ClassSize.sizeOf(this.columnFamily, this.columnFamily.length); + } + if (this.tableName != null){ + size += ClassSize.sizeOf(this.tableName, this.tableName.length); + } return size; } @@ -233,9 +256,17 @@ public String toString() { sb.append(" includesTags="); sb.append(includesTags); sb.append(" compressAlgo="); sb.append(compressAlgo); sb.append(" compressTags="); sb.append(compressTags); - sb.append(" cryptoContext=[ "); sb.append(cryptoContext); sb.append(" ]"); + sb.append(" cryptoContext=[ "); sb.append(cryptoContext); + sb.append(" ]"); + if (tableName != null) { + sb.append(", tableName="); + sb.append(Bytes.toString(tableName)); + } + if (columnFamily != null) { + sb.append(", columnFamily="); + sb.append(Bytes.toString(columnFamily)); + } sb.append(" ]"); return sb.toString(); } - } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java index d620d553ae5e..10ee69ff0dc7 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java @@ -54,6 +54,8 @@ public class HFileContextBuilder { private long fileCreateTime = 0; private String hfileName = null; + private byte[] columnFamily = null; + private byte[] tableName = null; public HFileContextBuilder() {} @@ -73,6 +75,8 @@ public HFileContextBuilder(final HFileContext hfc) { this.cryptoContext = hfc.getEncryptionContext(); this.fileCreateTime = hfc.getFileCreateTime(); this.hfileName = hfc.getHFileName(); + this.columnFamily = hfc.getColumnFamily(); + this.tableName = hfc.getTableName(); } public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) { @@ -135,9 +139,19 @@ public HFileContextBuilder withHFileName(String name) { return this; } + public HFileContextBuilder withColumnFamily(byte[] columnFamily){ + this.columnFamily = columnFamily; + return this; + } + + public HFileContextBuilder withTableName(byte[] tableName){ + this.tableName = tableName; + return this; + } + public HFileContext build() { return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression, compressTags, checksumType, bytesPerChecksum, blocksize, encoding, cryptoContext, - fileCreateTime, hfileName); + fileCreateTime, hfileName, columnFamily, tableName); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java index 93e18370407f..1d9b9ca48813 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java @@ -201,8 +201,8 @@ protected boolean checkKey(final Cell cell) throws IOException { int keyComp = comparator.compareOnlyKeyPortion(lastCell, cell); if (keyComp > 0) { - throw new IOException("Added a key not lexically larger than" - + " previous. Current cell = " + cell + ", lastCell = " + lastCell); + String message = getLexicalErrorMessage(cell); + throw new IOException(message); } else if (keyComp == 0) { isDuplicateKey = true; } @@ -210,6 +210,18 @@ protected boolean checkKey(final Cell cell) throws IOException { return isDuplicateKey; } + private String getLexicalErrorMessage(Cell cell) { + StringBuilder sb = new StringBuilder(); + sb.append("Added a key not lexically larger than previous. Current cell = "); + sb.append(cell); + sb.append(", lastCell = "); + sb.append(lastCell); + //file context includes HFile path and optionally table and CF of file being written + sb.append("fileContext="); + sb.append(hFileContext); + return sb.toString(); + } + /** Checks the given value for validity. */ protected void checkValue(final byte[] value, final int offset, final int length) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index d177402cf557..114d64250fcd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -1292,6 +1292,8 @@ HFileBlock getBlockForCaching(CacheConfig cacheConf) { .withCompressTags(fileContext.isCompressTags()) .withIncludesMvcc(fileContext.isIncludesMvcc()) .withIncludesTags(fileContext.isIncludesTags()) + .withColumnFamily(fileContext.getColumnFamily()) + .withTableName(fileContext.getTableName()) .build(); return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(), getUncompressedSizeWithoutHeader(), prevOffset, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index e8f3c1f6bbbd..202cd55c81ef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -316,8 +316,8 @@ private WriterLength getNewWriter(byte[] family, Configuration conf, .withCompression(compression) .withChecksumType(HStore.getChecksumType(conf)) .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)) - .withBlockSize(blockSize); - + .withBlockSize(blockSize) + .withColumnFamily(family); if (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS) { contextBuilder.withIncludesTags(true); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 9c0897fd7c71..f2038cf3c880 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1158,6 +1158,9 @@ private HFileContext createFileContext(Compression.Algorithm compression, .withDataBlockEncoding(family.getDataBlockEncoding()) .withEncryptionContext(cryptoContext) .withCreateTime(EnvironmentEdgeManager.currentTime()) + .withColumnFamily(family.getName()) + .withTableName(region.getTableDesc(). + getTableName().getName()) .build(); return hFileContext; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index a1137cf61e12..91b25211a262 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -32,6 +32,8 @@ 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.CellUtil; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -44,7 +46,10 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Writer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Writable; +import org.junit.Assert; +import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.Mockito; /** * test hfile features. @@ -95,6 +100,44 @@ public void testEmptyHFile() throws IOException { assertNull(r.getLastKey()); } + @Test + public void testCorruptOutOfOrderHFileWrite() throws IOException { + Path path = new Path(ROOT_DIR, "testCorruptOutOfOrderHFileWrite"); + FSDataOutputStream mockedOutputStream = Mockito.mock(FSDataOutputStream.class); + String columnFamily = "MyColumnFamily"; + String tableName = "MyTableName"; + HFileContext fileContext = new HFileContextBuilder() + .withHFileName("testCorruptOutOfOrderHFileWriteHFile") + .withBlockSize(minBlockSize) + .withColumnFamily(Bytes.toBytes(columnFamily)) + .withTableName(Bytes.toBytes(tableName)) + .withHBaseCheckSum(false) + .withCompression(Compression.Algorithm.NONE) + .withCompressTags(false) + .build(); + HFileWriterV3 writer = new HFileWriterV3(conf, cacheConf, fs, path, mockedOutputStream, + new KeyValue.KVComparator(), fileContext); + byte[] row = Bytes.toBytes("foo"); + byte[] qualifier = Bytes.toBytes("qualifier"); + byte[] cf = Bytes.toBytes(columnFamily); + byte[] val = Bytes.toBytes("fooVal"); + long firstTS = 100L; + long secondTS = 101L; + Cell firstCell = CellUtil.createCell(row,cf, qualifier, firstTS, Type.Put.getCode(), val); + Cell secondCell= CellUtil.createCell(row,cf, qualifier, secondTS, Type.Put.getCode(), val); + //second Cell will sort "higher" than the first because later timestamps should come first + writer.append(firstCell); + try { + writer.append(secondCell); + } catch(IOException ie){ + String message = ie.getMessage(); + Assert.assertTrue(message.contains("not lexically larger")); + Assert.assertTrue(message.contains(tableName)); + Assert.assertTrue(message.contains(columnFamily)); + return; + } + Assert.fail("Exception wasn't thrown even though Cells were appended in the wrong order!"); + } /** * Create 0-length hfile and show that it fails */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index f029a39d94db..4cd222577bd6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -144,6 +145,7 @@ public class TestStore { */ @Before public void setUp() throws IOException { + qualifiers.clear(); qualifiers.add(qf1); qualifiers.add(qf3); qualifiers.add(qf5); @@ -1449,6 +1451,16 @@ public void hook(int size) { } } + @Test + public void testHFileContextSetWithCFAndTable() throws Exception { + init(this.name.getMethodName()); + StoreFile.Writer writer = store.createWriterInTmp(10000L, + Compression.Algorithm.NONE, false, true, false, true); + HFileContext hFileContext = writer.getHFileWriter().getFileContext(); + assertArrayEquals(family, hFileContext.getColumnFamily()); + assertArrayEquals(table, hFileContext.getTableName()); + } + private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); HColumnDescriptor hcd = new HColumnDescriptor(family); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java index 028937c37a62..e72bec0b175d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java @@ -104,6 +104,7 @@ public static void createHFile( HFileContext meta = new HFileContextBuilder() .withIncludesTags(withTag) .withDataBlockEncoding(encoding) + .withColumnFamily(family) .build(); HFile.Writer writer = HFile.getWriterFactory(configuration, new CacheConfig(configuration)) .withPath(fs, path) From eee337f5c3352874ff225497066bf6f584f3ed0b Mon Sep 17 00:00:00 2001 From: Baiqiang Zhao Date: Tue, 19 Nov 2019 18:55:40 +0800 Subject: [PATCH 29/51] HBASE-23278 Add a table-level compaction progress display on the UI (#817) Signed-off-by: Guangxu Cheng --- .../resources/hbase-webapps/master/table.jsp | 542 ++++++++++++------ 1 file changed, 366 insertions(+), 176 deletions(-) diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index b45d633d08e1..3d5e2303bc8b 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -159,10 +159,7 @@ if ( fqtn != null ) { table = new HTable(conf, fqtn); if (table.getTableDescriptor().getRegionReplication() > 1) { - tableHeader = "

Table Regions

NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd Key
"; withReplica = true; - } else { - tableHeader = "

Table Regions

NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd KeyReplicaID
"; } if ( !readOnly && action != null ) { %> @@ -213,70 +210,168 @@ if ( fqtn != null ) { <% if(fqtn.equals(TableName.META_TABLE_NAME.getNameAsString())) { %> -<%= tableHeader %> - -<% - // NOTE: Presumes meta with one or more replicas - for (int j = 0; j < numMetaReplicas; j++) { - HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica( - HRegionInfo.FIRST_META_REGIONINFO, j); - ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1); - for (int i = 0; i < 1; i++) { - String url = ""; - String readReq = "N/A"; - String writeReq = "N/A"; - String fileSize = "N/A"; - String fileCount = "N/A"; - String memSize = "N/A"; - float locality = 0.0f; +

Table Regions

+
+ +
+
+
NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd Key
+ + + + + + + + + + + + + <% + if (withReplica) { + %> + + <% + } + %> + + + + <% + // NOTE: Presumes meta with one or more replicas + for (int j = 0; j < numMetaReplicas; j++) { + HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica( + HRegionInfo.FIRST_META_REGIONINFO, j); + ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1); + for (int i = 0; i < 1; i++) { + String url = ""; + String readReq = "N/A"; + String writeReq = "N/A"; + String fileSize = "N/A"; + String fileCount = "N/A"; + String memSize = "N/A"; + float locality = 0.0f; - if (metaLocation != null) { - ServerLoad sl = master.getServerManager().getLoad(metaLocation); - // The host name portion should be safe, but I don't know how we handle IDNs so err on the side of failing safely. - url = "//" + URLEncoder.encode(metaLocation.getHostname()) + ":" + master.getRegionServerInfoPort(metaLocation) + "/"; - if (sl != null) { - Map map = sl.getRegionsLoad(); - if (map.containsKey(meta.getRegionName())) { - RegionLoad load = map.get(meta.getRegionName()); - readReq = String.format("%,1d", load.getReadRequestsCount()); - writeReq = String.format("%,1d", load.getWriteRequestsCount()); - fileSize = StringUtils.byteDesc(load.getStorefileSizeMB()*1024l*1024); - fileCount = String.format("%,1d", load.getStorefiles()); - memSize = StringUtils.byteDesc(load.getMemStoreSizeMB()*1024l*1024); - locality = load.getDataLocality(); - } - } - } -%> - - <% - String metaLocationString = metaLocation != null ? - StringEscapeUtils.escapeHtml(metaLocation.getHostname().toString()) - + ":" + master.getRegionServerInfoPort(metaLocation) : - "(null)"; - %> - - - - - - - - - - -<% - if (withReplica) { -%> - -<% - } -%> - -<% } %> -<%} %> - -
NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd KeyReplicaID
<%= escapeXml(meta.getRegionNameAsString()) %><%= metaLocationString %><%= readReq%><%= writeReq%><%= fileSize%><%= fileCount%><%= memSize%><%= locality%><%= escapeXml(Bytes.toString(meta.getStartKey())) %><%= escapeXml(Bytes.toString(meta.getEndKey())) %><%= meta.getReplicaId() %>
+ if (metaLocation != null) { + ServerLoad sl = master.getServerManager().getLoad(metaLocation); + // The host name portion should be safe, but I don't know how we handle IDNs so err on the side of failing safely. + url = "//" + URLEncoder.encode(metaLocation.getHostname()) + ":" + master.getRegionServerInfoPort(metaLocation) + "/"; + if (sl != null) { + Map map = sl.getRegionsLoad(); + if (map.containsKey(meta.getRegionName())) { + RegionLoad load = map.get(meta.getRegionName()); + readReq = String.format("%,1d", load.getReadRequestsCount()); + writeReq = String.format("%,1d", load.getWriteRequestsCount()); + fileSize = StringUtils.byteDesc(load.getStorefileSizeMB()*1024l*1024); + fileCount = String.format("%,1d", load.getStorefiles()); + memSize = StringUtils.byteDesc(load.getMemStoreSizeMB()*1024l*1024); + locality = load.getDataLocality(); + } + } + } + %> + + <% + String metaLocationString = metaLocation != null ? + StringEscapeUtils.escapeHtml(metaLocation.getHostname().toString()) + + ":" + master.getRegionServerInfoPort(metaLocation) : + "(null)"; + %> + <%= escapeXml(meta.getRegionNameAsString()) %> + <%= metaLocationString %> + <%= readReq%> + <%= writeReq%> + <%= fileSize%> + <%= fileCount%> + <%= memSize%> + <%= locality%> + <%= escapeXml(Bytes.toString(meta.getStartKey())) %> + <%= escapeXml(Bytes.toString(meta.getEndKey())) %> + <% + if (withReplica) { + %> + <%= meta.getReplicaId() %> + <% + } + %> + + <% } %> + <%} %> + + + +
+ + + + + + + + + + + + + <% + // NOTE: Presumes meta with one or more replicas + for (int j = 0; j < numMetaReplicas; j++) { + HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica( + HRegionInfo.FIRST_META_REGIONINFO, j); + ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1); + for (int i = 0; i < 1; i++) { + String url = ""; + long compactingKVs = 0; + long compactedKVs = 0; + String compactionProgress = ""; + + if (metaLocation != null) { + ServerLoad sl = master.getServerManager().getLoad(metaLocation); + // The host name portion should be safe, but I don't know how we handle IDNs so err on the side of failing safely. + url = "//" + URLEncoder.encode(metaLocation.getHostname()) + ":" + master.getRegionServerInfoPort(metaLocation) + "/"; + if (sl != null) { + Map map = sl.getRegionsLoad(); + if (map.containsKey(meta.getRegionName())) { + RegionLoad load = map.get(meta.getRegionName()); + compactingKVs = load.getTotalCompactingKVs(); + compactedKVs = load.getCurrentCompactedKVs(); + if (compactingKVs > 0) { + compactionProgress = String.format("%.2f", 100 * ((float) + compactedKVs / compactingKVs)) + "%"; + } + } + } + } + %> + + <% + String metaLocationString = metaLocation != null ? + StringEscapeUtils.escapeHtml(metaLocation.getHostname().toString()) + + ":" + master.getRegionServerInfoPort(metaLocation) : + "(null)"; + %> + + + + + + + + <% } %> + <%} %> + +
NameRegion ServerNum. Compacting KVsNum. Compacted KVsRemaining KVsCompaction Progress
<%= escapeXml(meta.getRegionNameAsString()) %><%= metaLocationString %><%= String.format("%,1d", compactingKVs)%><%= String.format("%,1d", compactedKVs)%><%= String.format("%,1d", compactingKVs - compactedKVs)%><%= compactionProgress%>
+
+ + <%} else { Admin admin = master.getConnection().getAdmin(); try { %> @@ -363,6 +458,9 @@ if ( fqtn != null ) { long totalSize = 0; long totalStoreFileCount = 0; long totalMemSize = 0; + long totalCompactingKVs = 0; + long totalCompactedKVs = 0; + String percentDone = ""; String urlRegionServer = null; Map regDistribution = new TreeMap(); Map primaryRegDistribution = new TreeMap(); @@ -388,6 +486,8 @@ if ( fqtn != null ) { totalStoreFileCount += regionload.getStorefiles(); totalMemSize += regionload.getMemStoreSizeMB(); totalStoreFileSizeMB += regionload.getStorefileSizeMB(); + totalCompactingKVs += regionload.getTotalCompactingKVs(); + totalCompactedKVs += regionload.getCurrentCompactedKVs(); } else { RegionLoad load0 = new RegionLoad(ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(HBaseProtos.RegionSpecifier.newBuilder().setValue(ByteString.copyFrom(regionInfo.getRegionName())).build()).build()); regionsToLoad.put(regionInfo, load0); @@ -401,121 +501,209 @@ if ( fqtn != null ) { regionsToLoad.put(regionInfo, load0); } } + if (totalCompactingKVs > 0) { + percentDone = String.format("%.2f", 100 * + ((float) totalCompactedKVs / totalCompactingKVs)) + "%"; + } if(regions != null && regions.size() > 0) { %>

Table Regions

- - - - - - - - - - - - - -<% - if (withReplica) { -%> - -<% - } -%> - - - +
+ +
+
+
Name(<%= String.format("%,1d", regions.size())%>)Region ServerReadRequests
(<%= String.format("%,1d", totalReadReq)%>)
WriteRequests
(<%= String.format("%,1d", totalWriteReq)%>)
StorefileSize
(<%= StringUtils.byteDesc(totalSize*1024l*1024)%>)
Num.Storefiles
(<%= String.format("%,1d", totalStoreFileCount)%>)
MemSize
(<%= StringUtils.byteDesc(totalMemSize*1024l*1024)%>)
LocalityStart KeyEnd KeyReplicaID
+ + + + + + + + + + + + + <% + if (withReplica) { + %> + + <% + } + %> + + + + <% + List> entryList = new ArrayList>(regionsToLoad.entrySet()); + numRegions = regions.size(); + int numRegionsRendered = 0; + // render all regions + if (numRegionsToRender < 0) { + numRegionsToRender = numRegions; + } + for (Map.Entry hriEntry : entryList) { + HRegionInfo regionInfo = hriEntry.getKey(); + ServerName addr = regions.get(regionInfo); + RegionLoad load = hriEntry.getValue(); + String readReq = "N/A"; + String writeReq = "N/A"; + String regionSize = "N/A"; + String fileCount = "N/A"; + String memSize = "N/A"; + float locality = 0.0f; + if(load != null) { + readReq = String.format("%,1d", load.getReadRequestsCount()); + writeReq = String.format("%,1d", load.getWriteRequestsCount()); + regionSize = StringUtils.byteDesc(load.getStorefileSizeMB()*1024l*1024); + fileCount = String.format("%,1d", load.getStorefiles()); + memSize = StringUtils.byteDesc(load.getMemStoreSizeMB()*1024l*1024); + locality = load.getDataLocality(); + } -<% - List> entryList = new ArrayList>(regionsToLoad.entrySet()); - numRegions = regions.size(); - int numRegionsRendered = 0; - // render all regions - if (numRegionsToRender < 0) { - numRegionsToRender = numRegions; - } - for (Map.Entry hriEntry : entryList) { - HRegionInfo regionInfo = hriEntry.getKey(); - ServerName addr = regions.get(regionInfo); - RegionLoad load = hriEntry.getValue(); - String readReq = "N/A"; - String writeReq = "N/A"; - String regionSize = "N/A"; - String fileCount = "N/A"; - String memSize = "N/A"; - float locality = 0.0f; - if(load != null) { - readReq = String.format("%,1d", load.getReadRequestsCount()); - writeReq = String.format("%,1d", load.getWriteRequestsCount()); - regionSize = StringUtils.byteDesc(load.getStorefileSizeMB()*1024l*1024); - fileCount = String.format("%,1d", load.getStorefiles()); - memSize = StringUtils.byteDesc(load.getMemStoreSizeMB()*1024l*1024); - locality = load.getDataLocality(); - } + if (addr != null) { + ServerLoad sl = master.getServerManager().getLoad(addr); + // This port might be wrong if RS actually ended up using something else. + urlRegionServer = + "//" + URLEncoder.encode(addr.getHostname()) + ":" + master.getRegionServerInfoPort(addr) + "/"; + if(sl != null) { + Integer i = regDistribution.get(addr); + if (null == i) i = Integer.valueOf(0); + regDistribution.put(addr, i + 1); + if (withReplica && RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) { + i = primaryRegDistribution.get(addr); + if (null == i) i = Integer.valueOf(0); + primaryRegDistribution.put(addr, i+1); + } + } + } + if (numRegionsRendered < numRegionsToRender) { + numRegionsRendered++; + %> + + + <% + if (urlRegionServer != null) { + %> + + <% + } else { + %> + + <% + } + %> + + + + + + + + + <% + if (withReplica) { + %> + + <% + } + %> + + <% } %> + <% } %> + +
Name(<%= String.format("%,1d", regions.size())%>)Region ServerReadRequests
(<%= String.format("%,1d", totalReadReq)%>)
WriteRequests
(<%= String.format("%,1d", totalWriteReq)%>)
StorefileSize
(<%= StringUtils.byteDesc(totalSize*1024l*1024)%>)
Num.Storefiles
(<%= String.format("%,1d", totalStoreFileCount)%>)
MemSize
(<%= StringUtils.byteDesc(totalMemSize*1024l*1024)%>)
LocalityStart KeyEnd KeyReplicaID
<%= escapeXml(Bytes.toStringBinary(regionInfo.getRegionName())) %> + <%= StringEscapeUtils.escapeHtml(addr.getHostname().toString()) + ":" + master.getRegionServerInfoPort(addr) %> + not deployed<%= readReq%><%= writeReq%><%= regionSize%><%= fileCount%><%= memSize%><%= locality%><%= escapeXml(Bytes.toStringBinary(regionInfo.getStartKey()))%><%= escapeXml(Bytes.toStringBinary(regionInfo.getEndKey()))%><%= regionInfo.getReplicaId() %>
+ <% if (numRegions > numRegionsRendered) { + String allRegionsUrl = "?name=" + URLEncoder.encode(fqtn,"UTF-8") + "&numRegions=all"; + %> +

This table has <%= numRegions %> regions in total, in order to improve the page load time, + only <%= numRegionsRendered %> regions are displayed here, click + here to see all regions.

+ <% } %> + +
+ + + + + + + + + + + + + <% + numRegionsRendered = 0; + for (Map.Entry hriEntry : entryList) { + HRegionInfo regionInfo = hriEntry.getKey(); + ServerName addr = regions.get(regionInfo); + RegionLoad load = hriEntry.getValue(); + long compactingKVs = 0; + long compactedKVs = 0; + String compactionProgress = ""; + if(load != null) { + compactingKVs = load.getTotalCompactingKVs(); + compactedKVs = load.getCurrentCompactedKVs(); + if (compactingKVs > 0) { + compactionProgress = String.format("%.2f", 100 * ((float) + compactedKVs / compactingKVs)) + "%"; + } + } - if (addr != null) { - ServerLoad sl = master.getServerManager().getLoad(addr); - // This port might be wrong if RS actually ended up using something else. - urlRegionServer = - "//" + URLEncoder.encode(addr.getHostname()) + ":" + master.getRegionServerInfoPort(addr) + "/"; - if(sl != null) { - Integer i = regDistribution.get(addr); - if (null == i) i = Integer.valueOf(0); - regDistribution.put(addr, i + 1); - if (withReplica && RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) { - i = primaryRegDistribution.get(addr); - if (null == i) i = Integer.valueOf(0); - primaryRegDistribution.put(addr, i+1); - } - } - } - if (numRegionsRendered < numRegionsToRender) { - numRegionsRendered++; -%> - - - <% - if (urlRegionServer != null) { - %> - - <% - } else { - %> - - <% - } - %> - - - - - - - - - <% - if (withReplica) { - %> - - <% - } - %> - -<% } %> -<% } %> - -
Name(<%= String.format("%,1d", regions.size())%>)Region ServerNum. Compacting KVs
(<%= String.format("%,1d", totalCompactingKVs)%>)
Num. Compacted KVs
(<%= String.format("%,1d", totalCompactedKVs)%>)
Remaining KVs
(<%= String.format("%,1d", totalCompactingKVs - totalCompactedKVs)%>)
Compaction Progress
(<%= percentDone %>)
<%= escapeXml(Bytes.toStringBinary(regionInfo.getRegionName())) %> - <%= StringEscapeUtils.escapeHtml(addr.getHostname().toString()) + ":" + master.getRegionServerInfoPort(addr) %> - not deployed<%= readReq%><%= writeReq%><%= regionSize%><%= fileCount%><%= memSize%><%= locality%><%= escapeXml(Bytes.toStringBinary(regionInfo.getStartKey()))%><%= escapeXml(Bytes.toStringBinary(regionInfo.getEndKey()))%><%= regionInfo.getReplicaId() %>
-<% if (numRegions > numRegionsRendered) { - String allRegionsUrl = "?name=" + URLEncoder.encode(fqtn,"UTF-8") + "&numRegions=all"; -%> -

This table has <%= numRegions %> regions in total, in order to improve the page load time, - only <%= numRegionsRendered %> regions are displayed here, click - here to see all regions.

-<% } %> + if (addr != null) { + // This port might be wrong if RS actually ended up using something else. + urlRegionServer = + "//" + URLEncoder.encode(addr.getHostname()) + ":" + master.getRegionServerInfoPort(addr) + "/"; + } + if (numRegionsRendered < numRegionsToRender) { + numRegionsRendered++; + %> + + <%= escapeXml(Bytes.toStringBinary(regionInfo.getRegionName())) %> + <% + if (urlRegionServer != null) { + %> + + <%= StringEscapeUtils.escapeHtml(addr.getHostname().toString()) + ":" + master.getRegionServerInfoPort(addr) %> + + <% + } else { + %> + not deployed + <% + } + %> + <%= String.format("%,1d", compactingKVs)%> + <%= String.format("%,1d", compactedKVs)%> + <%= String.format("%,1d", compactingKVs - compactedKVs)%> + <%= compactionProgress%> + + <% } %> + <% } %> + + + <% if (numRegions > numRegionsRendered) { + String allRegionsUrl = "?name=" + URLEncoder.encode(fqtn,"UTF-8") + "&numRegions=all"; + %> +

This table has <%= numRegions %> regions in total, in order to improve the page load time, + only <%= numRegionsRendered %> regions are displayed here, click + here to see all regions.

+ <% } %> +
+ +

Regions by Region Server

<% if (withReplica) { @@ -652,6 +840,8 @@ $(document).ready(function() $("#regionServerTable").tablesorter(); $("#regionServerDetailsTable").tablesorter(); $("#tableRegionTable").tablesorter(); + $("#tableCompactStatsTable").tablesorter(); + $("#metaTableCompactStatsTable").tablesorter(); } ); From eb5e94aced6b4feec10b2ffac56e94a994d9c157 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 21 Nov 2019 09:47:05 -0800 Subject: [PATCH 30/51] HBASE-23259: Populate master address end points in cluster/rs configs (#807) (#858) All the clients need to know the master RPC end points while using master based registry for creating cluster connections. This patch amends the test cluster utility to populate these configs in the base configuration object used to spin up the cluster. The config key added here ("hbase.master.addrs") is used in the subsequent patches for HBASE-18095. Signed-off-by: Nick Dimiduk (cherry picked from commit 834ccb4bf6c22fc2a8aab172490fba75c7a40f1c) --- .../org/apache/hadoop/hbase/HConstants.java | 5 ++ .../hadoop/hbase/LocalHBaseCluster.java | 12 ++++- .../hadoop/hbase/TestHBaseTestingUtility.java | 54 ++++++++++++++++--- 3 files changed, 64 insertions(+), 7 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 16d96a2fc5e5..4607de986a06 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -175,6 +175,11 @@ public enum OperationStatusCode { /** Configuration key for master web API port */ public static final String MASTER_INFO_PORT = "hbase.master.info.port"; + /** Configuration key for the list of master host:ports **/ + public static final String MASTER_ADDRS_KEY = "hbase.master.addrs"; + + public static final String MASTER_ADDRS_DEFAULT = "localhost:" + DEFAULT_MASTER_PORT; + /** Parameter name for the master type being backup (waits for primary to go inactive). */ public static final String MASTER_TYPE_BACKUP = "hbase.master.backup"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index 042037e159bf..4f9cd3b8f264 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; +import com.google.common.base.Joiner; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -161,6 +162,15 @@ public LocalHBaseCluster(final Configuration conf, final int noMasters, for (int i = 0; i < noMasters; i++) { addMaster(new Configuration(conf), i); } + + // Populate the master address host ports in the config. This is needed if a master based + // registry is configured for client metadata services (HBASE-18095) + List masterHostPorts = new ArrayList<>(); + for (JVMClusterUtil.MasterThread masterThread: getMasters()) { + masterHostPorts.add(masterThread.getMaster().getServerName().getAddress().toString()); + } + conf.set(HConstants.MASTER_ADDRS_KEY, Joiner.on(",").join(masterHostPorts)); + // Start the HRegionServers. this.regionServerClass = (Class)conf.getClass(HConstants.REGION_SERVER_IMPL, @@ -214,7 +224,7 @@ public JVMClusterUtil.MasterThread addMaster() throws IOException { } public JVMClusterUtil.MasterThread addMaster(Configuration c, final int index) - throws IOException { + throws IOException { // Create each master with its own Configuration instance so each has // its HConnection instance rather than share (see HBASE_INSTANCES down in // the guts of HConnectionManager. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java index 82d9ae015626..96fa8ea31be9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,21 +17,19 @@ */ package org.apache.hadoop.hbase; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; - import java.io.File; import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Random; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -420,5 +417,50 @@ public void testMiniZooKeeperWithMultipleClientPorts() throws Exception { hbt.shutdownMiniMapReduceCluster(); } + + @Test + public void testOverridingOfDefaultPorts() throws Exception { + // confirm that default port properties being overridden to random + Configuration defaultConfig = HBaseConfiguration.create(); + defaultConfig.setInt(HConstants.MASTER_INFO_PORT, HConstants.DEFAULT_MASTER_INFOPORT); + defaultConfig.setInt(HConstants.REGIONSERVER_INFO_PORT, + HConstants.DEFAULT_REGIONSERVER_INFOPORT); + HBaseTestingUtility htu = new HBaseTestingUtility(defaultConfig); + try { + MiniHBaseCluster defaultCluster = htu.startMiniCluster(); + final String masterHostPort = + defaultCluster.getMaster().getServerName().getAddress().toString(); + assertNotEquals(HConstants.DEFAULT_MASTER_INFOPORT, + defaultCluster.getConfiguration().getInt(HConstants.MASTER_INFO_PORT, 0)); + assertNotEquals(HConstants.DEFAULT_REGIONSERVER_INFOPORT, + defaultCluster.getConfiguration().getInt(HConstants.REGIONSERVER_INFO_PORT, 0)); + assertEquals(masterHostPort, + defaultCluster.getConfiguration().get(HConstants.MASTER_ADDRS_KEY)); + } finally { + htu.shutdownMiniCluster(); + } + + // confirm that nonDefault (custom) port settings are NOT overridden + Configuration altConfig = HBaseConfiguration.create(); + final int nonDefaultMasterInfoPort = 3333; + final int nonDefaultRegionServerPort = 4444; + altConfig.setInt(HConstants.MASTER_INFO_PORT, nonDefaultMasterInfoPort); + altConfig.setInt(HConstants.REGIONSERVER_INFO_PORT, nonDefaultRegionServerPort); + altConfig.setBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, false); + htu = new HBaseTestingUtility(altConfig); + try { + MiniHBaseCluster customCluster = htu.startMiniCluster(); + final String masterHostPort = + customCluster.getMaster().getServerName().getAddress().toString(); + assertEquals(nonDefaultMasterInfoPort, + customCluster.getConfiguration().getInt(HConstants.MASTER_INFO_PORT, 0)); + assertEquals(nonDefaultRegionServerPort, + customCluster.getConfiguration().getInt(HConstants.REGIONSERVER_INFO_PORT, 0)); + assertEquals(masterHostPort, + customCluster.getConfiguration().get(HConstants.MASTER_ADDRS_KEY)); + } finally { + htu.shutdownMiniCluster(); + } + } } From 38ae0b561dc3ad2a1fafd58ad264332fbbf99c8f Mon Sep 17 00:00:00 2001 From: Nick Dimiduk Date: Thu, 21 Nov 2019 11:09:00 -0800 Subject: [PATCH 31/51] HBASE-23234 Provide .editorconfig based on checkstyle configuration (#846) This file is generated using IntelliJ, following these steps: #. Open Preferences > Editor > Code Style #. Select (config wheel) > Import Schema > CheckStyle Configuration #. Select `hbase-checkstyle/src/main/resources/hbase/checkstyle.xml` #. Select (config wheel) > Export > EditorConfig File Signed-off-by: Sean Busbey --- .editorconfig | 693 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 693 insertions(+) create mode 100644 .editorconfig diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 000000000000..011673016c26 --- /dev/null +++ b/.editorconfig @@ -0,0 +1,693 @@ +# +# 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. +# + +[*] +charset = utf-8 +end_of_line = lf +indent_size = 2 +indent_style = space +insert_final_newline = true +max_line_length = 100 +tab_width = 2 +ij_continuation_indent_size = 2 +ij_formatter_off_tag = @formatter:off +ij_formatter_on_tag = @formatter:on +ij_formatter_tags_enabled = false +ij_smart_tabs = false +ij_wrap_on_typing = false + +[*.css] +ij_css_align_closing_brace_with_properties = false +ij_css_blank_lines_around_nested_selector = 1 +ij_css_blank_lines_between_blocks = 1 +ij_css_brace_placement = 0 +ij_css_hex_color_long_format = false +ij_css_hex_color_lower_case = false +ij_css_hex_color_short_format = false +ij_css_hex_color_upper_case = false +ij_css_keep_blank_lines_in_code = 2 +ij_css_keep_indents_on_empty_lines = false +ij_css_keep_single_line_blocks = false +ij_css_properties_order = font,font-family,font-size,font-weight,font-style,font-variant,font-size-adjust,font-stretch,line-height,position,z-index,top,right,bottom,left,display,visibility,float,clear,overflow,overflow-x,overflow-y,clip,zoom,align-content,align-items,align-self,flex,flex-flow,flex-basis,flex-direction,flex-grow,flex-shrink,flex-wrap,justify-content,order,box-sizing,width,min-width,max-width,height,min-height,max-height,margin,margin-top,margin-right,margin-bottom,margin-left,padding,padding-top,padding-right,padding-bottom,padding-left,table-layout,empty-cells,caption-side,border-spacing,border-collapse,list-style,list-style-position,list-style-type,list-style-image,content,quotes,counter-reset,counter-increment,resize,cursor,user-select,nav-index,nav-up,nav-right,nav-down,nav-left,transition,transition-delay,transition-timing-function,transition-duration,transition-property,transform,transform-origin,animation,animation-name,animation-duration,animation-play-state,animation-timing-function,animation-delay,animation-iteration-count,animation-direction,text-align,text-align-last,vertical-align,white-space,text-decoration,text-emphasis,text-emphasis-color,text-emphasis-style,text-emphasis-position,text-indent,text-justify,letter-spacing,word-spacing,text-outline,text-transform,text-wrap,text-overflow,text-overflow-ellipsis,text-overflow-mode,word-wrap,word-break,tab-size,hyphens,pointer-events,opacity,color,border,border-width,border-style,border-color,border-top,border-top-width,border-top-style,border-top-color,border-right,border-right-width,border-right-style,border-right-color,border-bottom,border-bottom-width,border-bottom-style,border-bottom-color,border-left,border-left-width,border-left-style,border-left-color,border-radius,border-top-left-radius,border-top-right-radius,border-bottom-right-radius,border-bottom-left-radius,border-image,border-image-source,border-image-slice,border-image-width,border-image-outset,border-image-repeat,outline,outline-width,outline-style,outline-color,outline-offset,background,background-color,background-image,background-repeat,background-attachment,background-position,background-position-x,background-position-y,background-clip,background-origin,background-size,box-decoration-break,box-shadow,text-shadow +ij_css_space_after_colon = true +ij_css_space_before_opening_brace = true +ij_css_value_alignment = 0 + +[*.java] +ij_java_align_consecutive_assignments = false +ij_java_align_consecutive_variable_declarations = false +ij_java_align_group_field_declarations = false +ij_java_align_multiline_annotation_parameters = false +ij_java_align_multiline_array_initializer_expression = false +ij_java_align_multiline_assignment = false +ij_java_align_multiline_binary_operation = false +ij_java_align_multiline_chained_methods = false +ij_java_align_multiline_extends_list = false +ij_java_align_multiline_for = true +ij_java_align_multiline_method_parentheses = false +ij_java_align_multiline_parameters = false +ij_java_align_multiline_parameters_in_calls = false +ij_java_align_multiline_parenthesized_expression = false +ij_java_align_multiline_resources = false +ij_java_align_multiline_ternary_operation = false +ij_java_align_multiline_throws_list = false +ij_java_align_subsequent_simple_methods = false +ij_java_align_throws_keyword = false +ij_java_annotation_parameter_wrap = off +ij_java_array_initializer_new_line_after_left_brace = false +ij_java_array_initializer_right_brace_on_new_line = false +ij_java_array_initializer_wrap = normal +ij_java_assert_statement_colon_on_next_line = false +ij_java_assert_statement_wrap = off +ij_java_assignment_wrap = normal +ij_java_binary_operation_sign_on_next_line = true +ij_java_binary_operation_wrap = normal +ij_java_blank_lines_after_anonymous_class_header = 0 +ij_java_blank_lines_after_class_header = 0 +ij_java_blank_lines_after_imports = 1 +ij_java_blank_lines_after_package = 1 +ij_java_blank_lines_around_class = 1 +ij_java_blank_lines_around_field = 0 +ij_java_blank_lines_around_field_in_interface = 0 +ij_java_blank_lines_around_initializer = 1 +ij_java_blank_lines_around_method = 1 +ij_java_blank_lines_around_method_in_interface = 1 +ij_java_blank_lines_before_class_end = 0 +ij_java_blank_lines_before_imports = 1 +ij_java_blank_lines_before_method_body = 0 +ij_java_blank_lines_before_package = 0 +ij_java_block_brace_style = end_of_line +ij_java_block_comment_at_first_column = true +ij_java_call_parameters_new_line_after_left_paren = false +ij_java_call_parameters_right_paren_on_new_line = false +ij_java_call_parameters_wrap = normal +ij_java_case_statement_on_separate_line = true +ij_java_catch_on_new_line = false +ij_java_class_annotation_wrap = normal +ij_java_class_brace_style = end_of_line +ij_java_class_count_to_use_import_on_demand = 999 +ij_java_class_names_in_javadoc = 1 +ij_java_do_not_indent_top_level_class_members = false +ij_java_do_not_wrap_after_single_annotation = false +ij_java_do_while_brace_force = always +ij_java_doc_add_blank_line_after_description = true +ij_java_doc_add_blank_line_after_param_comments = false +ij_java_doc_add_blank_line_after_return = false +ij_java_doc_add_p_tag_on_empty_lines = true +ij_java_doc_align_exception_comments = true +ij_java_doc_align_param_comments = true +ij_java_doc_do_not_wrap_if_one_line = false +ij_java_doc_enable_formatting = true +ij_java_doc_enable_leading_asterisks = true +ij_java_doc_indent_on_continuation = false +ij_java_doc_keep_empty_lines = false +ij_java_doc_keep_empty_parameter_tag = true +ij_java_doc_keep_empty_return_tag = true +ij_java_doc_keep_empty_throws_tag = true +ij_java_doc_keep_invalid_tags = true +ij_java_doc_param_description_on_new_line = false +ij_java_doc_preserve_line_breaks = false +ij_java_doc_use_throws_not_exception_tag = true +ij_java_else_on_new_line = false +ij_java_entity_dd_suffix = EJB +ij_java_entity_eb_suffix = Bean +ij_java_entity_hi_suffix = Home +ij_java_entity_lhi_prefix = Local +ij_java_entity_lhi_suffix = Home +ij_java_entity_li_prefix = Local +ij_java_entity_pk_class = java.lang.String +ij_java_entity_vo_suffix = VO +ij_java_enum_constants_wrap = off +ij_java_extends_keyword_wrap = normal +ij_java_extends_list_wrap = normal +ij_java_field_annotation_wrap = normal +ij_java_finally_on_new_line = false +ij_java_for_brace_force = always +ij_java_for_statement_new_line_after_left_paren = false +ij_java_for_statement_right_paren_on_new_line = false +ij_java_for_statement_wrap = off +ij_java_generate_final_locals = false +ij_java_generate_final_parameters = false +ij_java_if_brace_force = always +ij_java_imports_layout = $*,*,org.apache.hbase.thirdparty.**,org.apache.hadoop.hbase.shaded.** +ij_java_indent_case_from_switch = true +ij_java_insert_inner_class_imports = false +ij_java_insert_override_annotation = true +ij_java_keep_blank_lines_before_right_brace = 1 +ij_java_keep_blank_lines_between_package_declaration_and_header = 2 +ij_java_keep_blank_lines_in_code = 1 +ij_java_keep_blank_lines_in_declarations = 1 +ij_java_keep_control_statement_in_one_line = true +ij_java_keep_first_column_comment = false +ij_java_keep_indents_on_empty_lines = false +ij_java_keep_line_breaks = false +ij_java_keep_multiple_expressions_in_one_line = false +ij_java_keep_simple_blocks_in_one_line = false +ij_java_keep_simple_classes_in_one_line = false +ij_java_keep_simple_lambdas_in_one_line = false +ij_java_keep_simple_methods_in_one_line = false +ij_java_lambda_brace_style = end_of_line +ij_java_layout_static_imports_separately = true +ij_java_line_comment_add_space = false +ij_java_line_comment_at_first_column = true +ij_java_message_dd_suffix = EJB +ij_java_message_eb_suffix = Bean +ij_java_method_annotation_wrap = normal +ij_java_method_brace_style = end_of_line +ij_java_method_call_chain_wrap = normal +ij_java_method_parameters_new_line_after_left_paren = false +ij_java_method_parameters_right_paren_on_new_line = false +ij_java_method_parameters_wrap = normal +ij_java_modifier_list_wrap = false +ij_java_names_count_to_use_import_on_demand = 999 +ij_java_parameter_annotation_wrap = normal +ij_java_parentheses_expression_new_line_after_left_paren = false +ij_java_parentheses_expression_right_paren_on_new_line = false +ij_java_place_assignment_sign_on_next_line = false +ij_java_prefer_longer_names = true +ij_java_prefer_parameters_wrap = false +ij_java_repeat_synchronized = true +ij_java_replace_instanceof_and_cast = false +ij_java_replace_null_check = true +ij_java_replace_sum_lambda_with_method_ref = true +ij_java_resource_list_new_line_after_left_paren = false +ij_java_resource_list_right_paren_on_new_line = false +ij_java_resource_list_wrap = on_every_item +ij_java_session_dd_suffix = EJB +ij_java_session_eb_suffix = Bean +ij_java_session_hi_suffix = Home +ij_java_session_lhi_prefix = Local +ij_java_session_lhi_suffix = Home +ij_java_session_li_prefix = Local +ij_java_session_si_suffix = Service +ij_java_space_after_closing_angle_bracket_in_type_argument = false +ij_java_space_after_colon = true +ij_java_space_after_comma = true +ij_java_space_after_comma_in_type_arguments = true +ij_java_space_after_for_semicolon = true +ij_java_space_after_quest = true +ij_java_space_after_type_cast = true +ij_java_space_before_annotation_array_initializer_left_brace = false +ij_java_space_before_annotation_parameter_list = false +ij_java_space_before_array_initializer_left_brace = true +ij_java_space_before_catch_keyword = true +ij_java_space_before_catch_left_brace = true +ij_java_space_before_catch_parentheses = true +ij_java_space_before_class_left_brace = true +ij_java_space_before_colon = true +ij_java_space_before_colon_in_foreach = true +ij_java_space_before_comma = false +ij_java_space_before_do_left_brace = true +ij_java_space_before_else_keyword = true +ij_java_space_before_else_left_brace = true +ij_java_space_before_finally_keyword = true +ij_java_space_before_finally_left_brace = true +ij_java_space_before_for_left_brace = true +ij_java_space_before_for_parentheses = true +ij_java_space_before_for_semicolon = false +ij_java_space_before_if_left_brace = true +ij_java_space_before_if_parentheses = true +ij_java_space_before_method_call_parentheses = false +ij_java_space_before_method_left_brace = true +ij_java_space_before_method_parentheses = false +ij_java_space_before_opening_angle_bracket_in_type_parameter = false +ij_java_space_before_quest = true +ij_java_space_before_switch_left_brace = true +ij_java_space_before_switch_parentheses = true +ij_java_space_before_synchronized_left_brace = true +ij_java_space_before_synchronized_parentheses = true +ij_java_space_before_try_left_brace = true +ij_java_space_before_try_parentheses = true +ij_java_space_before_type_parameter_list = false +ij_java_space_before_while_keyword = true +ij_java_space_before_while_left_brace = true +ij_java_space_before_while_parentheses = true +ij_java_space_inside_one_line_enum_braces = false +ij_java_space_within_empty_array_initializer_braces = false +ij_java_space_within_empty_method_call_parentheses = false +ij_java_space_within_empty_method_parentheses = false +ij_java_spaces_around_additive_operators = true +ij_java_spaces_around_assignment_operators = true +ij_java_spaces_around_bitwise_operators = true +ij_java_spaces_around_equality_operators = true +ij_java_spaces_around_lambda_arrow = true +ij_java_spaces_around_logical_operators = true +ij_java_spaces_around_method_ref_dbl_colon = false +ij_java_spaces_around_multiplicative_operators = true +ij_java_spaces_around_relational_operators = true +ij_java_spaces_around_shift_operators = true +ij_java_spaces_around_type_bounds_in_type_parameters = true +ij_java_spaces_around_unary_operator = false +ij_java_spaces_within_angle_brackets = false +ij_java_spaces_within_annotation_parentheses = false +ij_java_spaces_within_array_initializer_braces = true +ij_java_spaces_within_braces = false +ij_java_spaces_within_brackets = false +ij_java_spaces_within_cast_parentheses = false +ij_java_spaces_within_catch_parentheses = false +ij_java_spaces_within_for_parentheses = false +ij_java_spaces_within_if_parentheses = false +ij_java_spaces_within_method_call_parentheses = false +ij_java_spaces_within_method_parentheses = false +ij_java_spaces_within_parentheses = false +ij_java_spaces_within_switch_parentheses = false +ij_java_spaces_within_synchronized_parentheses = false +ij_java_spaces_within_try_parentheses = false +ij_java_spaces_within_while_parentheses = false +ij_java_special_else_if_treatment = true +ij_java_subclass_name_suffix = Impl +ij_java_ternary_operation_signs_on_next_line = false +ij_java_ternary_operation_wrap = on_every_item +ij_java_test_name_suffix = Test +ij_java_throws_keyword_wrap = normal +ij_java_throws_list_wrap = normal +ij_java_use_external_annotations = false +ij_java_use_fq_class_names = false +ij_java_use_single_class_imports = true +ij_java_variable_annotation_wrap = normal +ij_java_visibility = public +ij_java_while_brace_force = always +ij_java_while_on_new_line = false +ij_java_wrap_comments = false +ij_java_wrap_first_method_in_call_chain = false +ij_java_wrap_long_lines = false + +[*.proto] +ij_proto_keep_indents_on_empty_lines = false + +[.editorconfig] +ij_editorconfig_align_group_field_declarations = false +ij_editorconfig_space_after_colon = false +ij_editorconfig_space_after_comma = true +ij_editorconfig_space_before_colon = false +ij_editorconfig_space_before_comma = false +ij_editorconfig_spaces_around_assignment_operators = true + +[{*.cjs,*.js}] +ij_javascript_align_imports = false +ij_javascript_align_multiline_array_initializer_expression = false +ij_javascript_align_multiline_binary_operation = false +ij_javascript_align_multiline_chained_methods = false +ij_javascript_align_multiline_extends_list = false +ij_javascript_align_multiline_for = true +ij_javascript_align_multiline_parameters = true +ij_javascript_align_multiline_parameters_in_calls = false +ij_javascript_align_multiline_ternary_operation = false +ij_javascript_align_object_properties = 0 +ij_javascript_align_union_types = false +ij_javascript_align_var_statements = 0 +ij_javascript_array_initializer_new_line_after_left_brace = false +ij_javascript_array_initializer_right_brace_on_new_line = false +ij_javascript_array_initializer_wrap = off +ij_javascript_assignment_wrap = off +ij_javascript_binary_operation_sign_on_next_line = false +ij_javascript_binary_operation_wrap = off +ij_javascript_blacklist_imports = rxjs/Rx,node_modules/**/*,@angular/material,@angular/material/typings/** +ij_javascript_blank_lines_after_imports = 1 +ij_javascript_blank_lines_around_class = 1 +ij_javascript_blank_lines_around_field = 0 +ij_javascript_blank_lines_around_function = 1 +ij_javascript_blank_lines_around_method = 1 +ij_javascript_block_brace_style = end_of_line +ij_javascript_call_parameters_new_line_after_left_paren = false +ij_javascript_call_parameters_right_paren_on_new_line = false +ij_javascript_call_parameters_wrap = off +ij_javascript_catch_on_new_line = false +ij_javascript_chained_call_dot_on_new_line = true +ij_javascript_class_brace_style = end_of_line +ij_javascript_comma_on_new_line = false +ij_javascript_do_while_brace_force = never +ij_javascript_else_on_new_line = false +ij_javascript_enforce_trailing_comma = keep +ij_javascript_extends_keyword_wrap = off +ij_javascript_extends_list_wrap = off +ij_javascript_field_prefix = _ +ij_javascript_file_name_style = relaxed +ij_javascript_finally_on_new_line = false +ij_javascript_for_brace_force = never +ij_javascript_for_statement_new_line_after_left_paren = false +ij_javascript_for_statement_right_paren_on_new_line = false +ij_javascript_for_statement_wrap = off +ij_javascript_force_quote_style = false +ij_javascript_force_semicolon_style = false +ij_javascript_function_expression_brace_style = end_of_line +ij_javascript_if_brace_force = never +ij_javascript_import_merge_members = global +ij_javascript_import_prefer_absolute_path = global +ij_javascript_import_sort_members = true +ij_javascript_import_sort_module_name = false +ij_javascript_import_use_node_resolution = true +ij_javascript_imports_wrap = on_every_item +ij_javascript_indent_case_from_switch = true +ij_javascript_indent_chained_calls = true +ij_javascript_indent_package_children = 0 +ij_javascript_jsx_attribute_value = braces +ij_javascript_keep_blank_lines_in_code = 2 +ij_javascript_keep_first_column_comment = true +ij_javascript_keep_indents_on_empty_lines = false +ij_javascript_keep_line_breaks = true +ij_javascript_keep_simple_blocks_in_one_line = false +ij_javascript_keep_simple_methods_in_one_line = false +ij_javascript_line_comment_add_space = true +ij_javascript_line_comment_at_first_column = false +ij_javascript_method_brace_style = end_of_line +ij_javascript_method_call_chain_wrap = off +ij_javascript_method_parameters_new_line_after_left_paren = false +ij_javascript_method_parameters_right_paren_on_new_line = false +ij_javascript_method_parameters_wrap = off +ij_javascript_object_literal_wrap = on_every_item +ij_javascript_parentheses_expression_new_line_after_left_paren = false +ij_javascript_parentheses_expression_right_paren_on_new_line = false +ij_javascript_place_assignment_sign_on_next_line = false +ij_javascript_prefer_as_type_cast = false +ij_javascript_prefer_parameters_wrap = false +ij_javascript_reformat_c_style_comments = false +ij_javascript_space_after_colon = true +ij_javascript_space_after_comma = true +ij_javascript_space_after_dots_in_rest_parameter = false +ij_javascript_space_after_generator_mult = true +ij_javascript_space_after_property_colon = true +ij_javascript_space_after_quest = true +ij_javascript_space_after_type_colon = true +ij_javascript_space_after_unary_not = false +ij_javascript_space_before_async_arrow_lparen = true +ij_javascript_space_before_catch_keyword = true +ij_javascript_space_before_catch_left_brace = true +ij_javascript_space_before_catch_parentheses = true +ij_javascript_space_before_class_lbrace = true +ij_javascript_space_before_class_left_brace = true +ij_javascript_space_before_colon = true +ij_javascript_space_before_comma = false +ij_javascript_space_before_do_left_brace = true +ij_javascript_space_before_else_keyword = true +ij_javascript_space_before_else_left_brace = true +ij_javascript_space_before_finally_keyword = true +ij_javascript_space_before_finally_left_brace = true +ij_javascript_space_before_for_left_brace = true +ij_javascript_space_before_for_parentheses = true +ij_javascript_space_before_for_semicolon = false +ij_javascript_space_before_function_left_parenth = true +ij_javascript_space_before_generator_mult = false +ij_javascript_space_before_if_left_brace = true +ij_javascript_space_before_if_parentheses = true +ij_javascript_space_before_method_call_parentheses = false +ij_javascript_space_before_method_left_brace = true +ij_javascript_space_before_method_parentheses = false +ij_javascript_space_before_property_colon = false +ij_javascript_space_before_quest = true +ij_javascript_space_before_switch_left_brace = true +ij_javascript_space_before_switch_parentheses = true +ij_javascript_space_before_try_left_brace = true +ij_javascript_space_before_type_colon = false +ij_javascript_space_before_unary_not = false +ij_javascript_space_before_while_keyword = true +ij_javascript_space_before_while_left_brace = true +ij_javascript_space_before_while_parentheses = true +ij_javascript_spaces_around_additive_operators = true +ij_javascript_spaces_around_arrow_function_operator = true +ij_javascript_spaces_around_assignment_operators = true +ij_javascript_spaces_around_bitwise_operators = true +ij_javascript_spaces_around_equality_operators = true +ij_javascript_spaces_around_logical_operators = true +ij_javascript_spaces_around_multiplicative_operators = true +ij_javascript_spaces_around_relational_operators = true +ij_javascript_spaces_around_shift_operators = true +ij_javascript_spaces_around_unary_operator = false +ij_javascript_spaces_within_array_initializer_brackets = false +ij_javascript_spaces_within_brackets = false +ij_javascript_spaces_within_catch_parentheses = false +ij_javascript_spaces_within_for_parentheses = false +ij_javascript_spaces_within_if_parentheses = false +ij_javascript_spaces_within_imports = false +ij_javascript_spaces_within_interpolation_expressions = false +ij_javascript_spaces_within_method_call_parentheses = false +ij_javascript_spaces_within_method_parentheses = false +ij_javascript_spaces_within_object_literal_braces = false +ij_javascript_spaces_within_object_type_braces = true +ij_javascript_spaces_within_parentheses = false +ij_javascript_spaces_within_switch_parentheses = false +ij_javascript_spaces_within_type_assertion = false +ij_javascript_spaces_within_union_types = true +ij_javascript_spaces_within_while_parentheses = false +ij_javascript_special_else_if_treatment = true +ij_javascript_ternary_operation_signs_on_next_line = false +ij_javascript_ternary_operation_wrap = off +ij_javascript_union_types_wrap = on_every_item +ij_javascript_use_chained_calls_group_indents = false +ij_javascript_use_double_quotes = true +ij_javascript_use_explicit_js_extension = global +ij_javascript_use_path_mapping = always +ij_javascript_use_public_modifier = false +ij_javascript_use_semicolon_after_statement = true +ij_javascript_var_declaration_wrap = normal +ij_javascript_while_brace_force = never +ij_javascript_while_on_new_line = false +ij_javascript_wrap_comments = false + +[{*.gradle,*.groovy,*.gant,*.gdsl,*.gy,*.gson,Jenkinsfile*}] +ij_groovy_align_group_field_declarations = false +ij_groovy_align_multiline_array_initializer_expression = false +ij_groovy_align_multiline_assignment = false +ij_groovy_align_multiline_binary_operation = false +ij_groovy_align_multiline_chained_methods = false +ij_groovy_align_multiline_extends_list = false +ij_groovy_align_multiline_for = true +ij_groovy_align_multiline_method_parentheses = false +ij_groovy_align_multiline_parameters = true +ij_groovy_align_multiline_parameters_in_calls = false +ij_groovy_align_multiline_resources = true +ij_groovy_align_multiline_ternary_operation = false +ij_groovy_align_multiline_throws_list = false +ij_groovy_align_throws_keyword = false +ij_groovy_array_initializer_new_line_after_left_brace = false +ij_groovy_array_initializer_right_brace_on_new_line = false +ij_groovy_array_initializer_wrap = off +ij_groovy_assert_statement_wrap = off +ij_groovy_assignment_wrap = off +ij_groovy_binary_operation_wrap = off +ij_groovy_blank_lines_after_class_header = 0 +ij_groovy_blank_lines_after_imports = 1 +ij_groovy_blank_lines_after_package = 1 +ij_groovy_blank_lines_around_class = 1 +ij_groovy_blank_lines_around_field = 0 +ij_groovy_blank_lines_around_field_in_interface = 0 +ij_groovy_blank_lines_around_method = 1 +ij_groovy_blank_lines_around_method_in_interface = 1 +ij_groovy_blank_lines_before_imports = 1 +ij_groovy_blank_lines_before_method_body = 0 +ij_groovy_blank_lines_before_package = 0 +ij_groovy_block_brace_style = end_of_line +ij_groovy_block_comment_at_first_column = true +ij_groovy_call_parameters_new_line_after_left_paren = false +ij_groovy_call_parameters_right_paren_on_new_line = false +ij_groovy_call_parameters_wrap = off +ij_groovy_catch_on_new_line = false +ij_groovy_class_annotation_wrap = split_into_lines +ij_groovy_class_brace_style = end_of_line +ij_groovy_do_while_brace_force = never +ij_groovy_else_on_new_line = false +ij_groovy_enum_constants_wrap = off +ij_groovy_extends_keyword_wrap = off +ij_groovy_extends_list_wrap = off +ij_groovy_field_annotation_wrap = split_into_lines +ij_groovy_finally_on_new_line = false +ij_groovy_for_brace_force = never +ij_groovy_for_statement_new_line_after_left_paren = false +ij_groovy_for_statement_right_paren_on_new_line = false +ij_groovy_for_statement_wrap = off +ij_groovy_if_brace_force = never +ij_groovy_indent_case_from_switch = true +ij_groovy_keep_blank_lines_before_right_brace = 2 +ij_groovy_keep_blank_lines_in_code = 2 +ij_groovy_keep_blank_lines_in_declarations = 2 +ij_groovy_keep_control_statement_in_one_line = true +ij_groovy_keep_first_column_comment = true +ij_groovy_keep_indents_on_empty_lines = false +ij_groovy_keep_line_breaks = true +ij_groovy_keep_multiple_expressions_in_one_line = false +ij_groovy_keep_simple_blocks_in_one_line = false +ij_groovy_keep_simple_classes_in_one_line = true +ij_groovy_keep_simple_lambdas_in_one_line = true +ij_groovy_keep_simple_methods_in_one_line = true +ij_groovy_lambda_brace_style = end_of_line +ij_groovy_line_comment_add_space = false +ij_groovy_line_comment_at_first_column = true +ij_groovy_method_annotation_wrap = split_into_lines +ij_groovy_method_brace_style = end_of_line +ij_groovy_method_call_chain_wrap = off +ij_groovy_method_parameters_new_line_after_left_paren = false +ij_groovy_method_parameters_right_paren_on_new_line = false +ij_groovy_method_parameters_wrap = off +ij_groovy_modifier_list_wrap = false +ij_groovy_parameter_annotation_wrap = off +ij_groovy_parentheses_expression_new_line_after_left_paren = false +ij_groovy_parentheses_expression_right_paren_on_new_line = false +ij_groovy_prefer_parameters_wrap = false +ij_groovy_resource_list_new_line_after_left_paren = false +ij_groovy_resource_list_right_paren_on_new_line = false +ij_groovy_resource_list_wrap = off +ij_groovy_space_after_colon = true +ij_groovy_space_after_comma = true +ij_groovy_space_after_comma_in_type_arguments = true +ij_groovy_space_after_for_semicolon = true +ij_groovy_space_after_quest = true +ij_groovy_space_after_type_cast = true +ij_groovy_space_before_annotation_parameter_list = false +ij_groovy_space_before_array_initializer_left_brace = false +ij_groovy_space_before_catch_keyword = true +ij_groovy_space_before_catch_left_brace = true +ij_groovy_space_before_catch_parentheses = true +ij_groovy_space_before_class_left_brace = true +ij_groovy_space_before_colon = true +ij_groovy_space_before_comma = false +ij_groovy_space_before_do_left_brace = true +ij_groovy_space_before_else_keyword = true +ij_groovy_space_before_else_left_brace = true +ij_groovy_space_before_finally_keyword = true +ij_groovy_space_before_finally_left_brace = true +ij_groovy_space_before_for_left_brace = true +ij_groovy_space_before_for_parentheses = true +ij_groovy_space_before_for_semicolon = false +ij_groovy_space_before_if_left_brace = true +ij_groovy_space_before_if_parentheses = true +ij_groovy_space_before_method_call_parentheses = false +ij_groovy_space_before_method_left_brace = true +ij_groovy_space_before_method_parentheses = false +ij_groovy_space_before_quest = true +ij_groovy_space_before_switch_left_brace = true +ij_groovy_space_before_switch_parentheses = true +ij_groovy_space_before_synchronized_left_brace = true +ij_groovy_space_before_synchronized_parentheses = true +ij_groovy_space_before_try_left_brace = true +ij_groovy_space_before_try_parentheses = true +ij_groovy_space_before_while_keyword = true +ij_groovy_space_before_while_left_brace = true +ij_groovy_space_before_while_parentheses = true +ij_groovy_space_within_empty_array_initializer_braces = false +ij_groovy_space_within_empty_method_call_parentheses = false +ij_groovy_spaces_around_additive_operators = true +ij_groovy_spaces_around_assignment_operators = true +ij_groovy_spaces_around_bitwise_operators = true +ij_groovy_spaces_around_equality_operators = true +ij_groovy_spaces_around_lambda_arrow = true +ij_groovy_spaces_around_logical_operators = true +ij_groovy_spaces_around_multiplicative_operators = true +ij_groovy_spaces_around_relational_operators = true +ij_groovy_spaces_around_shift_operators = true +ij_groovy_spaces_within_annotation_parentheses = false +ij_groovy_spaces_within_array_initializer_braces = false +ij_groovy_spaces_within_braces = true +ij_groovy_spaces_within_brackets = false +ij_groovy_spaces_within_cast_parentheses = false +ij_groovy_spaces_within_catch_parentheses = false +ij_groovy_spaces_within_for_parentheses = false +ij_groovy_spaces_within_if_parentheses = false +ij_groovy_spaces_within_method_call_parentheses = false +ij_groovy_spaces_within_method_parentheses = false +ij_groovy_spaces_within_parentheses = false +ij_groovy_spaces_within_switch_parentheses = false +ij_groovy_spaces_within_synchronized_parentheses = false +ij_groovy_spaces_within_try_parentheses = false +ij_groovy_spaces_within_while_parentheses = false +ij_groovy_special_else_if_treatment = true +ij_groovy_ternary_operation_wrap = off +ij_groovy_throws_keyword_wrap = off +ij_groovy_throws_list_wrap = off +ij_groovy_variable_annotation_wrap = off +ij_groovy_while_brace_force = never +ij_groovy_while_on_new_line = false +ij_groovy_wrap_long_lines = false + +[{*.html,*.sht,*.shtm,*.shtml,*.ng,*.htm}] +ij_html_add_new_line_before_tags = body,div,p,form,h1,h2,h3 +ij_html_align_attributes = true +ij_html_align_text = false +ij_html_attribute_wrap = normal +ij_html_block_comment_at_first_column = true +ij_html_do_not_align_children_of_min_lines = 0 +ij_html_do_not_break_if_inline_tags = title,h1,h2,h3,h4,h5,h6,p +ij_html_do_not_indent_children_of_tags = html,body,thead,tbody,tfoot +ij_html_enforce_quotes = false +ij_html_inline_tags = a,abbr,acronym,b,basefont,bdo,big,br,cite,cite,code,dfn,em,font,i,img,input,kbd,label,q,s,samp,select,small,span,strike,strong,sub,sup,textarea,tt,u,var +ij_html_keep_blank_lines = 2 +ij_html_keep_indents_on_empty_lines = false +ij_html_keep_line_breaks = true +ij_html_keep_line_breaks_in_text = true +ij_html_keep_whitespaces = false +ij_html_keep_whitespaces_inside = span,pre,textarea +ij_html_line_comment_at_first_column = true +ij_html_new_line_after_last_attribute = never +ij_html_new_line_before_first_attribute = never +ij_html_quote_style = double +ij_html_remove_new_line_before_tags = br +ij_html_space_after_tag_name = false +ij_html_space_around_equality_in_attribute = false +ij_html_space_inside_empty_tag = false +ij_html_text_wrap = normal + +[{*.jhm,*.xjb,*.rng,*.wsdd,*.wsdl,*.fxml,*.plan,*.bpmn,*.pom,*.xslt,*.jrxml,*.ant,*.xul,*.xsl,*.xsd,*.tld,*.jnlp,*.wadl,*.xml}] +ij_xml_block_comment_at_first_column = true +ij_xml_keep_indents_on_empty_lines = false +ij_xml_line_comment_at_first_column = true + +[{*.vsl,*.vm,*.ft}] +ij_vtl_keep_indents_on_empty_lines = false + +[{*.xjsp,*.tagf,*.tag,*.jsf,*.jsp,*.jspf}] +ij_jsp_jsp_prefer_comma_separated_import_list = false +ij_jsp_keep_indents_on_empty_lines = false + +[{*.yml,*.yaml}] +ij_yaml_keep_indents_on_empty_lines = false +ij_yaml_keep_line_breaks = true + +[{*.zsh,*.bash,*.sh}] +ij_shell_binary_ops_start_line = false +ij_shell_keep_column_alignment_padding = false +ij_shell_minify_program = false +ij_shell_redirect_followed_by_space = false +ij_shell_switch_cases_indented = false + +[{.asciidoctorconfig,*.ad,*.adoc,*.asciidoc}] +ij_asciidoc_formatting_enabled = true +ij_asciidoc_one_sentence_per_line = true + +[{messages.*,spring.schemas,messages,spring.handlers,*.properties}] +ij_properties_align_group_field_declarations = false + +[{rcov,rake,cucumber,rails,spec,spork,capfile,gemfile,rakefile,guardfile,isolate,vagrantfile,Puppetfile,*.thor,*.gemspec,*.rb,*.rake,*.rbw,*.ru,*.jbuilder}] +ij_continuation_indent_size = 4 +ij_ruby_align_group_field_declarations = false +ij_ruby_align_multiline_parameters = true +ij_ruby_blank_lines_around_method = 1 +ij_ruby_convert_brace_block_by_enter = true +ij_ruby_force_newlines_around_visibility_mods = true +ij_ruby_indent_private_methods = false +ij_ruby_indent_protected_methods = false +ij_ruby_indent_public_methods = false +ij_ruby_indent_when_cases = false +ij_ruby_keep_blank_lines_in_declarations = 2 +ij_ruby_keep_indents_on_empty_lines = false +ij_ruby_keep_line_breaks = true +ij_ruby_parentheses_around_method_arguments = true +ij_ruby_spaces_around_hashrocket = true +ij_ruby_spaces_around_other_operators = true +ij_ruby_spaces_around_range_operators = false +ij_ruby_spaces_around_relational_operators = true +ij_ruby_spaces_within_array_initializer_braces = true +ij_ruby_spaces_within_braces = false From da9f6bf81877293df2af9d564d7ee51d088c087d Mon Sep 17 00:00:00 2001 From: Karthik Palanisamy Date: Sat, 23 Nov 2019 07:13:47 -0800 Subject: [PATCH 32/51] HBASE-23237 Prevent Negative values in metrics requestsPerSecond (#866) Signed-off-by: Guangxu Cheng Signed-off-by: Josh Elser --- .../hbase/regionserver/HRegionServer.java | 6 +- .../MetricsRegionServerWrapperImpl.java | 58 ++++++++---- .../TestRequestsPerSecondMetric.java | 92 +++++++++++++++++++ 3 files changed, 138 insertions(+), 18 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRequestsPerSecondMetric.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 2ba44292607a..86245841a118 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -369,6 +369,7 @@ public class HRegionServer extends HasThread implements public static final String REGIONSERVER = "regionserver"; MetricsRegionServer metricsRegionServer; + MetricsRegionServerWrapperImpl metricsRegionServerImpl; MetricsTable metricsTable; private SpanReceiverHost spanReceiverHost; @@ -1496,8 +1497,8 @@ protected void handleReportForDutyResponse(final RegionServerStartupResponse c) this.cacheConfig = new CacheConfig(conf); this.walFactory = setupWALAndReplication(); // Init in here rather than in constructor after thread name has been set - this.metricsRegionServer = new MetricsRegionServer( - new MetricsRegionServerWrapperImpl(this), conf); + this.metricsRegionServerImpl = new MetricsRegionServerWrapperImpl(this); + this.metricsRegionServer = new MetricsRegionServer(metricsRegionServerImpl, conf); this.metricsTable = new MetricsTable(new MetricsTableWrapperAggregateImpl(this)); // Now that we have a metrics source, start the pause monitor this.pauseMonitor = new JvmPauseMonitor(conf, getMetrics().getMetricsSource()); @@ -3114,6 +3115,7 @@ public Region getFromOnlineRegions(final String encodedRegionName) { @Override public boolean removeFromOnlineRegions(final Region r, ServerName destination) { Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); + metricsRegionServerImpl.requestsCountCache.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { long closeSeqNum = r.getMaxFlushedSeqId(); if (closeSeqNum == HConstants.NO_SEQNUM) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 061267d8d32d..948c255df281 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -17,8 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -84,6 +87,8 @@ class MetricsRegionServerWrapperImpl private volatile long majorCompactedCellsSize = 0; private volatile long blockedRequestsCount = 0L; private volatile long averageRegionSize = 0L; + protected final Map> + requestsCountCache = new ConcurrentHashMap>(); private CacheStats cacheStats; private CacheStats l1Stats = null; @@ -586,9 +591,6 @@ public long getMajorCompactedCellsSize() { public class RegionServerMetricsWrapperRunnable implements Runnable { private long lastRan = 0; - private long lastRequestCount = 0; - private long lastReadRequestsCount = 0; - private long lastWriteRequestsCount = 0; @Override synchronized public void run() { @@ -628,7 +630,40 @@ synchronized public void run() { long tempBlockedRequestsCount = 0L; int regionCount = 0; - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + + long currentReadRequestsCount = 0; + long currentWriteRequestsCount = 0; + long lastReadRequestsCount = 0; + long lastWriteRequestsCount = 0; + long readRequestsDelta = 0; + long writeRequestsDelta = 0; + long totalReadRequestsDelta = 0; + long totalWriteRequestsDelta = 0; + String encodedRegionName; + for (Region r : regionServer.getOnlineRegionsLocalContext()) { + encodedRegionName = r.getRegionInfo().getEncodedName(); + currentReadRequestsCount = r.getReadRequestsCount(); + currentWriteRequestsCount = r.getWriteRequestsCount(); + if (requestsCountCache.containsKey(encodedRegionName)) { + lastReadRequestsCount = requestsCountCache.get(encodedRegionName).get(0); + lastWriteRequestsCount = requestsCountCache.get(encodedRegionName).get(1); + readRequestsDelta = currentReadRequestsCount - lastReadRequestsCount; + writeRequestsDelta = currentWriteRequestsCount - lastWriteRequestsCount; + totalReadRequestsDelta += readRequestsDelta; + totalWriteRequestsDelta += writeRequestsDelta; + //Update cache for our next comparision + requestsCountCache.get(encodedRegionName).set(0,currentReadRequestsCount); + requestsCountCache.get(encodedRegionName).set(1,currentWriteRequestsCount); + } else { + // List[0] -> readRequestCount + // List[1] -> writeRequestCount + ArrayList requests = new ArrayList(2); + requests.add(currentReadRequestsCount); + requests.add(currentWriteRequestsCount); + requestsCountCache.put(encodedRegionName, requests); + totalReadRequestsDelta += currentReadRequestsCount; + totalWriteRequestsDelta += currentWriteRequestsCount; + } tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL(); tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL(); tempReadRequestsCount += r.getReadRequestsCount(); @@ -697,25 +732,16 @@ synchronized public void run() { // If we've time traveled keep the last requests per second. if ((currentTime - lastRan) > 0) { - long currentRequestCount = getTotalRowActionRequestCount(); - requestsPerSecond = (currentRequestCount - lastRequestCount) / + requestsPerSecond = (totalReadRequestsDelta + totalWriteRequestsDelta) / ((currentTime - lastRan) / 1000.0); - lastRequestCount = currentRequestCount; - - long intervalReadRequestsCount = tempReadRequestsCount - lastReadRequestsCount; - long intervalWriteRequestsCount = tempWriteRequestsCount - lastWriteRequestsCount; - double readRequestsRatePerMilliSecond = ((double)intervalReadRequestsCount/ + double readRequestsRatePerMilliSecond = ((double)totalReadRequestsDelta/ (double)period); - double writeRequestsRatePerMilliSecond = ((double)intervalWriteRequestsCount/ + double writeRequestsRatePerMilliSecond = ((double)totalWriteRequestsDelta/ (double)period); readRequestsRate = readRequestsRatePerMilliSecond * 1000.0; writeRequestsRate = writeRequestsRatePerMilliSecond * 1000.0; - - lastReadRequestsCount = tempReadRequestsCount; - lastWriteRequestsCount = tempWriteRequestsCount; - } lastRan = currentTime; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRequestsPerSecondMetric.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRequestsPerSecondMetric.java new file mode 100644 index 000000000000..6f27fd786fbc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRequestsPerSecondMetric.java @@ -0,0 +1,92 @@ +/** + * 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; + + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Validate requestsPerSecond metric. + */ +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestRequestsPerSecondMetric { + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + private static final long METRICS_PERIOD = 2000L; + private static Configuration conf; + + + @BeforeClass + public static void setup() throws Exception { + conf = UTIL.getConfiguration(); + conf.setLong(HConstants.REGIONSERVER_METRICS_PERIOD, METRICS_PERIOD); + UTIL.startMiniCluster(1); + } + + @AfterClass + public static void teardown() throws Exception { + UTIL.shutdownMiniCluster(); + } + + + @Test + /** + * This test will confirm no negative value in requestsPerSecond metric during any region + * transition(close region/remove region/move region). + * Firstly, load 2000 random rows for 25 regions and will trigger a metric. + * Now, metricCache will have a current read and write requests count. + * Next, we disable a table and all of its 25 regions will be closed. + * As part of region close, his metric will also be removed from metricCache. + * prior to HBASE-23237, we do not remove/reset his metric so we incorrectly compute + * (currentRequestCount - lastRequestCount) which result into negative value. + * + * @throws IOException + * @throws InterruptedException + */ + public void testNoNegativeSignAtRequestsPerSecond() throws IOException, InterruptedException { + final TableName TABLENAME = TableName.valueOf("t"); + final String FAMILY = "f"; + Admin admin = UTIL.getHBaseAdmin(); + UTIL.createMultiRegionTable(TABLENAME, FAMILY.getBytes(),25); + Table table = admin.getConnection().getTable(TABLENAME); + HRegionServer regionServer = UTIL.getMiniHBaseCluster().getRegionServer(0); + MetricsRegionServerWrapperImpl metricsWrapper = + new MetricsRegionServerWrapperImpl(regionServer); + MetricsRegionServerWrapperImpl.RegionServerMetricsWrapperRunnable metricsServer + = metricsWrapper.new RegionServerMetricsWrapperRunnable(); + metricsServer.run(); + UTIL.loadNumericRows(table, FAMILY.getBytes(), 1, 2000); + Thread.sleep(METRICS_PERIOD); + metricsServer.run(); + admin.disableTable(TABLENAME); + Thread.sleep(METRICS_PERIOD); + metricsServer.run(); + Assert.assertTrue(metricsWrapper.getRequestsPerSecond() > -1); + } +} From af2ac03e3a1656cdd5ef9de05667c1f71406b35f Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Mon, 2 Dec 2019 06:39:24 -0600 Subject: [PATCH 33/51] HBASE-23337 Release scripts should rely on maven for deploy. (#887) - switch to nexus-staging-maven-plugin for asf-release - cleaned up some tabs in the root pom (differs from master because there are no release scripts here.) Signed-off-by: stack (cherry picked from commit 97e01070001ef81558b4dae3a3610d0c73651cb9) --- pom.xml | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/pom.xml b/pom.xml index 92c435d99b6d..ec70ba80928a 100644 --- a/pom.xml +++ b/pom.xml @@ -2157,6 +2157,28 @@ ${hbase-surefire.cygwin-argLine} + + + apache-release + + + + + org.sonatype.plugins + nexus-staging-maven-plugin + 1.6.8 + true + + https://repository.apache.org/ + apache.releases.https + + + + + release From 737eaa635a8d6e81e5ebac3b31c81096e24d3635 Mon Sep 17 00:00:00 2001 From: Toshihiro Suzuki Date: Wed, 4 Dec 2019 21:24:03 +0900 Subject: [PATCH 34/51] HBASE-23359 RS going down with NPE when splitting a region with compaction disabled in branch-1 (#899) Signed-off-by: Balazs Meszaros Signed-off-by Anoop Sam John --- .../apache/hadoop/hbase/regionserver/CompactSplitThread.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java index ed4bd0da8e7e..f3ff124b143c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java @@ -653,7 +653,9 @@ public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) { if (runnable instanceof CompactionRunner) { CompactionRunner runner = (CompactionRunner)runnable; LOG.debug("Compaction Rejected: " + runner); - runner.store.cancelRequestedCompaction(runner.compaction); + if (runner.compaction != null) { + runner.store.cancelRequestedCompaction(runner.compaction); + } } } } From ec55c2a5d976ae1b2f797854419146bbef2fccc2 Mon Sep 17 00:00:00 2001 From: Toshihiro Suzuki Date: Wed, 4 Dec 2019 21:44:42 +0900 Subject: [PATCH 35/51] HBASE-22096 /storeFile.jsp shows CorruptHFileException when the storeFile is a reference file Signed-off-by: Lijin Bin --- .../resources/hbase-webapps/regionserver/storeFile.jsp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/storeFile.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/storeFile.jsp index 05d8783ed406..0b7117c5535f 100644 --- a/hbase-server/src/main/resources/hbase-webapps/regionserver/storeFile.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/storeFile.jsp @@ -21,16 +21,18 @@ import="java.io.ByteArrayOutputStream" import="java.io.PrintStream" import="org.apache.hadoop.conf.Configuration" + import="org.apache.hadoop.fs.FileSystem" import="org.apache.hadoop.fs.Path" import="org.apache.hadoop.hbase.HBaseConfiguration" import="org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter" import="org.apache.hadoop.hbase.regionserver.HRegionServer" - import="org.apache.hadoop.hbase.regionserver.StoreFile" + import="org.apache.hadoop.hbase.regionserver.StoreFileInfo" %> <% String storeFile = request.getParameter("name"); HRegionServer rs = (HRegionServer) getServletContext().getAttribute(HRegionServer.REGIONSERVER); Configuration conf = rs.getConfiguration(); + FileSystem fs = FileSystem.get(conf); %>