From ef1962ee12767795a8d6d7497978a48f08965da5 Mon Sep 17 00:00:00 2001 From: guluo Date: Thu, 30 Nov 2023 22:02:46 +0800 Subject: [PATCH 001/109] NPE when flushing a non-existing column family --- .../hadoop/hbase/regionserver/HRegion.java | 4 ++- .../hbase/regionserver/TestHRegion.java | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 0dc96747dd36..d93a4f444cda 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -2597,7 +2597,9 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque private Collection getSpecificStores(List families) { Collection specificStoresToFlush = new ArrayList<>(); for (byte[] family : families) { - specificStoresToFlush.add(stores.get(family)); + if (stores.keySet().contains(family)) { + specificStoresToFlush.add(stores.get(family)); + } } return specificStoresToFlush; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 6a1c285bf8db..cfc0b089b8ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5806,6 +5806,36 @@ public void testFlushResult() throws IOException { } } + @Test + public void testFlushWithSpecifiedFamily() throws IOException { + byte[] family = Bytes.toBytes("family"); + this.region = initHRegion(tableName, method, family); + + Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); + this.region.put(put); + + HRegion.FlushResult fr = + this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); + assertFalse(fr.isFlushSucceeded()); + assertFalse(fr.isCompactionNeeded()); + } + + @Test + public void testFlushWithSpecifiedNoSuchFamily() throws IOException { + byte[] family = Bytes.toBytes("family"); + byte[] noSuchFamily = Bytes.toBytes("noSuchFamily"); + this.region = initHRegion(tableName, method, family); + + Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); + this.region.put(put); + + HRegion.FlushResult fr = + this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); + LOG.error(fr.getResult().toString()); + assertFalse(fr.isFlushSucceeded()); + assertFalse(fr.isCompactionNeeded()); + } + protected Configuration initSplit() { // Always compact if there is more than one store file. CONF.setInt("hbase.hstore.compactionThreshold", 2); From 8fbdc7da726813a6c1a4880babcd072e76c113ba Mon Sep 17 00:00:00 2001 From: guluo Date: Fri, 1 Dec 2023 00:16:02 +0800 Subject: [PATCH 002/109] Update incorrect assertions --- .../apache/hadoop/hbase/regionserver/TestHRegion.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index cfc0b089b8ab..86307904c358 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5816,8 +5816,8 @@ public void testFlushWithSpecifiedFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); - assertFalse(fr.isFlushSucceeded()); - assertFalse(fr.isCompactionNeeded()); + assertTrue(fr.isFlushSucceeded()); + assertTrue(fr.isCompactionNeeded()); } @Test @@ -5831,9 +5831,8 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); - LOG.error(fr.getResult().toString()); - assertFalse(fr.isFlushSucceeded()); - assertFalse(fr.isCompactionNeeded()); + assertTrue(fr.isFlushSucceeded()); + assertTrue(fr.isCompactionNeeded()); } protected Configuration initSplit() { From 343683ef747a76e202813966943757795c114b16 Mon Sep 17 00:00:00 2001 From: guluo Date: Fri, 1 Dec 2023 00:36:29 +0800 Subject: [PATCH 003/109] Update assertions --- .../java/org/apache/hadoop/hbase/regionserver/TestHRegion.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 86307904c358..7f65332c51bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5817,7 +5817,6 @@ public void testFlushWithSpecifiedFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); assertTrue(fr.isFlushSucceeded()); - assertTrue(fr.isCompactionNeeded()); } @Test @@ -5832,7 +5831,6 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); assertTrue(fr.isFlushSucceeded()); - assertTrue(fr.isCompactionNeeded()); } protected Configuration initSplit() { From 7f3e40026892575151b2d780a9fd5f5305fe96b9 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Thu, 30 Nov 2023 21:09:05 -0800 Subject: [PATCH 004/109] Revert "HBASE-28204 Canary can take lot more time If region starts with delete markers (#5522)" This reverts commit ce9eabe61661599d0b424026841eaf0087d84805. --- .../apache/hadoop/hbase/tool/CanaryTool.java | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java index d0cd199ecdc9..d5676263c820 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java @@ -510,38 +510,38 @@ public Void call() { private Void readColumnFamily(Table table, ColumnFamilyDescriptor column) { byte[] startKey = null; - Scan scan = new Scan(); + Get get = null; + Scan scan = null; ResultScanner rs = null; StopWatch stopWatch = new StopWatch(); startKey = region.getStartKey(); // Can't do a get on empty start row so do a Scan of first element if any instead. if (startKey.length > 0) { - // There are 4 types of region for any table. - // 1. Start and End key are empty. (Table with Single region) - // 2. Start key is empty. (First region of the table) - // 3. End key is empty. (Last region of the table) - // 4. Region with Start & End key. (All the regions between first & last region of the - // table.) - // - // Since Scan only takes Start and/or End Row and doesn't accept the region ID, - // we set the start row when Regions are of type 3 OR 4 as mentioned above. - // For type 1 and 2, We don't need to set this option. - scan.withStartRow(startKey); + get = new Get(startKey); + get.setCacheBlocks(false); + get.setFilter(new FirstKeyOnlyFilter()); + get.addFamily(column.getName()); + } else { + scan = new Scan(); + LOG.debug("rawScan {} for {}", rawScanEnabled, region.getTable()); + scan.setRaw(rawScanEnabled); + scan.setCaching(1); + scan.setCacheBlocks(false); + scan.setFilter(new FirstKeyOnlyFilter()); + scan.addFamily(column.getName()); + scan.setMaxResultSize(1L); + scan.setOneRowLimit(); } - LOG.debug("rawScan {} for {}", rawScanEnabled, region.getTable()); - scan.setRaw(rawScanEnabled); - scan.setCaching(1); - scan.setCacheBlocks(false); - scan.setFilter(new FirstKeyOnlyFilter()); - scan.addFamily(column.getName()); - scan.setMaxResultSize(1L); - scan.setOneRowLimit(); LOG.debug("Reading from {} {} {} {}", region.getTable(), region.getRegionNameAsString(), column.getNameAsString(), Bytes.toStringBinary(startKey)); try { stopWatch.start(); - rs = table.getScanner(scan); - rs.next(); + if (startKey.length > 0) { + table.get(get); + } else { + rs = table.getScanner(scan); + rs.next(); + } stopWatch.stop(); this.readWriteLatency.add(stopWatch.getTime()); sink.publishReadTiming(serverName, region, column, stopWatch.getTime()); From 02217e1abc0f467d5d5c6e3b5fd2304b2813df49 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Fri, 1 Dec 2023 08:23:52 -0500 Subject: [PATCH 005/109] HBASE-28222 Leak in ExportSnapshot during verifySnapshot on S3A (#5554) Revert "HBASE-12819 ExportSnapshot doesn't close FileSystem instances" This reverts commit ee32eebeab38be4e171c6aaf362aff9a584a37f3 Signed-off-by: Wellington Chevreuil --- .../hadoop/hbase/snapshot/ExportSnapshot.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java index f2a8e00fea5e..c6f655c37306 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java @@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; @@ -211,14 +210,12 @@ public void setup(Context context) throws IOException { outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); try { - srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); inputFs = FileSystem.get(inputRoot.toUri(), srcConf); } catch (IOException e) { throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e); } try { - destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); outputFs = FileSystem.get(outputRoot.toUri(), destConf); } catch (IOException e) { throw new IOException("Could not get the output FileSystem with root=" + outputRoot, e); @@ -241,12 +238,6 @@ public void setup(Context context) throws IOException { } } - @Override - protected void cleanup(Context context) { - IOUtils.closeStream(inputFs); - IOUtils.closeStream(outputFs); - } - @Override public void map(BytesWritable key, NullWritable value, Context context) throws InterruptedException, IOException { @@ -990,10 +981,8 @@ public int doWork() throws IOException { } Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); - srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); FileSystem inputFs = FileSystem.get(inputRoot.toUri(), srcConf); Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); - destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); FileSystem outputFs = FileSystem.get(outputRoot.toUri(), destConf); boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false) || conf.get(SnapshotDescriptionUtils.SNAPSHOT_WORKING_DIR) != null; @@ -1149,9 +1138,6 @@ public int doWork() throws IOException { } outputFs.delete(outputSnapshotDir, true); return 1; - } finally { - IOUtils.closeStream(inputFs); - IOUtils.closeStream(outputFs); } } From 7dd4d0c532a0d4e60037c193b2ce22f0b03ca01e Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 1 Dec 2023 23:31:33 +0800 Subject: [PATCH 006/109] HBASE-28212 Addendum fix TestShell (#5555) We added a new field in Procedure so the json output is also changed thus we need to change the assertion --- .../src/test/ruby/shell/list_locks_test.rb | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb index 20a910c485dd..89c6940db2a6 100644 --- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb +++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb @@ -81,7 +81,8 @@ def create_shared_lock(proc_id) "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", " \ - "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}], " \ + "\"executed\"=>false" \ "}\n\n", output) end @@ -101,7 +102,8 @@ def create_shared_lock(proc_id) "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", " \ - "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}], " \ + "\"executed\"=>false" \ "}\n\n", output) end @@ -119,7 +121,8 @@ def create_shared_lock(proc_id) "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"1\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", " \ - "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}], " \ + "\"executed\"=>false" \ "}\n\n" \ "TABLE(hbase:namespace)\n" \ "Lock type: SHARED, count: 1\n\n", @@ -143,7 +146,8 @@ def create_shared_lock(proc_id) "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"2\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", " \ - "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}], " \ + "\"executed\"=>false" \ "}\n\n", output) end @@ -168,7 +172,8 @@ def create_shared_lock(proc_id) "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"3\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", " \ - "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}], " \ + "\"executed\"=>false" \ "}\n\n", output) end @@ -198,14 +203,14 @@ def create_shared_lock(proc_id) "\"lastUpdate\"=>\"0\", \"stateMessage\"=>[{" \ "\"lockType\"=>\"EXCLUSIVE\", " \ "\"tableName\"=>{\"namespace\"=>\"bnM0\", \"qualifier\"=>\"dGFibGU0\"" \ - "}, \"description\"=>\"description\"}]}\n" \ + "}, \"description\"=>\"description\"}], \"executed\"=>false}\n" \ "Waiting procedures\n" \ "{\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ "\"procId\"=>\"2\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ "\"lastUpdate\"=>\"0\", \"stateMessage\"=>[{" \ "\"lockType\"=>\"SHARED\", " \ "\"tableName\"=>{\"namespace\"=>\"bnM0\", \"qualifier\"=>\"dGFibGU0\"}, " \ - "\"description\"=>\"description\"}]}\n" \ + "\"description\"=>\"description\"}], \"executed\"=>false}\n" \ "1 row(s)\n\n", output) end From 8631714705231aa5db5397d92e4ea9d0e5625129 Mon Sep 17 00:00:00 2001 From: hiping-tech <58875741+hiping-tech@users.noreply.github.com> Date: Mon, 4 Dec 2023 15:51:28 +0800 Subject: [PATCH 007/109] HBASE-28226 Add logic to check for RegionStateNode null pointer in FlushRegionProcedure (#5548) Co-authored-by: lvhaiping.lhp Signed-off-by: Duo Zhang --- .../hadoop/hbase/master/procedure/FlushRegionProcedure.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/FlushRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/FlushRegionProcedure.java index 67f0442b618a..88f7e652cbff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/FlushRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/FlushRegionProcedure.java @@ -88,6 +88,11 @@ protected Procedure[] execute(MasterProcedureEnv env) RegionStates regionStates = env.getAssignmentManager().getRegionStates(); RegionStateNode regionNode = regionStates.getRegionStateNode(region); + if (regionNode == null) { + LOG.debug("Region {} is not in region states, it is very likely that it has been cleared by" + + " other procedures such as merge or split, so skip {}. See HBASE-28226", region, this); + return null; + } regionNode.lock(); try { if (!regionNode.isInState(State.OPEN) || regionNode.isInTransition()) { From cf798adeccd575169a1e1e723cd6e1496c380c3f Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Mon, 4 Dec 2023 16:52:57 +0800 Subject: [PATCH 008/109] HBASE-28199 Phase I: Suspend TRSP and SCP when updating meta (#5520) Signed-off-by: Yu Li --- .../hbase/procedure2/ProcedureExecutor.java | 57 ++++-- .../master/assignment/AssignmentManager.java | 150 ++++++++-------- .../assignment/RegionRemoteProcedureBase.java | 46 +++-- .../master/assignment/RegionStateNode.java | 33 +++- .../assignment/RegionStateNodeLock.java | 166 ++++++++++++++++++ .../master/assignment/RegionStateStore.java | 67 +++++-- .../TransitRegionStateProcedure.java | 126 +++++++++---- .../master/procedure/MasterProcedureEnv.java | 8 + .../procedure/ServerCrashProcedure.java | 57 +++++- .../procedure/TruncateRegionProcedure.java | 2 +- ...eplicationQueueFromZkToTableProcedure.java | 78 ++++---- .../replication/ReplicationPeerManager.java | 2 +- .../hbase/procedure2/ProcedureFutureUtil.java | 112 ++++++++++++ .../master/assignment/MockMasterServices.java | 4 +- .../assignment/TestAssignmentManagerUtil.java | 3 +- .../TestOpenRegionProcedureBackoff.java | 7 +- .../assignment/TestRaceBetweenSCPAndTRSP.java | 13 +- .../assignment/TestRegionStateNodeLock.java | 139 +++++++++++++++ .../master/assignment/TestRollbackSCP.java | 8 +- .../procedure/TestProcedurePriority.java | 20 ++- 20 files changed, 879 insertions(+), 219 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNodeLock.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateNodeLock.java diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java index 3099c64e00f6..5aa11811122b 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java @@ -32,8 +32,10 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -237,6 +239,12 @@ public interface ProcedureExecutorListener { */ private TimeoutExecutorThread workerMonitorExecutor; + private ExecutorService forceUpdateExecutor; + + // A thread pool for executing some asynchronous tasks for procedures, you can find references to + // getAsyncTaskExecutor to see the usage + private ExecutorService asyncTaskExecutor; + private int corePoolSize; private int maxPoolSize; @@ -247,9 +255,6 @@ public interface ProcedureExecutorListener { */ private final ProcedureScheduler scheduler; - private final Executor forceUpdateExecutor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Force-Update-PEWorker-%d").build()); - private final AtomicLong lastProcId = new AtomicLong(-1); private final AtomicLong workerId = new AtomicLong(0); private final AtomicInteger activeExecutorCount = new AtomicInteger(0); @@ -317,19 +322,6 @@ public ProcedureExecutor(final Configuration conf, final TEnvironment environmen this.conf = conf; this.checkOwnerSet = conf.getBoolean(CHECK_OWNER_SET_CONF_KEY, DEFAULT_CHECK_OWNER_SET); refreshConfiguration(conf); - store.registerListener(new ProcedureStoreListener() { - - @Override - public void forceUpdate(long[] procIds) { - Arrays.stream(procIds).forEach(procId -> forceUpdateExecutor.execute(() -> { - try { - forceUpdateProcedure(procId); - } catch (IOException e) { - LOG.warn("Failed to force update procedure with pid={}", procId); - } - })); - } - }); } private void load(final boolean abortOnCorruption) throws IOException { @@ -614,6 +606,28 @@ public void init(int numThreads, boolean abortOnCorruption) throws IOException { this.timeoutExecutor = new TimeoutExecutorThread<>(this, threadGroup, "ProcExecTimeout"); this.workerMonitorExecutor = new TimeoutExecutorThread<>(this, threadGroup, "WorkerMonitor"); + int size = Math.max(2, Runtime.getRuntime().availableProcessors()); + ThreadPoolExecutor executor = new ThreadPoolExecutor(size, size, 1, TimeUnit.MINUTES, + new LinkedBlockingQueue(), new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(getClass().getSimpleName() + "-Async-Task-Executor-%d").build()); + executor.allowCoreThreadTimeOut(true); + this.asyncTaskExecutor = executor; + forceUpdateExecutor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Force-Update-PEWorker-%d").build()); + store.registerListener(new ProcedureStoreListener() { + + @Override + public void forceUpdate(long[] procIds) { + Arrays.stream(procIds).forEach(procId -> forceUpdateExecutor.execute(() -> { + try { + forceUpdateProcedure(procId); + } catch (IOException e) { + LOG.warn("Failed to force update procedure with pid={}", procId); + } + })); + } + }); + // Create the workers workerId.set(0); workerThreads = new CopyOnWriteArrayList<>(); @@ -678,6 +692,8 @@ public void stop() { scheduler.stop(); timeoutExecutor.sendStopSignal(); workerMonitorExecutor.sendStopSignal(); + forceUpdateExecutor.shutdown(); + asyncTaskExecutor.shutdown(); } public void join() { @@ -2055,6 +2071,13 @@ public IdLock getProcExecutionLock() { return procExecutionLock; } + /** + * Get a thread pool for executing some asynchronous tasks + */ + public ExecutorService getAsyncTaskExecutor() { + return asyncTaskExecutor; + } + // ========================================================================== // Worker Thread // ========================================================================== diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 804757959d5c..474b95a2a69b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -82,6 +83,7 @@ import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.VersionInfo; @@ -1989,71 +1991,78 @@ public RegionInfo getRegionInfo(final String encodedRegionName) { // Should only be called in TransitRegionStateProcedure(and related procedures), as the locking // and pre-assumptions are very tricky. // ============================================================================================ - private void transitStateAndUpdate(RegionStateNode regionNode, RegionState.State newState, - RegionState.State... expectedStates) throws IOException { + private CompletableFuture transitStateAndUpdate(RegionStateNode regionNode, + RegionState.State newState, RegionState.State... expectedStates) { RegionState.State state = regionNode.getState(); - regionNode.transitionState(newState, expectedStates); - boolean succ = false; try { - regionStateStore.updateRegionLocation(regionNode); - succ = true; - } finally { - if (!succ) { + regionNode.transitionState(newState, expectedStates); + } catch (UnexpectedStateException e) { + return FutureUtils.failedFuture(e); + } + CompletableFuture future = regionStateStore.updateRegionLocation(regionNode); + FutureUtils.addListener(future, (r, e) -> { + if (e != null) { // revert regionNode.setState(state); } - } + }); + return future; } // should be called within the synchronized block of RegionStateNode - void regionOpening(RegionStateNode regionNode) throws IOException { + CompletableFuture regionOpening(RegionStateNode regionNode) { // As in SCP, for performance reason, there is no TRSP attached with this region, we will not // update the region state, which means that the region could be in any state when we want to // assign it after a RS crash. So here we do not pass the expectedStates parameter. - transitStateAndUpdate(regionNode, State.OPENING); - regionStates.addRegionToServer(regionNode); - // update the operation count metrics - metrics.incrementOperationCounter(); + return transitStateAndUpdate(regionNode, State.OPENING).thenAccept(r -> { + regionStates.addRegionToServer(regionNode); + // update the operation count metrics + metrics.incrementOperationCounter(); + }); } // should be called under the RegionStateNode lock // The parameter 'giveUp' means whether we will try to open the region again, if it is true, then // we will persist the FAILED_OPEN state into hbase:meta. - void regionFailedOpen(RegionStateNode regionNode, boolean giveUp) throws IOException { + CompletableFuture regionFailedOpen(RegionStateNode regionNode, boolean giveUp) { RegionState.State state = regionNode.getState(); ServerName regionLocation = regionNode.getRegionLocation(); - if (giveUp) { - regionNode.setState(State.FAILED_OPEN); - regionNode.setRegionLocation(null); - boolean succ = false; - try { - regionStateStore.updateRegionLocation(regionNode); - succ = true; - } finally { - if (!succ) { - // revert - regionNode.setState(state); - regionNode.setRegionLocation(regionLocation); - } + if (!giveUp) { + if (regionLocation != null) { + regionStates.removeRegionFromServer(regionLocation, regionNode); } + return CompletableFuture.completedFuture(null); } - if (regionLocation != null) { - regionStates.removeRegionFromServer(regionLocation, regionNode); - } + regionNode.setState(State.FAILED_OPEN); + regionNode.setRegionLocation(null); + CompletableFuture future = regionStateStore.updateRegionLocation(regionNode); + FutureUtils.addListener(future, (r, e) -> { + if (e == null) { + if (regionLocation != null) { + regionStates.removeRegionFromServer(regionLocation, regionNode); + } + } else { + // revert + regionNode.setState(state); + regionNode.setRegionLocation(regionLocation); + } + }); + return future; } // should be called under the RegionStateNode lock - void regionClosing(RegionStateNode regionNode) throws IOException { - transitStateAndUpdate(regionNode, State.CLOSING, STATES_EXPECTED_ON_CLOSING); - - RegionInfo hri = regionNode.getRegionInfo(); - // Set meta has not initialized early. so people trying to create/edit tables will wait - if (isMetaRegion(hri)) { - setMetaAssigned(hri, false); - } - regionStates.addRegionToServer(regionNode); - // update the operation count metrics - metrics.incrementOperationCounter(); + CompletableFuture regionClosing(RegionStateNode regionNode) { + return transitStateAndUpdate(regionNode, State.CLOSING, STATES_EXPECTED_ON_CLOSING) + .thenAccept(r -> { + RegionInfo hri = regionNode.getRegionInfo(); + // Set meta has not initialized early. so people trying to create/edit tables will wait + if (isMetaRegion(hri)) { + setMetaAssigned(hri, false); + } + regionStates.addRegionToServer(regionNode); + // update the operation count metrics + metrics.incrementOperationCounter(); + }); } // for open and close, they will first be persist to the procedure store in @@ -2062,7 +2071,8 @@ void regionClosing(RegionStateNode regionNode) throws IOException { // RegionRemoteProcedureBase is woken up, we will persist the RegionStateNode to hbase:meta. // should be called under the RegionStateNode lock - void regionOpenedWithoutPersistingToMeta(RegionStateNode regionNode) throws IOException { + void regionOpenedWithoutPersistingToMeta(RegionStateNode regionNode) + throws UnexpectedStateException { regionNode.transitionState(State.OPEN, STATES_EXPECTED_ON_OPEN); RegionInfo regionInfo = regionNode.getRegionInfo(); regionStates.addRegionToServer(regionNode); @@ -2070,7 +2080,8 @@ void regionOpenedWithoutPersistingToMeta(RegionStateNode regionNode) throws IOEx } // should be called under the RegionStateNode lock - void regionClosedWithoutPersistingToMeta(RegionStateNode regionNode) throws IOException { + void regionClosedWithoutPersistingToMeta(RegionStateNode regionNode) + throws UnexpectedStateException { ServerName regionLocation = regionNode.getRegionLocation(); regionNode.transitionState(State.CLOSED, STATES_EXPECTED_ON_CLOSED); regionNode.setRegionLocation(null); @@ -2080,40 +2091,41 @@ void regionClosedWithoutPersistingToMeta(RegionStateNode regionNode) throws IOEx } } + // should be called under the RegionStateNode lock + CompletableFuture persistToMeta(RegionStateNode regionNode) { + return regionStateStore.updateRegionLocation(regionNode).thenAccept(r -> { + RegionInfo regionInfo = regionNode.getRegionInfo(); + if (isMetaRegion(regionInfo) && regionNode.getState() == State.OPEN) { + // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it + // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager + // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state + // on table that contains state. + setMetaAssigned(regionInfo, true); + } + }); + } + // should be called under the RegionStateNode lock // for SCP - public void regionClosedAbnormally(RegionStateNode regionNode) throws IOException { + public CompletableFuture regionClosedAbnormally(RegionStateNode regionNode) { RegionState.State state = regionNode.getState(); ServerName regionLocation = regionNode.getRegionLocation(); - regionNode.transitionState(State.ABNORMALLY_CLOSED); + regionNode.setState(State.ABNORMALLY_CLOSED); regionNode.setRegionLocation(null); - boolean succ = false; - try { - regionStateStore.updateRegionLocation(regionNode); - succ = true; - } finally { - if (!succ) { + CompletableFuture future = regionStateStore.updateRegionLocation(regionNode); + FutureUtils.addListener(future, (r, e) -> { + if (e == null) { + if (regionLocation != null) { + regionNode.setLastHost(regionLocation); + regionStates.removeRegionFromServer(regionLocation, regionNode); + } + } else { // revert regionNode.setState(state); regionNode.setRegionLocation(regionLocation); } - } - if (regionLocation != null) { - regionNode.setLastHost(regionLocation); - regionStates.removeRegionFromServer(regionLocation, regionNode); - } - } - - void persistToMeta(RegionStateNode regionNode) throws IOException { - regionStateStore.updateRegionLocation(regionNode); - RegionInfo regionInfo = regionNode.getRegionInfo(); - if (isMetaRegion(regionInfo) && regionNode.getState() == State.OPEN) { - // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it - // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager - // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state - // on table that contains state. - setMetaAssigned(regionInfo, true); - } + }); + return future; } // ============================================================================================ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java index 6b6da9e33965..d27e0068b0ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -29,6 +30,7 @@ import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureEvent; +import org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; @@ -73,6 +75,8 @@ public abstract class RegionRemoteProcedureBase extends Procedure future; + protected RegionRemoteProcedureBase() { } @@ -268,11 +272,21 @@ private void unattach(MasterProcedureEnv env) { getParent(env).unattachRemoteProc(this); } + private CompletableFuture getFuture() { + return future; + } + + private void setFuture(CompletableFuture f) { + future = f; + } + @Override protected Procedure[] execute(MasterProcedureEnv env) throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException { RegionStateNode regionNode = getRegionNode(env); - regionNode.lock(); + if (future == null) { + regionNode.lock(this); + } try { switch (state) { case REGION_REMOTE_PROCEDURE_DISPATCH: { @@ -294,16 +308,29 @@ protected Procedure[] execute(MasterProcedureEnv env) throw new ProcedureSuspendedException(); } case REGION_REMOTE_PROCEDURE_REPORT_SUCCEED: - env.getAssignmentManager().persistToMeta(regionNode); - unattach(env); + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + () -> unattach(env)) + ) { + return null; + } + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getAssignmentManager().persistToMeta(regionNode), env, () -> unattach(env)); return null; case REGION_REMOTE_PROCEDURE_DISPATCH_FAIL: // the remote call is failed so we do not need to change the region state, just return. unattach(env); return null; case REGION_REMOTE_PROCEDURE_SERVER_CRASH: - env.getAssignmentManager().regionClosedAbnormally(regionNode); - unattach(env); + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + () -> unattach(env)) + ) { + return null; + } + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getAssignmentManager().regionClosedAbnormally(regionNode), env, + () -> unattach(env)); return null; default: throw new IllegalStateException("Unknown state: " + state); @@ -314,12 +341,11 @@ protected Procedure[] execute(MasterProcedureEnv env) } long backoff = retryCounter.getBackoffTimeAndIncrementAttempts(); LOG.warn("Failed updating meta, suspend {}secs {}; {};", backoff / 1000, this, regionNode, e); - setTimeout(Math.toIntExact(backoff)); - setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); - skipPersistence(); - throw new ProcedureSuspendedException(); + throw suspend(Math.toIntExact(backoff), true); } finally { - regionNode.unlock(); + if (future == null) { + regionNode.unlock(this); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java index 91c0222facd1..de00ca92e4c6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java @@ -19,8 +19,6 @@ import java.util.Arrays; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -30,6 +28,7 @@ import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState.State; +import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureEvent; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; @@ -75,7 +74,7 @@ public AssignmentProcedureEvent(final RegionInfo regionInfo) { } } - final Lock lock = new ReentrantLock(); + private final RegionStateNodeLock lock; private final RegionInfo regionInfo; private final ProcedureEvent event; private final ConcurrentMap ritMap; @@ -106,6 +105,7 @@ public AssignmentProcedureEvent(final RegionInfo regionInfo) { this.regionInfo = regionInfo; this.event = new AssignmentProcedureEvent(regionInfo); this.ritMap = ritMap; + this.lock = new RegionStateNodeLock(regionInfo); } /** @@ -319,6 +319,9 @@ public void checkOnline() throws DoNotRetryRegionException { } } + // The below 3 methods are for normal locking operation, where the thread owner is the current + // thread. Typically you just need to use these 3 methods, and use try..finally to release the + // lock in the finally block public void lock() { lock.lock(); } @@ -330,4 +333,28 @@ public boolean tryLock() { public void unlock() { lock.unlock(); } + + // The below 3 methods are for locking region state node when executing procedures, where we may + // do some time consuming work under the lock, for example, updating meta. As we may suspend the + // procedure while holding the lock and then release it when the procedure is back, in another + // thread, so we need to use the procedure itself as owner, instead of the current thread. You can + // see the usage in TRSP, SCP, and RegionRemoteProcedureBase for more details. + // Notice that, this does not mean you must use these 3 methods when locking region state node in + // procedure, you are free to use the above 3 methods if you do not want to hold the lock when + // suspending the procedure. + public void lock(Procedure proc) { + lock.lock(proc); + } + + public boolean tryLock(Procedure proc) { + return lock.tryLock(proc); + } + + public void unlock(Procedure proc) { + lock.unlock(proc); + } + + boolean isLocked() { + return lock.isLocked(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNodeLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNodeLock.java new file mode 100644 index 000000000000..a672425c8ed2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNodeLock.java @@ -0,0 +1,166 @@ +/* + * 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.assignment; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A lock implementation which supports unlock by another thread. + *

+ * This is because we need to hold region state node lock while updating region state to meta(for + * keeping consistency), so it is better to yield the procedure to release the procedure worker. But + * after waking up the procedure, we may use another procedure worker to execute the procedure, + * which means we need to unlock by another thread. See HBASE-28196 for more details. + */ +@InterfaceAudience.Private +class RegionStateNodeLock { + + // for better logging message + private final RegionInfo regionInfo; + + private final Lock lock = new ReentrantLock(); + + private final Condition cond = lock.newCondition(); + + private Object owner; + + private int count; + + RegionStateNodeLock(RegionInfo regionInfo) { + this.regionInfo = regionInfo; + } + + private void lock0(Object lockBy) { + lock.lock(); + try { + for (;;) { + if (owner == null) { + owner = lockBy; + count = 1; + return; + } + if (owner == lockBy) { + count++; + return; + } + cond.awaitUninterruptibly(); + } + } finally { + lock.unlock(); + } + } + + private boolean tryLock0(Object lockBy) { + if (!lock.tryLock()) { + return false; + } + try { + if (owner == null) { + owner = lockBy; + count = 1; + return true; + } + if (owner == lockBy) { + count++; + return true; + } + return false; + } finally { + lock.unlock(); + } + } + + private void unlock0(Object unlockBy) { + lock.lock(); + try { + if (owner == null) { + throw new IllegalMonitorStateException("RegionStateNode " + regionInfo + " is not locked"); + } + if (owner != unlockBy) { + throw new IllegalMonitorStateException("RegionStateNode " + regionInfo + " is locked by " + + owner + ", can not be unlocked by " + unlockBy); + } + count--; + if (count == 0) { + owner = null; + cond.signal(); + } + } finally { + lock.unlock(); + } + } + + /** + * Normal lock, will set the current thread as owner. Typically you should use try...finally to + * call unlock in the finally block. + */ + void lock() { + lock0(Thread.currentThread()); + } + + /** + * Normal tryLock, will set the current thread as owner. Typically you should use try...finally to + * call unlock in the finally block. + */ + boolean tryLock() { + return tryLock0(Thread.currentThread()); + } + + /** + * Normal unLock, will use the current thread as owner. Typically you should use try...finally to + * call unlock in the finally block. + */ + void unlock() { + unlock0(Thread.currentThread()); + } + + /** + * Lock by a procedure. You can release the lock in another thread. + */ + void lock(Procedure proc) { + lock0(proc); + } + + /** + * TryLock by a procedure. You can release the lock in another thread. + */ + boolean tryLock(Procedure proc) { + return tryLock0(proc); + } + + /** + * Unlock by a procedure. You do not need to call this method in the same thread with lock. + */ + void unlock(Procedure proc) { + unlock0(proc); + } + + boolean isLocked() { + lock.lock(); + try { + return owner != null; + } finally { + lock.unlock(); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 3561e0cd055b..4d506365f238 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -176,7 +176,7 @@ public static void visitMetaEntry(final RegionStateVisitor visitor, final Result } } - void updateRegionLocation(RegionStateNode regionStateNode) throws IOException { + private Put generateUpdateRegionLocationPut(RegionStateNode regionStateNode) throws IOException { long time = EnvironmentEdgeManager.currentTime(); long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum() @@ -221,11 +221,34 @@ && hasGlobalReplicationScope(regionInfo.getTable()) .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name())) .build()); LOG.info(info.toString()); - updateRegionLocation(regionInfo, state, put); + return put; + } + + CompletableFuture updateRegionLocation(RegionStateNode regionStateNode) { + Put put; + try { + put = generateUpdateRegionLocationPut(regionStateNode); + } catch (IOException e) { + return FutureUtils.failedFuture(e); + } + RegionInfo regionInfo = regionStateNode.getRegionInfo(); + State state = regionStateNode.getState(); + CompletableFuture future = updateRegionLocation(regionInfo, state, put); if (regionInfo.isMetaRegion() && regionInfo.isFirst()) { // mirror the meta location to zookeeper - mirrorMetaLocation(regionInfo, regionLocation, state); + // we store meta location in master local region which means the above method is + // synchronous(we just wrap the result with a CompletableFuture to make it look like + // asynchronous), so it is OK to just call this method directly here + assert future.isDone(); + if (!future.isCompletedExceptionally()) { + try { + mirrorMetaLocation(regionInfo, regionStateNode.getRegionLocation(), state); + } catch (IOException e) { + return FutureUtils.failedFuture(e); + } + } } + return future; } private void mirrorMetaLocation(RegionInfo regionInfo, ServerName serverName, State state) @@ -249,25 +272,31 @@ private void removeMirrorMetaLocation(int oldReplicaCount, int newReplicaCount) } } - private void updateRegionLocation(RegionInfo regionInfo, State state, Put put) - throws IOException { - try { - if (regionInfo.isMetaRegion()) { + private CompletableFuture updateRegionLocation(RegionInfo regionInfo, State state, + Put put) { + CompletableFuture future; + if (regionInfo.isMetaRegion()) { + try { masterRegion.update(r -> r.put(put)); - } else { - try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) { - table.put(put); - } + future = CompletableFuture.completedFuture(null); + } catch (Exception e) { + future = FutureUtils.failedFuture(e); } - } catch (IOException e) { - // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host! - // In tests we abort the Master! - String msg = String.format("FAILED persisting region=%s state=%s", - regionInfo.getShortNameToLog(), state); - LOG.error(msg, e); - master.abort(msg, e); - throw e; + } else { + AsyncTable table = master.getAsyncConnection().getTable(TableName.META_TABLE_NAME); + future = table.put(put); } + FutureUtils.addListener(future, (r, e) -> { + if (e != null) { + // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host! + // In tests we abort the Master! + String msg = String.format("FAILED persisting region=%s state=%s", + regionInfo.getShortNameToLog(), state); + LOG.error(msg, e); + master.abort(msg, e); + } + }); + return future; } private long getOpenSeqNumForParentRegion(RegionInfo region) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java index 81397915647d..911c0f3111e9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java @@ -24,6 +24,7 @@ import edu.umd.cs.findbugs.annotations.Nullable; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.ServerName; @@ -38,11 +39,13 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure; import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -137,6 +140,8 @@ public class TransitRegionStateProcedure private long forceRetainmentTotalWait; + private CompletableFuture future; + public TransitRegionStateProcedure() { } @@ -268,21 +273,54 @@ private void queueAssign(MasterProcedureEnv env, RegionStateNode regionNode) } } - private void openRegion(MasterProcedureEnv env, RegionStateNode regionNode) throws IOException { + private CompletableFuture getFuture() { + return future; + } + + private void setFuture(CompletableFuture f) { + future = f; + } + + private void openRegionAfterUpdatingMeta(ServerName loc) { + addChildProcedure(new OpenRegionProcedure(this, getRegion(), loc)); + setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED); + } + + private void openRegion(MasterProcedureEnv env, RegionStateNode regionNode) + throws IOException, ProcedureSuspendedException { ServerName loc = regionNode.getRegionLocation(); + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + () -> openRegionAfterUpdatingMeta(loc)) + ) { + return; + } if (loc == null || BOGUS_SERVER_NAME.equals(loc)) { LOG.warn("No location specified for {}, jump back to state {} to get one", getRegion(), RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE); setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE); throw new HBaseIOException("Failed to open region, the location is null or bogus."); } - env.getAssignmentManager().regionOpening(regionNode); - addChildProcedure(new OpenRegionProcedure(this, getRegion(), loc)); - setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED); + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getAssignmentManager().regionOpening(regionNode), env, + () -> openRegionAfterUpdatingMeta(loc)); + } + + private void regionFailedOpenAfterUpdatingMeta(MasterProcedureEnv env, + RegionStateNode regionNode) { + setFailure(getClass().getSimpleName(), new RetriesExhaustedException( + "Max attempts " + env.getAssignmentManager().getAssignMaxAttempts() + " exceeded")); + regionNode.unsetProcedure(this); } private Flow confirmOpened(MasterProcedureEnv env, RegionStateNode regionNode) - throws IOException { + throws IOException, ProcedureSuspendedException { + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + () -> regionFailedOpenAfterUpdatingMeta(env, regionNode)) + ) { + return Flow.NO_MORE_STATE; + } if (regionNode.isInState(State.OPEN)) { retryCounter = null; if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED) { @@ -306,14 +344,16 @@ private Flow confirmOpened(MasterProcedureEnv env, RegionStateNode regionNode) LOG.info("Retry={} of max={}; {}; {}", retries, maxAttempts, this, regionNode.toShortString()); if (retries >= maxAttempts) { - env.getAssignmentManager().regionFailedOpen(regionNode, true); - setFailure(getClass().getSimpleName(), new RetriesExhaustedException( - "Max attempts " + env.getAssignmentManager().getAssignMaxAttempts() + " exceeded")); - regionNode.unsetProcedure(this); + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getAssignmentManager().regionFailedOpen(regionNode, true), env, + () -> regionFailedOpenAfterUpdatingMeta(env, regionNode)); return Flow.NO_MORE_STATE; } - env.getAssignmentManager().regionFailedOpen(regionNode, false); + // if not giving up, we will not update meta, so the returned CompletableFuture should be a fake + // one, which should have been completed already + CompletableFuture future = env.getAssignmentManager().regionFailedOpen(regionNode, false); + assert future.isDone(); // we failed to assign the region, force a new plan forceNewPlan = true; regionNode.setRegionLocation(null); @@ -329,17 +369,29 @@ private Flow confirmOpened(MasterProcedureEnv env, RegionStateNode regionNode) } } - private void closeRegion(MasterProcedureEnv env, RegionStateNode regionNode) throws IOException { + private void closeRegionAfterUpdatingMeta(RegionStateNode regionNode) { + CloseRegionProcedure closeProc = isSplit + ? new CloseRegionProcedure(this, getRegion(), regionNode.getRegionLocation(), assignCandidate, + true) + : new CloseRegionProcedure(this, getRegion(), regionNode.getRegionLocation(), assignCandidate, + evictCache); + addChildProcedure(closeProc); + setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED); + } + + private void closeRegion(MasterProcedureEnv env, RegionStateNode regionNode) + throws IOException, ProcedureSuspendedException { + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + () -> closeRegionAfterUpdatingMeta(regionNode)) + ) { + return; + } if (regionNode.isInState(State.OPEN, State.CLOSING, State.MERGING, State.SPLITTING)) { // this is the normal case - env.getAssignmentManager().regionClosing(regionNode); - CloseRegionProcedure closeProc = isSplit - ? new CloseRegionProcedure(this, getRegion(), regionNode.getRegionLocation(), - assignCandidate, true) - : new CloseRegionProcedure(this, getRegion(), regionNode.getRegionLocation(), - assignCandidate, evictCache); - addChildProcedure(closeProc); - setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED); + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getAssignmentManager().regionClosing(regionNode), env, + () -> closeRegionAfterUpdatingMeta(regionNode)); } else { forceNewPlan = true; regionNode.setRegionLocation(null); @@ -393,11 +445,18 @@ protected Procedure[] execute(MasterProcedureEnv env) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { RegionStateNode regionNode = env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(getRegion()); - regionNode.lock(); + if (future == null) { + // if future is not null, we will not release the regionNode lock, so do not need to lock it + // again + regionNode.lock(this); + } try { return super.execute(env); } finally { - regionNode.unlock(); + if (future == null) { + // release the lock if there is no pending updating meta operation + regionNode.unlock(this); + } } } @@ -452,10 +511,7 @@ protected Flow executeFromState(MasterProcedureEnv env, RegionStateTransitionSta "Failed transition, suspend {}secs {}; {}; waiting on rectified condition fixed " + "by other Procedure or operator intervention", backoff / 1000, this, regionNode.toShortString(), e); - setTimeout(Math.toIntExact(backoff)); - setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); - skipPersistence(); - throw new ProcedureSuspendedException(); + throw suspend(Math.toIntExact(backoff), true); } } @@ -492,15 +548,25 @@ public void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode, } // Should be called with RegionStateNode locked - public void serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode, - ServerName serverName, boolean forceNewPlan) throws IOException { + public CompletableFuture serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode, + ServerName serverName, boolean forceNewPlan) { this.forceNewPlan = forceNewPlan; if (remoteProc != null) { // this means we are waiting for the sub procedure, so wake it up - remoteProc.serverCrashed(env, regionNode, serverName); + try { + remoteProc.serverCrashed(env, regionNode, serverName); + } catch (Exception e) { + return FutureUtils.failedFuture(e); + } + return CompletableFuture.completedFuture(null); } else { - // we are in RUNNING state, just update the region state, and we will process it later. - env.getAssignmentManager().regionClosedAbnormally(regionNode); + if (regionNode.isInState(State.ABNORMALLY_CLOSED)) { + // should be a retry, where we have already changed the region state to abnormally closed + return CompletableFuture.completedFuture(null); + } else { + // we are in RUNNING state, just update the region state, and we will process it later. + return env.getAssignmentManager().regionClosedAbnormally(regionNode); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java index f7f4146bd0d5..218d3096d8df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; +import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -86,6 +87,13 @@ public MasterProcedureEnv(final MasterServices master, this.remoteDispatcher = remoteDispatcher; } + /** + * Get a thread pool for executing some asynchronous tasks + */ + public ExecutorService getAsyncTaskExecutor() { + return master.getMasterProcedureExecutor().getAsyncTaskExecutor(); + } + public User getRequestUser() { return RpcServer.getRequestUser().orElse(Superusers.getSystemUser()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java index 97976756d828..901cc38a7be7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; @@ -107,6 +109,10 @@ public class ServerCrashProcedure extends // progress will not update the state because the actual state is overwritten by its next state private ServerCrashState currentRunningState = getInitialState(); + private CompletableFuture updateMetaFuture; + + private int processedRegions = 0; + /** * Call this constructor queuing up a Procedure. * @param serverName Name of the crashed server. @@ -532,6 +538,14 @@ protected boolean isMatchingRegionLocation(RegionStateNode rsn) { return this.serverName.equals(rsn.getRegionLocation()); } + private CompletableFuture getUpdateMetaFuture() { + return updateMetaFuture; + } + + private void setUpdateMetaFuture(CompletableFuture f) { + updateMetaFuture = f; + } + /** * Assign the regions on the crashed RS to other Rses. *

@@ -542,14 +556,30 @@ protected boolean isMatchingRegionLocation(RegionStateNode rsn) { * We will also check whether the table for a region is enabled, if not, we will skip assigning * it. */ - private void assignRegions(MasterProcedureEnv env, List regions) throws IOException { + private void assignRegions(MasterProcedureEnv env, List regions) + throws IOException, ProcedureSuspendedException { AssignmentManager am = env.getMasterServices().getAssignmentManager(); boolean retainAssignment = env.getMasterConfiguration().getBoolean(MASTER_SCP_RETAIN_ASSIGNMENT, DEFAULT_MASTER_SCP_RETAIN_ASSIGNMENT); - for (RegionInfo region : regions) { + // Since we may suspend in the middle of this loop, so here we use processedRegions to record + // the progress, so next time we can locate the correct region + // We do not need to persist the processedRegions when serializing the procedure, as when master + // restarts, the sub procedure list will be cleared when rescheduling this SCP again, so we need + // to start from beginning. + for (int n = regions.size(); processedRegions < n; processedRegions++) { + RegionInfo region = regions.get(processedRegions); RegionStateNode regionNode = am.getRegionStates().getOrCreateRegionStateNode(region); - regionNode.lock(); + if (updateMetaFuture == null) { + regionNode.lock(this); + } try { + if ( + ProcedureFutureUtil.checkFuture(this, this::getUpdateMetaFuture, + this::setUpdateMetaFuture, () -> { + }) + ) { + continue; + } // This is possible, as when a server is dead, TRSP will fail to schedule a RemoteProcedure // and then try to assign the region to a new RS. And before it has updated the region // location to the new RS, we may have already called the am.getRegionsOnServer so we will @@ -572,8 +602,10 @@ private void assignRegions(MasterProcedureEnv env, List regions) thr } if (regionNode.getProcedure() != null) { LOG.info("{} found RIT {}; {}", this, regionNode.getProcedure(), regionNode); - regionNode.getProcedure().serverCrashed(env, regionNode, getServerName(), - !retainAssignment); + ProcedureFutureUtil.suspendIfNecessary(this, this::setUpdateMetaFuture, regionNode + .getProcedure().serverCrashed(env, regionNode, getServerName(), !retainAssignment), env, + () -> { + }); continue; } if ( @@ -583,7 +615,9 @@ private void assignRegions(MasterProcedureEnv env, List regions) thr // We need to change the state here otherwise the TRSP scheduled by DTP will try to // close the region from a dead server and will never succeed. Please see HBASE-23636 // for more details. - env.getAssignmentManager().regionClosedAbnormally(regionNode); + ProcedureFutureUtil.suspendIfNecessary(this, this::setUpdateMetaFuture, + env.getAssignmentManager().regionClosedAbnormally(regionNode), env, () -> { + }); LOG.info("{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED.", this, regionNode); continue; @@ -599,11 +633,20 @@ private void assignRegions(MasterProcedureEnv env, List regions) thr TransitRegionStateProcedure proc = TransitRegionStateProcedure.assign(env, region, !retainAssignment, null); regionNode.setProcedure(proc); + // It is OK to still use addChildProcedure even if we suspend in the middle of this loop, as + // the subProcList will only be cleared when we successfully returned from the + // executeFromState method. This means we will submit all the TRSPs after we successfully + // finished this loop addChildProcedure(proc); } finally { - regionNode.unlock(); + if (updateMetaFuture == null) { + regionNode.unlock(this); + } } } + // we will call this method two times if the region server carries meta, so we need to reset it + // to 0 after successfully finished the above loop + processedRegions = 0; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index 5e907c1681ac..9730391baf22 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java @@ -110,8 +110,8 @@ assert getRegion().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID || isFailed() private void deleteRegionFromFileSystem(final MasterProcedureEnv env) throws IOException { RegionStateNode regionNode = env.getAssignmentManager().getRegionStates().getRegionStateNode(getRegion()); + regionNode.lock(); try { - regionNode.lock(); final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName()); HRegionFileSystem.deleteRegionFromFileSystem(env.getMasterConfiguration(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java index c88d613e5260..cff1b3879360 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.procedure.GlobalProcedureInterface; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface; +import org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; @@ -43,8 +44,6 @@ import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration; -import org.apache.hadoop.hbase.util.FutureUtils; -import org.apache.hadoop.hbase.util.IdLock; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.yetus.audience.InterfaceAudience; @@ -73,7 +72,7 @@ public class MigrateReplicationQueueFromZkToTableProcedure private List disabledPeerIds; - private CompletableFuture future; + private CompletableFuture future; private ExecutorService executor; @@ -84,6 +83,14 @@ public String getGlobalId() { return getClass().getSimpleName(); } + private CompletableFuture getFuture() { + return future; + } + + private void setFuture(CompletableFuture f) { + future = f; + } + private ProcedureSuspendedException suspend(Configuration conf, LongConsumer backoffConsumer) throws ProcedureSuspendedException { if (retryCounter == null) { @@ -153,6 +160,12 @@ private void waitUntilNoPeerProcedure(MasterProcedureEnv env) throws ProcedureSu LOG.info("No pending peer procedures found, continue..."); } + private void finishMigartion() { + shutdownExecutorService(); + setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING); + resetRetry(); + } + @Override protected Flow executeFromState(MasterProcedureEnv env, MigrateReplicationQueueFromZkToTableState state) @@ -195,52 +208,23 @@ protected Flow executeFromState(MasterProcedureEnv env, setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE); return Flow.HAS_MORE_STATE; case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE: - if (future != null) { - // should have finished when we arrive here - assert future.isDone(); - try { - future.get(); - } catch (Exception e) { - future = null; - throw suspend(env.getMasterConfiguration(), - backoff -> LOG.warn("failed to migrate queue data, sleep {} secs and retry later", - backoff / 1000, e)); + try { + if ( + ProcedureFutureUtil.checkFuture(this, this::getFuture, this::setFuture, + this::finishMigartion) + ) { + return Flow.HAS_MORE_STATE; } - shutdownExecutorService(); - setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING); - resetRetry(); - return Flow.HAS_MORE_STATE; + ProcedureFutureUtil.suspendIfNecessary(this, this::setFuture, + env.getReplicationPeerManager() + .migrateQueuesFromZk(env.getMasterServices().getZooKeeper(), getExecutorService()), + env, this::finishMigartion); + } catch (IOException e) { + throw suspend(env.getMasterConfiguration(), + backoff -> LOG.warn("failed to migrate queue data, sleep {} secs and retry later", + backoff / 1000, e)); } - future = env.getReplicationPeerManager() - .migrateQueuesFromZk(env.getMasterServices().getZooKeeper(), getExecutorService()); - FutureUtils.addListener(future, (r, e) -> { - // should acquire procedure execution lock to make sure that the procedure executor has - // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be - // race and cause unexpected result - IdLock procLock = - env.getMasterServices().getMasterProcedureExecutor().getProcExecutionLock(); - IdLock.Entry lockEntry; - try { - lockEntry = procLock.getLockEntry(getProcId()); - } catch (IOException ioe) { - LOG.error("Error while acquiring execution lock for procedure {}" - + " when trying to wake it up, aborting...", this, ioe); - env.getMasterServices().abort("Can not acquire procedure execution lock", e); - return; - } - try { - setTimeoutFailure(env); - } finally { - procLock.releaseLockEntry(lockEntry); - } - }); - // here we set timeout to -1 so the ProcedureExecutor will not schedule a Timer for us - setTimeout(-1); - setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); - // skip persistence is a must now since when restarting, if the procedure is in - // WAITING_TIMEOUT state and has -1 as timeout, it will block there forever... - skipPersistence(); - throw new ProcedureSuspendedException(); + return Flow.HAS_MORE_STATE; case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING: long rsWithLowerVersion = env.getMasterServices().getServerManager().getOnlineServers().values().stream() diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java index 988c519f781d..322b5bb7fc78 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java @@ -797,7 +797,7 @@ private CompletableFuture runAsync(ExceptionalRunnable task, ExecutorService /** * Submit the migration tasks to the given {@code executor}. */ - CompletableFuture migrateQueuesFromZk(ZKWatcher zookeeper, ExecutorService executor) { + CompletableFuture migrateQueuesFromZk(ZKWatcher zookeeper, ExecutorService executor) { // the replication queue table creation is asynchronous and will be triggered by addPeer, so // here we need to manually initialize it since we will not call addPeer. try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java new file mode 100644 index 000000000000..8ca4cba245da --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java @@ -0,0 +1,112 @@ +/* + * 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.procedure2; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Supplier; +import org.apache.commons.lang3.mutable.MutableBoolean; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.util.FutureUtils; +import org.apache.hadoop.hbase.util.IdLock; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A helper class for switching procedure out(yielding) while it is doing some time consuming + * operation, such as updating meta, where we can get a {@link CompletableFuture} about the + * operation. + */ +@InterfaceAudience.Private +public final class ProcedureFutureUtil { + + private static final Logger LOG = LoggerFactory.getLogger(ProcedureFutureUtil.class); + + private ProcedureFutureUtil() { + } + + public static boolean checkFuture(Procedure proc, Supplier> getFuture, + Consumer> setFuture, Runnable actionAfterDone) throws IOException { + CompletableFuture future = getFuture.get(); + if (future == null) { + return false; + } + // reset future + setFuture.accept(null); + FutureUtils.get(future); + actionAfterDone.run(); + return true; + } + + public static void suspendIfNecessary(Procedure proc, + Consumer> setFuture, CompletableFuture future, + MasterProcedureEnv env, Runnable actionAfterDone) + throws IOException, ProcedureSuspendedException { + MutableBoolean completed = new MutableBoolean(false); + Thread currentThread = Thread.currentThread(); + FutureUtils.addListener(future, (r, e) -> { + if (Thread.currentThread() == currentThread) { + LOG.debug("The future has completed while adding callback, give up suspending procedure {}", + proc); + // this means the future has already been completed, as we call the callback directly while + // calling addListener, so here we just set completed to true without doing anything + completed.setTrue(); + return; + } + LOG.debug("Going to wake up procedure {} because future has completed", proc); + // This callback may be called inside netty's event loop, so we should not block it for a long + // time. The worker executor will hold the execution lock while executing the procedure, and + // we may persist the procedure state inside the lock, which is a time consuming operation. + // And what makes things worse is that, we persist procedure state to master local region, + // where the AsyncFSWAL implementation will use the same netty's event loop for dealing with + // I/O, which could even cause dead lock. + env.getAsyncTaskExecutor().execute(() -> { + // should acquire procedure execution lock to make sure that the procedure executor has + // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be + // race and cause unexpected result + IdLock procLock = + env.getMasterServices().getMasterProcedureExecutor().getProcExecutionLock(); + IdLock.Entry lockEntry; + try { + lockEntry = procLock.getLockEntry(proc.getProcId()); + } catch (IOException ioe) { + LOG.error("Error while acquiring execution lock for procedure {}" + + " when trying to wake it up, aborting...", proc, ioe); + env.getMasterServices().abort("Can not acquire procedure execution lock", e); + return; + } + try { + env.getProcedureScheduler().addFront(proc); + } finally { + procLock.releaseLockEntry(lockEntry); + } + }); + }); + if (completed.getValue()) { + FutureUtils.get(future); + actionAfterDone.run(); + } else { + // suspend the procedure + setFuture.accept(future); + proc.skipPersistence(); + throw new ProcedureSuspendedException(); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java index c601425e5f0a..6bc4c9d14e6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CoordinatedStateManager; @@ -305,7 +306,8 @@ public MockRegionStateStore(MasterServices master, MasterRegion masterRegion) { } @Override - public void updateRegionLocation(RegionStateNode regionNode) throws IOException { + public CompletableFuture updateRegionLocation(RegionStateNode regionNode) { + return CompletableFuture.completedFuture(null); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java index eb6f069474a9..69381b37e38c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -80,7 +79,7 @@ public void tearDownAfterTest() throws IOException { for (RegionInfo region : UTIL.getAdmin().getRegions(TABLE_NAME)) { RegionStateNode regionNode = AM.getRegionStates().getRegionStateNode(region); // confirm that we have released the lock - assertFalse(((ReentrantLock) regionNode.lock).isLocked()); + assertFalse(regionNode.isLocked()); TransitRegionStateProcedure proc = regionNode.getProcedure(); if (proc != null) { regionNode.unsetProcedure(proc); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 2757e0dd9f20..2f88f6087dd4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -61,11 +62,11 @@ public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion } @Override - void persistToMeta(RegionStateNode regionNode) throws IOException { + CompletableFuture persistToMeta(RegionStateNode regionNode) { if (FAIL) { - throw new IOException("Inject Error!"); + return FutureUtils.failedFuture(new IOException("Inject Error!")); } - super.persistToMeta(regionNode); + return super.persistToMeta(regionNode); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java index 1d13912fb72c..05179c5eadb7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import org.apache.hadoop.conf.Configuration; @@ -38,6 +39,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -76,8 +78,14 @@ public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion } @Override - void regionOpening(RegionStateNode regionNode) throws IOException { - super.regionOpening(regionNode); + CompletableFuture regionOpening(RegionStateNode regionNode) { + CompletableFuture future = super.regionOpening(regionNode); + try { + // wait until the operation done, then trigger later processing, to make the test more + // stable + FutureUtils.get(future); + } catch (IOException e) { + } if (regionNode.getRegionInfo().getTable().equals(NAME) && ARRIVE_REGION_OPENING != null) { ARRIVE_REGION_OPENING.countDown(); ARRIVE_REGION_OPENING = null; @@ -86,6 +94,7 @@ void regionOpening(RegionStateNode regionNode) throws IOException { } catch (InterruptedException e) { } } + return future; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateNodeLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateNodeLock.java new file mode 100644 index 000000000000..c308b69c98cf --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateNodeLock.java @@ -0,0 +1,139 @@ +/* + * 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.assignment; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.AtomicUtils; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, SmallTests.class }) +public class TestRegionStateNodeLock { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRegionStateNodeLock.class); + + private final RegionInfo regionInfo = + RegionInfoBuilder.newBuilder(TableName.valueOf("test")).build(); + + private RegionStateNodeLock lock; + + @Before + public void setUp() { + lock = new RegionStateNodeLock(regionInfo); + } + + @Test + public void testLockByThread() { + assertFalse(lock.isLocked()); + assertThrows(IllegalMonitorStateException.class, () -> lock.unlock()); + lock.lock(); + assertTrue(lock.isLocked()); + // reentrant + assertTrue(lock.tryLock()); + lock.unlock(); + assertTrue(lock.isLocked()); + lock.unlock(); + assertFalse(lock.isLocked()); + } + + @Test + public void testLockByProc() { + NoopProcedure proc = new NoopProcedure(); + assertFalse(lock.isLocked()); + assertThrows(IllegalMonitorStateException.class, () -> lock.unlock(proc)); + lock.lock(proc); + assertTrue(lock.isLocked()); + // reentrant + assertTrue(lock.tryLock(proc)); + lock.unlock(proc); + assertTrue(lock.isLocked()); + lock.unlock(proc); + assertFalse(lock.isLocked()); + } + + @Test + public void testLockProcThenThread() { + NoopProcedure proc = new NoopProcedure(); + assertFalse(lock.isLocked()); + lock.lock(proc); + assertFalse(lock.tryLock()); + assertThrows(IllegalMonitorStateException.class, () -> lock.unlock()); + long startNs = System.nanoTime(); + new Thread(() -> { + Threads.sleepWithoutInterrupt(2000); + lock.unlock(proc); + }).start(); + lock.lock(); + long costNs = System.nanoTime() - startNs; + assertThat(TimeUnit.NANOSECONDS.toMillis(costNs), greaterThanOrEqualTo(1800L)); + assertTrue(lock.isLocked()); + lock.unlock(); + assertFalse(lock.isLocked()); + } + + @Test + public void testLockMultiThread() throws InterruptedException { + int nThreads = 10; + AtomicLong concurrency = new AtomicLong(0); + AtomicLong maxConcurrency = new AtomicLong(0); + Thread[] threads = new Thread[nThreads]; + for (int i = 0; i < nThreads; i++) { + threads[i] = new Thread(() -> { + for (int j = 0; j < 1000; j++) { + lock.lock(); + try { + long c = concurrency.incrementAndGet(); + AtomicUtils.updateMax(maxConcurrency, c); + concurrency.decrementAndGet(); + } finally { + lock.unlock(); + } + Threads.sleepWithoutInterrupt(1); + } + }); + } + for (Thread t : threads) { + t.start(); + } + for (Thread t : threads) { + t.join(); + } + assertEquals(0, concurrency.get()); + assertEquals(1, maxConcurrency.get()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index 3d1a2c4caa94..cd73e09af6db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -85,7 +87,7 @@ public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion } @Override - void persistToMeta(RegionStateNode regionNode) throws IOException { + CompletableFuture persistToMeta(RegionStateNode regionNode) { TransitRegionStateProcedure proc = regionNode.getProcedure(); if (!regionNode.getRegionInfo().isMetaRegion() && proc.hasParent()) { Procedure p = @@ -96,10 +98,10 @@ void persistToMeta(RegionStateNode regionNode) throws IOException { ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdateInRollback( getMaster().getMasterProcedureExecutor(), true); } - throw new RuntimeException("inject code bug"); + return FutureUtils.failedFuture(new RuntimeException("inject code bug")); } } - super.persistToMeta(regionNode); + return super.persistToMeta(regionNode); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index 3319a761eb4c..d1e7dc147615 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -55,10 +56,16 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; + /** * Test to ensure that the priority for procedures and stuck checker can partially solve the problem * describe in HBASE-19976, that is, RecoverMetaProcedure can finally be executed within a certain * period of time. + *

+ * As of HBASE-28199, we no longer block a worker when updating meta now, so this test can not test + * adding procedure worker now, but it could still be used to make sure that we could make progress + * when meta is gone and we have a lot of pending TRSPs. */ @Category({ MasterTests.class, LargeTests.class }) public class TestProcedurePriority { @@ -129,6 +136,7 @@ public static void setUp() throws Exception { } UTIL.getAdmin().balance(BalanceRequest.newBuilder().setIgnoreRegionsInTransition(true).build()); UTIL.waitUntilNoRegionsInTransition(); + UTIL.getAdmin().balancerSwitch(false, true); } @AfterClass @@ -144,22 +152,26 @@ public void test() throws Exception { HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); FAIL = true; UTIL.getMiniHBaseCluster().killRegionServer(rsNoMeta.getServerName()); - // wait until all the worker thread are stuck, which means that the stuck checker will start to - // add new worker thread. ProcedureExecutor executor = UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + // wait until we have way more TRSPs than the core pool size, and then make sure we can recover + // normally UTIL.waitFor(60000, new ExplainingPredicate() { @Override public boolean evaluate() throws Exception { - return executor.getWorkerThreadCount() > CORE_POOL_SIZE; + return executor.getProcedures().stream().filter(p -> !p.isFinished()) + .filter(p -> p.getState() != ProcedureState.INITIALIZING) + .filter(p -> p instanceof TransitRegionStateProcedure).count() > 5 * CORE_POOL_SIZE; } @Override public String explainFailure() throws Exception { - return "Stuck checker does not add new worker thread"; + return "Not enough TRSPs scheduled"; } }); + // sleep more time to make sure the TRSPs have been executed + Thread.sleep(10000); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMetaThread.getRegionServer().getServerName()); rsWithMetaThread.join(); FAIL = false; From bc0f7a41b38de453bf3cc188d54d227e0b7f3ba9 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Mon, 4 Dec 2023 16:33:34 +0530 Subject: [PATCH 009/109] HBASE-28209: Create a jmx metrics to expose the oldWALs directory size (#5528) Signed-off-by: Wellington Chevreuil --- .../org/apache/hadoop/hbase/HConstants.java | 7 ++ .../src/main/resources/hbase-default.xml | 6 ++ .../hbase/master/MetricsMasterSource.java | 2 + .../hbase/master/MetricsMasterSourceImpl.java | 4 +- .../hbase/master/MetricsMasterWrapper.java | 5 ++ .../apache/hadoop/hbase/master/HMaster.java | 6 ++ .../hadoop/hbase/master/MasterWalManager.java | 12 +++ .../master/MetricsMasterWrapperImpl.java | 8 ++ .../hbase/master/OldWALsDirSizeChore.java | 53 +++++++++++ .../hbase/master/TestMasterMetrics.java | 2 + .../master/TestMasterMetricsWrapper.java | 1 + .../hbase/master/TestOldWALsDirSizeChore.java | 90 +++++++++++++++++++ .../asciidoc/_chapters/hbase-default.adoc | 11 +++ 13 files changed, 206 insertions(+), 1 deletion(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/OldWALsDirSizeChore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOldWALsDirSizeChore.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 2aa9ecf69ec4..5b53d2b2c0d3 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 @@ -153,6 +153,13 @@ public enum OperationStatusCode { /** Default value for the balancer period */ public static final int DEFAULT_HBASE_BALANCER_PERIOD = 300000; + /** Config for the oldWALs directory size updater period */ + public static final String HBASE_OLDWAL_DIR_SIZE_UPDATER_PERIOD = + "hbase.master.oldwals.dir.updater.period"; + + /** Default value for the oldWALs directory size updater period */ + public static final int DEFAULT_HBASE_OLDWAL_DIR_SIZE_UPDATER_PERIOD = 300000; + /** * Config key for enable/disable automatically separate child regions to different region servers * in the procedure of split regions. One child will be kept to the server where parent region is diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index 17a9853d2ad3..1bf63b136e04 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -606,6 +606,12 @@ possible configurations would overwhelm and obscure the important. Period at which the region balancer runs in the Master, in milliseconds. + + hbase.master.oldwals.dir.updater.period + 300000 + Period at which the oldWALs directory size calculator/updater will run in the + Master, in milliseconds. + hbase.regions.slop 0.2 diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java index 4a5b97ae66bf..d606ed630881 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java @@ -70,6 +70,7 @@ public interface MetricsMasterSource extends BaseSource { String CLUSTER_REQUESTS_NAME = "clusterRequests"; String CLUSTER_READ_REQUESTS_NAME = "clusterReadRequests"; String CLUSTER_WRITE_REQUESTS_NAME = "clusterWriteRequests"; + String OLD_WAL_DIR_SIZE_NAME = "oldWALsDirSize"; String MASTER_ACTIVE_TIME_DESC = "Master Active Time"; String MASTER_START_TIME_DESC = "Master Start Time"; String MASTER_FINISHED_INITIALIZATION_TIME_DESC = @@ -91,6 +92,7 @@ public interface MetricsMasterSource extends BaseSource { String OFFLINE_REGION_COUNT_DESC = "Number of Offline Regions"; String SERVER_CRASH_METRIC_PREFIX = "serverCrash"; + String OLD_WAL_DIR_SIZE_DESC = "size of old WALs directory in bytes"; /** * Increment the number of requests the cluster has seen. diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java index e0abf77bea44..011e66312aa3 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java @@ -129,7 +129,9 @@ public void getMetrics(MetricsCollector metricsCollector, boolean all) { .tag(Interns.info(SERVER_NAME_NAME, SERVER_NAME_DESC), masterWrapper.getServerName()) .tag(Interns.info(CLUSTER_ID_NAME, CLUSTER_ID_DESC), masterWrapper.getClusterId()) .tag(Interns.info(IS_ACTIVE_MASTER_NAME, IS_ACTIVE_MASTER_DESC), - String.valueOf(masterWrapper.getIsActiveMaster())); + String.valueOf(masterWrapper.getIsActiveMaster())) + .addGauge(Interns.info(OLD_WAL_DIR_SIZE_NAME, OLD_WAL_DIR_SIZE_DESC), + masterWrapper.getOldWALsDirSize()); } metricsRegistry.snapshot(metricsRecordBuilder, all); diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java index a900edf115e3..83419e2d5501 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java @@ -153,4 +153,9 @@ public interface MetricsMasterWrapper { * @return pair of count for online regions and offline regions */ PairOfSameType getRegionCounts(); + + /** + * Get the size of old WALs directory in bytes. + */ + long getOldWALsDirSize(); } 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 0dca3a0111e3..b492b177e426 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 @@ -458,6 +458,7 @@ public class HMaster extends HBaseServerBase implements Maste private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier; private QuotaObserverChore quotaObserverChore; private SnapshotQuotaObserverChore snapshotQuotaChore; + private OldWALsDirSizeChore oldWALsDirSizeChore; private ProcedureExecutor procedureExecutor; private ProcedureStore procedureStore; @@ -1362,6 +1363,10 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE this.rollingUpgradeChore = new RollingUpgradeChore(this); getChoreService().scheduleChore(rollingUpgradeChore); + + this.oldWALsDirSizeChore = new OldWALsDirSizeChore(this); + getChoreService().scheduleChore(this.oldWALsDirSizeChore); + status.markComplete("Progress after master initialized complete"); } @@ -1894,6 +1899,7 @@ protected void stopChores() { shutdownChore(hbckChore); shutdownChore(regionsRecoveryChore); shutdownChore(rollingUpgradeChore); + shutdownChore(oldWALsDirSizeChore); } /** Returns Get remote side's InetAddress */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java index aca04a8ac83a..a2c929cb79d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java @@ -89,6 +89,9 @@ public boolean accept(Path p) { // create the split log lock private final Lock splitLogLock = new ReentrantLock(); + // old WALs directory size in bytes + private long oldWALsDirSize; + /** * Superceded by {@link SplitWALManager}; i.e. procedure-based WAL splitting rather than 'classic' * zk-coordinated WAL splitting. @@ -114,6 +117,7 @@ public MasterWalManager(Configuration conf, FileSystem fs, Path rootDir, MasterS this.services = services; this.splitLogManager = new SplitLogManager(services, conf); this.oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME); + this.oldWALsDirSize = 0; } public void stop() { @@ -134,6 +138,14 @@ Path getOldLogDir() { return this.oldLogDir; } + public void updateOldWALsDirSize() throws IOException { + this.oldWALsDirSize = fs.getContentSummary(this.oldLogDir).getLength(); + } + + public long getOldWALsDirSize() { + return this.oldWALsDirSize; + } + public FileSystem getFileSystem() { return this.fs; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java index 923c663807f1..ff6f5b8e5df8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java @@ -238,4 +238,12 @@ public PairOfSameType getRegionCounts() { return new PairOfSameType<>(0, 0); } } + + @Override + public long getOldWALsDirSize() { + if (master == null || !master.isInitialized()) { + return 0; + } + return master.getMasterWalManager().getOldWALsDirSize(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OldWALsDirSizeChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OldWALsDirSizeChore.java new file mode 100644 index 000000000000..b2f0622b7d28 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/OldWALsDirSizeChore.java @@ -0,0 +1,53 @@ +/* + * 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 java.io.IOException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This chore is used to update the 'oldWALsDirSize' variable in {@link MasterWalManager} through + * the {@link MasterWalManager#updateOldWALsDirSize()} method. + */ +@InterfaceAudience.Private +public class OldWALsDirSizeChore extends ScheduledChore { + private static final Logger LOG = LoggerFactory.getLogger(OldWALsDirSizeChore.class); + + private final MasterServices master; + + public OldWALsDirSizeChore(MasterServices master) { + super(master.getServerName() + "-OldWALsDirSizeChore", master, + master.getConfiguration().getInt(HConstants.HBASE_OLDWAL_DIR_SIZE_UPDATER_PERIOD, + HConstants.DEFAULT_HBASE_OLDWAL_DIR_SIZE_UPDATER_PERIOD)); + this.master = master; + } + + @Override + protected void chore() { + try { + this.master.getMasterWalManager().updateOldWALsDirSize(); + } catch (IOException e) { + LOG.error("Got exception while trying to update the old WALs Directory size counter: " + + e.getMessage(), e); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java index 3966b1f6bec9..09618b3d899e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java @@ -183,6 +183,8 @@ public void testDefaultMasterMetrics() throws Exception { metricsHelper.assertCounter(MetricsMasterSource.SERVER_CRASH_METRIC_PREFIX + "SubmittedCount", 0, masterSource); + metricsHelper.assertGauge("oldWALsDirSize", master.getMasterWalManager().getOldWALsDirSize(), + masterSource); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java index f73ebde89c11..d1389ee68e9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java @@ -85,6 +85,7 @@ public void testInfo() throws IOException { assertEquals(master.getMasterCoprocessors().length, info.getCoprocessors().length); assertEquals(master.getServerManager().getOnlineServersList().size(), info.getNumRegionServers()); + assertEquals(master.getMasterWalManager().getOldWALsDirSize(), info.getOldWALsDirSize()); int regionServerCount = NUM_RS; assertEquals(regionServerCount, info.getNumRegionServers()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOldWALsDirSizeChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOldWALsDirSizeChore.java new file mode 100644 index 000000000000..7bd4ec5a1c24 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOldWALsDirSizeChore.java @@ -0,0 +1,90 @@ +/* + * 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 static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.master.assignment.MockMasterServices; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests for OldWALsDirSizeChore Here we are using the {@link MockMasterServices} to mock the Hbase + * Master. Chore's won't be running automatically; we need to run every time. + */ +@Category({ MasterTests.class, SmallTests.class }) +public class TestOldWALsDirSizeChore { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOldWALsDirSizeChore.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestOldWALsDirSizeChore.class); + + private MockMasterServices master; + + private static final HBaseTestingUtil HBASE_TESTING_UTILITY = new HBaseTestingUtil(); + + @Before + public void setUp() throws Exception { + master = new MockMasterServices(HBASE_TESTING_UTILITY.getConfiguration()); + master.start(10, null); + } + + @After + public void tearDown() throws Exception { + master.stop("tearDown"); + } + + @Test + public void testOldWALsDirSizeChore() throws IOException { + // Assume the OldWALs directory size is initially zero as the chore hasn't run yet + long currentOldWALsDirSize = master.getMasterWalManager().getOldWALsDirSize(); + assertEquals("Initial OldWALs directory size should be zero before running the chore", 0, + currentOldWALsDirSize); + + int dummyFileSize = 50 * 1024 * 1024; // 50MB + byte[] dummyData = new byte[dummyFileSize]; + + // Create a dummy file in the OldWALs directory + Path dummyFileInOldWALsDir = new Path(master.getMasterWalManager().getOldLogDir(), "dummy.txt"); + try (FSDataOutputStream outputStream = + master.getMasterWalManager().getFileSystem().create(dummyFileInOldWALsDir)) { + outputStream.write(dummyData); + } + + // Run the OldWALsDirSizeChore to update the directory size + OldWALsDirSizeChore oldWALsDirSizeChore = new OldWALsDirSizeChore(master); + oldWALsDirSizeChore.chore(); + + // Verify that the OldWALs directory size has increased by the file size + assertEquals("OldWALs directory size after chore should be as expected", dummyFileSize, + master.getMasterWalManager().getOldWALsDirSize()); + } +} diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc index 69fb4a0eae66..03391cc38b1a 100644 --- a/src/main/asciidoc/_chapters/hbase-default.adoc +++ b/src/main/asciidoc/_chapters/hbase-default.adoc @@ -761,6 +761,17 @@ Period at which the region balancer runs in the Master. `300000` +[[hbase.master.oldwals.dir.updater.period]] +*`hbase.master.oldwals.dir.updater.period`*:: ++ +.Description +Period at which the oldWALs directory size calculator/updater will run in the Master. + ++ +.Default +`300000` + + [[hbase.regions.slop]] *`hbase.regions.slop`*:: + From 25e9228e2c0a9a752db02e48d55010e0197fd203 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Mon, 4 Dec 2023 09:25:09 -0500 Subject: [PATCH 010/109] HBASE-28215: region reopen procedure batching/throttling (#5534) Signed-off-by: Bryan Beaudreault --- .../procedure/ModifyTableProcedure.java | 14 +- .../ReopenTableRegionsProcedure.java | 151 ++++++++++-- ...openTableRegionsProcedureBatchBackoff.java | 103 +++++++++ ...stReopenTableRegionsProcedureBatching.java | 216 ++++++++++++++++++ 4 files changed, 458 insertions(+), 26 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatchBackoff.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatching.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index ff0d7d2cc94b..f7314349ee2c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -17,6 +17,11 @@ */ package org.apache.hadoop.hbase.master.procedure; +import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT; +import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY; +import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_SIZE_MAX_DISABLED; +import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_SIZE_MAX_KEY; + import java.io.IOException; import java.util.Arrays; import java.util.Collections; @@ -26,6 +31,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ConcurrentTableModificationException; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; @@ -219,7 +225,13 @@ protected Flow executeFromState(final MasterProcedureEnv env, final ModifyTableS break; case MODIFY_TABLE_REOPEN_ALL_REGIONS: if (isTableEnabled(env)) { - addChildProcedure(new ReopenTableRegionsProcedure(getTableName())); + Configuration conf = env.getMasterConfiguration(); + long backoffMillis = conf.getLong(PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY, + PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT); + int batchSizeMax = + conf.getInt(PROGRESSIVE_BATCH_SIZE_MAX_KEY, PROGRESSIVE_BATCH_SIZE_MAX_DISABLED); + addChildProcedure( + new ReopenTableRegionsProcedure(getTableName(), backoffMillis, batchSizeMax)); } setNextState(ModifyTableState.MODIFY_TABLE_ASSIGN_NEW_REPLICAS); break; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java index 4efb1768b0ce..353636e6ddd6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.procedure; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -53,6 +54,17 @@ public class ReopenTableRegionsProcedure private static final Logger LOG = LoggerFactory.getLogger(ReopenTableRegionsProcedure.class); + public static final String PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY = + "hbase.reopen.table.regions.progressive.batch.backoff.ms"; + public static final long PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT = 0L; + public static final String PROGRESSIVE_BATCH_SIZE_MAX_KEY = + "hbase.reopen.table.regions.progressive.batch.size.max"; + public static final int PROGRESSIVE_BATCH_SIZE_MAX_DISABLED = -1; + private static final int PROGRESSIVE_BATCH_SIZE_MAX_DEFAULT_VALUE = Integer.MAX_VALUE; + + // this minimum prevents a max which would break this procedure + private static final int MINIMUM_BATCH_SIZE_MAX = 1; + private TableName tableName; // Specify specific regions of a table to reopen. @@ -61,20 +73,46 @@ public class ReopenTableRegionsProcedure private List regions = Collections.emptyList(); + private List currentRegionBatch = Collections.emptyList(); + private RetryCounter retryCounter; + private long reopenBatchBackoffMillis; + private int reopenBatchSize; + private int reopenBatchSizeMax; + private long regionsReopened = 0; + private long batchesProcessed = 0; + public ReopenTableRegionsProcedure() { - regionNames = Collections.emptyList(); + this(null); } public ReopenTableRegionsProcedure(TableName tableName) { - this.tableName = tableName; - this.regionNames = Collections.emptyList(); + this(tableName, Collections.emptyList()); } public ReopenTableRegionsProcedure(final TableName tableName, final List regionNames) { + this(tableName, regionNames, PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT, + PROGRESSIVE_BATCH_SIZE_MAX_DISABLED); + } + + public ReopenTableRegionsProcedure(final TableName tableName, long reopenBatchBackoffMillis, + int reopenBatchSizeMax) { + this(tableName, Collections.emptyList(), reopenBatchBackoffMillis, reopenBatchSizeMax); + } + + public ReopenTableRegionsProcedure(final TableName tableName, final List regionNames, + long reopenBatchBackoffMillis, int reopenBatchSizeMax) { this.tableName = tableName; this.regionNames = regionNames; + this.reopenBatchBackoffMillis = reopenBatchBackoffMillis; + if (reopenBatchSizeMax == PROGRESSIVE_BATCH_SIZE_MAX_DISABLED) { + this.reopenBatchSize = Integer.MAX_VALUE; + this.reopenBatchSizeMax = Integer.MAX_VALUE; + } else { + this.reopenBatchSize = 1; + this.reopenBatchSizeMax = Math.max(reopenBatchSizeMax, MINIMUM_BATCH_SIZE_MAX); + } } @Override @@ -87,6 +125,30 @@ public TableOperationType getTableOperationType() { return TableOperationType.REGION_EDIT; } + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public long getRegionsReopened() { + return regionsReopened; + } + + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public long getBatchesProcessed() { + return batchesProcessed; + } + + @RestrictedApi(explanation = "Should only be called internally or in tests", link = "", + allowedOnPath = ".*(/src/test/.*|ReopenTableRegionsProcedure).java") + protected int progressBatchSize() { + int previousBatchSize = reopenBatchSize; + reopenBatchSize = Math.min(reopenBatchSizeMax, 2 * reopenBatchSize); + if (reopenBatchSize < previousBatchSize) { + // the batch size should never decrease. this must be overflow, so just use max + reopenBatchSize = reopenBatchSizeMax; + } + return reopenBatchSize; + } + private boolean canSchedule(MasterProcedureEnv env, HRegionLocation loc) { if (loc.getSeqNum() < 0) { return false; @@ -114,7 +176,13 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS); return Flow.HAS_MORE_STATE; case REOPEN_TABLE_REGIONS_REOPEN_REGIONS: - for (HRegionLocation loc : regions) { + // if we didn't finish reopening the last batch yet, let's keep trying until we do. + // at that point, the batch will be empty and we can generate a new batch + if (!regions.isEmpty() && currentRegionBatch.isEmpty()) { + currentRegionBatch = regions.stream().limit(reopenBatchSize).collect(Collectors.toList()); + batchesProcessed++; + } + for (HRegionLocation loc : currentRegionBatch) { RegionStateNode regionNode = env.getAssignmentManager().getRegionStates().getRegionStateNode(loc.getRegion()); // this possible, maybe the region has already been merged or split, see HBASE-20921 @@ -133,39 +201,72 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState regionNode.unlock(); } addChildProcedure(proc); + regionsReopened++; } setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_CONFIRM_REOPENED); return Flow.HAS_MORE_STATE; case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED: - regions = regions.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened) - .filter(l -> l != null).collect(Collectors.toList()); + // update region lists based on what's been reopened + regions = filterReopened(env, regions); + currentRegionBatch = filterReopened(env, currentRegionBatch); + + // existing batch didn't fully reopen, so try to resolve that first. + // since this is a retry, don't do the batch backoff + if (!currentRegionBatch.isEmpty()) { + return reopenIfSchedulable(env, currentRegionBatch, false); + } + if (regions.isEmpty()) { return Flow.NO_MORE_STATE; } - if (regions.stream().anyMatch(loc -> canSchedule(env, loc))) { - retryCounter = null; - setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS); - return Flow.HAS_MORE_STATE; - } - // We can not schedule TRSP for all the regions need to reopen, wait for a while and retry - // again. - if (retryCounter == null) { - retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration()); - } - long backoff = retryCounter.getBackoffTimeAndIncrementAttempts(); - LOG.info( - "There are still {} region(s) which need to be reopened for table {} are in " - + "OPENING state, suspend {}secs and try again later", - regions.size(), tableName, backoff / 1000); - setTimeout(Math.toIntExact(backoff)); - setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); - skipPersistence(); - throw new ProcedureSuspendedException(); + + // current batch is finished, schedule more regions + return reopenIfSchedulable(env, regions, true); default: throw new UnsupportedOperationException("unhandled state=" + state); } } + private List filterReopened(MasterProcedureEnv env, + List regionsToCheck) { + return regionsToCheck.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened) + .filter(l -> l != null).collect(Collectors.toList()); + } + + private Flow reopenIfSchedulable(MasterProcedureEnv env, List regionsToReopen, + boolean shouldBatchBackoff) throws ProcedureSuspendedException { + if (regionsToReopen.stream().anyMatch(loc -> canSchedule(env, loc))) { + retryCounter = null; + setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS); + if (shouldBatchBackoff && reopenBatchBackoffMillis > 0) { + progressBatchSize(); + setBackoffState(reopenBatchBackoffMillis); + throw new ProcedureSuspendedException(); + } else { + return Flow.HAS_MORE_STATE; + } + } + + // We can not schedule TRSP for all the regions need to reopen, wait for a while and retry + // again. + if (retryCounter == null) { + retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration()); + } + long backoffMillis = retryCounter.getBackoffTimeAndIncrementAttempts(); + LOG.info( + "There are still {} region(s) which need to be reopened for table {}. {} are in " + + "OPENING state, suspend {}secs and try again later", + regions.size(), tableName, currentRegionBatch.size(), backoffMillis / 1000); + setBackoffState(backoffMillis); + throw new ProcedureSuspendedException(); + } + + private void setBackoffState(long millis) { + setTimeout(Math.toIntExact(millis)); + setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT); + skipPersistence(); + } + private List getRegionLocationsForReopen(List tableRegionsForReopen) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatchBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatchBackoff.java new file mode 100644 index 000000000000..fbabb1fa22cc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatchBackoff.java @@ -0,0 +1,103 @@ +/* + * 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.procedure; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.ServerManager; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Confirm that we will rate limit reopen batches when reopening all table regions. This can avoid + * the pain associated with reopening too many regions at once. + */ +@Category({ MasterTests.class, MediumTests.class }) +public class TestReopenTableRegionsProcedureBatchBackoff { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReopenTableRegionsProcedureBatchBackoff.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + private static TableName TABLE_NAME = TableName.valueOf("BatchBackoff"); + private static final int BACKOFF_MILLIS_PER_RS = 3_000; + private static final int REOPEN_BATCH_SIZE = 1; + + private static byte[] CF = Bytes.toBytes("cf"); + + @BeforeClass + public static void setUp() throws Exception { + Configuration conf = UTIL.getConfiguration(); + conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1); + UTIL.startMiniCluster(1); + UTIL.createMultiRegionTable(TABLE_NAME, CF, 10); + } + + @AfterClass + public static void tearDown() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @Test + public void testRegionBatchBackoff() throws IOException { + ProcedureExecutor procExec = + UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + List regions = UTIL.getAdmin().getRegions(TABLE_NAME); + assertTrue(10 <= regions.size()); + ReopenTableRegionsProcedure proc = + new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, REOPEN_BATCH_SIZE); + procExec.submitProcedure(proc); + Instant startedAt = Instant.now(); + ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); + Instant stoppedAt = Instant.now(); + assertTrue(Duration.between(startedAt, stoppedAt).toMillis() + > (long) regions.size() * BACKOFF_MILLIS_PER_RS); + } + + @Test + public void testRegionBatchNoBackoff() throws IOException { + ProcedureExecutor procExec = + UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + List regions = UTIL.getAdmin().getRegions(TABLE_NAME); + assertTrue(10 <= regions.size()); + int noBackoffMillis = 0; + ReopenTableRegionsProcedure proc = + new ReopenTableRegionsProcedure(TABLE_NAME, noBackoffMillis, REOPEN_BATCH_SIZE); + procExec.submitProcedure(proc); + ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, + (long) regions.size() * BACKOFF_MILLIS_PER_RS); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatching.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatching.java new file mode 100644 index 000000000000..8ea9b3c6a309 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestReopenTableRegionsProcedureBatching.java @@ -0,0 +1,216 @@ +/* + * 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.procedure; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.RegionState.State; +import org.apache.hadoop.hbase.master.ServerManager; +import org.apache.hadoop.hbase.master.assignment.AssignmentManager; +import org.apache.hadoop.hbase.master.assignment.RegionStateNode; +import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; + +/** + * Confirm that we will batch region reopens when reopening all table regions. This can avoid the + * pain associated with reopening too many regions at once. + */ +@Category({ MasterTests.class, MediumTests.class }) +public class TestReopenTableRegionsProcedureBatching { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReopenTableRegionsProcedureBatching.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + private static final int BACKOFF_MILLIS_PER_RS = 0; + private static final int REOPEN_BATCH_SIZE_MAX = 1; + + private static TableName TABLE_NAME = TableName.valueOf("Batching"); + + private static byte[] CF = Bytes.toBytes("cf"); + + @BeforeClass + public static void setUp() throws Exception { + Configuration conf = UTIL.getConfiguration(); + conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1); + UTIL.startMiniCluster(1); + UTIL.createMultiRegionTable(TABLE_NAME, CF); + } + + @AfterClass + public static void tearDown() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @Test + public void testSmallMaxBatchSize() throws IOException { + AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); + ProcedureExecutor procExec = + UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + List regions = UTIL.getAdmin().getRegions(TABLE_NAME); + assertTrue(2 <= regions.size()); + Set stuckRegions = + regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); + ReopenTableRegionsProcedure proc = + new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, REOPEN_BATCH_SIZE_MAX); + procExec.submitProcedure(proc); + UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); + + // the first batch should be small + confirmBatchSize(1, stuckRegions, proc); + ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); + + // other batches should also be small + assertTrue(proc.getBatchesProcessed() >= regions.size()); + + // all regions should only be opened once + assertEquals(proc.getRegionsReopened(), regions.size()); + } + + @Test + public void testDefaultMaxBatchSize() throws IOException { + AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); + ProcedureExecutor procExec = + UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + List regions = UTIL.getAdmin().getRegions(TABLE_NAME); + assertTrue(2 <= regions.size()); + Set stuckRegions = + regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); + ReopenTableRegionsProcedure proc = new ReopenTableRegionsProcedure(TABLE_NAME); + procExec.submitProcedure(proc); + UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); + + // the first batch should be large + confirmBatchSize(regions.size(), stuckRegions, proc); + ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); + + // all regions should only be opened once + assertEquals(proc.getRegionsReopened(), regions.size()); + } + + @Test + public void testNegativeBatchSizeDoesNotBreak() throws IOException { + AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); + ProcedureExecutor procExec = + UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); + List regions = UTIL.getAdmin().getRegions(TABLE_NAME); + assertTrue(2 <= regions.size()); + Set stuckRegions = + regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); + ReopenTableRegionsProcedure proc = + new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, -100); + procExec.submitProcedure(proc); + UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); + + // the first batch should be small + confirmBatchSize(1, stuckRegions, proc); + ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); + + // other batches should also be small + assertTrue(proc.getBatchesProcessed() >= regions.size()); + + // all regions should only be opened once + assertEquals(proc.getRegionsReopened(), regions.size()); + } + + @Test + public void testBatchSizeDoesNotOverflow() { + ReopenTableRegionsProcedure proc = + new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, Integer.MAX_VALUE); + int currentBatchSize = 1; + while (currentBatchSize < Integer.MAX_VALUE) { + currentBatchSize = proc.progressBatchSize(); + assertTrue(currentBatchSize > 0); + } + } + + private void confirmBatchSize(int expectedBatchSize, Set stuckRegions, + ReopenTableRegionsProcedure proc) { + while (true) { + if (proc.getBatchesProcessed() == 0) { + continue; + } + stuckRegions.forEach(this::unstickRegion); + UTIL.waitFor(5000, () -> expectedBatchSize == proc.getRegionsReopened()); + break; + } + } + + static class StuckRegion { + final TransitRegionStateProcedure trsp; + final RegionStateNode regionNode; + final long openSeqNum; + + public StuckRegion(TransitRegionStateProcedure trsp, RegionStateNode regionNode, + long openSeqNum) { + this.trsp = trsp; + this.regionNode = regionNode; + this.openSeqNum = openSeqNum; + } + } + + private StuckRegion stickRegion(AssignmentManager am, + ProcedureExecutor procExec, RegionInfo regionInfo) { + RegionStateNode regionNode = am.getRegionStates().getRegionStateNode(regionInfo); + TransitRegionStateProcedure trsp = + TransitRegionStateProcedure.unassign(procExec.getEnvironment(), regionInfo); + regionNode.lock(); + long openSeqNum; + try { + openSeqNum = regionNode.getOpenSeqNum(); + regionNode.setState(State.OPENING); + regionNode.setOpenSeqNum(-1L); + regionNode.setProcedure(trsp); + } finally { + regionNode.unlock(); + } + return new StuckRegion(trsp, regionNode, openSeqNum); + } + + private void unstickRegion(StuckRegion stuckRegion) { + stuckRegion.regionNode.lock(); + try { + stuckRegion.regionNode.setState(State.OPEN); + stuckRegion.regionNode.setOpenSeqNum(stuckRegion.openSeqNum); + stuckRegion.regionNode.unsetProcedure(stuckRegion.trsp); + } finally { + stuckRegion.regionNode.unlock(); + } + } +} From 6e421e9d94ac092e5381021926f203fc57b76f0a Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Wed, 6 Dec 2023 12:53:32 -0500 Subject: [PATCH 011/109] HBASE-28206 [JDK17] JVM crashes intermittently on aarch64 (#5561) Signed-off-by: Duo Zhang --- .../MetricsRegionServerWrapperImpl.java | 639 +++++++++--------- .../TestMetricsRegionServerAggregate.java | 227 +++++++ 2 files changed, 535 insertions(+), 331 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerAggregate.java 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 6c7fc504b5fd..ef0ee71f7fdc 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 @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.OptionalDouble; -import java.util.OptionalLong; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -72,62 +71,16 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { private CacheStats cacheStats; private CacheStats l1Stats = null; private CacheStats l2Stats = null; - - private volatile long numStores = 0; private volatile long numWALFiles = 0; private volatile long walFileSize = 0; - private volatile long numStoreFiles = 0; - private volatile long memstoreSize = 0; - private volatile long onHeapMemstoreSize = 0; - private volatile long offHeapMemstoreSize = 0; - private volatile long storeFileSize = 0; - private volatile double storeFileSizeGrowthRate = 0; - private volatile long maxStoreFileCount = 0; - private volatile long maxStoreFileAge = 0; - private volatile long minStoreFileAge = 0; - private volatile long avgStoreFileAge = 0; - private volatile long numReferenceFiles = 0; - private volatile double requestsPerSecond = 0.0; - private volatile long readRequestsCount = 0; - private volatile double readRequestsRatePerSecond = 0; - private volatile long cpRequestsCount = 0; - private volatile long filteredReadRequestsCount = 0; - private volatile long writeRequestsCount = 0; - private volatile double writeRequestsRatePerSecond = 0; - private volatile long checkAndMutateChecksFailed = 0; - private volatile long checkAndMutateChecksPassed = 0; - private volatile long storefileIndexSize = 0; - private volatile long totalStaticIndexSize = 0; - private volatile long totalStaticBloomSize = 0; - private volatile long bloomFilterRequestsCount = 0; - private volatile long bloomFilterNegativeResultsCount = 0; - private volatile long bloomFilterEligibleRequestsCount = 0; - private volatile long numMutationsWithoutWAL = 0; - private volatile long dataInMemoryWithoutWAL = 0; - private volatile double percentFileLocal = 0; - private volatile double percentFileLocalSecondaryRegions = 0; - private volatile long flushedCellsCount = 0; - private volatile long compactedCellsCount = 0; - private volatile long majorCompactedCellsCount = 0; - private volatile long flushedCellsSize = 0; - private volatile long compactedCellsSize = 0; - private volatile long majorCompactedCellsSize = 0; - private volatile long cellsCountCompactedToMob = 0; - private volatile long cellsCountCompactedFromMob = 0; - private volatile long cellsSizeCompactedToMob = 0; - private volatile long cellsSizeCompactedFromMob = 0; - private volatile long mobFlushCount = 0; - private volatile long mobFlushedCellsCount = 0; - private volatile long mobFlushedCellsSize = 0; - private volatile long mobScanCellsCount = 0; - private volatile long mobScanCellsSize = 0; private volatile long mobFileCacheAccessCount = 0; private volatile long mobFileCacheMissCount = 0; private volatile double mobFileCacheHitRatio = 0; private volatile long mobFileCacheEvictedCount = 0; private volatile long mobFileCacheCount = 0; - private volatile long blockedRequestsCount = 0L; - private volatile long averageRegionSize = 0L; + + private volatile RegionMetricAggregate aggregate = new RegionMetricAggregate(null); + protected final Map> requestsCountCache = new ConcurrentHashMap>(); @@ -249,7 +202,7 @@ public long getTotalRequestCount() { @Override public long getTotalRowActionRequestCount() { - return readRequestsCount + writeRequestsCount; + return aggregate.readRequestsCount + aggregate.writeRequestsCount; } @Override @@ -462,7 +415,7 @@ public void forceRecompute() { @Override public long getNumStores() { - return numStores; + return aggregate.numStores; } @Override @@ -491,92 +444,92 @@ public long getNumWALSlowAppend() { @Override public long getNumStoreFiles() { - return numStoreFiles; + return aggregate.numStoreFiles; } @Override public long getMaxStoreFiles() { - return maxStoreFileCount; + return aggregate.maxStoreFileCount; } @Override public long getMaxStoreFileAge() { - return maxStoreFileAge; + return aggregate.maxStoreFileAge; } @Override public long getMinStoreFileAge() { - return minStoreFileAge; + return aggregate.minStoreFileAge; } @Override public long getAvgStoreFileAge() { - return avgStoreFileAge; + return aggregate.avgStoreFileAge; } @Override public long getNumReferenceFiles() { - return numReferenceFiles; + return aggregate.numReferenceFiles; } @Override public long getMemStoreSize() { - return memstoreSize; + return aggregate.memstoreSize; } @Override public long getOnHeapMemStoreSize() { - return onHeapMemstoreSize; + return aggregate.onHeapMemstoreSize; } @Override public long getOffHeapMemStoreSize() { - return offHeapMemstoreSize; + return aggregate.offHeapMemstoreSize; } @Override public long getStoreFileSize() { - return storeFileSize; + return aggregate.storeFileSize; } @Override public double getStoreFileSizeGrowthRate() { - return storeFileSizeGrowthRate; + return aggregate.storeFileSizeGrowthRate; } @Override public double getRequestsPerSecond() { - return requestsPerSecond; + return aggregate.requestsPerSecond; } @Override public long getReadRequestsCount() { - return readRequestsCount; + return aggregate.readRequestsCount; } @Override public long getCpRequestsCount() { - return cpRequestsCount; + return aggregate.cpRequestsCount; } @Override public double getReadRequestsRatePerSecond() { - return readRequestsRatePerSecond; + return aggregate.readRequestsRatePerSecond; } @Override public long getFilteredReadRequestsCount() { - return filteredReadRequestsCount; + return aggregate.filteredReadRequestsCount; } @Override public long getWriteRequestsCount() { - return writeRequestsCount; + return aggregate.writeRequestsCount; } @Override public double getWriteRequestsRatePerSecond() { - return writeRequestsRatePerSecond; + return aggregate.writeRequestsRatePerSecond; } @Override @@ -606,62 +559,62 @@ public long getRpcMutateRequestsCount() { @Override public long getCheckAndMutateChecksFailed() { - return checkAndMutateChecksFailed; + return aggregate.checkAndMutateChecksFailed; } @Override public long getCheckAndMutateChecksPassed() { - return checkAndMutateChecksPassed; + return aggregate.checkAndMutateChecksPassed; } @Override public long getStoreFileIndexSize() { - return storefileIndexSize; + return aggregate.storefileIndexSize; } @Override public long getTotalStaticIndexSize() { - return totalStaticIndexSize; + return aggregate.totalStaticIndexSize; } @Override public long getTotalStaticBloomSize() { - return totalStaticBloomSize; + return aggregate.totalStaticBloomSize; } @Override public long getBloomFilterRequestsCount() { - return bloomFilterRequestsCount; + return aggregate.bloomFilterRequestsCount; } @Override public long getBloomFilterNegativeResultsCount() { - return bloomFilterNegativeResultsCount; + return aggregate.bloomFilterNegativeResultsCount; } @Override public long getBloomFilterEligibleRequestsCount() { - return bloomFilterEligibleRequestsCount; + return aggregate.bloomFilterEligibleRequestsCount; } @Override public long getNumMutationsWithoutWAL() { - return numMutationsWithoutWAL; + return aggregate.numMutationsWithoutWAL; } @Override public long getDataInMemoryWithoutWAL() { - return dataInMemoryWithoutWAL; + return aggregate.dataInMemoryWithoutWAL; } @Override public double getPercentFileLocal() { - return percentFileLocal; + return aggregate.percentFileLocal; } @Override public double getPercentFileLocalSecondaryRegions() { - return percentFileLocalSecondaryRegions; + return aggregate.percentFileLocalSecondaryRegions; } @Override @@ -674,77 +627,77 @@ public long getUpdatesBlockedTime() { @Override public long getFlushedCellsCount() { - return flushedCellsCount; + return aggregate.flushedCellsCount; } @Override public long getCompactedCellsCount() { - return compactedCellsCount; + return aggregate.compactedCellsCount; } @Override public long getMajorCompactedCellsCount() { - return majorCompactedCellsCount; + return aggregate.majorCompactedCellsCount; } @Override public long getFlushedCellsSize() { - return flushedCellsSize; + return aggregate.flushedCellsSize; } @Override public long getCompactedCellsSize() { - return compactedCellsSize; + return aggregate.compactedCellsSize; } @Override public long getMajorCompactedCellsSize() { - return majorCompactedCellsSize; + return aggregate.majorCompactedCellsSize; } @Override public long getCellsCountCompactedFromMob() { - return cellsCountCompactedFromMob; + return aggregate.cellsCountCompactedFromMob; } @Override public long getCellsCountCompactedToMob() { - return cellsCountCompactedToMob; + return aggregate.cellsCountCompactedToMob; } @Override public long getCellsSizeCompactedFromMob() { - return cellsSizeCompactedFromMob; + return aggregate.cellsSizeCompactedFromMob; } @Override public long getCellsSizeCompactedToMob() { - return cellsSizeCompactedToMob; + return aggregate.cellsSizeCompactedToMob; } @Override public long getMobFlushCount() { - return mobFlushCount; + return aggregate.mobFlushCount; } @Override public long getMobFlushedCellsCount() { - return mobFlushedCellsCount; + return aggregate.mobFlushedCellsCount; } @Override public long getMobFlushedCellsSize() { - return mobFlushedCellsSize; + return aggregate.mobFlushedCellsSize; } @Override public long getMobScanCellsCount() { - return mobScanCellsCount; + return aggregate.mobScanCellsCount; } @Override public long getMobScanCellsSize() { - return mobScanCellsSize; + return aggregate.mobScanCellsSize; } @Override @@ -777,6 +730,247 @@ public int getActiveScanners() { return regionServer.getRpcServices().getScannersCount(); } + private static final class RegionMetricAggregate { + private long numStores = 0; + private long numStoreFiles = 0; + private long memstoreSize = 0; + private long onHeapMemstoreSize = 0; + private long offHeapMemstoreSize = 0; + private long storeFileSize = 0; + private double storeFileSizeGrowthRate = 0; + private long maxStoreFileCount = 0; + private long maxStoreFileAge = 0; + private long minStoreFileAge = Long.MAX_VALUE; + private long avgStoreFileAge = 0; + private long numReferenceFiles = 0; + + private long cpRequestsCount = 0; + private double requestsPerSecond = 0.0; + private long readRequestsCount = 0; + private double readRequestsRatePerSecond = 0; + private long filteredReadRequestsCount = 0; + private long writeRequestsCount = 0; + private double writeRequestsRatePerSecond = 0; + private long checkAndMutateChecksFailed = 0; + private long checkAndMutateChecksPassed = 0; + private long storefileIndexSize = 0; + private long totalStaticIndexSize = 0; + private long totalStaticBloomSize = 0; + private long bloomFilterRequestsCount = 0; + private long bloomFilterNegativeResultsCount = 0; + private long bloomFilterEligibleRequestsCount = 0; + private long numMutationsWithoutWAL = 0; + private long dataInMemoryWithoutWAL = 0; + private double percentFileLocal = 0; + private double percentFileLocalSecondaryRegions = 0; + private long flushedCellsCount = 0; + private long compactedCellsCount = 0; + private long majorCompactedCellsCount = 0; + private long flushedCellsSize = 0; + private long compactedCellsSize = 0; + private long majorCompactedCellsSize = 0; + private long cellsCountCompactedToMob = 0; + private long cellsCountCompactedFromMob = 0; + private long cellsSizeCompactedToMob = 0; + private long cellsSizeCompactedFromMob = 0; + private long mobFlushCount = 0; + private long mobFlushedCellsCount = 0; + private long mobFlushedCellsSize = 0; + private long mobScanCellsCount = 0; + private long mobScanCellsSize = 0; + private long blockedRequestsCount = 0L; + private long averageRegionSize = 0L; + private long totalReadRequestsDelta = 0; + private long totalWriteRequestsDelta = 0; + + private RegionMetricAggregate(RegionMetricAggregate other) { + if (other != null) { + requestsPerSecond = other.requestsPerSecond; + readRequestsRatePerSecond = other.readRequestsRatePerSecond; + writeRequestsRatePerSecond = other.writeRequestsRatePerSecond; + } + } + + private void aggregate(HRegionServer regionServer, + Map> requestsCountCache) { + HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); + HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions = new HDFSBlocksDistribution(); + + long avgAgeNumerator = 0; + long numHFiles = 0; + int regionCount = 0; + + for (HRegion r : regionServer.getOnlineRegionsLocalContext()) { + Deltas deltas = calculateReadWriteDeltas(r, requestsCountCache); + totalReadRequestsDelta += deltas.readRequestsCountDelta; + totalWriteRequestsDelta += deltas.writeRequestsCountDelta; + + numMutationsWithoutWAL += r.getNumMutationsWithoutWAL(); + dataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL(); + cpRequestsCount += r.getCpRequestsCount(); + readRequestsCount += r.getReadRequestsCount(); + filteredReadRequestsCount += r.getFilteredReadRequestsCount(); + writeRequestsCount += r.getWriteRequestsCount(); + checkAndMutateChecksFailed += r.getCheckAndMutateChecksFailed(); + checkAndMutateChecksPassed += r.getCheckAndMutateChecksPassed(); + blockedRequestsCount += r.getBlockedRequestsCount(); + + StoreFileStats storeFileStats = aggregateStores(r.getStores()); + numHFiles += storeFileStats.numHFiles; + avgAgeNumerator += storeFileStats.avgAgeNumerator; + + HDFSBlocksDistribution distro = r.getHDFSBlocksDistribution(); + hdfsBlocksDistribution.add(distro); + if (r.getRegionInfo().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { + hdfsBlocksDistributionSecondaryRegions.add(distro); + } + + regionCount++; + } + + float localityIndex = + hdfsBlocksDistribution.getBlockLocalityIndex(regionServer.getServerName().getHostname()); + percentFileLocal = Double.isNaN(localityIndex) ? 0 : (localityIndex * 100); + + float localityIndexSecondaryRegions = hdfsBlocksDistributionSecondaryRegions + .getBlockLocalityIndex(regionServer.getServerName().getHostname()); + percentFileLocalSecondaryRegions = + Double.isNaN(localityIndexSecondaryRegions) ? 0 : (localityIndexSecondaryRegions * 100); + + if (regionCount > 0) { + averageRegionSize = (memstoreSize + storeFileSize) / regionCount; + } + + // if there were no store files, we'll never have updated this with Math.min + // so set it to 0, which is a better value to display in case of no storefiles + if (minStoreFileAge == Long.MAX_VALUE) { + this.minStoreFileAge = 0; + } + + if (numHFiles != 0) { + avgStoreFileAge = avgAgeNumerator / numHFiles; + } + } + + private static final class Deltas { + private final long readRequestsCountDelta; + private final long writeRequestsCountDelta; + + private Deltas(long readRequestsCountDelta, long writeRequestsCountDelta) { + this.readRequestsCountDelta = readRequestsCountDelta; + this.writeRequestsCountDelta = writeRequestsCountDelta; + } + } + + private Deltas calculateReadWriteDeltas(HRegion r, + Map> requestsCountCache) { + String encodedRegionName = r.getRegionInfo().getEncodedName(); + long currentReadRequestsCount = r.getReadRequestsCount(); + long currentWriteRequestsCount = r.getWriteRequestsCount(); + if (requestsCountCache.containsKey(encodedRegionName)) { + long lastReadRequestsCount = requestsCountCache.get(encodedRegionName).get(0); + long lastWriteRequestsCount = requestsCountCache.get(encodedRegionName).get(1); + + // Update cache for our next comparison + requestsCountCache.get(encodedRegionName).set(0, currentReadRequestsCount); + requestsCountCache.get(encodedRegionName).set(1, currentWriteRequestsCount); + + long readRequestsDelta = currentReadRequestsCount - lastReadRequestsCount; + long writeRequestsDelta = currentWriteRequestsCount - lastWriteRequestsCount; + return new Deltas(readRequestsDelta, writeRequestsDelta); + } else { + // List[0] -> readRequestCount + // List[1] -> writeRequestCount + ArrayList requests = new ArrayList(2); + requests.add(currentReadRequestsCount); + requests.add(currentWriteRequestsCount); + requestsCountCache.put(encodedRegionName, requests); + return new Deltas(currentReadRequestsCount, currentWriteRequestsCount); + } + } + + public void updateRates(long timeSinceLastRun, long expectedPeriod, long lastStoreFileSize) { + requestsPerSecond = + (totalReadRequestsDelta + totalWriteRequestsDelta) / (timeSinceLastRun / 1000.0); + + double readRequestsRatePerMilliSecond = (double) totalReadRequestsDelta / expectedPeriod; + double writeRequestsRatePerMilliSecond = (double) totalWriteRequestsDelta / expectedPeriod; + + readRequestsRatePerSecond = readRequestsRatePerMilliSecond * 1000.0; + writeRequestsRatePerSecond = writeRequestsRatePerMilliSecond * 1000.0; + + long intervalStoreFileSize = storeFileSize - lastStoreFileSize; + storeFileSizeGrowthRate = (double) intervalStoreFileSize * 1000.0 / expectedPeriod; + } + + private static final class StoreFileStats { + private final long numHFiles; + private final long avgAgeNumerator; + + private StoreFileStats(long numHFiles, long avgAgeNumerator) { + this.numHFiles = numHFiles; + this.avgAgeNumerator = avgAgeNumerator; + } + } + + private StoreFileStats aggregateStores(List stores) { + numStores += stores.size(); + long numHFiles = 0; + long avgAgeNumerator = 0; + for (Store store : stores) { + numStoreFiles += store.getStorefilesCount(); + memstoreSize += store.getMemStoreSize().getDataSize(); + onHeapMemstoreSize += store.getMemStoreSize().getHeapSize(); + offHeapMemstoreSize += store.getMemStoreSize().getOffHeapSize(); + storeFileSize += store.getStorefilesSize(); + maxStoreFileCount = Math.max(maxStoreFileCount, store.getStorefilesCount()); + + maxStoreFileAge = + Math.max(store.getMaxStoreFileAge().orElse(maxStoreFileAge), maxStoreFileAge); + minStoreFileAge = + Math.min(store.getMinStoreFileAge().orElse(minStoreFileAge), minStoreFileAge); + + long storeHFiles = store.getNumHFiles(); + numHFiles += storeHFiles; + numReferenceFiles += store.getNumReferenceFiles(); + + OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge(); + if (storeAvgStoreFileAge.isPresent()) { + avgAgeNumerator = + (long) (avgAgeNumerator + storeAvgStoreFileAge.getAsDouble() * storeHFiles); + } + + storefileIndexSize += store.getStorefilesRootLevelIndexSize(); + totalStaticBloomSize += store.getTotalStaticBloomSize(); + totalStaticIndexSize += store.getTotalStaticIndexSize(); + bloomFilterRequestsCount += store.getBloomFilterRequestsCount(); + bloomFilterNegativeResultsCount += store.getBloomFilterNegativeResultsCount(); + bloomFilterEligibleRequestsCount += store.getBloomFilterEligibleRequestsCount(); + flushedCellsCount += store.getFlushedCellsCount(); + compactedCellsCount += store.getCompactedCellsCount(); + majorCompactedCellsCount += store.getMajorCompactedCellsCount(); + flushedCellsSize += store.getFlushedCellsSize(); + compactedCellsSize += store.getCompactedCellsSize(); + majorCompactedCellsSize += store.getMajorCompactedCellsSize(); + if (store instanceof HMobStore) { + HMobStore mobStore = (HMobStore) store; + cellsCountCompactedToMob += mobStore.getCellsCountCompactedToMob(); + cellsCountCompactedFromMob += mobStore.getCellsCountCompactedFromMob(); + cellsSizeCompactedToMob += mobStore.getCellsSizeCompactedToMob(); + cellsSizeCompactedFromMob += mobStore.getCellsSizeCompactedFromMob(); + mobFlushCount += mobStore.getMobFlushCount(); + mobFlushedCellsCount += mobStore.getMobFlushedCellsCount(); + mobFlushedCellsSize += mobStore.getMobFlushedCellsSize(); + mobScanCellsCount += mobStore.getMobScanCellsCount(); + mobScanCellsSize += mobStore.getMobScanCellsSize(); + } + } + + return new StoreFileStats(numHFiles, avgAgeNumerator); + } + + } + /** * This is the runnable that will be executed on the executor every PERIOD number of seconds It * will take metrics/numbers from all of the regions and use them to compute point in time @@ -790,170 +984,8 @@ public class RegionServerMetricsWrapperRunnable implements Runnable { @Override synchronized public void run() { try { - HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); - HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions = - new HDFSBlocksDistribution(); - - long tempNumStores = 0, tempNumStoreFiles = 0, tempStoreFileSize = 0; - long tempMemstoreSize = 0, tempOnHeapMemstoreSize = 0, tempOffHeapMemstoreSize = 0; - long tempMaxStoreFileAge = 0, tempNumReferenceFiles = 0; - long tempMaxStoreFileCount = 0; - long avgAgeNumerator = 0, numHFiles = 0; - long tempMinStoreFileAge = Long.MAX_VALUE; - long tempFilteredReadRequestsCount = 0, tempCpRequestsCount = 0; - long tempCheckAndMutateChecksFailed = 0; - long tempCheckAndMutateChecksPassed = 0; - long tempStorefileIndexSize = 0; - long tempTotalStaticIndexSize = 0; - long tempTotalStaticBloomSize = 0; - long tempBloomFilterRequestsCount = 0; - long tempBloomFilterNegativeResultsCount = 0; - long tempBloomFilterEligibleRequestsCount = 0; - long tempNumMutationsWithoutWAL = 0; - long tempDataInMemoryWithoutWAL = 0; - double tempPercentFileLocal = 0; - double tempPercentFileLocalSecondaryRegions = 0; - long tempFlushedCellsCount = 0; - long tempCompactedCellsCount = 0; - long tempMajorCompactedCellsCount = 0; - long tempFlushedCellsSize = 0; - long tempCompactedCellsSize = 0; - long tempMajorCompactedCellsSize = 0; - long tempCellsCountCompactedToMob = 0; - long tempCellsCountCompactedFromMob = 0; - long tempCellsSizeCompactedToMob = 0; - long tempCellsSizeCompactedFromMob = 0; - long tempMobFlushCount = 0; - long tempMobFlushedCellsCount = 0; - long tempMobFlushedCellsSize = 0; - long tempMobScanCellsCount = 0; - long tempMobScanCellsSize = 0; - long tempBlockedRequestsCount = 0; - int regionCount = 0; - - long tempReadRequestsCount = 0; - long tempWriteRequestsCount = 0; - 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 (HRegion 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; - } - tempReadRequestsCount += r.getReadRequestsCount(); - tempWriteRequestsCount += r.getWriteRequestsCount(); - tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL(); - tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL(); - tempCpRequestsCount += r.getCpRequestsCount(); - tempFilteredReadRequestsCount += r.getFilteredReadRequestsCount(); - tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed(); - tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed(); - tempBlockedRequestsCount += r.getBlockedRequestsCount(); - List storeList = r.getStores(); - tempNumStores += storeList.size(); - for (Store store : storeList) { - tempNumStoreFiles += store.getStorefilesCount(); - tempMemstoreSize += store.getMemStoreSize().getDataSize(); - tempOnHeapMemstoreSize += store.getMemStoreSize().getHeapSize(); - tempOffHeapMemstoreSize += store.getMemStoreSize().getOffHeapSize(); - tempStoreFileSize += store.getStorefilesSize(); - - tempMaxStoreFileCount = Math.max(tempMaxStoreFileCount, store.getStorefilesCount()); - - OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge(); - if ( - storeMaxStoreFileAge.isPresent() - && storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge - ) { - tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong(); - } - - OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge(); - if ( - storeMinStoreFileAge.isPresent() - && storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge - ) { - tempMinStoreFileAge = storeMinStoreFileAge.getAsLong(); - } - - long storeHFiles = store.getNumHFiles(); - numHFiles += storeHFiles; - tempNumReferenceFiles += store.getNumReferenceFiles(); - - OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge(); - if (storeAvgStoreFileAge.isPresent()) { - avgAgeNumerator = - (long) (avgAgeNumerator + storeAvgStoreFileAge.getAsDouble() * storeHFiles); - } - - tempStorefileIndexSize += store.getStorefilesRootLevelIndexSize(); - tempTotalStaticBloomSize += store.getTotalStaticBloomSize(); - tempTotalStaticIndexSize += store.getTotalStaticIndexSize(); - tempBloomFilterRequestsCount += store.getBloomFilterRequestsCount(); - tempBloomFilterNegativeResultsCount += store.getBloomFilterNegativeResultsCount(); - tempBloomFilterEligibleRequestsCount += store.getBloomFilterEligibleRequestsCount(); - tempFlushedCellsCount += store.getFlushedCellsCount(); - tempCompactedCellsCount += store.getCompactedCellsCount(); - tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount(); - tempFlushedCellsSize += store.getFlushedCellsSize(); - tempCompactedCellsSize += store.getCompactedCellsSize(); - tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize(); - if (store instanceof HMobStore) { - HMobStore mobStore = (HMobStore) store; - tempCellsCountCompactedToMob += mobStore.getCellsCountCompactedToMob(); - tempCellsCountCompactedFromMob += mobStore.getCellsCountCompactedFromMob(); - tempCellsSizeCompactedToMob += mobStore.getCellsSizeCompactedToMob(); - tempCellsSizeCompactedFromMob += mobStore.getCellsSizeCompactedFromMob(); - tempMobFlushCount += mobStore.getMobFlushCount(); - tempMobFlushedCellsCount += mobStore.getMobFlushedCellsCount(); - tempMobFlushedCellsSize += mobStore.getMobFlushedCellsSize(); - tempMobScanCellsCount += mobStore.getMobScanCellsCount(); - tempMobScanCellsSize += mobStore.getMobScanCellsSize(); - } - } - - HDFSBlocksDistribution distro = r.getHDFSBlocksDistribution(); - hdfsBlocksDistribution.add(distro); - if (r.getRegionInfo().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { - hdfsBlocksDistributionSecondaryRegions.add(distro); - } - regionCount++; - } - - float localityIndex = - hdfsBlocksDistribution.getBlockLocalityIndex(regionServer.getServerName().getHostname()); - tempPercentFileLocal = Double.isNaN(tempBlockedRequestsCount) ? 0 : (localityIndex * 100); - - float localityIndexSecondaryRegions = hdfsBlocksDistributionSecondaryRegions - .getBlockLocalityIndex(regionServer.getServerName().getHostname()); - tempPercentFileLocalSecondaryRegions = - Double.isNaN(localityIndexSecondaryRegions) ? 0 : (localityIndexSecondaryRegions * 100); + RegionMetricAggregate newVal = new RegionMetricAggregate(aggregate); + newVal.aggregate(regionServer, requestsCountCache); // Compute the number of requests per second long currentTime = EnvironmentEdgeManager.currentTime(); @@ -963,24 +995,14 @@ synchronized public void run() { if (lastRan == 0) { lastRan = currentTime - period; } - // If we've time traveled keep the last requests per second. - if ((currentTime - lastRan) > 0) { - requestsPerSecond = - (totalReadRequestsDelta + totalWriteRequestsDelta) / ((currentTime - lastRan) / 1000.0); - - double readRequestsRatePerMilliSecond = (double) totalReadRequestsDelta / period; - double writeRequestsRatePerMilliSecond = (double) totalWriteRequestsDelta / period; - - readRequestsRatePerSecond = readRequestsRatePerMilliSecond * 1000.0; - writeRequestsRatePerSecond = writeRequestsRatePerMilliSecond * 1000.0; - - long intervalStoreFileSize = tempStoreFileSize - lastStoreFileSize; - storeFileSizeGrowthRate = (double) intervalStoreFileSize * 1000.0 / period; - lastStoreFileSize = tempStoreFileSize; + long timeSinceLastRun = currentTime - lastRan; + // If we've time traveled keep the last requests per second. + if (timeSinceLastRun > 0) { + newVal.updateRates(timeSinceLastRun, period, lastStoreFileSize); } - lastRan = currentTime; + aggregate = newVal; List providers = regionServer.getWalFactory().getAllWALProviders(); for (WALProvider provider : providers) { @@ -988,58 +1010,6 @@ synchronized public void run() { walFileSize += provider.getLogFileSize(); } - // Copy over computed values so that no thread sees half computed values. - numStores = tempNumStores; - numStoreFiles = tempNumStoreFiles; - memstoreSize = tempMemstoreSize; - onHeapMemstoreSize = tempOnHeapMemstoreSize; - offHeapMemstoreSize = tempOffHeapMemstoreSize; - storeFileSize = tempStoreFileSize; - maxStoreFileCount = tempMaxStoreFileCount; - maxStoreFileAge = tempMaxStoreFileAge; - if (regionCount > 0) { - averageRegionSize = (memstoreSize + storeFileSize) / regionCount; - } - if (tempMinStoreFileAge != Long.MAX_VALUE) { - minStoreFileAge = tempMinStoreFileAge; - } - - if (numHFiles != 0) { - avgStoreFileAge = avgAgeNumerator / numHFiles; - } - - numReferenceFiles = tempNumReferenceFiles; - readRequestsCount = tempReadRequestsCount; - cpRequestsCount = tempCpRequestsCount; - filteredReadRequestsCount = tempFilteredReadRequestsCount; - writeRequestsCount = tempWriteRequestsCount; - checkAndMutateChecksFailed = tempCheckAndMutateChecksFailed; - checkAndMutateChecksPassed = tempCheckAndMutateChecksPassed; - storefileIndexSize = tempStorefileIndexSize; - totalStaticIndexSize = tempTotalStaticIndexSize; - totalStaticBloomSize = tempTotalStaticBloomSize; - bloomFilterRequestsCount = tempBloomFilterRequestsCount; - bloomFilterNegativeResultsCount = tempBloomFilterNegativeResultsCount; - bloomFilterEligibleRequestsCount = tempBloomFilterEligibleRequestsCount; - numMutationsWithoutWAL = tempNumMutationsWithoutWAL; - dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL; - percentFileLocal = tempPercentFileLocal; - percentFileLocalSecondaryRegions = tempPercentFileLocalSecondaryRegions; - flushedCellsCount = tempFlushedCellsCount; - compactedCellsCount = tempCompactedCellsCount; - majorCompactedCellsCount = tempMajorCompactedCellsCount; - flushedCellsSize = tempFlushedCellsSize; - compactedCellsSize = tempCompactedCellsSize; - majorCompactedCellsSize = tempMajorCompactedCellsSize; - cellsCountCompactedToMob = tempCellsCountCompactedToMob; - cellsCountCompactedFromMob = tempCellsCountCompactedFromMob; - cellsSizeCompactedToMob = tempCellsSizeCompactedToMob; - cellsSizeCompactedFromMob = tempCellsSizeCompactedFromMob; - mobFlushCount = tempMobFlushCount; - mobFlushedCellsCount = tempMobFlushedCellsCount; - mobFlushedCellsSize = tempMobFlushedCellsSize; - mobScanCellsCount = tempMobScanCellsCount; - mobScanCellsSize = tempMobScanCellsSize; mobFileCacheAccessCount = mobFileCache != null ? mobFileCache.getAccessCount() : 0L; mobFileCacheMissCount = mobFileCache != null ? mobFileCache.getMissCount() : 0L; mobFileCacheHitRatio = mobFileCache != null ? mobFileCache.getHitRatio() : 0.0; @@ -1048,7 +1018,9 @@ synchronized public void run() { } mobFileCacheEvictedCount = mobFileCache != null ? mobFileCache.getEvictedFileCount() : 0L; mobFileCacheCount = mobFileCache != null ? mobFileCache.getCacheSize() : 0; - blockedRequestsCount = tempBlockedRequestsCount; + + lastStoreFileSize = aggregate.storeFileSize; + lastRan = currentTime; } catch (Throwable e) { LOG.warn("Caught exception! Will suppress and retry.", e); } @@ -1094,12 +1066,12 @@ public long getZeroCopyBytesRead() { @Override public long getBlockedRequestsCount() { - return blockedRequestsCount; + return aggregate.blockedRequestsCount; } @Override public long getAverageRegionSize() { - return averageRegionSize; + return aggregate.averageRegionSize; } @Override @@ -1226,4 +1198,9 @@ public long getByteBuffAllocatorTotalBufferCount() { public long getByteBuffAllocatorUsedBufferCount() { return this.allocator.getUsedBufferCount(); } + + // Visible for testing + long getPeriod() { + return period; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerAggregate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerAggregate.java new file mode 100644 index 000000000000..428416833875 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerAggregate.java @@ -0,0 +1,227 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.List; +import java.util.OptionalDouble; +import java.util.OptionalLong; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HDFSBlocksDistribution; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.ipc.RpcServerInterface; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; +import org.mockito.stubbing.Answer; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category({ SmallTests.class, RegionServerTests.class }) +public class TestMetricsRegionServerAggregate { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMetricsRegionServerAggregate.class); + + @Test + public void test() { + AtomicInteger retVal = new AtomicInteger(0); + Answer defaultAnswer = invocation -> { + Class returnType = invocation.getMethod().getReturnType(); + + if (returnType.equals(Integer.TYPE) || returnType.equals(Integer.class)) { + return retVal.get(); + } else if (returnType.equals(Long.TYPE) || returnType.equals(Long.class)) { + return (long) retVal.get(); + } + return Mockito.RETURNS_DEFAULTS.answer(invocation); + }; + + ServerName serverName = mock(ServerName.class); + when(serverName.getHostname()).thenReturn("foo"); + WALFactory walFactory = mock(WALFactory.class); + RpcServerInterface rpcServer = mock(RpcServerInterface.class); + AtomicInteger storeFileCount = new AtomicInteger(1); + HRegion regionOne = getMockedRegion(defaultAnswer, "a", "foo", true, storeFileCount); + HRegion regionTwo = getMockedRegion(defaultAnswer, "b", "bar", true, storeFileCount); + HRegion regionThree = getMockedRegion(defaultAnswer, "c", "foo", false, storeFileCount); + HRegion regionFour = getMockedRegion(defaultAnswer, "d", "bar", false, storeFileCount); + List regions = Lists.newArrayList(regionOne, regionTwo, regionThree, regionFour); + + int numStoresPerRegion = 2; + for (HRegion region : regions) { + // if adding more stores, update numStoresPerRegion so that tests below continue working + assertEquals(numStoresPerRegion, region.getStores().size()); + } + + HRegionServer regionServer = mock(HRegionServer.class, defaultAnswer); + when(regionServer.getWalFactory()).thenReturn(walFactory); + when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions); + when(regionServer.getServerName()).thenReturn(serverName); + Configuration conf = HBaseConfiguration.create(); + int metricsPeriodSec = 600; + // set a very long period so that it doesn't actually run during our very quick test + conf.setLong(HConstants.REGIONSERVER_METRICS_PERIOD, metricsPeriodSec * 1000); + when(regionServer.getConfiguration()).thenReturn(conf); + when(regionServer.getRpcServer()).thenReturn(rpcServer); + + MetricsRegionServerWrapperImpl wrapper = new MetricsRegionServerWrapperImpl(regionServer); + + // we need to control the edge because rate calculations expect a + // stable interval relative to the configured period + ManualEnvironmentEdge edge = new ManualEnvironmentEdge(); + EnvironmentEdgeManager.injectEdge(edge); + + try { + for (int i = 1; i <= 10; i++) { + edge.incValue(wrapper.getPeriod()); + retVal.incrementAndGet(); + wrapper.forceRecompute(); + + int numRegions = regions.size(); + int totalStores = numRegions * numStoresPerRegion; + + // there are N regions, and each has M stores. everything gets aggregated, so + // multiply expected values accordingly + int expectedForRegions = retVal.get() * numRegions; + int expectedForStores = retVal.get() * totalStores; + + assertEquals(totalStores, wrapper.getNumStores()); + assertEquals(expectedForStores, wrapper.getFlushedCellsCount()); + assertEquals(expectedForStores, wrapper.getCompactedCellsCount()); + assertEquals(expectedForStores, wrapper.getMajorCompactedCellsCount()); + assertEquals(expectedForStores, wrapper.getFlushedCellsSize()); + assertEquals(expectedForStores, wrapper.getCompactedCellsSize()); + assertEquals(expectedForStores, wrapper.getMajorCompactedCellsSize()); + assertEquals(expectedForRegions, wrapper.getCellsCountCompactedFromMob()); + assertEquals(expectedForRegions, wrapper.getCellsCountCompactedToMob()); + assertEquals(expectedForRegions, wrapper.getCellsSizeCompactedFromMob()); + assertEquals(expectedForRegions, wrapper.getCellsSizeCompactedToMob()); + assertEquals(expectedForRegions, wrapper.getMobFlushCount()); + assertEquals(expectedForRegions, wrapper.getMobFlushedCellsCount()); + assertEquals(expectedForRegions, wrapper.getMobFlushedCellsSize()); + assertEquals(expectedForRegions, wrapper.getMobScanCellsCount()); + assertEquals(expectedForRegions, wrapper.getMobScanCellsSize()); + assertEquals(expectedForRegions, wrapper.getCheckAndMutateChecksFailed()); + assertEquals(expectedForRegions, wrapper.getCheckAndMutateChecksPassed()); + assertEquals(expectedForStores, wrapper.getStoreFileIndexSize()); + assertEquals(expectedForStores, wrapper.getTotalStaticIndexSize()); + assertEquals(expectedForStores, wrapper.getTotalStaticBloomSize()); + assertEquals(expectedForStores, wrapper.getBloomFilterRequestsCount()); + assertEquals(expectedForStores, wrapper.getBloomFilterNegativeResultsCount()); + assertEquals(expectedForStores, wrapper.getBloomFilterEligibleRequestsCount()); + assertEquals(expectedForRegions, wrapper.getNumMutationsWithoutWAL()); + assertEquals(expectedForRegions, wrapper.getDataInMemoryWithoutWAL()); + assertEquals(expectedForRegions, wrapper.getAverageRegionSize()); + assertEquals(expectedForRegions, wrapper.getBlockedRequestsCount()); + assertEquals(expectedForStores, wrapper.getNumReferenceFiles()); + assertEquals(expectedForStores, wrapper.getMemStoreSize()); + assertEquals(expectedForStores, wrapper.getOnHeapMemStoreSize()); + assertEquals(expectedForStores, wrapper.getOffHeapMemStoreSize()); + assertEquals(expectedForStores, wrapper.getStoreFileSize()); + assertEquals(expectedForRegions, wrapper.getReadRequestsCount()); + assertEquals(expectedForRegions, wrapper.getCpRequestsCount()); + assertEquals(expectedForRegions, wrapper.getFilteredReadRequestsCount()); + assertEquals(expectedForRegions, wrapper.getWriteRequestsCount()); + assertEquals(expectedForRegions * 2, wrapper.getTotalRowActionRequestCount()); + + // If we have N regions, each with M stores. That's N*M stores in total. In creating those + // stores, we increment the number and age of storefiles for each one. So the first + // store has 1 file of 1 age, then 2 files of 2 age, etc. + // formula for 1+2+3..+n + assertEquals((totalStores * (totalStores + 1)) / 2, wrapper.getNumStoreFiles()); + assertEquals(totalStores, wrapper.getMaxStoreFiles()); + assertEquals(totalStores, wrapper.getMaxStoreFileAge()); + assertEquals(1, wrapper.getMinStoreFileAge()); + assertEquals(totalStores / 2, wrapper.getAvgStoreFileAge()); + + // there are four regions, two are primary and the other two secondary + // for each type, one region has 100% locality, the other has 0%. + // this just proves we correctly aggregate for each + assertEquals(50.0, wrapper.getPercentFileLocal(), 0.0001); + assertEquals(50.0, wrapper.getPercentFileLocalSecondaryRegions(), 0.0001); + + // readRequestCount and writeRequestCount are tracking the value of i, which increases by 1 + // each interval. There are N regions, so the delta each interval is N*i=N. So the rate is + // simply N / period. + assertEquals((double) numRegions / metricsPeriodSec, wrapper.getReadRequestsRatePerSecond(), + 0.0001); + assertEquals((double) numRegions / metricsPeriodSec, + wrapper.getWriteRequestsRatePerSecond(), 0.0001); + // total of above, so multiply by 2 + assertEquals((double) numRegions / metricsPeriodSec * 2, wrapper.getRequestsPerSecond(), + 0.0001); + // Similar logic to above, except there are M totalStores and each one is of + // size tracking i. So the rate is just M / period. + assertEquals((double) totalStores / metricsPeriodSec, wrapper.getStoreFileSizeGrowthRate(), + 0.0001); + } + } finally { + EnvironmentEdgeManager.reset(); + } + } + + private HRegion getMockedRegion(Answer defaultAnswer, String name, String localOnHost, + boolean isPrimary, AtomicInteger storeFileCount) { + RegionInfo regionInfo = mock(RegionInfo.class); + when(regionInfo.getEncodedName()).thenReturn(name); + if (!isPrimary) { + when(regionInfo.getReplicaId()).thenReturn(RegionInfo.DEFAULT_REPLICA_ID + 1); + } + HDFSBlocksDistribution distribution = new HDFSBlocksDistribution(); + distribution.addHostsAndBlockWeight(new String[] { localOnHost }, 100); + + HStore store = getMockedStore(HStore.class, defaultAnswer, storeFileCount); + HMobStore mobStore = getMockedStore(HMobStore.class, defaultAnswer, storeFileCount); + + HRegion region = mock(HRegion.class, defaultAnswer); + when(region.getRegionInfo()).thenReturn(regionInfo); + when(region.getHDFSBlocksDistribution()).thenReturn(distribution); + when(region.getStores()).thenReturn(Lists.newArrayList(store, mobStore)); + return region; + } + + private T getMockedStore(Class clazz, Answer defaultAnswer, + AtomicInteger storeFileCount) { + T store = mock(clazz, defaultAnswer); + int storeFileCountVal = storeFileCount.getAndIncrement(); + when(store.getStorefilesCount()).thenReturn(storeFileCountVal); + when(store.getAvgStoreFileAge()).thenReturn(OptionalDouble.of(storeFileCountVal)); + when(store.getMaxStoreFileAge()).thenReturn(OptionalLong.of(storeFileCountVal)); + when(store.getMinStoreFileAge()).thenReturn(OptionalLong.of(storeFileCountVal)); + MemStoreSize memStore = mock(MemStoreSize.class, defaultAnswer); + when(store.getMemStoreSize()).thenReturn(memStore); + return store; + } + +} From 82a2ce10f24a828b2c4960ba85b714a0203c8441 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Sat, 9 Dec 2023 21:55:11 +0800 Subject: [PATCH 012/109] HBASE-28248 Race between RegionRemoteProcedureBase and rollback operation could lead to ROLLEDBACK state be persisent to procedure store (#5567) Signed-off-by: GeorryHuang Signed-off-by: Yi Mei --- .../assignment/RegionRemoteProcedureBase.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java index d27e0068b0ca..f6668d9c14b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseStateData; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; /** @@ -183,7 +184,20 @@ protected abstract void updateTransitionWithoutPersistingToMeta(MasterProcedureE // A bit strange but the procedure store will throw RuntimeException if we can not persist the // state, so upper layer should take care of this... private void persistAndWake(MasterProcedureEnv env, RegionStateNode regionNode) { - env.getMasterServices().getMasterProcedureExecutor().getStore().update(this); + // The synchronization here is to guard with ProcedureExecutor.executeRollback, as here we will + // not hold the procedure execution lock, but we should not persist a procedure in ROLLEDBACK + // state to the procedure store. + // The ProcedureStore.update must be inside the lock, so here the check for procedure state and + // update could be atomic. In ProcedureExecutor.cleanupAfterRollbackOneStep, we will set the + // state to ROLLEDBACK, which will hold the same lock too as the Procedure.setState method is + // synchronized. This is the key to keep us safe. + synchronized (this) { + if (getState() == ProcedureState.ROLLEDBACK) { + LOG.warn("Procedure {} has already been rolled back, skip persistent", this); + return; + } + env.getMasterServices().getMasterProcedureExecutor().getStore().update(this); + } regionNode.getProcedureEvent().wake(env.getProcedureScheduler()); } From 25c639f9d6deb52367089e2631004f4d6862daea Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Mon, 11 Dec 2023 10:17:33 +0000 Subject: [PATCH 013/109] HBASE-28251 [SFT] Add description for specifying SFT impl during snapshot recovery (#5570) Signed-off-by: Duo Zhang Signed-off-by: Nihal Jain Signed-off-by: Peter Somogyi --- .../_chapters/bulk_data_generator_tool.adoc | 8 ++++---- .../_chapters/store_file_tracking.adoc | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/src/main/asciidoc/_chapters/bulk_data_generator_tool.adoc b/src/main/asciidoc/_chapters/bulk_data_generator_tool.adoc index 3ac6ca693121..b04fcdeb7264 100644 --- a/src/main/asciidoc/_chapters/bulk_data_generator_tool.adoc +++ b/src/main/asciidoc/_chapters/bulk_data_generator_tool.adoc @@ -18,8 +18,8 @@ * limitations under the License. */ //// - -== Bulk Data Generator Tool +[[BulkDataGeneratorTool]] += Bulk Data Generator Tool :doctype: book :numbered: :toc: left @@ -29,7 +29,7 @@ This is a random data generator tool for HBase tables leveraging Hbase bulk load. It can create pre-splited HBase table and the generated data is *uniformly distributed* to all the regions of the table. -=== How to Use +== Usage [source] ---- @@ -53,7 +53,7 @@ hbase org.apache.hadoop.hbase.util.bulkdatagenerator.BulkDataGeneratorTool -t TE hbase org.apache.hadoop.hbase.util.bulkdatagenerator.BulkDataGeneratorTool -t TEST_TABLE -mc 10 -r 100 -sc 10 -Dmapreduce.map.memory.mb=8192 ---- -=== How it Works +== Overview ==== Table Schema Tool generates a HBase table with single column family, i.e. *cf* and 9 columns i.e. diff --git a/src/main/asciidoc/_chapters/store_file_tracking.adoc b/src/main/asciidoc/_chapters/store_file_tracking.adoc index 74d802f386c5..b6c1f7e73399 100644 --- a/src/main/asciidoc/_chapters/store_file_tracking.adoc +++ b/src/main/asciidoc/_chapters/store_file_tracking.adoc @@ -143,3 +143,22 @@ example, that would be as follows: ---- alter 'my-table', CONFIGURATION => {'hbase.store.file-tracker.impl' => 'FILE'} ---- + +### Specifying trackers during snapshot recovery + +It's also possible to specify a given store file tracking implementation when recovering a snapshot +using the _CLONE_SFT_ option of _clone_snasphot_ command. This is useful when recovering old +snapshots, taken prior to a change in the global configuration, or if the snapshot has been +imported from a different cluster that had a different store file tracking setting. +Because snapshots preserve table and colum family descriptors, a simple restore would reload +the original configuration, requiring the additional steps described above to convert the +table/column family to the desired tracker implementation. +An example of how to use _clone_snapshot_ to specify the *FILE* tracker implementation +is shown below: + +---- +clone_snapshot 'snapshotName', 'namespace:tableName', {CLONE_SFT=>'FILE'} +---- + +NOTE: The option to specify the tracker during snapshot recovery is only available for the +_clone_snapshot_ command. The _restore_snapshot_ command does not support this parameter. From 78c5ac372550835133935a3022a0142880476297 Mon Sep 17 00:00:00 2001 From: Fantasy-Jay <13631435453@163.com> Date: Mon, 11 Dec 2023 23:22:13 +0800 Subject: [PATCH 014/109] HBASE-28190 Add slow sync log rolling test in TestAsyncLogRolling. (#5507) Signed-off-by: Duo Zhang --- .../hbase/regionserver/wal/AbstractFSWAL.java | 4 + .../hadoop/hbase/regionserver/wal/FSHLog.java | 8 - .../wal/AbstractTestLogRolling.java | 106 +++++++- .../regionserver/wal/TestAsyncLogRolling.java | 65 +++++ .../regionserver/wal/TestLogRolling.java | 234 ++++-------------- 5 files changed, 218 insertions(+), 199 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index acf3231d4e90..1a5b5384b01f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -2245,6 +2245,10 @@ private static void split(final Configuration conf, final Path p) throws IOExcep WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf)); } + W getWriter() { + return this.writer; + } + private static void usage() { System.err.println("Usage: AbstractFSWAL "); System.err.println("Arguments:"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index d0d5ce5f2e17..131f284557af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -603,14 +603,6 @@ DatanodeInfo[] getPipeline() { return new DatanodeInfo[0]; } - Writer getWriter() { - return this.writer; - } - - void setWriter(Writer writer) { - this.writer = writer; - } - @Override protected Writer createCombinedWriter(Writer localWriter, Writer remoteWriter) { // put remote writer first as usually it will cost more time to finish, so we write to it first diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 940dbebf614b..2a5aec458828 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -20,9 +20,13 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -31,6 +35,7 @@ import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Get; @@ -48,8 +53,10 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -59,6 +66,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + /** * Test log deletion as logs are rolled. */ @@ -74,6 +83,10 @@ public abstract class AbstractTestLogRolling { protected static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); @Rule public final TestName name = new TestName(); + protected static int syncLatencyMillis; + private static int rowNum = 1; + private static final AtomicBoolean slowSyncHookCalled = new AtomicBoolean(); + protected static ScheduledExecutorService EXECUTOR; public AbstractTestLogRolling() { this.server = null; @@ -118,6 +131,17 @@ public static void setUpBeforeClass() throws Exception { // disable low replication check for log roller to get a more stable result // TestWALOpenAfterDNRollingStart will test this option. conf.setLong("hbase.regionserver.hlog.check.lowreplication.interval", 24L * 60 * 60 * 1000); + + // For slow sync threshold test: roll after 5 slow syncs in 10 seconds + conf.setInt(FSHLog.SLOW_SYNC_ROLL_THRESHOLD, 5); + conf.setInt(FSHLog.SLOW_SYNC_ROLL_INTERVAL_MS, 10 * 1000); + // For slow sync threshold test: roll once after a sync above this threshold + conf.setInt(FSHLog.ROLL_ON_SYNC_TIME_MS, 5000); + + // Slow sync executor. + EXECUTOR = Executors + .newSingleThreadScheduledExecutor(new ThreadFactoryBuilder().setNameFormat("Slow-sync-%d") + .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); } @Before @@ -139,6 +163,11 @@ public void tearDown() throws Exception { TEST_UTIL.shutdownMiniCluster(); } + @AfterClass + public static void tearDownAfterClass() { + EXECUTOR.shutdownNow(); + } + private void startAndWriteData() throws IOException, InterruptedException { this.server = cluster.getRegionServerThreads().get(0).getRegionServer(); @@ -158,6 +187,74 @@ private void startAndWriteData() throws IOException, InterruptedException { } } + private static void setSyncLatencyMillis(int latency) { + syncLatencyMillis = latency; + } + + protected final AbstractFSWAL getWALAndRegisterSlowSyncHook(RegionInfo region) + throws IOException { + // Get a reference to the wal. + final AbstractFSWAL log = (AbstractFSWAL) server.getWAL(region); + + // Register a WALActionsListener to observe if a SLOW_SYNC roll is requested + log.registerWALActionsListener(new WALActionsListener() { + @Override + public void logRollRequested(RollRequestReason reason) { + switch (reason) { + case SLOW_SYNC: + slowSyncHookCalled.lazySet(true); + break; + default: + break; + } + } + }); + return log; + } + + protected final void checkSlowSync(AbstractFSWAL log, Table table, int slowSyncLatency, + int writeCount, boolean slowSync) throws Exception { + if (slowSyncLatency > 0) { + setSyncLatencyMillis(slowSyncLatency); + setSlowLogWriter(log.conf); + } else { + setDefaultLogWriter(log.conf); + } + + // Set up for test + log.rollWriter(true); + slowSyncHookCalled.set(false); + + final WALProvider.WriterBase oldWriter = log.getWriter(); + + // Write some data + for (int i = 0; i < writeCount; i++) { + writeData(table, rowNum++); + } + + if (slowSync) { + TEST_UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate() { + @Override + public boolean evaluate() throws Exception { + return log.getWriter() != oldWriter; + } + + @Override + public String explainFailure() throws Exception { + return "Waited too long for our test writer to get rolled out"; + } + }); + + assertTrue("Should have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); + } else { + assertFalse("Should not have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); + } + } + + protected abstract void setSlowLogWriter(Configuration conf); + + protected abstract void setDefaultLogWriter(Configuration conf); + /** * Tests that log rolling doesn't hang when no data is written. */ @@ -239,12 +336,10 @@ void validateData(Table table, int rownum) throws IOException { */ @Test public void testCompactionRecordDoesntBlockRolling() throws Exception { - Table table = null; // When the hbase:meta table can be opened, the region servers are running - Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); - try { - table = createTestTable(getName()); + try (Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + Table table = createTestTable(getName())) { server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); HRegion region = server.getRegions(table.getName()).get(0); @@ -286,9 +381,6 @@ public void testCompactionRecordDoesntBlockRolling() throws Exception { log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added. assertEquals("Should have 1 WALs at the end", 1, AbstractFSWALProvider.getNumRolledLogFiles(log)); - } finally { - if (t != null) t.close(); - if (table != null) table.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java index 9dc27a693a7f..804e93eb8f56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java @@ -20,10 +20,17 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; @@ -36,6 +43,9 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; + @Category({ VerySlowRegionServerTests.class, LargeTests.class }) public class TestAsyncLogRolling extends AbstractTestLogRolling { @@ -51,6 +61,61 @@ public static void setUpBeforeClass() throws Exception { AbstractTestLogRolling.setUpBeforeClass(); } + public static class SlowSyncLogWriter extends AsyncProtobufLogWriter { + + public SlowSyncLogWriter(EventLoopGroup eventLoopGroup, Class channelClass) { + super(eventLoopGroup, channelClass); + } + + @Override + public CompletableFuture sync(boolean forceSync) { + CompletableFuture future = new CompletableFuture<>(); + super.sync(forceSync).whenCompleteAsync((lengthAfterFlush, error) -> { + EXECUTOR.schedule(() -> { + if (error != null) { + future.completeExceptionally(error); + } else { + future.complete(lengthAfterFlush); + } + }, syncLatencyMillis, TimeUnit.MILLISECONDS); + }); + return future; + } + } + + @Override + protected void setSlowLogWriter(Configuration conf) { + conf.set(AsyncFSWALProvider.WRITER_IMPL, SlowSyncLogWriter.class.getName()); + } + + @Override + protected void setDefaultLogWriter(Configuration conf) { + conf.set(AsyncFSWALProvider.WRITER_IMPL, AsyncProtobufLogWriter.class.getName()); + } + + @Test + public void testSlowSyncLogRolling() throws Exception { + // Create the test table + TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); + admin.createTable(desc); + try (Table table = TEST_UTIL.getConnection().getTable(desc.getTableName())) { + server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); + RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); + final AbstractFSWAL log = getWALAndRegisterSlowSyncHook(region); + + // Set default log writer, no additional latency to any sync on the hlog. + checkSlowSync(log, table, -1, 10, false); + + // Adds 5000 ms of latency to any sync on the hlog. This will trip the other threshold. + // Write some data. Should only take one sync. + checkSlowSync(log, table, 5000, 1, true); + + // Set default log writer, no additional latency to any sync on the hlog. + checkSlowSync(log, table, -1, 10, false); + } + } + @Test public void testLogRollOnDatanodeDeath() throws IOException, InterruptedException { dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 3, true, null, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index f07a02cb25d1..9caa47e8614b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -56,10 +55,9 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.apache.hadoop.hbase.wal.WALStreamReader; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -98,192 +96,30 @@ public static void setUpBeforeClass() throws Exception { conf.setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3); conf.set(WALFactory.WAL_PROVIDER, "filesystem"); AbstractTestLogRolling.setUpBeforeClass(); - - // For slow sync threshold test: roll after 5 slow syncs in 10 seconds - TEST_UTIL.getConfiguration().setInt(FSHLog.SLOW_SYNC_ROLL_THRESHOLD, 5); - TEST_UTIL.getConfiguration().setInt(FSHLog.SLOW_SYNC_ROLL_INTERVAL_MS, 10 * 1000); - // For slow sync threshold test: roll once after a sync above this threshold - TEST_UTIL.getConfiguration().setInt(FSHLog.ROLL_ON_SYNC_TIME_MS, 5000); } - @Test - public void testSlowSyncLogRolling() throws Exception { - // Create the test table - TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); - admin.createTable(desc); - Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); - int row = 1; - try { - // Get a reference to the FSHLog - server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); - RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); - final FSHLog log = (FSHLog) server.getWAL(region); - - // Register a WALActionsListener to observe if a SLOW_SYNC roll is requested - - final AtomicBoolean slowSyncHookCalled = new AtomicBoolean(); - log.registerWALActionsListener(new WALActionsListener() { - @Override - public void logRollRequested(WALActionsListener.RollRequestReason reason) { - switch (reason) { - case SLOW_SYNC: - slowSyncHookCalled.lazySet(true); - break; - default: - break; - } - } - }); - - // Write some data - - for (int i = 0; i < 10; i++) { - writeData(table, row++); - } - - assertFalse("Should not have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); - - // Set up for test - slowSyncHookCalled.set(false); - - // Wrap the current writer with the anonymous class below that adds 200 ms of - // latency to any sync on the hlog. This should be more than sufficient to trigger - // slow sync warnings. - final Writer oldWriter1 = log.getWriter(); - final Writer newWriter1 = new Writer() { - @Override - public void close() throws IOException { - oldWriter1.close(); - } - - @Override - public void sync(boolean forceSync) throws IOException { - try { - Thread.sleep(200); - } catch (InterruptedException e) { - InterruptedIOException ex = new InterruptedIOException(); - ex.initCause(e); - throw ex; - } - oldWriter1.sync(forceSync); - } - - @Override - public void append(Entry entry) throws IOException { - oldWriter1.append(entry); - } - - @Override - public long getLength() { - return oldWriter1.getLength(); - } - - @Override - public long getSyncedLength() { - return oldWriter1.getSyncedLength(); - } - }; - log.setWriter(newWriter1); - - // Write some data. - // We need to write at least 5 times, but double it. We should only request - // a SLOW_SYNC roll once in the current interval. - for (int i = 0; i < 10; i++) { - writeData(table, row++); - } - - // Wait for our wait injecting writer to get rolled out, as needed. - - TEST_UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate() { - @Override - public boolean evaluate() throws Exception { - return log.getWriter() != newWriter1; - } - - @Override - public String explainFailure() throws Exception { - return "Waited too long for our test writer to get rolled out"; - } - }); - - assertTrue("Should have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); - - // Set up for test - slowSyncHookCalled.set(false); - - // Wrap the current writer with the anonymous class below that adds 5000 ms of - // latency to any sync on the hlog. - // This will trip the other threshold. - final Writer oldWriter2 = (Writer) log.getWriter(); - final Writer newWriter2 = new Writer() { - @Override - public void close() throws IOException { - oldWriter2.close(); - } - - @Override - public void sync(boolean forceSync) throws IOException { - try { - Thread.sleep(5000); - } catch (InterruptedException e) { - InterruptedIOException ex = new InterruptedIOException(); - ex.initCause(e); - throw ex; - } - oldWriter2.sync(forceSync); - } - - @Override - public void append(Entry entry) throws IOException { - oldWriter2.append(entry); - } - - @Override - public long getLength() { - return oldWriter2.getLength(); - } - - @Override - public long getSyncedLength() { - return oldWriter2.getSyncedLength(); - } - }; - log.setWriter(newWriter2); - - // Write some data. Should only take one sync. - - writeData(table, row++); - - // Wait for our wait injecting writer to get rolled out, as needed. - - TEST_UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate() { - @Override - public boolean evaluate() throws Exception { - return log.getWriter() != newWriter2; - } - - @Override - public String explainFailure() throws Exception { - return "Waited too long for our test writer to get rolled out"; - } - }); - - assertTrue("Should have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); - - // Set up for test - slowSyncHookCalled.set(false); - - // Write some data - for (int i = 0; i < 10; i++) { - writeData(table, row++); + public static class SlowSyncLogWriter extends ProtobufLogWriter { + @Override + public void sync(boolean forceSync) throws IOException { + try { + Thread.sleep(syncLatencyMillis); + } catch (InterruptedException e) { + InterruptedIOException ex = new InterruptedIOException(); + ex.initCause(e); + throw ex; } + super.sync(forceSync); + } + } - assertFalse("Should not have triggered log roll due to SLOW_SYNC", slowSyncHookCalled.get()); + @Override + protected void setSlowLogWriter(Configuration conf) { + conf.set(FSHLogProvider.WRITER_IMPL, SlowSyncLogWriter.class.getName()); + } - } finally { - table.close(); - } + @Override + protected void setDefaultLogWriter(Configuration conf) { + conf.set(FSHLogProvider.WRITER_IMPL, ProtobufLogWriter.class.getName()); } void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout) @@ -313,6 +149,36 @@ void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, } } + @Test + public void testSlowSyncLogRolling() throws Exception { + // Create the test table + TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(getName())) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); + admin.createTable(desc); + try (Table table = TEST_UTIL.getConnection().getTable(desc.getTableName())) { + server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); + RegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo(); + final AbstractFSWAL log = getWALAndRegisterSlowSyncHook(region); + + // Set default log writer, no additional latency to any sync on the hlog. + checkSlowSync(log, table, -1, 10, false); + + // Adds 200 ms of latency to any sync on the hlog. This should be more than sufficient to + // trigger slow sync warnings. + // Write some data. + // We need to write at least 5 times, but double it. We should only request + // a SLOW_SYNC roll once in the current interval. + checkSlowSync(log, table, 200, 10, true); + + // Adds 5000 ms of latency to any sync on the hlog. This will trip the other threshold. + // Write some data. Should only take one sync. + checkSlowSync(log, table, 5000, 1, true); + + // Set default log writer, no additional latency to any sync on the hlog. + checkSlowSync(log, table, -1, 10, false); + } + } + /** * Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 & * syncFs() support (HDFS-200) From 5503962350d6a4fbb7d7e21c4f73a070707d847b Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Tue, 12 Dec 2023 10:41:40 +0100 Subject: [PATCH 015/109] HBASE-28252 Add sun.net.dns and sun.net.util to the JDK11+ module exports in the hbase script (#5571) Signed-off-by: Duo Zhang Signed-off-by: Balazs Meszaros --- bin/hbase | 5 +++-- bin/hbase-config.sh | 8 ++++++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/bin/hbase b/bin/hbase index 60cfb8afef9a..30b4e94a89ae 100755 --- a/bin/hbase +++ b/bin/hbase @@ -496,7 +496,7 @@ add_jdk11_deps_to_classpath() { } add_jdk11_jvm_flags() { - HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED" + HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED" } add_opentelemetry_agent() { @@ -786,7 +786,7 @@ fi # Add lib/jdk11 jars to the classpath if [ "${DEBUG}" = "true" ]; then - echo "Deciding on addition of lib/jdk11 jars to the classpath" + echo "Deciding on addition of lib/jdk11 jars to the classpath and setting JVM module flags" fi addJDK11Jars=false @@ -879,6 +879,7 @@ export CLASSPATH if [ "${DEBUG}" = "true" ]; then echo "classpath=${CLASSPATH}" >&2 HBASE_OPTS="${HBASE_OPTS} -Xdiag" + echo "HBASE_OPTS=${HBASE_OPTS}" fi # resolve the command arguments diff --git a/bin/hbase-config.sh b/bin/hbase-config.sh index 104e9a0b67c3..0e8b3feed213 100644 --- a/bin/hbase-config.sh +++ b/bin/hbase-config.sh @@ -178,8 +178,12 @@ EOF fi function read_java_version() { - properties="$("${JAVA_HOME}/bin/java" -XshowSettings:properties -version 2>&1)" - echo "${properties}" | "${GREP}" java.runtime.version | head -1 | "${SED}" -e 's/.* = \([^ ]*\)/\1/' + # Avoid calling java repeatedly + if [ -z "$read_java_version_cached" ]; then + properties="$("${JAVA_HOME}/bin/java" -XshowSettings:properties -version 2>&1)" + read_java_version_cached="$(echo "${properties}" | "${GREP}" java.runtime.version | head -1 | "${SED}" -e 's/.* = \([^ ]*\)/\1/')" + fi + echo "$read_java_version_cached" } # Inspect the system properties exposed by this JVM to identify the major From f406c6bbf03ad3d5ca6bbd14944b8846c9ceb914 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Tue, 12 Dec 2023 15:13:40 +0100 Subject: [PATCH 016/109] HBASE-28247 Add java.base/sun.net.dns and java.base/sun.net.util export to jdk11 JVM test flags (#5569) Signed-off-by: Nihal Jain Signed-off-by: Balazs Meszaros --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2b1da88e056f..85bcf65108e5 100644 --- a/pom.xml +++ b/pom.xml @@ -981,7 +981,9 @@ --add-opens java.base/java.util=ALL-UNNAMED --add-opens java.base/java.util.concurrent=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED - --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED + --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED + --add-exports java.base/sun.net.dns=ALL-UNNAMED + --add-exports java.base/sun.net.util=ALL-UNNAMED --add-opens java.base/jdk.internal.util.random=ALL-UNNAMED ${hbase-surefire.argLine} @{jacocoArgLine} From 3d117125892ee36e8a66171fba3a223c09bc0b9a Mon Sep 17 00:00:00 2001 From: hiping-tech <58875741+hiping-tech@users.noreply.github.com> Date: Tue, 12 Dec 2023 23:13:01 +0800 Subject: [PATCH 017/109] HBASE-28241 The snapshot operation encountered an NPE and failed. (#5560) Fixed the check for an ongoing Snapshot before proceeding with the merge/split region operation. Co-authored-by: lvhaiping.lhp Signed-off-by: Duo Zhang Signed-off-by: Hui Ruan --- .../MergeTableRegionsProcedure.java | 2 +- .../assignment/SplitTableRegionProcedure.java | 3 +- .../TestMergeTableRegionsProcedure.java | 44 +++++++++++++++ .../TestSplitTableRegionProcedure.java | 53 +++++++++++++++++++ .../procedure/TestSnapshotProcedure.java | 24 +++++++++ 5 files changed, 124 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index c0b47b0bc246..7d4ec71d35b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -442,7 +442,7 @@ protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) { private boolean prepareMergeRegion(final MasterProcedureEnv env) throws IOException { // Fail if we are taking snapshot for the given table TableName tn = regionsToMerge[0].getTable(); - if (env.getMasterServices().getSnapshotManager().isTakingSnapshot(tn)) { + if (env.getMasterServices().getSnapshotManager().isTableTakingAnySnapshot(tn)) { throw new MergeRegionException("Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) + ", because we are snapshotting " + tn); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index a0118cbd7b05..2e2182b25d29 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -505,7 +505,8 @@ private byte[] getSplitRow() { public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException { // Fail if we are taking snapshot for the given table if ( - env.getMasterServices().getSnapshotManager().isTakingSnapshot(getParentRegion().getTable()) + env.getMasterServices().getSnapshotManager() + .isTableTakingAnySnapshot(getParentRegion().getTable()) ) { setFailure(new IOException("Skip splitting region " + getParentRegion().getShortNameToLog() + ", because we are taking snapshot for the table " + getParentRegion().getTable())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java index abc6fc45ad30..c0c4e355f2bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.assignment; +import static org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.assertProcFailed; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,16 +34,20 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.SnapshotType; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility; +import org.apache.hadoop.hbase.master.procedure.TestSnapshotProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; @@ -59,6 +64,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + @Category({ MasterTests.class, LargeTests.class }) public class TestMergeTableRegionsProcedure { @@ -347,6 +355,42 @@ public void testMergeWithoutPONR() throws Exception { assertRegionCount(tableName, initialRegionCount - 1); } + @Test + public void testMergingRegionWhileTakingSnapshot() throws Exception { + final TableName tableName = TableName.valueOf("testMergingRegionWhileTakingSnapshot"); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + List tableRegions = createTable(tableName); + + ProcedureTestingUtility.waitNoProcedureRunning(procExec); + + SnapshotDescription snapshot = + new SnapshotDescription("SnapshotProcedureTest", tableName, SnapshotType.FLUSH); + SnapshotProtos.SnapshotDescription snapshotProto = + ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot); + snapshotProto = SnapshotDescriptionUtils.validate(snapshotProto, + UTIL.getHBaseCluster().getMaster().getConfiguration()); + long snapshotProcId = procExec.submitProcedure( + new TestSnapshotProcedure.DelaySnapshotProcedure(procExec.getEnvironment(), snapshotProto)); + UTIL.getHBaseCluster().getMaster().getSnapshotManager().registerSnapshotProcedure(snapshotProto, + snapshotProcId); + + RegionInfo[] regionsToMerge = new RegionInfo[2]; + regionsToMerge[0] = tableRegions.get(0); + regionsToMerge[1] = tableRegions.get(1); + + long mergeProcId = procExec.submitProcedure( + new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true)); + + ProcedureTestingUtility + .waitProcedure(UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor(), mergeProcId); + ProcedureTestingUtility.waitProcedure( + UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor(), snapshotProcId); + + assertProcFailed(procExec, mergeProcId); + assertEquals(initialRegionCount, UTIL.getAdmin().getRegions(tableName).size()); + } + private List createTable(final TableName tableName) throws Exception { TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java index 6e25dbab48ce..6ec36e75bea2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java @@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.SnapshotType; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -50,10 +52,12 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility; +import org.apache.hadoop.hbase.master.procedure.TestSnapshotProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -69,6 +73,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + @Category({ MasterTests.class, MediumTests.class }) public class TestSplitTableRegionProcedure { @@ -550,6 +557,52 @@ public void testSplitWithoutPONR() throws Exception { verify(tableName, splitRowNum); } + @Test + public void testSplitRegionWhileTakingSnapshot() throws Exception { + final TableName tableName = TableName.valueOf(name.getMethodName()); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + + RegionInfo[] regions = MasterProcedureTestingUtility.createTable(procExec, tableName, null, + columnFamilyName1, columnFamilyName2); + int splitRowNum = startRowNum + rowCount / 2; + byte[] splitKey = Bytes.toBytes("" + splitRowNum); + + assertTrue("not able to find a splittable region", regions != null); + assertTrue("not able to find a splittable region", regions.length == 1); + ProcedureTestingUtility.waitNoProcedureRunning(procExec); + + // task snapshot + SnapshotDescription snapshot = + new SnapshotDescription("SnapshotProcedureTest", tableName, SnapshotType.FLUSH); + SnapshotProtos.SnapshotDescription snapshotProto = + ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot); + snapshotProto = SnapshotDescriptionUtils.validate(snapshotProto, + UTIL.getHBaseCluster().getMaster().getConfiguration()); + long snapshotProcId = procExec.submitProcedure( + new TestSnapshotProcedure.DelaySnapshotProcedure(procExec.getEnvironment(), snapshotProto)); + UTIL.getHBaseCluster().getMaster().getSnapshotManager().registerSnapshotProcedure(snapshotProto, + snapshotProcId); + + // collect AM metrics before test + collectAssignmentManagerMetrics(); + + // Split region of the table + long procId = procExec.submitProcedure( + new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey)); + // Wait the completion + ProcedureTestingUtility.waitProcedure(procExec, procId); + ProcedureTestingUtility.waitProcedure(procExec, snapshotProcId); + + ProcedureTestingUtility.assertProcFailed(procExec, procId); + ProcedureTestingUtility.assertProcNotFailed(procExec, snapshotProcId); + + assertTrue(UTIL.getMiniHBaseCluster().getRegions(tableName).size() == 1); + assertTrue(UTIL.countRows(tableName) == 0); + + assertEquals(splitSubmittedCount + 1, splitProcMetrics.getSubmittedCounter().getCount()); + assertEquals(splitFailedCount + 1, splitProcMetrics.getFailedCounter().getCount()); + } + private void deleteData(final TableName tableName, final int startDeleteRowNum) throws IOException, InterruptedException { Table t = UTIL.getConnection().getTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java index 04442eb771d8..84a3e84763b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.hbase.master.procedure; +import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS; import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Optional; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -51,6 +53,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SnapshotState; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; @@ -70,6 +73,27 @@ public class TestSnapshotProcedure { protected SnapshotDescription snapshot; protected SnapshotProtos.SnapshotDescription snapshotProto; + public static final class DelaySnapshotProcedure extends SnapshotProcedure { + public DelaySnapshotProcedure() { + } + + public DelaySnapshotProcedure(final MasterProcedureEnv env, + final SnapshotProtos.SnapshotDescription snapshot) { + super(env, snapshot); + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, + MasterProcedureProtos.SnapshotState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + Flow flow = super.executeFromState(env, state); + if (state == SNAPSHOT_SNAPSHOT_ONLINE_REGIONS) { + TimeUnit.SECONDS.sleep(20); + } + return flow; + } + } + @Before public void setup() throws Exception { TEST_UTIL = new HBaseTestingUtil(); From 29bfc610d0433f720a34bc47aadca1433bbb1882 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 13 Dec 2023 14:52:12 +0800 Subject: [PATCH 018/109] HBASE-28244 ProcedureTestingUtility.restart is broken sometimes after HBASE-28199 (#5563) Signed-off-by: Duo Zhang --- .../hbase/procedure2/ProcedureExecutor.java | 46 ++++++++++++++----- .../hbase/procedure2/ProcedureFutureUtil.java | 13 +++++- 2 files changed, 46 insertions(+), 13 deletions(-) diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java index 5aa11811122b..e01a27d74675 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java @@ -35,6 +35,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -605,15 +606,23 @@ public void init(int numThreads, boolean abortOnCorruption) throws IOException { this.threadGroup = new ThreadGroup("PEWorkerGroup"); this.timeoutExecutor = new TimeoutExecutorThread<>(this, threadGroup, "ProcExecTimeout"); this.workerMonitorExecutor = new TimeoutExecutorThread<>(this, threadGroup, "WorkerMonitor"); + ThreadFactory backingThreadFactory = new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(threadGroup, r); + } + }; int size = Math.max(2, Runtime.getRuntime().availableProcessors()); - ThreadPoolExecutor executor = new ThreadPoolExecutor(size, size, 1, TimeUnit.MINUTES, - new LinkedBlockingQueue(), new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat(getClass().getSimpleName() + "-Async-Task-Executor-%d").build()); + ThreadPoolExecutor executor = + new ThreadPoolExecutor(size, size, 1, TimeUnit.MINUTES, new LinkedBlockingQueue(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(getClass().getSimpleName() + "-Async-Task-Executor-%d") + .setThreadFactory(backingThreadFactory).build()); executor.allowCoreThreadTimeOut(true); this.asyncTaskExecutor = executor; - forceUpdateExecutor = Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Force-Update-PEWorker-%d").build()); + forceUpdateExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Force-Update-PEWorker-%d").setThreadFactory(backingThreadFactory).build()); store.registerListener(new ProcedureStoreListener() { @Override @@ -684,10 +693,10 @@ public void startWorkers() throws IOException { } public void stop() { - if (!running.getAndSet(false)) { - return; - } - + // it is possible that we fail in init, while loading procedures, so we will not set running to + // true but we should have already started the ProcedureScheduler, and also the two + // ExecutorServices, so here we do not check running state, just stop them + running.set(false); LOG.info("Stopping"); scheduler.stop(); timeoutExecutor.sendStopSignal(); @@ -708,14 +717,29 @@ public void join() { for (WorkerThread worker : workerThreads) { worker.awaitTermination(); } + try { + if (!forceUpdateExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + LOG.warn("There are still pending tasks in forceUpdateExecutor"); + } + } catch (InterruptedException e) { + LOG.warn("interrupted while waiting for forceUpdateExecutor termination", e); + Thread.currentThread().interrupt(); + } + try { + if (!asyncTaskExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + LOG.warn("There are still pending tasks in asyncTaskExecutor"); + } + } catch (InterruptedException e) { + LOG.warn("interrupted while waiting for asyncTaskExecutor termination", e); + Thread.currentThread().interrupt(); + } // Destroy the Thread Group for the executors // TODO: Fix. #join is not place to destroy resources. try { threadGroup.destroy(); } catch (IllegalThreadStateException e) { - LOG.error("ThreadGroup {} contains running threads; {}: See STDOUT", this.threadGroup, - e.getMessage()); + LOG.error("ThreadGroup {} contains running threads; {}: See STDOUT", this.threadGroup, e); // This dumps list of threads on STDOUT. this.threadGroup.list(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java index 8ca4cba245da..997063c3097d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFutureUtil.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.function.Consumer; import java.util.function.Supplier; import org.apache.commons.lang3.mutable.MutableBoolean; @@ -61,6 +62,14 @@ public static void suspendIfNecessary(Procedure proc, throws IOException, ProcedureSuspendedException { MutableBoolean completed = new MutableBoolean(false); Thread currentThread = Thread.currentThread(); + // This is for testing. In ProcedureTestingUtility, we will restart a ProcedureExecutor and + // reuse it, for performance, so we need to make sure that all the procedure have been stopped. + // But here, the callback of this future is not executed in a PEWorker, so in ProcedureExecutor + // we have no way to stop it. So here, we will get the asyncTaskExecutor first, in the PEWorker + // thread, where the ProcedureExecutor should have not been stopped yet, then when calling the + // callback, if the ProcedureExecutor have already been stopped and restarted, the + // asyncTaskExecutor will also be shutdown so we can not add anything back to the scheduler. + ExecutorService asyncTaskExecutor = env.getAsyncTaskExecutor(); FutureUtils.addListener(future, (r, e) -> { if (Thread.currentThread() == currentThread) { LOG.debug("The future has completed while adding callback, give up suspending procedure {}", @@ -77,7 +86,7 @@ public static void suspendIfNecessary(Procedure proc, // And what makes things worse is that, we persist procedure state to master local region, // where the AsyncFSWAL implementation will use the same netty's event loop for dealing with // I/O, which could even cause dead lock. - env.getAsyncTaskExecutor().execute(() -> { + asyncTaskExecutor.execute(() -> { // should acquire procedure execution lock to make sure that the procedure executor has // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be // race and cause unexpected result @@ -89,7 +98,7 @@ public static void suspendIfNecessary(Procedure proc, } catch (IOException ioe) { LOG.error("Error while acquiring execution lock for procedure {}" + " when trying to wake it up, aborting...", proc, ioe); - env.getMasterServices().abort("Can not acquire procedure execution lock", e); + env.getMasterServices().abort("Can not acquire procedure execution lock", ioe); return; } try { From 74405eaa964f633f614916449f1a0de26fde1742 Mon Sep 17 00:00:00 2001 From: guluo Date: Mon, 11 Dec 2023 21:42:29 +0800 Subject: [PATCH 019/109] flush failed-fast if there are non-existing families at client side --- .../hbase/client/RawAsyncHBaseAdmin.java | 79 ++++++++++++++----- .../hadoop/hbase/regionserver/HRegion.java | 18 ++++- .../hbase/client/TestFlushFromClient.java | 36 +++++++++ .../hbase/regionserver/TestHRegion.java | 8 +- 4 files changed, 115 insertions(+), 26 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 103a64e520a1..4005eaf9450b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -81,6 +81,7 @@ import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.quotas.QuotaTableUtil; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; @@ -966,6 +967,8 @@ public CompletableFuture flush(TableName tableName) { @Override public CompletableFuture flush(TableName tableName, byte[] columnFamily) { + Preconditions.checkNotNull(columnFamily, + "columnFamily is null, If you don't specify a columnFamily, use flush(TableName) instead."); return flush(tableName, Collections.singletonList(columnFamily)); } @@ -975,6 +978,8 @@ public CompletableFuture flush(TableName tableName, List columnFam // If the server version is lower than the client version, it's possible that the // flushTable method is not present in the server side, if so, we need to fall back // to the old implementation. + Preconditions.checkNotNull(columnFamilyList, + "columnFamily is null, If you don't specify a columnFamily, use flush(TableName) instead."); List columnFamilies = columnFamilyList.stream() .filter(cf -> cf != null && cf.length > 0).distinct().collect(Collectors.toList()); FlushTableRequest request = RequestConverter.buildFlushTableRequest(tableName, columnFamilies, @@ -983,21 +988,40 @@ public CompletableFuture flush(TableName tableName, List columnFam tableName, request, (s, c, req, done) -> s.flushTable(c, req, done), (resp) -> resp.getProcId(), new FlushTableProcedureBiConsumer(tableName)); CompletableFuture future = new CompletableFuture<>(); - addListener(procFuture, (ret, error) -> { + addListener(getDescriptor(tableName), (tDesc, error) -> { if (error != null) { - if (error instanceof TableNotFoundException || error instanceof TableNotEnabledException) { - future.completeExceptionally(error); - } else if (error instanceof DoNotRetryIOException) { - // usually this is caused by the method is not present on the server or - // the hbase hadoop version does not match the running hadoop version. - // if that happens, we need fall back to the old flush implementation. - LOG.info("Unrecoverable error in master side. Fallback to FlushTableProcedure V1", error); - legacyFlush(future, tableName, columnFamilies); + future.completeExceptionally(error); + } else { + List nonFaimilies = columnFamilies.stream().filter(cf -> !tDesc.hasColumnFamily(cf)) + .map(cf -> Bytes.toString(cf)).collect(Collectors.toList()); + if (nonFaimilies.size() > 0) { + String noSuchFamiliesMsg = + String.format("There are non-existing families %s, we cannot flush the table %s", + nonFaimilies, tableName.getNameAsString()); + future.completeExceptionally(new NoSuchColumnFamilyException(noSuchFamiliesMsg)); } else { - future.completeExceptionally(error); + addListener(procFuture, (ret, error2) -> { + if (error2 != null) { + if ( + error2 instanceof TableNotFoundException + || error2 instanceof TableNotEnabledException + ) { + future.completeExceptionally(error2); + } else if (error2 instanceof DoNotRetryIOException) { + // usually this is caused by the method is not present on the server or + // the hbase hadoop version does not match the running hadoop version. + // if that happens, we need fall back to the old flush implementation. + LOG.info("Unrecoverable error in master side. Fallback to FlushTableProcedure V1", + error2); + legacyFlush(future, tableName, columnFamilies); + } else { + future.completeExceptionally(error2); + } + } else { + future.complete(ret); + } + }); } - } else { - future.complete(ret); } }); return future; @@ -1071,14 +1095,29 @@ CompletableFuture flushRegionInternal(byte[] regionName, by .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName))); return; } - addListener(flush(serverName, location.getRegion(), columnFamily, writeFlushWALMarker), - (ret, err2) -> { - if (err2 != null) { - future.completeExceptionally(err2); - } else { - future.complete(ret); - } - }); + TableName tableName = location.getRegion().getTable(); + addListener(getDescriptor(tableName), (tDesc, error2) -> { + if (error2 != null) { + future.completeExceptionally(error2); + return; + } + if (columnFamily != null && !tDesc.hasColumnFamily(columnFamily)) { + String noSuchFamiliedMsg = String.format( + "There are non-existing family %s, we cannot flush the region %s, in table %s", + Bytes.toString(columnFamily), location.getRegion().getRegionNameAsString(), + tableName.getNameAsString()); + future.completeExceptionally(new NoSuchColumnFamilyException(noSuchFamiliedMsg)); + return; + } + addListener(flush(serverName, location.getRegion(), columnFamily, writeFlushWALMarker), + (ret, error3) -> { + if (error3 != null) { + future.completeExceptionally(error3); + } else { + future.complete(ret); + } + }); + }); }); return future; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index d93a4f444cda..a6a049081d37 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -2512,6 +2512,20 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque LOG.debug(msg); return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); } + // cannot flush non-existing column families, and fail-fast + if (families != null) { + List noSuchFamilies = + families.stream().filter(cf -> !getTableDescriptor().hasColumnFamily(cf)) + .map(cf -> Bytes.toString(cf)).collect(Collectors.toList()); + if (noSuchFamilies.size() > 0) { + String noSuchFamiliesMsg = String.format( + "There are non-existing families %s, we cannot flush the region %s, in table %s.", + noSuchFamilies, getRegionInfo().getRegionNameAsString(), + getTableDescriptor().getTableName().getNameAsString()); + LOG.warn(noSuchFamiliesMsg); + return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, noSuchFamiliesMsg, false); + } + } MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this); status.setStatus("Acquiring readlock on region"); // block waiting for the lock for flushing cache @@ -2597,9 +2611,7 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque private Collection getSpecificStores(List families) { Collection specificStoresToFlush = new ArrayList<>(); for (byte[] family : families) { - if (stores.keySet().contains(family)) { - specificStoresToFlush.add(stores.get(family)); - } + specificStoresToFlush.add(stores.get(family)); } return specificStoresToFlush; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java index 4756a5b22304..6659b6164f01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java @@ -19,10 +19,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -30,9 +34,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.io.IOUtils; import org.junit.After; @@ -43,6 +49,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +73,9 @@ public class TestFlushFromClient { @Rule public TestName name = new TestName(); + @Rule + public ExpectedException exception = ExpectedException.none(); + public TableName tableName; @BeforeClass @@ -184,6 +194,32 @@ public void testAsyncFlushRegionFamily() throws Exception { } } + @Test + public void testAsyncFlushTableWithNonExistingFamilies() throws IOException { + AsyncAdmin admin = asyncConn.getAdmin(); + List families = new ArrayList<>(); + families.add(FAMILY_1); + families.add(FAMILY_2); + families.add(Bytes.toBytes("non_family01")); + families.add(Bytes.toBytes("non_family02")); + CompletableFuture future = CompletableFuture.allOf(admin.flush(tableName, families)); + exception.expect(NoSuchColumnFamilyException.class); + FutureUtils.get(future); + } + + @Test + public void testAsyncFlushRegionWithNonExistingFamily() throws IOException { + AsyncAdmin admin = asyncConn.getAdmin(); + List regions = getRegionInfo(); + assertNotNull(regions); + assertTrue(regions.size() > 0); + HRegion region = regions.get(0); + CompletableFuture future = CompletableFuture.allOf( + admin.flushRegion(region.getRegionInfo().getRegionName(), Bytes.toBytes("non_family"))); + exception.expect(NoSuchColumnFamilyException.class); + FutureUtils.get(future); + } + @Test public void testFlushRegionServer() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 7f65332c51bd..3e4199439f7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5828,9 +5828,11 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); this.region.put(put); - HRegion.FlushResult fr = - this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); - assertTrue(fr.isFlushSucceeded()); + HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family, noSuchFamily), false, + FlushLifeCycleTracker.DUMMY); + assertEquals(HRegion.FlushResult.Result.CANNOT_FLUSH, fr.getResult()); + assertFalse("Cannot flush the Region because of non-existing column families.", + fr.isFlushSucceeded()); } protected Configuration initSplit() { From 9eb6cc4f66e02a9b9a59d345b199fd7638843415 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Thu, 14 Dec 2023 07:46:36 -0500 Subject: [PATCH 020/109] HBASE-28029 Netty SSL throughput improvement (#5580) Signed-off-by: Duo Zhang Reviewed-by: Nihal Jain --- .../org/apache/hadoop/hbase/io/crypto/tls/X509Util.java | 9 +++++++++ .../java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java | 5 +++++ 2 files changed, 14 insertions(+) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java index 41acfbbf48f4..fff32866fb9a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java @@ -105,6 +105,15 @@ public final class X509Util { public static final String HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT = "hbase.server.netty.tls.supportplaintext"; + /** + * Set the SSL wrapSize for netty. This is only a maximum wrap size. Buffers smaller than this + * will not be consolidated, but buffers larger than this will be split into multiple wrap + * buffers. The netty default of 16k is not great for hbase which tends to return larger payloads + * than that, meaning most responses end up getting chunked up. This leads to more memory + * contention in netty's PoolArena. See https://github.com/netty/netty/pull/13551 + */ + public static final String HBASE_SERVER_NETTY_TLS_WRAP_SIZE = "hbase.server.netty.tls.wrapSize"; + public static final int DEFAULT_HBASE_SERVER_NETTY_TLS_WRAP_SIZE = 1024 * 1024; // // Client-side specific configs // diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index 722ee1d28c91..ceff84a90e11 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.DEFAULT_HBASE_SERVER_NETTY_TLS_WRAP_SIZE; import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.HBASE_SERVER_NETTY_TLS_ENABLED; import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT; +import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.HBASE_SERVER_NETTY_TLS_WRAP_SIZE; import static org.apache.hadoop.hbase.io.crypto.tls.X509Util.TLS_CONFIG_REVERSE_DNS_LOOKUP_ENABLED; import java.io.IOException; @@ -408,6 +410,9 @@ private void initSSL(ChannelPipeline p, boolean supportPlaintext) sslHandler = nettySslContext.newHandler(p.channel().alloc()); } + sslHandler.setWrapDataSize( + conf.getInt(HBASE_SERVER_NETTY_TLS_WRAP_SIZE, DEFAULT_HBASE_SERVER_NETTY_TLS_WRAP_SIZE)); + p.addLast("ssl", sslHandler); LOG.debug("SSL handler added for channel: {}", p.channel()); } From 4ffdcea08f67cb1ae76f17b9d6a2653a45f2781c Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Thu, 14 Dec 2023 21:52:01 +0530 Subject: [PATCH 021/109] HBASE-28245 Sync internal protobuf version for hbase to be same as hbase-thirdparty (#5564) Signed-off-by: Duo Zhang --- hbase-examples/pom.xml | 2 +- hbase-protocol-shaded/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hbase-examples/pom.xml b/hbase-examples/pom.xml index 1a5ca5bd09aa..5a7d8f957da0 100644 --- a/hbase-examples/pom.xml +++ b/hbase-examples/pom.xml @@ -33,7 +33,7 @@ - 3.21.12 + 3.24.3 diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml index 1926e049cb9b..b2d9f79bbd08 100644 --- a/hbase-protocol-shaded/pom.xml +++ b/hbase-protocol-shaded/pom.xml @@ -34,7 +34,7 @@ - 3.21.12 + 3.24.3 false @@ -3159,6 +3162,7 @@ testapidocs User API The HBase Application Programmer's API + Apache HBaseâ„¢ ${project.version} API org.apache.hadoop.hbase.backup*:org.apache.hadoop.hbase.catalog:org.apache.hadoop.hbase.client.coprocessor:org.apache.hadoop.hbase.client.metrics:org.apache.hadoop.hbase.codec*:org.apache.hadoop.hbase.constraint:org.apache.hadoop.hbase.coprocessor.*:org.apache.hadoop.hbase.executor:org.apache.hadoop.hbase.fs:*.generated.*:org.apache.hadoop.hbase.io.hfile.*:org.apache.hadoop.hbase.mapreduce.hadoopbackport:org.apache.hadoop.hbase.mapreduce.replication:org.apache.hadoop.hbase.master.*:org.apache.hadoop.hbase.metrics*:org.apache.hadoop.hbase.migration:org.apache.hadoop.hbase.monitoring:org.apache.hadoop.hbase.p*:org.apache.hadoop.hbase.regionserver.compactions:org.apache.hadoop.hbase.regionserver.handler:org.apache.hadoop.hbase.regionserver.snapshot:org.apache.hadoop.hbase.replication.*:org.apache.hadoop.hbase.rest.filter:org.apache.hadoop.hbase.rest.model:org.apache.hadoop.hbase.rest.p*:org.apache.hadoop.hbase.security.*:org.apache.hadoop.hbase.thrift*:org.apache.hadoop.hbase.tmpl.*:org.apache.hadoop.hbase.tool:org.apache.hadoop.hbase.trace:org.apache.hadoop.hbase.util.byterange*:org.apache.hadoop.hbase.util.test:org.apache.hadoop.hbase.util.vint:org.apache.hadoop.metrics2*:org.apache.hadoop.hbase.io.compress* false From 4e0af3b7ae7cd751f5c544de136783d606c9f825 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 3 Jan 2024 18:01:07 +0800 Subject: [PATCH 040/109] HBASE-28290 Addendum also change title for test javadoc (#5601) Signed-off-by: Nihal Jain Signed-off-by: Peter Somogyi --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 20efab111c09..2fbd2769029a 100644 --- a/pom.xml +++ b/pom.xml @@ -3044,7 +3044,7 @@ testdevapidocs Test Developer API The full HBase API test code, including private and unstable APIs - Apache HBaseâ„¢ ${project.version} API + Apache HBaseâ„¢ ${project.version} API **/generated/* **/protobuf/* @@ -3160,9 +3160,9 @@ true testapidocs - User API + Test User API The HBase Application Programmer's API - Apache HBaseâ„¢ ${project.version} API + Apache HBaseâ„¢ ${project.version} API org.apache.hadoop.hbase.backup*:org.apache.hadoop.hbase.catalog:org.apache.hadoop.hbase.client.coprocessor:org.apache.hadoop.hbase.client.metrics:org.apache.hadoop.hbase.codec*:org.apache.hadoop.hbase.constraint:org.apache.hadoop.hbase.coprocessor.*:org.apache.hadoop.hbase.executor:org.apache.hadoop.hbase.fs:*.generated.*:org.apache.hadoop.hbase.io.hfile.*:org.apache.hadoop.hbase.mapreduce.hadoopbackport:org.apache.hadoop.hbase.mapreduce.replication:org.apache.hadoop.hbase.master.*:org.apache.hadoop.hbase.metrics*:org.apache.hadoop.hbase.migration:org.apache.hadoop.hbase.monitoring:org.apache.hadoop.hbase.p*:org.apache.hadoop.hbase.regionserver.compactions:org.apache.hadoop.hbase.regionserver.handler:org.apache.hadoop.hbase.regionserver.snapshot:org.apache.hadoop.hbase.replication.*:org.apache.hadoop.hbase.rest.filter:org.apache.hadoop.hbase.rest.model:org.apache.hadoop.hbase.rest.p*:org.apache.hadoop.hbase.security.*:org.apache.hadoop.hbase.thrift*:org.apache.hadoop.hbase.tmpl.*:org.apache.hadoop.hbase.tool:org.apache.hadoop.hbase.trace:org.apache.hadoop.hbase.util.byterange*:org.apache.hadoop.hbase.util.test:org.apache.hadoop.hbase.util.vint:org.apache.hadoop.metrics2*:org.apache.hadoop.hbase.io.compress* false From 4a545581ca92a4a0e54a777f0e846e5c7af75b9f Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Wed, 3 Jan 2024 18:56:44 +0530 Subject: [PATCH 041/109] HBASE-28249 Bump jruby to 9.3.13.0 and related joni and jcodings to 2.2.1 and 1.0.58 respectively (#5568) - Also fix "NoMethodError: undefined method runtime for JRuby:Module" caused due to bump Signed-off-by: Rajeshbabu Chintaguntla --- hbase-shell/src/main/ruby/jar-bootstrap.rb | 8 ++++++++ pom.xml | 6 +++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/hbase-shell/src/main/ruby/jar-bootstrap.rb b/hbase-shell/src/main/ruby/jar-bootstrap.rb index ed8f9b3a3a29..63cb0a755449 100644 --- a/hbase-shell/src/main/ruby/jar-bootstrap.rb +++ b/hbase-shell/src/main/ruby/jar-bootstrap.rb @@ -37,6 +37,14 @@ # hbase hacking. include Java +# Required to access JRuby-specific internal features, such as `JRuby.runtime` +# Loading 'java' was automatically loading 'jruby' until JRuby 9.2. +# But, it has changed since JRuby 9.3. JRuby 9.3+ needs loading 'jruby' explicitly. +# +# See also: https://github.com/jruby/jruby/issues/7221#issuecomment-1133646241 +# +require 'jruby' + # Some goodies for hirb. Should these be left up to the user's discretion? if $stdin.tty? require 'irb/completion' diff --git a/pom.xml b/pom.xml index 2fbd2769029a..bbecc3a72d86 100644 --- a/pom.xml +++ b/pom.xml @@ -846,7 +846,7 @@ 2.1.1 2.3.2 3.0.1-b08 - 9.3.9.0 + 9.3.13.0 4.13.2 1.3 1.15.0 @@ -863,8 +863,8 @@ 2.4.1 1.5.4 - 2.1.43 - 1.0.57 + 2.2.1 + 1.0.58 2.12.2 1.76 1.5.1 From 119885415c2f1770d95fdf832d249cbf7a0c80f4 Mon Sep 17 00:00:00 2001 From: mrzhao Date: Fri, 5 Jan 2024 17:15:17 +0800 Subject: [PATCH 042/109] HBASE-28259 Add java.base/java.io=ALL-UNNAMED open to jdk11_jvm_flags (#5581) Signed-off-by: Wellington Chevreuil Signed-off-by: Nihal Jain --- bin/hbase | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/hbase b/bin/hbase index 30b4e94a89ae..e329c5070899 100755 --- a/bin/hbase +++ b/bin/hbase @@ -496,7 +496,7 @@ add_jdk11_deps_to_classpath() { } add_jdk11_jvm_flags() { - HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED" + HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED" } add_opentelemetry_agent() { From 849eee46c937bc9bed801d90e2402f4b1a901de0 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Sat, 6 Jan 2024 16:47:17 +0800 Subject: [PATCH 043/109] HBASE-28290 Addendum use 'Test API' for test javadoc (#5602) Signed-off-by: Nihal Jain --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index bbecc3a72d86..ae6a75fbae92 100644 --- a/pom.xml +++ b/pom.xml @@ -3044,7 +3044,7 @@ testdevapidocs Test Developer API The full HBase API test code, including private and unstable APIs - Apache HBaseâ„¢ ${project.version} API + Apache HBaseâ„¢ ${project.version} Test API **/generated/* **/protobuf/* @@ -3161,8 +3161,8 @@ true testapidocs Test User API - The HBase Application Programmer's API - Apache HBaseâ„¢ ${project.version} API + The HBase Application Programmer's API test code + Apache HBaseâ„¢ ${project.version} Test API org.apache.hadoop.hbase.backup*:org.apache.hadoop.hbase.catalog:org.apache.hadoop.hbase.client.coprocessor:org.apache.hadoop.hbase.client.metrics:org.apache.hadoop.hbase.codec*:org.apache.hadoop.hbase.constraint:org.apache.hadoop.hbase.coprocessor.*:org.apache.hadoop.hbase.executor:org.apache.hadoop.hbase.fs:*.generated.*:org.apache.hadoop.hbase.io.hfile.*:org.apache.hadoop.hbase.mapreduce.hadoopbackport:org.apache.hadoop.hbase.mapreduce.replication:org.apache.hadoop.hbase.master.*:org.apache.hadoop.hbase.metrics*:org.apache.hadoop.hbase.migration:org.apache.hadoop.hbase.monitoring:org.apache.hadoop.hbase.p*:org.apache.hadoop.hbase.regionserver.compactions:org.apache.hadoop.hbase.regionserver.handler:org.apache.hadoop.hbase.regionserver.snapshot:org.apache.hadoop.hbase.replication.*:org.apache.hadoop.hbase.rest.filter:org.apache.hadoop.hbase.rest.model:org.apache.hadoop.hbase.rest.p*:org.apache.hadoop.hbase.security.*:org.apache.hadoop.hbase.thrift*:org.apache.hadoop.hbase.tmpl.*:org.apache.hadoop.hbase.tool:org.apache.hadoop.hbase.trace:org.apache.hadoop.hbase.util.byterange*:org.apache.hadoop.hbase.util.test:org.apache.hadoop.hbase.util.vint:org.apache.hadoop.metrics2*:org.apache.hadoop.hbase.io.compress* false From 4aeabdcc7156f21aea4155972b0d0d8c5dff4966 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Sat, 6 Jan 2024 16:47:56 +0800 Subject: [PATCH 044/109] HBASE-28277 Move minimum hadoop 3 support to 3.3.x for 2.6+ (#5598) Signed-off-by: Bryan Beaudreault --- dev-support/hbase-personality.sh | 11 +++++++++-- pom.xml | 2 +- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh index 67aa2d1d168f..c7131c45e2f1 100755 --- a/dev-support/hbase-personality.sh +++ b/dev-support/hbase-personality.sh @@ -602,13 +602,20 @@ function hadoopcheck_rebuild else hbase_hadoop3_versions="3.1.1 3.1.2 3.1.3 3.1.4 3.2.0 3.2.1 3.2.2 3.2.3 3.2.4 3.3.0 3.3.1 3.3.2 3.3.3 3.3.4 3.3.5 3.3.6" fi - else - yetus_info "Setting Hadoop 3 versions to test based on branch-2.5+/master/feature branch rules" + elif [[ "${PATCH_BRANCH}" = branch-2.5 ]]; then + yetus_info "Setting Hadoop 3 versions to test based on branch-2.5 rules" if [[ "${QUICK_HADOOPCHECK}" == "true" ]]; then hbase_hadoop3_versions="3.2.4 3.3.6" else hbase_hadoop3_versions="3.2.3 3.2.4 3.3.2 3.3.3 3.3.4 3.3.5 3.3.6" fi + else + yetus_info "Setting Hadoop 3 versions to test based on branch-2.6+/master/feature branch rules" + if [[ "${QUICK_HADOOPCHECK}" == "true" ]]; then + hbase_hadoop3_versions="3.3.6" + else + hbase_hadoop3_versions="3.3.5 3.3.6" + fi fi export MAVEN_OPTS="${MAVEN_OPTS}" diff --git a/pom.xml b/pom.xml index ae6a75fbae92..7d013de887a2 100644 --- a/pom.xml +++ b/pom.xml @@ -803,7 +803,7 @@ 3.5.0 ${compileSource} - 3.2.4 + 3.3.5 ${hadoop-three.version} From 1176e92cc646111313c2fa6a7a8e9092aef1f5c9 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Sun, 7 Jan 2024 20:00:36 +0530 Subject: [PATCH 045/109] HBASE-28295 Fews tests are failing due to NCDFE: org/bouncycastle/operator/OperatorCreationException (#5608) Signed-off-by: Duo Zhang --- hbase-mapreduce/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml index 3d9877dbf787..47e3f40363c5 100644 --- a/hbase-mapreduce/pom.xml +++ b/hbase-mapreduce/pom.xml @@ -216,6 +216,11 @@ bcprov-jdk18on test + + org.bouncycastle + bcpkix-jdk18on + test + From 71cecb0d2f3e63bf4cb4b85347753a75ffa18c00 Mon Sep 17 00:00:00 2001 From: Charles Connell Date: Mon, 8 Jan 2024 05:15:21 -0500 Subject: [PATCH 046/109] HBASE-26268 Provide coprocessor hooks for updateConfiguration and clearRegionBlockCache (#5593) Co-authored-by: Charles Connell Signed-off-by: Nick Dimiduk --- .../apache/hadoop/hbase/HBaseServerBase.java | 41 ++++++++++++++---- .../hbase/coprocessor/MasterObserver.java | 21 +++++++++ .../coprocessor/RegionServerObserver.java | 43 +++++++++++++++++++ .../apache/hadoop/hbase/master/HMaster.java | 5 +++ .../hbase/master/MasterCoprocessorHost.java | 18 ++++++++ .../hbase/regionserver/HRegionServer.java | 5 +++ .../hbase/regionserver/RSRpcServices.java | 28 +++++++----- .../RegionServerCoprocessorHost.java | 37 ++++++++++++++++ .../security/access/AccessController.java | 23 ++++++++++ .../hbase/coprocessor/TestMasterObserver.java | 34 +++++++++++++++ .../security/access/TestAccessController.java | 35 +++++++++++++++ 11 files changed, 271 insertions(+), 19 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java index 36f4f3addf5c..f64f2947ee96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java @@ -46,16 +46,19 @@ import org.apache.hadoop.hbase.conf.ConfigurationManager; import org.apache.hadoop.hbase.conf.ConfigurationObserver; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.http.InfoServer; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder; import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager; import org.apache.hadoop.hbase.regionserver.MemStoreLAB; +import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ShutdownHook; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; @@ -184,14 +187,14 @@ public abstract class HBaseServerBase> extends protected final NettyEventLoopGroupConfig eventLoopGroupConfig; - /** - * If running on Windows, do windows-specific setup. - */ - private static void setupWindows(final Configuration conf, ConfigurationManager cm) { + private void setupSignalHandlers() { if (!SystemUtils.IS_OS_WINDOWS) { HBasePlatformDependent.handle("HUP", (number, name) -> { - conf.reloadConfiguration(); - cm.notifyAllObservers(conf); + try { + updateConfiguration(); + } catch (IOException e) { + LOG.error("Problem while reloading configuration", e); + } }); } } @@ -276,7 +279,7 @@ public HBaseServerBase(Configuration conf, String name) throws IOException { new ZKWatcher(conf, getProcessName() + ":" + addr.getPort(), this, canCreateBaseZNode()); this.configurationManager = new ConfigurationManager(); - setupWindows(conf, configurationManager); + setupSignalHandlers(); initializeFileSystem(); @@ -614,11 +617,31 @@ public ConfigurationManager getConfigurationManager() { /** * Reload the configuration from disk. */ - public void updateConfiguration() { + public void updateConfiguration() throws IOException { LOG.info("Reloading the configuration from disk."); // Reload the configuration from disk. + preUpdateConfiguration(); conf.reloadConfiguration(); configurationManager.notifyAllObservers(conf); + postUpdateConfiguration(); + } + + private void preUpdateConfiguration() throws IOException { + CoprocessorHost coprocessorHost = getCoprocessorHost(); + if (coprocessorHost instanceof RegionServerCoprocessorHost) { + ((RegionServerCoprocessorHost) coprocessorHost).preUpdateConfiguration(conf); + } else if (coprocessorHost instanceof MasterCoprocessorHost) { + ((MasterCoprocessorHost) coprocessorHost).preUpdateConfiguration(conf); + } + } + + private void postUpdateConfiguration() throws IOException { + CoprocessorHost coprocessorHost = getCoprocessorHost(); + if (coprocessorHost instanceof RegionServerCoprocessorHost) { + ((RegionServerCoprocessorHost) coprocessorHost).postUpdateConfiguration(conf); + } else if (coprocessorHost instanceof MasterCoprocessorHost) { + ((MasterCoprocessorHost) coprocessorHost).postUpdateConfiguration(conf); + } } @Override @@ -626,6 +649,8 @@ public String toString() { return getServerName().toString(); } + protected abstract CoprocessorHost getCoprocessorHost(); + protected abstract boolean canCreateBaseZNode(); protected abstract String getProcessName(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index 820fef71fd07..d0e451508b43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.MetaMutationAnnotation; @@ -1873,4 +1874,24 @@ default void preHasUserPermissions(ObserverContext default void postHasUserPermissions(ObserverContext ctx, String userName, List permissions) throws IOException { } + + /** + * Called before reloading the HMaster's {@link Configuration} from disk + * @param ctx the coprocessor instance's environment + * @param preReloadConf the {@link Configuration} in use prior to reload + * @throws IOException if you need to signal an IO error + */ + default void preUpdateMasterConfiguration(ObserverContext ctx, + Configuration preReloadConf) throws IOException { + } + + /** + * Called after reloading the HMaster's {@link Configuration} from disk + * @param ctx the coprocessor instance's environment + * @param postReloadConf the {@link Configuration} that was loaded + * @throws IOException if you need to signal an IO error + */ + default void postUpdateMasterConfiguration(ObserverContext ctx, + Configuration postReloadConf) throws IOException { + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java index 236667b4be7b..b6915ffaaeac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CacheEvictionStats; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -169,4 +171,45 @@ default void postReplicationSinkBatchMutate( } + /** + * Called before clearing the block caches for one or more regions + * @param ctx the coprocessor instance's environment + * @throws IOException if you need to signal an IO error + */ + default void preClearRegionBlockCache(ObserverContext ctx) + throws IOException { + } + + /** + * Called after clearing the block caches for one or more regions + * @param ctx the coprocessor instance's environment + * @param stats statistics about the cache evictions that happened + * @throws IOException if you need to signal an IO error + */ + default void postClearRegionBlockCache(ObserverContext ctx, + CacheEvictionStats stats) throws IOException { + } + + /** + * Called before reloading the RegionServer's {@link Configuration} from disk + * @param ctx the coprocessor instance's environment + * @param preReloadConf the {@link Configuration} in use prior to reload + * @throws IOException if you need to signal an IO error + */ + default void preUpdateRegionServerConfiguration( + ObserverContext ctx, Configuration preReloadConf) + throws IOException { + } + + /** + * Called after reloading the RegionServer's {@link Configuration} from disk + * @param ctx the coprocessor instance's environment + * @param postReloadConf the {@link Configuration} that was loaded + * @throws IOException if you need to signal an IO error + */ + default void postUpdateRegionServerConfiguration( + ObserverContext ctx, Configuration postReloadConf) + throws IOException { + } + } 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 8567f00cad0a..88b82f01069e 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 @@ -724,6 +724,11 @@ public MasterRpcServices getMasterRpcServices() { return rpcServices; } + @Override + protected MasterCoprocessorHost getCoprocessorHost() { + return getMasterCoprocessorHost(); + } + public boolean balanceSwitch(final boolean b) throws IOException { return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java index 3af69b362609..e3d269973f8f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java @@ -2114,4 +2114,22 @@ public void call(MasterObserver observer) throws IOException { } }); } + + public void preUpdateConfiguration(Configuration preReloadConf) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.preUpdateMasterConfiguration(this, preReloadConf); + } + }); + } + + public void postUpdateConfiguration(Configuration postReloadConf) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.postUpdateMasterConfiguration(this, postReloadConf); + } + }); + } } 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 b6c573a953f1..a77fa0cd879e 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 @@ -587,6 +587,11 @@ protected String getProcessName() { return REGIONSERVER; } + @Override + protected RegionServerCoprocessorHost getCoprocessorHost() { + return getRegionServerCoprocessorHost(); + } + @Override protected boolean canCreateBaseZNode() { return !clusterMode(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 4f04457e91b6..a43fac6993e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -3802,19 +3802,25 @@ public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(RpcController contr @Override public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller, ClearRegionBlockCacheRequest request) throws ServiceException { - rpcPreCheck("clearRegionBlockCache"); - ClearRegionBlockCacheResponse.Builder builder = ClearRegionBlockCacheResponse.newBuilder(); - CacheEvictionStatsBuilder stats = CacheEvictionStats.builder(); - List regions = getRegions(request.getRegionList(), stats); - for (HRegion region : regions) { - try { - stats = stats.append(this.server.clearRegionBlockCache(region)); - } catch (Exception e) { - stats.addException(region.getRegionInfo().getRegionName(), e); + try { + rpcPreCheck("clearRegionBlockCache"); + ClearRegionBlockCacheResponse.Builder builder = ClearRegionBlockCacheResponse.newBuilder(); + CacheEvictionStatsBuilder stats = CacheEvictionStats.builder(); + server.getRegionServerCoprocessorHost().preClearRegionBlockCache(); + List regions = getRegions(request.getRegionList(), stats); + for (HRegion region : regions) { + try { + stats = stats.append(this.server.clearRegionBlockCache(region)); + } catch (Exception e) { + stats.addException(region.getRegionInfo().getRegionName(), e); + } } + stats.withMaxCacheSize(server.getBlockCache().map(BlockCache::getMaxSize).orElse(0L)); + server.getRegionServerCoprocessorHost().postClearRegionBlockCache(stats.build()); + return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build(); + } catch (IOException e) { + throw new ServiceException(e); } - stats.withMaxCacheSize(server.getBlockCache().map(BlockCache::getMaxSize).orElse(0L)); - return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build(); } private void executeOpenRegionProcedures(OpenRegionRequest request, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java index af1e923760d9..06eabdad67d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.lang.reflect.InvocationTargetException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CacheEvictionStats; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Mutation; @@ -240,6 +241,42 @@ public void call(RegionServerObserver observer) throws IOException { }); } + public void preUpdateConfiguration(Configuration preReloadConf) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.preUpdateRegionServerConfiguration(this, preReloadConf); + } + }); + } + + public void postUpdateConfiguration(Configuration postReloadConf) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.postUpdateRegionServerConfiguration(this, postReloadConf); + } + }); + } + + public void preClearRegionBlockCache() throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.preClearRegionBlockCache(this); + } + }); + } + + public void postClearRegionBlockCache(CacheEvictionStats stats) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.postClearRegionBlockCache(this, stats); + } + }); + } + /** * Coprocessor environment extension providing access to region server related services. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 90a51e2cb03e..66a7b3a27032 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -2576,4 +2576,27 @@ public void preUpdateRSGroupConfig(final ObserverContext ctx) + throws IOException { + accessChecker.requirePermission(getActiveUser(ctx), "clearRegionBlockCache", null, + Permission.Action.ADMIN); + } + + @Override + public void preUpdateRegionServerConfiguration( + ObserverContext ctx, Configuration preReloadConf) + throws IOException { + accessChecker.requirePermission(getActiveUser(ctx), "updateConfiguration", null, + Permission.Action.ADMIN); + } + + @Override + public void preUpdateMasterConfiguration(ObserverContext ctx, + Configuration preReloadConf) throws IOException { + accessChecker.requirePermission(getActiveUser(ctx), "updateConfiguration", null, + Permission.Action.ADMIN); + } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index 9e65e58a56d2..cf647b9fe8a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -191,6 +191,8 @@ public static class CPMasterObserver implements MasterCoprocessor, MasterObserve private boolean postLockHeartbeatCalled; private boolean preMasterStoreFlushCalled; private boolean postMasterStoreFlushCalled; + private boolean preUpdateMasterConfigurationCalled; + private boolean postUpdateMasterConfigurationCalled; public void resetStates() { preCreateTableRegionInfosCalled = false; @@ -284,6 +286,8 @@ public void resetStates() { postLockHeartbeatCalled = false; preMasterStoreFlushCalled = false; postMasterStoreFlushCalled = false; + preUpdateMasterConfigurationCalled = false; + postUpdateMasterConfigurationCalled = false; } @Override @@ -1264,6 +1268,17 @@ public void postRollBackMergeRegionsAction( throws IOException { } + @Override + public void preUpdateMasterConfiguration(ObserverContext ctx, + Configuration preReloadConf) throws IOException { + preUpdateMasterConfigurationCalled = true; + } + + @Override + public void postUpdateMasterConfiguration(ObserverContext ctx, + Configuration postReloadConf) throws IOException { + postUpdateMasterConfigurationCalled = true; + } } private static HBaseTestingUtil UTIL = new HBaseTestingUtil(); @@ -1715,4 +1730,23 @@ public void testMasterStoreOperations() throws Exception { assertTrue("Master store flush called", cp.postMasterStoreFlushCalled); } } + + @Test + public void testUpdateConfiguration() throws Exception { + SingleProcessHBaseCluster cluster = UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + MasterCoprocessorHost host = master.getMasterCoprocessorHost(); + CPMasterObserver cp = host.findCoprocessor(CPMasterObserver.class); + cp.resetStates(); + assertFalse("No update configuration call", cp.preUpdateMasterConfigurationCalled); + assertFalse("No update configuration call", cp.postUpdateMasterConfigurationCalled); + + try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration()); + Admin admin = connection.getAdmin()) { + admin.updateConfiguration(); + + assertTrue("Update configuration called", cp.preUpdateMasterConfigurationCalled); + assertTrue("Update configuration called", cp.postUpdateMasterConfigurationCalled); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 11ae3b3ecf09..cc895d21ac61 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -3086,6 +3086,41 @@ public Object run() throws Exception { verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); } + @Test + public void testUpdateMasterConfiguration() throws Exception { + AccessTestAction action = () -> { + ACCESS_CONTROLLER.preUpdateMasterConfiguration(ObserverContextImpl.createAndPrepare(CP_ENV), + null); + return null; + }; + + verifyAllowed(action, SUPERUSER, USER_ADMIN); + verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + } + + @Test + public void testUpdateRegionServerConfiguration() throws Exception { + AccessTestAction action = () -> { + ACCESS_CONTROLLER + .preUpdateRegionServerConfiguration(ObserverContextImpl.createAndPrepare(RSCP_ENV), null); + return null; + }; + + verifyAllowed(action, SUPERUSER, USER_ADMIN); + verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + } + + @Test + public void testClearRegionBlockCache() throws Exception { + AccessTestAction action = () -> { + ACCESS_CONTROLLER.preClearRegionBlockCache(ObserverContextImpl.createAndPrepare(RSCP_ENV)); + return null; + }; + + verifyAllowed(action, SUPERUSER, USER_ADMIN); + verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + } + @Test public void testTransitSyncReplicationPeerState() throws Exception { AccessTestAction action = new AccessTestAction() { From 0916c72929e238735e3dc3ec09dbf935e8d6ad69 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Tue, 9 Jan 2024 16:50:29 +0100 Subject: [PATCH 047/109] HBASE-28261 Sync jvm11 module flags from hbase-surefire.jdk11.flags to bin/hbase (#5610) Signed-off-by: Nihal Jain Signed-off-by: Peter Somogyi --- bin/hbase | 3 ++- pom.xml | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/bin/hbase b/bin/hbase index e329c5070899..81379eaa587d 100755 --- a/bin/hbase +++ b/bin/hbase @@ -496,7 +496,8 @@ add_jdk11_deps_to_classpath() { } add_jdk11_jvm_flags() { - HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED" + # Keep in sync with hbase-surefire.jdk11.flags in the root pom.xml + HBASE_OPTS="$HBASE_OPTS -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED --add-opens java.base/java.lang.reflect=ALL-UNNAMED --add-opens java.base/java.util=ALL-UNNAMED --add-opens java.base/java.util.concurrent=ALL-UNNAMED --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED" } add_opentelemetry_agent() { diff --git a/pom.xml b/pom.xml index 7d013de887a2..6788884c2111 100644 --- a/pom.xml +++ b/pom.xml @@ -971,8 +971,11 @@ "-Djava.library.path=${hadoop.library.path};${java.library.path}" -Dorg.apache.hbase.thirdparty.io.netty.leakDetection.level=advanced -Dio.opentelemetry.context.enableStrictContext=true + -Dorg.apache.hbase.thirdparty.io.netty.tryReflectionSetAccessible=true --add-modules jdk.unsupported + --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/sun.nio.ch=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED @@ -984,6 +987,8 @@ --add-exports java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-exports java.base/sun.net.dns=ALL-UNNAMED --add-exports java.base/sun.net.util=ALL-UNNAMED + --add-opens java.base/jdk.internal.util.random=ALL-UNNAMED ${hbase-surefire.argLine} @{jacocoArgLine} From a09305d5854fc98300426271fad3b53a69d2ae71 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 10 Jan 2024 11:13:09 +0800 Subject: [PATCH 048/109] HBASE-28297 IntegrationTestImportTsv fails with ArrayIndexOfOutBounds (#5612) Signed-off-by: Nihal Jain --- .../hadoop/hbase/mapreduce/IntegrationTestImportTsv.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java index 35db989dd691..2e9ff8279365 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java @@ -60,6 +60,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; /** * Validate ImportTsv + BulkLoadFiles on a distributed cluster. @@ -85,6 +86,9 @@ public class IntegrationTestImportTsv extends Configured implements Tool { { byte[] family = Bytes.toBytes("d"); for (String line : Splitter.on('\n').split(simple_tsv)) { + if (Strings.isNullOrEmpty(line)) { + continue; + } String[] row = line.split("\t"); byte[] key = Bytes.toBytes(row[0]); long ts = Long.parseLong(row[1]); From 0db423101702b277b571f465f598aab3bfcce736 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Wed, 10 Jan 2024 17:13:42 +0530 Subject: [PATCH 049/109] [ADDENDUM] HBASE-28295 Few tests are failing due to NCDFE: org/bouncycastle/operator/OperatorCreationException (#5611) Signed-off-by: Duo Zhang --- hbase-backup/pom.xml | 6 ++++++ hbase-it/pom.xml | 6 ++++++ hbase-mapreduce/pom.xml | 1 + 3 files changed, 13 insertions(+) diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml index 478c8d8245ea..77d033d67fe6 100644 --- a/hbase-backup/pom.xml +++ b/hbase-backup/pom.xml @@ -105,6 +105,12 @@ + + + org.bouncycastle + bcpkix-jdk18on + test + org.apache.commons diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml index c3f884a16b0d..dbf34c404155 100644 --- a/hbase-it/pom.xml +++ b/hbase-it/pom.xml @@ -104,6 +104,12 @@ + + + org.bouncycastle + bcpkix-jdk18on + test + org.apache.hbase hbase-backup diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml index 47e3f40363c5..798250600e35 100644 --- a/hbase-mapreduce/pom.xml +++ b/hbase-mapreduce/pom.xml @@ -216,6 +216,7 @@ bcprov-jdk18on test + org.bouncycastle bcpkix-jdk18on From 8cab5a2dfb79304440d7c92a0d7170cd0f3e0739 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Thu, 11 Jan 2024 07:54:10 -0500 Subject: [PATCH 050/109] HBASE-28304 Add hbase-shaded-testing-util version to dependencyManagement (#5618) Signed-off-by: Duo Zhang --- hbase-shaded/hbase-shaded-testing-util-tester/pom.xml | 1 - pom.xml | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml b/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml index 928a3276df65..441d38ccd47f 100644 --- a/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml +++ b/hbase-shaded/hbase-shaded-testing-util-tester/pom.xml @@ -81,7 +81,6 @@ org.apache.hbase hbase-shaded-testing-util - ${project.version} test diff --git a/pom.xml b/pom.xml index 6788884c2111..a59379f8dc92 100644 --- a/pom.xml +++ b/pom.xml @@ -1284,6 +1284,11 @@ hbase-shaded-mapreduce ${project.version} + + org.apache.hbase + hbase-shaded-testing-util + ${project.version} + org.apache.hbase hbase-asyncfs From a7429a75f22e2228154eb30383fbffee86a2f103 Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Thu, 11 Jan 2024 23:44:35 +0530 Subject: [PATCH 051/109] HBASE-28301 IntegrationTestImportTsv fails with UnsupportedOperationException (#5613) Signed-off-by: Duo Zhang Signed-off-by: Peter Somogyi --- .../hbase/mapreduce/IntegrationTestImportTsv.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java index 2e9ff8279365..e5c1fbed1a56 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java @@ -145,12 +145,10 @@ protected void doLoadIncrementalHFiles(Path hfiles, TableName tableName) throws ToolRunner.run(new BulkLoadHFilesTool(getConf()), args)); Table table = null; - Scan scan = new Scan() { - { - setCacheBlocks(false); - setCaching(1000); - } - }; + Scan scan = new Scan(); + scan.setCacheBlocks(false); + scan.setCaching(1000); + try { table = util.getConnection().getTable(tableName); Iterator resultsIt = table.getScanner(scan).iterator(); From d57e4e238d6255a634762bd5f41db6d84803f703 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Jan 2024 14:50:29 +0800 Subject: [PATCH 052/109] HBASE-28308 Bump gitpython from 3.1.37 to 3.1.41 in /dev-support/flaky-tests (#5616) Bumps [gitpython](https://github.com/gitpython-developers/GitPython) from 3.1.37 to 3.1.41. - [Release notes](https://github.com/gitpython-developers/GitPython/releases) - [Changelog](https://github.com/gitpython-developers/GitPython/blob/main/CHANGES) - [Commits](https://github.com/gitpython-developers/GitPython/compare/3.1.37...3.1.41) --- updated-dependencies: - dependency-name: gitpython dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: Duo Zhang --- dev-support/flaky-tests/python-requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/flaky-tests/python-requirements.txt b/dev-support/flaky-tests/python-requirements.txt index 9ca8c66afb1a..0e09594aec8a 100644 --- a/dev-support/flaky-tests/python-requirements.txt +++ b/dev-support/flaky-tests/python-requirements.txt @@ -17,6 +17,6 @@ # requests==2.31.0 future==0.18.3 -gitpython==3.1.37 +gitpython==3.1.41 rbtools==4.0 jinja2==3.1.2 From 8ae77e2908cf1ce56d6b06f6492cfd731a8962ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Jan 2024 14:51:28 +0800 Subject: [PATCH 053/109] HBASE-28309 Bump gitpython in /dev-support/git-jira-release-audit (#5617) Bumps [gitpython](https://github.com/gitpython-developers/GitPython) from 3.1.37 to 3.1.41. - [Release notes](https://github.com/gitpython-developers/GitPython/releases) - [Changelog](https://github.com/gitpython-developers/GitPython/blob/main/CHANGES) - [Commits](https://github.com/gitpython-developers/GitPython/compare/3.1.37...3.1.41) --- updated-dependencies: - dependency-name: gitpython dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: Duo Zhang --- dev-support/git-jira-release-audit/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/git-jira-release-audit/requirements.txt b/dev-support/git-jira-release-audit/requirements.txt index 23a4b916fd4b..99fb2e2d0de9 100644 --- a/dev-support/git-jira-release-audit/requirements.txt +++ b/dev-support/git-jira-release-audit/requirements.txt @@ -23,7 +23,7 @@ cryptography==41.0.6 defusedxml==0.6.0 enlighten==1.4.0 gitdb2==2.0.6 -GitPython==3.1.37 +GitPython==3.1.41 idna==2.8 jira==2.0.0 oauthlib==3.1.0 From 6133d74318fc61f83faa865c4b307aa5cc7a8c29 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Jan 2024 14:52:56 +0800 Subject: [PATCH 054/109] HBASE-28310 Bump jinja2 from 3.1.2 to 3.1.3 in /dev-support/flaky-tests (#5619) Bumps [jinja2](https://github.com/pallets/jinja) from 3.1.2 to 3.1.3. - [Release notes](https://github.com/pallets/jinja/releases) - [Changelog](https://github.com/pallets/jinja/blob/main/CHANGES.rst) - [Commits](https://github.com/pallets/jinja/compare/3.1.2...3.1.3) --- updated-dependencies: - dependency-name: jinja2 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: Duo Zhang --- dev-support/flaky-tests/python-requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/flaky-tests/python-requirements.txt b/dev-support/flaky-tests/python-requirements.txt index 0e09594aec8a..8ef087b0036d 100644 --- a/dev-support/flaky-tests/python-requirements.txt +++ b/dev-support/flaky-tests/python-requirements.txt @@ -19,4 +19,4 @@ requests==2.31.0 future==0.18.3 gitpython==3.1.41 rbtools==4.0 -jinja2==3.1.2 +jinja2==3.1.3 From 4d2b8f8de668f0b5938c7665c9730133cd48b8da Mon Sep 17 00:00:00 2001 From: liuwenjing17 <154434091+liuwenjing17@users.noreply.github.com> Date: Sat, 13 Jan 2024 17:39:28 +0800 Subject: [PATCH 055/109] HBASE-28287 MOB HFiles are expired earlier than their reference data (#5599) Co-authored-by: liuwenjing3 Signed-off-by: Duo Zhang --- .../src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java | 1 + .../org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index e04d67a0aaaf..60f0f126ab60 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -280,6 +280,7 @@ public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, Table calendar.set(Calendar.HOUR_OF_DAY, 0); calendar.set(Calendar.MINUTE, 0); calendar.set(Calendar.SECOND, 0); + calendar.set(Calendar.MILLISECOND, 0); Date expireDate = calendar.getTime(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index f282c6f9d8f4..4bbc88681290 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -140,7 +140,8 @@ public void testCleaner() throws Exception { assertEquals("Before cleanup without delay 1", 1, firstFiles.length); String firstFile = firstFiles[0].getPath().getName(); - ts = EnvironmentEdgeManager.currentTime() - 1 * secondsOfDay() * 1000; // 1 day before + // 1.5 day before + ts = (long) (EnvironmentEdgeManager.currentTime() - 1.5 * secondsOfDay() * 1000); putKVAndFlush(table, row2, dummyData, ts); FileStatus[] secondFiles = TEST_UTIL.getTestFileSystem().listStatus(mobDirPath); // now there are 2 mob files From e3a0174e20542e661f787a874870b629a274daf5 Mon Sep 17 00:00:00 2001 From: haosen chen <99318736+haosenchen@users.noreply.github.com> Date: Sat, 13 Jan 2024 17:42:57 +0800 Subject: [PATCH 056/109] HBASE-28305 Add "Uncompressed StoreFileSize" column to the table.jsp (#5620) Co-authored-by: Haosen Chen Signed-off-by: Duo Zhang --- .../resources/hbase-webapps/master/table.jsp | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) 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 1d48a7561e1b..f0599b7aa64a 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -315,6 +315,7 @@ Region Server ReadRequests WriteRequests + Uncompressed StoreFileSize StorefileSize Num.Storefiles MemSize @@ -338,6 +339,7 @@ String hostAndPort = ""; String readReq = "N/A"; String writeReq = "N/A"; + String fileSizeUncompressed = ZEROMB; String fileSize = ZEROMB; String fileCount = "N/A"; String memSize = ZEROMB; @@ -356,6 +358,10 @@ if (rSize > 0) { fileSize = StringUtils.byteDesc((long) rSize); } + double rSizeUncompressed = load.getUncompressedStoreFileSize().get(Size.Unit.BYTE); + if (rSizeUncompressed > 0) { + fileSizeUncompressed = StringUtils.byteDesc((long) rSizeUncompressed); + } fileCount = String.format("%,1d", load.getStoreFileCount()); double mSize = load.getMemStoreSize().get(Size.Unit.BYTE); if (mSize > 0) { @@ -370,6 +376,7 @@ <%= StringEscapeUtils.escapeHtml4(hostAndPort) %> <%= readReq%> <%= writeReq%> + <%= fileSizeUncompressed%> <%= fileSize%> <%= fileCount%> <%= memSize%> @@ -834,6 +841,7 @@ <% long totalReadReq = 0; long totalWriteReq = 0; + long totalSizeUncompressed = 0; long totalSize = 0; long totalStoreFileCount = 0; long totalMemSize = 0; @@ -844,6 +852,7 @@ long totalBlocksLocalWithSsdWeight = 0; String totalCompactionProgress = ""; String totalMemSizeStr = ZEROMB; + String totalSizeUncompressedStr = ZEROMB; String totalSizeStr = ZEROMB; String totalLocality = ""; String totalLocalityForSsd = ""; @@ -865,6 +874,7 @@ if (regionMetrics != null) { totalReadReq += regionMetrics.getReadRequestCount(); totalWriteReq += regionMetrics.getWriteRequestCount(); + totalSizeUncompressed += regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE); totalSize += regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE); totalStoreFileCount += regionMetrics.getStoreFileCount(); totalMemSize += regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE); @@ -890,6 +900,9 @@ if (totalSize > 0) { totalSizeStr = StringUtils.byteDesc(totalSize*1024l*1024); } + if (totalSizeUncompressed > 0){ + totalSizeUncompressedStr = StringUtils.byteDesc(totalSizeUncompressed*1024l*1024); + } if (totalMemSize > 0) { totalMemSizeStr = StringUtils.byteDesc(totalMemSize*1024l*1024); } @@ -920,6 +933,7 @@ Region Server ReadRequests
(<%= String.format("%,1d", totalReadReq)%>) WriteRequests
(<%= String.format("%,1d", totalWriteReq)%>) + Uncompressed StoreFileSize
(<%= totalSizeUncompressedStr %>) StorefileSize
(<%= totalSizeStr %>) Num.Storefiles
(<%= String.format("%,1d", totalStoreFileCount)%>) MemSize
(<%= totalMemSizeStr %>) @@ -944,6 +958,7 @@ RegionMetrics load = hriEntry.getValue(); String readReq = "N/A"; String writeReq = "N/A"; + String regionSizeUncompressed = ZEROMB; String regionSize = ZEROMB; String fileCount = "N/A"; String memSize = ZEROMB; @@ -951,6 +966,10 @@ if (load != null) { readReq = String.format("%,1d", load.getReadRequestCount()); writeReq = String.format("%,1d", load.getWriteRequestCount()); + double rSizeUncompressed = load.getUncompressedStoreFileSize().get(Size.Unit.BYTE); + if (rSizeUncompressed > 0) { + regionSizeUncompressed = StringUtils.byteDesc((long)rSizeUncompressed); + } double rSize = load.getStoreFileSize().get(Size.Unit.BYTE); if (rSize > 0) { regionSize = StringUtils.byteDesc((long)rSize); @@ -987,6 +1006,7 @@ <%= buildRegionDeployedServerTag(regionInfo, master, regionsToServer) %> <%= readReq%> <%= writeReq%> + <%= regionSizeUncompressed%> <%= regionSize%> <%= fileCount%> <%= memSize%> From a683fcfbe8c5c84e58fa8670f4414f9d01ff43ed Mon Sep 17 00:00:00 2001 From: Nihal Jain Date: Sun, 14 Jan 2024 11:59:26 +0530 Subject: [PATCH 057/109] HBASE-27814 Add support for dump and process metrics servlet in REST InfoServer (#5215) Other changes: - Ensure info server stops during stop() - Extract header and footer. This would fix the log level page layout for rest web UI (See HBASE-20693) - Add hostname in the landing page instead of just port similar to other web UIs Signed-off-by: Nick Dimiduk --- .../hadoop/hbase/rest/RESTDumpServlet.java | 80 ++++++++ .../apache/hadoop/hbase/rest/RESTServer.java | 39 +++- .../resources/hbase-webapps/rest/footer.jsp | 32 +++ .../resources/hbase-webapps/rest/header.jsp | 74 +++++++ .../hbase-webapps/rest/processRest.jsp | 184 ++++++++++++++++++ .../resources/hbase-webapps/rest/rest.jsp | 80 ++------ 6 files changed, 422 insertions(+), 67 deletions(-) create mode 100644 hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTDumpServlet.java create mode 100644 hbase-rest/src/main/resources/hbase-webapps/rest/footer.jsp create mode 100644 hbase-rest/src/main/resources/hbase-webapps/rest/header.jsp create mode 100644 hbase-rest/src/main/resources/hbase-webapps/rest/processRest.jsp diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTDumpServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTDumpServlet.java new file mode 100644 index 000000000000..8bb306f7829a --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTDumpServlet.java @@ -0,0 +1,80 @@ +/* + * 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.rest; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.io.PrintWriter; +import java.util.Date; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.http.HttpServer; +import org.apache.hadoop.hbase.monitoring.StateDumpServlet; +import org.apache.hadoop.hbase.util.LogMonitoring; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class RESTDumpServlet extends StateDumpServlet { + private static final long serialVersionUID = 1L; + private static final String LINE = "==========================================================="; + + @Override + public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException { + if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) { + return; + } + + RESTServer restServer = (RESTServer) getServletContext().getAttribute(RESTServer.REST_SERVER); + assert restServer != null : "No REST Server in context!"; + + response.setContentType("text/plain"); + OutputStream os = response.getOutputStream(); + try (PrintWriter out = new PrintWriter(os)) { + + out.println("REST Server status for " + restServer.getServerName() + " as of " + new Date()); + + out.println("\n\nVersion Info:"); + out.println(LINE); + dumpVersionInfo(out); + + out.println("\n\nStacks:"); + out.println(LINE); + out.flush(); + PrintStream ps = new PrintStream(response.getOutputStream(), false, "UTF-8"); + Threads.printThreadInfo(ps, ""); + ps.flush(); + + out.println("\n\nREST Server configuration:"); + out.println(LINE); + Configuration conf = restServer.conf; + out.flush(); + conf.writeXml(os); + os.flush(); + + out.println("\n\nLogs"); + out.println(LINE); + long tailKb = getTailKbParam(request); + LogMonitoring.dumpTailOfLogs(out, tailKb); + + out.flush(); + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java index 886c81dc6680..42c00480526b 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.rest; import java.lang.management.ManagementFactory; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; @@ -29,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.http.ClickjackingPreventionFilter; import org.apache.hadoop.hbase.http.HttpServerUtil; import org.apache.hadoop.hbase.http.InfoServer; @@ -83,6 +85,7 @@ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) public class RESTServer implements Constants { static Logger LOG = LoggerFactory.getLogger("RESTServer"); + public static final String REST_SERVER = "rest"; static final String REST_CSRF_ENABLED_KEY = "hbase.rest.csrf.enabled"; static final boolean REST_CSRF_ENABLED_DEFAULT = false; @@ -112,6 +115,7 @@ public class RESTServer implements Constants { private final UserProvider userProvider; private Server server; private InfoServer infoServer; + private ServerName serverName; public RESTServer(Configuration conf) { RESTServer.conf = conf; @@ -163,8 +167,7 @@ private void addSecurityHeadersFilter(ServletContextHandler ctxHandler, Configur loginServerPrincipal(UserProvider userProvider, Configuration conf) throws Exception { Class containerClass = ServletContainer.class; if (userProvider.isHadoopSecurityEnabled() && userProvider.isHBaseSecurityEnabled()) { - String machineName = Strings.domainNamePointerToHostName(DNS.getDefaultHost( - conf.get(REST_DNS_INTERFACE, "default"), conf.get(REST_DNS_NAMESERVER, "default"))); + String machineName = getHostName(conf); String keytabFilename = conf.get(REST_KEYTAB_FILE); Preconditions.checkArgument(keytabFilename != null && !keytabFilename.isEmpty(), REST_KEYTAB_FILE + " should be set if security is enabled"); @@ -402,9 +405,14 @@ public synchronized void run() throws Exception { // Put up info server. int port = conf.getInt("hbase.rest.info.port", 8085); if (port >= 0) { - conf.setLong("startcode", EnvironmentEdgeManager.currentTime()); - String a = conf.get("hbase.rest.info.bindAddress", "0.0.0.0"); - this.infoServer = new InfoServer("rest", a, port, false, conf); + final long startCode = EnvironmentEdgeManager.currentTime(); + conf.setLong("startcode", startCode); + this.serverName = ServerName.valueOf(getHostName(conf), servicePort, startCode); + + String addr = conf.get("hbase.rest.info.bindAddress", "0.0.0.0"); + this.infoServer = new InfoServer(REST_SERVER, addr, port, false, conf); + this.infoServer.addPrivilegedServlet("dump", "/dump", RESTDumpServlet.class); + this.infoServer.setAttribute(REST_SERVER, this); this.infoServer.setAttribute("hbase.conf", conf); this.infoServer.start(); } @@ -412,6 +420,11 @@ public synchronized void run() throws Exception { server.start(); } + private static String getHostName(Configuration conf) throws UnknownHostException { + return Strings.domainNamePointerToHostName(DNS.getDefaultHost( + conf.get(REST_DNS_INTERFACE, "default"), conf.get(REST_DNS_NAMESERVER, "default"))); + } + public synchronized void join() throws Exception { if (server == null) { throw new IllegalStateException("Server is not running"); @@ -419,7 +432,19 @@ public synchronized void join() throws Exception { server.join(); } + private void stopInfoServer() { + if (this.infoServer != null) { + LOG.info("Stop info server"); + try { + this.infoServer.stop(); + } catch (Exception e) { + LOG.error("Failed to stop infoServer", e); + } + } + } + public synchronized void stop() throws Exception { + stopInfoServer(); if (server == null) { throw new IllegalStateException("Server is not running"); } @@ -443,6 +468,10 @@ public synchronized int getInfoPort() { return infoServer.getPort(); } + public ServerName getServerName() { + return serverName; + } + public Configuration getConf() { return conf; } diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/footer.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/footer.jsp new file mode 100644 index 000000000000..a642ac36eff7 --- /dev/null +++ b/hbase-rest/src/main/resources/hbase-webapps/rest/footer.jsp @@ -0,0 +1,32 @@ +<%-- +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +--%> + + + + + + + + diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/header.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/header.jsp new file mode 100644 index 000000000000..67f7656de592 --- /dev/null +++ b/hbase-rest/src/main/resources/hbase-webapps/rest/header.jsp @@ -0,0 +1,74 @@ +<%-- +/** +* 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. +*/ +--%> +<%@ page contentType="text/html;charset=UTF-8" + import="org.apache.hadoop.hbase.HBaseConfiguration"%> + + + + + + + <%= request.getParameter("pageTitle")%> + + + + + + + + + +

+ diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/processRest.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/processRest.jsp new file mode 100644 index 000000000000..2b2d35fbfb3f --- /dev/null +++ b/hbase-rest/src/main/resources/hbase-webapps/rest/processRest.jsp @@ -0,0 +1,184 @@ +<%-- +/** + * 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. + */ +--%> +<%@ page contentType="text/html;charset=UTF-8" + import="java.util.Date" + import="java.util.List" + import="javax.management.ObjectName" + import="java.lang.management.ManagementFactory" + import="java.lang.management.MemoryPoolMXBean" + import="java.lang.management.RuntimeMXBean" + import="java.lang.management.GarbageCollectorMXBean" + import="org.apache.hadoop.hbase.util.JSONMetricUtil" + import="org.apache.hadoop.hbase.procedure2.util.StringUtils" + import="org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix" +%> + +<% +RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean(); +ObjectName jvmMetrics = new ObjectName("Hadoop:service=HBase,name=JvmMetrics"); + +// There is always two of GC collectors +List gcBeans = JSONMetricUtil.getGcCollectorBeans(); +GarbageCollectorMXBean collector1 = null; +GarbageCollectorMXBean collector2 = null; +try { +collector1 = gcBeans.get(0); +collector2 = gcBeans.get(1); +} catch(IndexOutOfBoundsException e) {} +List mPools = JSONMetricUtil.getMemoryPools(); +pageContext.setAttribute("pageTitle", "Process info for PID: " + JSONMetricUtil.getProcessPID()); +%> + + + + + +
+
+ +
+ + + + + + + + + + + + + + +
StartedUptimePIDOwner
<%= new Date(runtimeBean.getStartTime()) %><%= StringUtils.humanTimeDiff(runtimeBean.getUptime()) %><%= JSONMetricUtil.getProcessPID() %><%= runtimeBean.getSystemProperties().get("user.name") %>
+
+
+
+ +
+ + + + + + + + + + + + + + + + + +
ThreadsNewThreadsRunableThreadsBlockedThreadsWaitingThreadsTimeWaitingThreadsTerminated
<%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsNew") %><%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsRunnable")%><%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsBlocked")%><%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsWaiting")%><%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsTimedWaiting")%><%= JSONMetricUtil.getValueFromMBean(jvmMetrics, "ThreadsTerminated")%>
+
+
+
+ +
+ <% if (gcBeans.size() == 2) { %> +
+ +
+
+ + + + + + + + + + + +
Collection CountCollection TimeLast duration
<%= collector1.getCollectionCount() %> <%= StringUtils.humanTimeDiff(collector1.getCollectionTime()) %> <%= StringUtils.humanTimeDiff(JSONMetricUtil.getLastGcDuration( + collector1.getObjectName())) %>
+
+
+ + + + + + + + + + + +
Collection CountCollection TimeLast duration
<%= collector2.getCollectionCount() %> <%= StringUtils.humanTimeDiff(collector2.getCollectionTime()) %> <%= StringUtils.humanTimeDiff(JSONMetricUtil.getLastGcDuration( + collector2.getObjectName())) %>
+
+
+
+ <%} else { %> +

Can not display GC Collector stats.

+ <%} %> + Total GC Collection time: <%= StringUtils.humanTimeDiff(collector1.getCollectionTime() + + collector2.getCollectionTime())%> +
+<% for(MemoryPoolMXBean mp:mPools) { +if(mp.getName().contains("Cache")) continue;%> +
+
+ +
+ + + + + + + + + + + + + + + + +
CommitedInitMaxUsedUtilization [%]
<%= TraditionalBinaryPrefix.long2String(mp.getUsage().getCommitted(), "B", 1) %><%= TraditionalBinaryPrefix.long2String(mp.getUsage().getInit(), "B", 1) %><%= TraditionalBinaryPrefix.long2String(mp.getUsage().getMax(), "B", 1) %><%= TraditionalBinaryPrefix.long2String(mp.getUsage().getUsed(), "B", 1) %><%= JSONMetricUtil.calcPercentage(mp.getUsage().getUsed(), + mp.getUsage().getCommitted()) %>
+
+<% } %> + + diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp index df8f0838d6cc..ce6725f283a7 100644 --- a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp +++ b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp @@ -18,70 +18,29 @@ */ --%> <%@ page contentType="text/html;charset=UTF-8" - import="org.apache.hadoop.conf.Configuration" - import="org.apache.hadoop.hbase.HBaseConfiguration" - import="org.apache.hadoop.hbase.rest.model.VersionModel" - import="org.apache.hadoop.hbase.util.VersionInfo" - import="java.util.Date"%> + import="org.apache.hadoop.conf.Configuration" + import="org.apache.hadoop.hbase.rest.RESTServer" + import="org.apache.hadoop.hbase.rest.model.VersionModel" + import="org.apache.hadoop.hbase.util.VersionInfo" + import="java.util.Date"%> + <% -Configuration conf = (Configuration)getServletContext().getAttribute("hbase.conf"); -long startcode = conf.getLong("startcode", System.currentTimeMillis()); -String listenPort = conf.get("hbase.rest.port", "8080"); -%> - - - - - - HBase REST Server: <%= listenPort %> - - + Configuration conf = (Configuration) getServletContext().getAttribute("hbase.conf"); + long startcode = conf.getLong("startcode", System.currentTimeMillis()); - - - - + final RESTServer restServer = (RESTServer) getServletContext().getAttribute(RESTServer.REST_SERVER); + final String hostName = restServer.getServerName().getHostname(); + pageContext.setAttribute("pageTitle", "HBase REST Server" + hostName); +%> - - + + +
@@ -124,9 +83,6 @@ String listenPort = conf.get("hbase.rest.port", "8080");
- - - - - + + From 7016c640960dd875f2daf78601dc68bfe14a1219 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Sun, 14 Jan 2024 17:18:54 -0500 Subject: [PATCH 058/109] HBASE-28307 Add hbase-openssl module and include in release binaries (#5623) Signed-off-by: Duo Zhang Signed-off-by: Nihal Jain --- hbase-assembly/pom.xml | 4 + .../hadoop/hbase/io/crypto/tls/X509Util.java | 3 +- hbase-extensions/hbase-openssl/pom.xml | 43 ++++++++ hbase-extensions/pom.xml | 99 +++++++++++++++++++ pom.xml | 11 +++ 5 files changed, 159 insertions(+), 1 deletion(-) create mode 100644 hbase-extensions/hbase-openssl/pom.xml create mode 100644 hbase-extensions/pom.xml diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml index 6a317045109b..a6e88cdbf77e 100644 --- a/hbase-assembly/pom.xml +++ b/hbase-assembly/pom.xml @@ -61,6 +61,10 @@ compile + + org.apache.hbase + hbase-openssl + org.apache.hbase hbase-server diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java index fff32866fb9a..46809050b5a3 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/tls/X509Util.java @@ -299,7 +299,8 @@ public static SslContext createSslContextForClient(Configuration config) * Adds SslProvider.OPENSSL if OpenSsl is available and enabled. In order to make it available, * one must ensure that a properly shaded netty-tcnative is on the classpath. Properly shaded * means relocated to be prefixed with "org.apache.hbase.thirdparty" like the rest of the netty - * classes. + * classes. We make available org.apache.hbase:hbase-openssl as a convenience module which one can + * use to pull in a shaded netty-tcnative statically linked against boringssl. */ private static boolean configureOpenSslIfAvailable(SslContextBuilder sslContextBuilder, Configuration conf) { diff --git a/hbase-extensions/hbase-openssl/pom.xml b/hbase-extensions/hbase-openssl/pom.xml new file mode 100644 index 000000000000..7158c2f4197e --- /dev/null +++ b/hbase-extensions/hbase-openssl/pom.xml @@ -0,0 +1,43 @@ + + + + 4.0.0 + + org.apache.hbase + hbase-extensions + ${revision} + ../pom.xml + + + hbase-openssl + jar + Apache HBase - OpenSSL support for TLS RPC + Includes tcnative bindings so that netty TLS can use OpenSSL + + + + org.apache.hbase.thirdparty + hbase-shaded-netty-tcnative + runtime + + + + diff --git a/hbase-extensions/pom.xml b/hbase-extensions/pom.xml new file mode 100644 index 000000000000..8a11e7754ea2 --- /dev/null +++ b/hbase-extensions/pom.xml @@ -0,0 +1,99 @@ + + + + 4.0.0 + + org.apache.hbase + hbase-build-configuration + ${revision} + ../hbase-build-configuration + + + hbase-extensions + pom + Apache HBase - Extensions + Parent for optional extension modules + + + hbase-openssl + + + + + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${project.basedir}/../../dev-support/spotbugs-exclude.xml + true + true + Max + + + + + maven-assembly-plugin + + true + + + + + + + + com.github.spotbugs + spotbugs-maven-plugin + + + + spotbugs + + false + + ${project.basedir}/../dev-support/spotbugs-exclude.xml + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + + maven-assembly-plugin + + true + + + + + + diff --git a/pom.xml b/pom.xml index a59379f8dc92..a32e73fb3377 100644 --- a/pom.xml +++ b/pom.xml @@ -758,6 +758,7 @@ hbase-asyncfs hbase-logging hbase-compression + hbase-extensions scm:git:git://gitbox.apache.org/repos/asf/hbase.git @@ -1331,6 +1332,11 @@ hbase-compression-zstd ${project.version} + + org.apache.hbase + hbase-openssl + ${project.version} + com.github.stephenc.findbugs @@ -1749,6 +1755,11 @@ hbase-shaded-netty ${hbase-thirdparty.version} + + org.apache.hbase.thirdparty + hbase-shaded-netty-tcnative + ${hbase-thirdparty.version} + org.apache.hbase.thirdparty hbase-shaded-protobuf From 3cfc92fa427441f1991df6ce5e6b95618710ea91 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 16 Jan 2024 12:02:01 +0800 Subject: [PATCH 059/109] HBASE-28265 Add 3.0.0-beta-1 to download page (#5624) Signed-off-by: Yi Mei Signed-off-by: Xin Sun --- src/site/xdoc/downloads.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml index dc7a3148b579..e9e9d97ebaed 100644 --- a/src/site/xdoc/downloads.xml +++ b/src/site/xdoc/downloads.xml @@ -45,27 +45,27 @@ under the License. - 3.0.0-alpha-4 + 3.0.0-beta-1 - 2023/06/07 + 2024/01/14 - 3.0.0-alpha-4 vs 2.0.0 + 3.0.0-beta-1 vs 2.0.0 - Changes + Changes - Release Notes + Release Notes - src (sha512 asc)
- bin (sha512 asc)
- client-bin (sha512 asc) + src (sha512 asc)
+ bin (sha512 asc)
+ client-bin (sha512 asc) - Testing only, not production ready + Feature freeze, passed a 10B ITBLL run, use with caution From f5fd60ea54f25e9d56c8b6d0e3783c42b169dfef Mon Sep 17 00:00:00 2001 From: jbewing Date: Tue, 16 Jan 2024 08:59:05 -0500 Subject: [PATCH 060/109] HBASE-28256 Enhance ByteBufferUtils.readVLong to read more bytes at a time (#5576) Signed-off-by: Bryan Beaudreault Signed-off-by: Duo Zhang --- .../hadoop/hbase/util/ByteBufferUtils.java | 83 ++++++++++++++----- .../hbase/util/TestByteBufferUtils.java | 18 ++++ 2 files changed, 78 insertions(+), 23 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 054de74d7d1e..d6b936323404 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -468,38 +468,75 @@ public static void writeVLong(ByteBuffer out, long i) { } } - private interface ByteVisitor { - byte get(); - } - - private static long readVLong(ByteVisitor visitor) { - byte firstByte = visitor.get(); + /** + * Similar to {@link WritableUtils#readVLong(java.io.DataInput)} but reads from a + * {@link ByteBuff}. + */ + public static long readVLong(ByteBuff buf) { + byte firstByte = buf.get(); int len = WritableUtils.decodeVIntSize(firstByte); if (len == 1) { return firstByte; + } else { + int remaining = len - 1; + long i = 0; + int offsetFromPos = 0; + if (remaining >= Bytes.SIZEOF_INT) { + // The int read has to be converted to unsigned long so the & op + i = (buf.getIntAfterPosition(offsetFromPos) & 0x00000000ffffffffL); + remaining -= Bytes.SIZEOF_INT; + offsetFromPos += Bytes.SIZEOF_INT; + } + if (remaining >= Bytes.SIZEOF_SHORT) { + short s = buf.getShortAfterPosition(offsetFromPos); + i = i << 16; + i = i | (s & 0xFFFF); + remaining -= Bytes.SIZEOF_SHORT; + offsetFromPos += Bytes.SIZEOF_SHORT; + } + for (int idx = 0; idx < remaining; idx++) { + byte b = buf.getByteAfterPosition(offsetFromPos + idx); + i = i << 8; + i = i | (b & 0xFF); + } + buf.skip(len - 1); + return WritableUtils.isNegativeVInt(firstByte) ? ~i : i; } - long i = 0; - for (int idx = 0; idx < len - 1; idx++) { - byte b = visitor.get(); - i = i << 8; - i = i | (b & 0xFF); - } - return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i); } /** * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a {@link ByteBuffer}. */ - public static long readVLong(ByteBuffer in) { - return readVLong(in::get); - } - - /** - * Similar to {@link WritableUtils#readVLong(java.io.DataInput)} but reads from a - * {@link ByteBuff}. - */ - public static long readVLong(ByteBuff in) { - return readVLong(in::get); + public static long readVLong(ByteBuffer buf) { + byte firstByte = buf.get(); + int len = WritableUtils.decodeVIntSize(firstByte); + if (len == 1) { + return firstByte; + } else { + int remaining = len - 1; + long i = 0; + int offsetFromPos = 0; + if (remaining >= Bytes.SIZEOF_INT) { + // The int read has to be converted to unsigned long so the & op + i = (buf.getInt(buf.position() + offsetFromPos) & 0x00000000ffffffffL); + remaining -= Bytes.SIZEOF_INT; + offsetFromPos += Bytes.SIZEOF_INT; + } + if (remaining >= Bytes.SIZEOF_SHORT) { + short s = buf.getShort(buf.position() + offsetFromPos); + i = i << 16; + i = i | (s & 0xFFFF); + remaining -= Bytes.SIZEOF_SHORT; + offsetFromPos += Bytes.SIZEOF_SHORT; + } + for (int idx = 0; idx < remaining; idx++) { + byte b = buf.get(buf.position() + offsetFromPos + idx); + i = i << 8; + i = i | (b & 0xFF); + } + buf.position(buf.position() + len - 1); + return WritableUtils.isNegativeVInt(firstByte) ? ~i : i; + } } /** diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java index eabfed2042ca..e07e75bffdb2 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.unsafe.HBasePlatformDependent; @@ -179,6 +180,23 @@ public void testReadWriteVLong() { ByteBufferUtils.writeVLong(b, l); b.flip(); assertEquals(l, ByteBufferUtils.readVLong(b)); + b.flip(); + assertEquals(l, ByteBufferUtils.readVLong(ByteBuff.wrap(b))); + } + } + + @Test + public void testReadWriteConsecutiveVLong() { + for (long l : testNumbers) { + ByteBuffer b = ByteBuffer.allocate(2 * MAX_VLONG_LENGTH); + ByteBufferUtils.writeVLong(b, l); + ByteBufferUtils.writeVLong(b, l - 4); + b.flip(); + assertEquals(l, ByteBufferUtils.readVLong(b)); + assertEquals(l - 4, ByteBufferUtils.readVLong(b)); + b.flip(); + assertEquals(l, ByteBufferUtils.readVLong(ByteBuff.wrap(b))); + assertEquals(l - 4, ByteBufferUtils.readVLong(ByteBuff.wrap(b))); } } From 696f58373be38a2e8c598f202cdcd874cc6306f4 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Tue, 16 Jan 2024 16:42:52 -0500 Subject: [PATCH 061/109] HBASE-28306 Add property to customize Version information (#5621) Signed-off-by: Duo Zhang --- hbase-common/pom.xml | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index fc3136e05558..dd30a7a6f581 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -31,6 +31,15 @@ Apache HBase - Common Common functionality for HBase + + + ${project.version} + + org.apache.hbase @@ -211,7 +220,7 @@ process-resources - + @@ -227,7 +236,7 @@ - + From 6017937ced115b1ad367faf9f836c3112f6b6f9d Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 17 Jan 2024 11:20:14 +0800 Subject: [PATCH 062/109] HBASE-28312 The bad auth exception can not be passed to client rpc calls properly (#5629) Signed-off-by: Bryan Beaudreault --- .../hbase/ipc/BlockingRpcConnection.java | 76 ++++++++++--------- .../apache/hadoop/hbase/ipc/ConnectionId.java | 2 +- .../org/apache/hadoop/hbase/ipc/IPCUtil.java | 19 ++++- .../hadoop/hbase/ipc/NettyRpcConnection.java | 2 +- .../hadoop/hbase/ipc/RpcConnection.java | 3 +- .../hadoop/hbase/ipc/DummyException.java | 27 +++++++ .../ipc/DummyFatalConnectionException.java | 27 +++++++ .../apache/hadoop/hbase/ipc/TestIPCUtil.java | 22 ++++++ .../ipc/NettyRpcServerPreambleHandler.java | 9 +++ .../hadoop/hbase/ipc/AbstractTestIPC.java | 23 ++++++ .../hbase/ipc/BadAuthNettyRpcConnection.java | 36 +++++++++ .../hadoop/hbase/ipc/TestBlockingIPC.java | 20 +++++ .../apache/hadoop/hbase/ipc/TestNettyIPC.java | 11 +++ .../{security => ipc}/TestNettyTlsIPC.java | 20 +++-- .../security/TestNettyTLSIPCFileWatcher.java | 7 +- .../hadoop/hbase/security/TestSaslTlsIPC.java | 2 +- 16 files changed, 254 insertions(+), 52 deletions(-) create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyException.java create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyFatalConnectionException.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/BadAuthNettyRpcConnection.java rename hbase-server/src/test/java/org/apache/hadoop/hbase/{security => ipc}/TestNettyTlsIPC.java (94%) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java index 81ad4d2f056d..f30b77c64fe9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java @@ -18,9 +18,7 @@ package org.apache.hadoop.hbase.ipc; import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader; -import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException; import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited; -import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException; import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; import static org.apache.hadoop.hbase.ipc.IPCUtil.write; @@ -68,6 +66,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator; @@ -657,6 +656,25 @@ private void readResponse() { // Read the header ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); int id = responseHeader.getCallId(); + if (LOG.isTraceEnabled()) { + LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader) + + ", totalSize: " + totalSize + " bytes"); + } + RemoteException remoteExc; + if (responseHeader.hasException()) { + ExceptionResponse exceptionResponse = responseHeader.getException(); + remoteExc = IPCUtil.createRemoteException(exceptionResponse); + if (IPCUtil.isFatalConnectionException(exceptionResponse)) { + // Here we will cleanup all calls so do not need to fall back, just return. + synchronized (this) { + closeConn(remoteExc); + } + return; + } + } else { + remoteExc = null; + } + call = calls.remove(id); // call.done have to be set before leaving this method expectedCall = (call != null && !call.isDone()); if (!expectedCall) { @@ -667,46 +685,34 @@ private void readResponse() { // this connection. int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader); int whatIsLeftToRead = totalSize - readSoFar; + LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead + + " bytes"); IOUtils.skipFully(in, whatIsLeftToRead); if (call != null) { call.callStats.setResponseSizeBytes(totalSize); - call.callStats - .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); } return; } - if (responseHeader.hasException()) { - ExceptionResponse exceptionResponse = responseHeader.getException(); - RemoteException re = createRemoteException(exceptionResponse); - call.setException(re); - call.callStats.setResponseSizeBytes(totalSize); - call.callStats - .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); - if (isFatalConnectionException(exceptionResponse)) { - synchronized (this) { - closeConn(re); - } - } - } else { - Message value = null; - if (call.responseDefaultType != null) { - Message.Builder builder = call.responseDefaultType.newBuilderForType(); - ProtobufUtil.mergeDelimitedFrom(builder, in); - value = builder.build(); - } - CellScanner cellBlockScanner = null; - if (responseHeader.hasCellBlockMeta()) { - int size = responseHeader.getCellBlockMeta().getLength(); - byte[] cellBlock = new byte[size]; - IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); - cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec, - this.compressor, cellBlock); - } - call.setResponse(value, cellBlockScanner); - call.callStats.setResponseSizeBytes(totalSize); - call.callStats - .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); + call.callStats.setResponseSizeBytes(totalSize); + if (remoteExc != null) { + call.setException(remoteExc); + return; + } + Message value = null; + if (call.responseDefaultType != null) { + Message.Builder builder = call.responseDefaultType.newBuilderForType(); + ProtobufUtil.mergeDelimitedFrom(builder, in); + value = builder.build(); + } + CellScanner cellBlockScanner = null; + if (responseHeader.hasCellBlockMeta()) { + int size = responseHeader.getCellBlockMeta().getLength(); + byte[] cellBlock = new byte[size]; + IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); + cellBlockScanner = + this.rpcClient.cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock); } + call.setResponse(value, cellBlockScanner); } catch (IOException e) { if (expectedCall) { call.setException(e); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java index 4de82e0c12af..f0b00c459d06 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java @@ -27,7 +27,7 @@ * uniquely identified by <remoteAddress, ticket, serviceName> */ @InterfaceAudience.Private -class ConnectionId { +public class ConnectionId { private static final int PRIME = 16777619; final User ticket; final String serviceName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java index d6df6c974ccf..bf4b833e856c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; @@ -62,6 +64,8 @@ @InterfaceAudience.Private class IPCUtil { + private static final Logger LOG = LoggerFactory.getLogger(IPCUtil.class); + /** * Write out header, param, and cell block if there is one. * @param dos Stream to write into @@ -159,8 +163,19 @@ static RemoteException createRemoteException(final ExceptionResponse e) { } /** Returns True if the exception is a fatal connection exception. */ - static boolean isFatalConnectionException(final ExceptionResponse e) { - return e.getExceptionClassName().equals(FatalConnectionException.class.getName()); + static boolean isFatalConnectionException(ExceptionResponse e) { + if (e.getExceptionClassName().equals(FatalConnectionException.class.getName())) { + return true; + } + // try our best to check for sub classes of FatalConnectionException + try { + return e.getExceptionClassName() != null && FatalConnectionException.class.isAssignableFrom( + Class.forName(e.getExceptionClassName(), false, IPCUtil.class.getClassLoader())); + // Class.forName may throw ExceptionInInitializerError so we have to catch Throwable here + } catch (Throwable t) { + LOG.debug("Can not get class object for {}", e.getExceptionClassName(), t); + return false; + } } static IOException toIOE(Throwable t) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java index 408ea347e7a3..4d8564be7a25 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -79,7 +79,7 @@ * @since 2.0.0 */ @InterfaceAudience.Private -class NettyRpcConnection extends RpcConnection { +public class NettyRpcConnection extends RpcConnection { private static final Logger LOG = LoggerFactory.getLogger(NettyRpcConnection.class); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java index 31698a1a1e8e..dbe6ed1648df 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java @@ -145,7 +145,8 @@ public void run(Timeout timeout) throws Exception { } } - protected final byte[] getConnectionHeaderPreamble() { + // will be overridden in tests + protected byte[] getConnectionHeaderPreamble() { // Assemble the preamble up in a buffer first and then send it. Writing individual elements, // they are getting sent across piecemeal according to wireshark and then server is messing // up the reading on occasion (the passed in stream is not buffered yet). diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyException.java new file mode 100644 index 000000000000..407c1248a98d --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyException.java @@ -0,0 +1,27 @@ +/* + * 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.ipc; + +/** + * Just a dummy exception for testing IPCUtil.isFatalConnectionException. + */ +public class DummyException extends Exception { + + private static final long serialVersionUID = 215191975455115118L; + +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyFatalConnectionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyFatalConnectionException.java new file mode 100644 index 000000000000..437b60b031b6 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/DummyFatalConnectionException.java @@ -0,0 +1,27 @@ +/* + * 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.ipc; + +/** + * Just a dummy exception for testing IPCUtil.isFatalConnectionException. + */ +public class DummyFatalConnectionException extends FatalConnectionException { + + private static final long serialVersionUID = -1966815615846798490L; + +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java index 67a8d15c1d02..d0e4044b0456 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -19,6 +19,7 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -44,6 +45,8 @@ import org.apache.hbase.thirdparty.io.netty.channel.DefaultEventLoop; import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; + @Category({ ClientTests.class, SmallTests.class }) public class TestIPCUtil { @@ -159,4 +162,23 @@ public void run() { eventLoop.shutdownGracefully().get(); } } + + @Test + public void testIsFatalConnectionException() { + // intentionally not reference the class object directly, so here we will not load the class, to + // make sure that in isFatalConnectionException, we can use initialized = false when calling + // Class.forName + ExceptionResponse resp = ExceptionResponse.newBuilder() + .setExceptionClassName("org.apache.hadoop.hbase.ipc.DummyFatalConnectionException").build(); + assertTrue(IPCUtil.isFatalConnectionException(resp)); + + resp = ExceptionResponse.newBuilder() + .setExceptionClassName("org.apache.hadoop.hbase.ipc.DummyException").build(); + assertFalse(IPCUtil.isFatalConnectionException(resp)); + + // class not found + resp = ExceptionResponse.newBuilder() + .setExceptionClassName("org.apache.hadoop.hbase.ipc.WhatEver").build(); + assertFalse(IPCUtil.isFatalConnectionException(resp)); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java index b79a67f986e8..02e1b5858117 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java @@ -38,6 +38,7 @@ class NettyRpcServerPreambleHandler extends SimpleChannelInboundHandler private final NettyRpcServer rpcServer; private final NettyServerRpcConnection conn; + private boolean processPreambleError; public NettyRpcServerPreambleHandler(NettyRpcServer rpcServer, NettyServerRpcConnection conn) { this.rpcServer = rpcServer; @@ -46,10 +47,18 @@ public NettyRpcServerPreambleHandler(NettyRpcServer rpcServer, NettyServerRpcCon @Override protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { + if (processPreambleError) { + // if we failed to process preamble, we will close the connection immediately, but it is + // possible that we have already received some bytes after the 'preamble' so when closing, the + // netty framework will still pass them here. So we set a flag here to just skip processing + // these broken messages. + return; + } ByteBuffer buf = ByteBuffer.allocate(msg.readableBytes()); msg.readBytes(buf); buf.flip(); if (!conn.processPreamble(buf)) { + processPreambleError = true; conn.close(); return; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index fe947d33110d..a93f54d4d9d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -29,9 +29,11 @@ import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -53,6 +55,7 @@ import java.net.InetSocketAddress; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; @@ -545,4 +548,24 @@ public void testTracingErrorIpc() throws IOException { hasTraceId(traceRule.getSpans().iterator().next().getTraceId())))); } } + + protected abstract AbstractRpcClient createBadAuthRpcClient(Configuration conf); + + @Test + public void testBadPreambleHeader() throws IOException, ServiceException { + Configuration clientConf = new Configuration(CONF); + RpcServer rpcServer = createRpcServer("testRpcServer", Collections.emptyList(), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); + try (AbstractRpcClient client = createBadAuthRpcClient(clientConf)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + ServiceException se = assertThrows(ServiceException.class, + () -> stub.echo(null, EchoRequestProto.newBuilder().setMessage("hello").build())); + IOException ioe = ProtobufUtil.handleRemoteException(se); + assertThat(ioe, instanceOf(BadAuthException.class)); + assertThat(ioe.getMessage(), containsString("authName=unknown")); + } finally { + rpcServer.stop(); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/BadAuthNettyRpcConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/BadAuthNettyRpcConnection.java new file mode 100644 index 000000000000..63554421dfb6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/BadAuthNettyRpcConnection.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.io.IOException; + +public class BadAuthNettyRpcConnection extends NettyRpcConnection { + + public BadAuthNettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) + throws IOException { + super(rpcClient, remoteId); + } + + @Override + protected byte[] getConnectionHeaderPreamble() { + byte[] header = super.getConnectionHeaderPreamble(); + // set an invalid auth code + header[header.length - 1] = -10; + return header; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java index 4d7d0996fabd..9544e8c35458 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java @@ -107,4 +107,24 @@ protected RpcServer createTestFailingRpcServer(String name, Configuration conf, RpcScheduler scheduler) throws IOException { return new TestFailingRpcServer(null, name, services, bindAddress, conf, scheduler); } + + @Override + protected AbstractRpcClient createBadAuthRpcClient(Configuration conf) { + return new BlockingRpcClient(conf) { + + @Override + protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new BlockingRpcConnection(this, remoteId) { + @Override + protected byte[] getConnectionHeaderPreamble() { + byte[] header = super.getConnectionHeaderPreamble(); + // set an invalid auth code + header[header.length - 1] = -10; + return header; + } + }; + } + + }; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java index 265ae7852f02..6feab5f2cac8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java @@ -146,4 +146,15 @@ protected RpcServer createTestFailingRpcServer(String name, Configuration conf, RpcScheduler scheduler) throws IOException { return new FailingNettyRpcServer(null, name, services, bindAddress, conf, scheduler); } + + @Override + protected AbstractRpcClient createBadAuthRpcClient(Configuration conf) { + return new NettyRpcClient(conf) { + + @Override + protected NettyRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new BadAuthNettyRpcConnection(this, remoteId); + } + }; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java similarity index 94% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java index f21e3b93bef5..4c654123e130 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTlsIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.security; +package org.apache.hadoop.hbase.ipc; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -37,13 +37,6 @@ import org.apache.hadoop.hbase.io.crypto.tls.X509TestContext; import org.apache.hadoop.hbase.io.crypto.tls.X509TestContextProvider; import org.apache.hadoop.hbase.io.crypto.tls.X509Util; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; -import org.apache.hadoop.hbase.ipc.AbstractTestIPC; -import org.apache.hadoop.hbase.ipc.FailingNettyRpcServer; -import org.apache.hadoop.hbase.ipc.NettyRpcClient; -import org.apache.hadoop.hbase.ipc.NettyRpcServer; -import org.apache.hadoop.hbase.ipc.RpcScheduler; -import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; @@ -193,4 +186,15 @@ protected RpcServer createTestFailingRpcServer(String name, RpcScheduler scheduler) throws IOException { return new FailingNettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler); } + + @Override + protected AbstractRpcClient createBadAuthRpcClient(Configuration conf) { + return new NettyRpcClient(conf) { + + @Override + protected NettyRpcConnection createConnection(ConnectionId remoteId) throws IOException { + return new BadAuthNettyRpcConnection(this, remoteId); + } + }; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java index 72fc7141680a..403a538f024b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestNettyTLSIPCFileWatcher.java @@ -109,14 +109,15 @@ public static List data() { @BeforeClass public static void setUpBeforeClass() throws IOException { Security.addProvider(new BouncyCastleProvider()); - File dir = new File(UTIL.getDataTestDir(TestNettyTlsIPC.class.getSimpleName()).toString()) - .getCanonicalFile(); + File dir = + new File(UTIL.getDataTestDir(TestNettyTLSIPCFileWatcher.class.getSimpleName()).toString()) + .getCanonicalFile(); FileUtils.forceMkdir(dir); // server must enable tls CONF.setBoolean(X509Util.HBASE_SERVER_NETTY_TLS_ENABLED, true); PROVIDER = new X509TestContextProvider(CONF, dir); EVENT_LOOP_GROUP_CONFIG = - NettyEventLoopGroupConfig.setup(CONF, TestNettyTlsIPC.class.getSimpleName()); + NettyEventLoopGroupConfig.setup(CONF, TestNettyTLSIPCFileWatcher.class.getSimpleName()); SERVER = mock(HBaseServerBase.class); when(SERVER.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java index 1477e8aa0fca..1120d56fb9fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSaslTlsIPC.java @@ -97,7 +97,7 @@ public static List data() { @BeforeClass public static void setUpBeforeClass() throws Exception { Security.addProvider(new BouncyCastleProvider()); - File dir = new File(TEST_UTIL.getDataTestDir(TestNettyTlsIPC.class.getSimpleName()).toString()) + File dir = new File(TEST_UTIL.getDataTestDir(TestSaslTlsIPC.class.getSimpleName()).toString()) .getCanonicalFile(); FileUtils.forceMkdir(dir); initKDCAndConf(); From 6b0ce08c82cfe2e7d34d6b62578e71aa2ef8b5f5 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 17 Jan 2024 13:36:06 +0800 Subject: [PATCH 063/109] HBASE-28312 Addendum NettyRpcConnection and ConnectionId do not need to be public --- .../src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java | 2 +- .../java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java index f0b00c459d06..4de82e0c12af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java @@ -27,7 +27,7 @@ * uniquely identified by <remoteAddress, ticket, serviceName> */ @InterfaceAudience.Private -public class ConnectionId { +class ConnectionId { private static final int PRIME = 16777619; final User ticket; final String serviceName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java index 4d8564be7a25..408ea347e7a3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -79,7 +79,7 @@ * @since 2.0.0 */ @InterfaceAudience.Private -public class NettyRpcConnection extends RpcConnection { +class NettyRpcConnection extends RpcConnection { private static final Logger LOG = LoggerFactory.getLogger(NettyRpcConnection.class); From c001ed3ba3c25a0d1a684c393092756b7ad9ee25 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 18 Jan 2024 16:59:01 +0800 Subject: [PATCH 064/109] HBASE-28316 Add BootstrapNodeService handlers (#5637) Signed-off-by: Bryan Beaudreault --- .../client/RegistryEndpointsRefresher.java | 2 +- .../hadoop/hbase/security/SecurityInfo.java | 6 + .../apache/hadoop/hbase/ipc/RpcServer.java | 7 ++ .../hbase/regionserver/RSRpcServices.java | 13 ++- .../hbase/security/HBasePolicyProvider.java | 7 +- .../hbase/client/TestBootstrapNodeUpdate.java | 103 ++++++++++++++++++ ...curityInfoAndHBasePolicyProviderMatch.java | 82 ++++++++++++++ 7 files changed, 216 insertions(+), 4 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecurityInfoAndHBasePolicyProviderMatch.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegistryEndpointsRefresher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegistryEndpointsRefresher.java index ac7cad275813..cdb2dd92b4fc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegistryEndpointsRefresher.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegistryEndpointsRefresher.java @@ -154,7 +154,7 @@ static RegistryEndpointsRefresher create(Configuration conf, String initialDelay TimeUnit.SECONDS.toMillis(conf.getLong(initialDelaySecsConfigName, periodicRefreshMs / 10))); long minTimeBetweenRefreshesMs = TimeUnit.SECONDS .toMillis(conf.getLong(minIntervalSecsConfigName, MIN_SECS_BETWEEN_REFRESHES_DEFAULT)); - Preconditions.checkArgument(minTimeBetweenRefreshesMs < periodicRefreshMs); + Preconditions.checkArgument(minTimeBetweenRefreshesMs <= periodicRefreshMs); return new RegistryEndpointsRefresher(initialDelayMs, periodicRefreshMs, minTimeBetweenRefreshesMs, refresher); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java index dbb4c83844a4..2e16d5646953 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java @@ -23,7 +23,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; @@ -50,6 +52,10 @@ public class SecurityInfo { new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN)); infos.put(RegistryProtos.ClientMetaService.getDescriptor().getName(), new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN)); + infos.put(BootstrapNodeProtos.BootstrapNodeService.getDescriptor().getName(), + new SecurityInfo(SecurityConstants.REGIONSERVER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN)); + infos.put(LockServiceProtos.LockService.getDescriptor().getName(), + new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN)); // NOTE: IF ADDING A NEW SERVICE, BE SURE TO UPDATE HBasePolicyProvider ALSO ELSE // new Service will not be found when all is Kerberized!!!! } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 6b4bf28bc959..d3ec4ff8c73e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -19,6 +19,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -881,4 +882,10 @@ public void setNamedQueueRecorder(NamedQueueRecorder namedQueueRecorder) { protected boolean needAuthorization() { return authorize; } + + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*/src/test/.*") + public List getServices() { + return services; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index a43fac6993e6..0fe6f6476a6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -336,8 +336,8 @@ public class RSRpcServices extends HBaseRpcServicesBase /** * Services launched in RSRpcServices. By default they are on but you can use the below booleans - * to selectively enable/disable either Admin or Client Service (Rare is the case where you would - * ever turn off one or the other). + * to selectively enable/disable these services (Rare is the case where you would ever turn off + * one or the other). */ public static final String REGIONSERVER_ADMIN_SERVICE_CONFIG = "hbase.regionserver.admin.executorService"; @@ -345,6 +345,8 @@ public class RSRpcServices extends HBaseRpcServicesBase "hbase.regionserver.client.executorService"; public static final String REGIONSERVER_CLIENT_META_SERVICE_CONFIG = "hbase.regionserver.client.meta.executorService"; + public static final String REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG = + "hbase.regionserver.bootstrap.nodes.executorService"; /** * An Rpc callback for closing a RegionScanner. @@ -1449,6 +1451,8 @@ protected List getServices() { boolean client = getConfiguration().getBoolean(REGIONSERVER_CLIENT_SERVICE_CONFIG, true); boolean clientMeta = getConfiguration().getBoolean(REGIONSERVER_CLIENT_META_SERVICE_CONFIG, true); + boolean bootstrapNodes = + getConfiguration().getBoolean(REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG, true); List bssi = new ArrayList<>(); if (client) { bssi.add(new BlockingServiceAndInterface(ClientService.newReflectiveBlockingService(this), @@ -1462,6 +1466,11 @@ protected List getServices() { bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this), ClientMetaService.BlockingInterface.class)); } + if (bootstrapNodes) { + bssi.add( + new BlockingServiceAndInterface(BootstrapNodeService.newReflectiveBlockingService(this), + BootstrapNodeService.BlockingInterface.class)); + } return new ImmutableList.Builder().addAll(bssi).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java index 91323a722150..30578a91909b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java @@ -25,7 +25,9 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService; @@ -44,8 +46,11 @@ public class HBasePolicyProvider extends PolicyProvider { new Service("security.client.protocol.acl", RegistryProtos.ClientMetaService.BlockingInterface.class), new Service("security.admin.protocol.acl", MasterService.BlockingInterface.class), + new Service("security.admin.protocol.acl", LockService.BlockingInterface.class), new Service("security.masterregion.protocol.acl", - RegionServerStatusService.BlockingInterface.class) }; + RegionServerStatusService.BlockingInterface.class), + new Service("security.regionserver.protocol.acl", + BootstrapNodeService.BlockingInterface.class) }; @Override public Service[] getServices() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java new file mode 100644 index 000000000000..d5b0ee18e594 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java @@ -0,0 +1,103 @@ +/* + * 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.client; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseRpcServicesBase; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; + +/** + * Make sure that we can update the bootstrap server from master to region server, and region server + * could also contact each other to update the bootstrap nodes. + */ +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestBootstrapNodeUpdate { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBootstrapNodeUpdate.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + private static RpcConnectionRegistry REGISTRY; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + Configuration conf = UTIL.getConfiguration(); + conf.setLong(BootstrapNodeManager.REQUEST_MASTER_INTERVAL_SECS, 5); + conf.setLong(BootstrapNodeManager.REQUEST_MASTER_MIN_INTERVAL_SECS, 1); + conf.setLong(BootstrapNodeManager.REQUEST_REGIONSERVER_INTERVAL_SECS, 1); + conf.setInt(HBaseRpcServicesBase.CLIENT_BOOTSTRAP_NODE_LIMIT, 2); + conf.setLong(RpcConnectionRegistry.INITIAL_REFRESH_DELAY_SECS, 5); + conf.setLong(RpcConnectionRegistry.PERIODIC_REFRESH_INTERVAL_SECS, 1); + conf.setLong(RpcConnectionRegistry.MIN_SECS_BETWEEN_REFRESHES, 1); + UTIL.startMiniCluster(3); + REGISTRY = new RpcConnectionRegistry(conf); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + Closeables.close(REGISTRY, true); + UTIL.shutdownMiniCluster(); + } + + @Test + public void testUpdate() throws Exception { + ServerName activeMasterServerName = REGISTRY.getActiveMaster().get(); + ServerName masterInConf = ServerName.valueOf(activeMasterServerName.getHostname(), + activeMasterServerName.getPort(), -1); + // we should have master in the beginning + assertThat(REGISTRY.getParsedServers(), hasItem(masterInConf)); + // and after refreshing, we will switch to use region servers + UTIL.waitFor(15000, () -> !REGISTRY.getParsedServers().contains(masterInConf) + && !REGISTRY.getParsedServers().contains(activeMasterServerName)); + Set parsedServers = REGISTRY.getParsedServers(); + assertEquals(2, parsedServers.size()); + // now kill one region server + ServerName serverToKill = parsedServers.iterator().next(); + UTIL.getMiniHBaseCluster().killRegionServer(serverToKill); + // wait until the region server disappears + // since the min node limit is 2, this means region server will still contact each other for + // getting bootstrap nodes, instead of requesting master directly, so this assert can make sure + // that the getAllBootstrapNodes works fine, and also the client can communicate with region + // server to update bootstrap nodes + UTIL.waitFor(30000, () -> !REGISTRY.getParsedServers().contains(serverToKill)); + // should still have 2 servers, the remaining 2 live region servers + assertEquals(2, parsedServers.size()); + // make sure the registry still works fine + assertNotNull(REGISTRY.getClusterId().get()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecurityInfoAndHBasePolicyProviderMatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecurityInfoAndHBasePolicyProviderMatch.java new file mode 100644 index 000000000000..4a664d4d4d0e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecurityInfoAndHBasePolicyProviderMatch.java @@ -0,0 +1,82 @@ +/* + * 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.security; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertNotNull; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; +import org.apache.hadoop.hbase.ipc.RpcServerInterface; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.security.authorize.Service; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Make sure that all rpc services for master and region server are properly configured in + * {@link SecurityInfo} and {@link HBasePolicyProvider}. + */ +@Category({ SecurityTests.class, SmallTests.class }) +public class TestSecurityInfoAndHBasePolicyProviderMatch { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSecurityInfoAndHBasePolicyProviderMatch.class); + + private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + UTIL.startMiniCluster(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + UTIL.shutdownMiniCluster(); + } + + private void assertServiceMatches(RpcServerInterface rpcServer) { + HBasePolicyProvider provider = new HBasePolicyProvider(); + Set> serviceClasses = + Stream.of(provider.getServices()).map(Service::getProtocol).collect(Collectors.toSet()); + for (BlockingServiceAndInterface bsai : ((RpcServer) rpcServer).getServices()) { + assertNotNull( + "no security info for " + bsai.getBlockingService().getDescriptorForType().getName(), + SecurityInfo.getInfo(bsai.getBlockingService().getDescriptorForType().getName())); + assertThat(serviceClasses, hasItem(bsai.getServiceInterface())); + } + } + + @Test + public void testMatches() { + assertServiceMatches( + UTIL.getMiniHBaseCluster().getMaster().getMasterRpcServices().getRpcServer()); + assertServiceMatches(UTIL.getMiniHBaseCluster().getRegionServer(0).getRpcServer()); + } +} From fdde2273006dc3b227d82b297b548885bb9cb48a Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Thu, 18 Jan 2024 07:41:25 -0500 Subject: [PATCH 065/109] HBASE-28319 Expose DelegatingRpcScheduler as IA.LimitedPrivate (#5638) Signed-off-by: Duo Zhang --- .../hadoop/hbase/ipc/DelegatingRpcScheduler.java | 11 +++++++++++ 1 file changed, 11 insertions(+) rename hbase-server/src/{test => main}/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java (86%) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java similarity index 86% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java index f8ac4a1bb9a1..52bd43cb7f62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java @@ -17,6 +17,17 @@ */ package org.apache.hadoop.hbase.ipc; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +/** + * Users of the hbase.region.server.rpc.scheduler.factory.class customization config can return an + * implementation which extends this class in order to minimize impact of breaking interface + * changes. + */ +@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX }) +@InterfaceStability.Evolving public class DelegatingRpcScheduler extends RpcScheduler { protected RpcScheduler delegate; From 2f6b6aaaaad6f036a5604378cfa1b84d4a245fca Mon Sep 17 00:00:00 2001 From: Xin Sun Date: Mon, 22 Jan 2024 14:02:11 +0800 Subject: [PATCH 066/109] HBASE-28324 TestRegionNormalizerWorkQueue#testTake is flaky (#5643) Signed-off-by: Duo Zhang --- .../normalizer/TestRegionNormalizerWorkQueue.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorkQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorkQueue.java index c6d14c191145..088df7e7376e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorkQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorkQueue.java @@ -22,7 +22,6 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.util.ArrayList; @@ -41,6 +40,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.ClassRule; @@ -186,6 +187,7 @@ public void testTake() throws Exception { final RegionNormalizerWorkQueue queue = new RegionNormalizerWorkQueue<>(); final ConcurrentLinkedQueue takeTimes = new ConcurrentLinkedQueue<>(); final AtomicBoolean finished = new AtomicBoolean(false); + final int count = 5; final Runnable consumer = () -> { try { while (!finished.get()) { @@ -199,11 +201,12 @@ public void testTake() throws Exception { CompletableFuture worker = CompletableFuture.runAsync(consumer); final long testStart = System.nanoTime(); - for (int i = 0; i < 5; i++) { + for (int i = 0; i < count; i++) { Thread.sleep(10); queue.put(i); } - + // should have timing information for 5 calls to take. + Waiter.waitFor(HBaseConfiguration.create(), 1000, () -> count == takeTimes.size()); // set finished = true and pipe one more value in case the thread needs an extra pass through // the loop. finished.set(true); @@ -211,9 +214,7 @@ public void testTake() throws Exception { worker.get(1, TimeUnit.SECONDS); final Iterator times = takeTimes.iterator(); - assertTrue("should have timing information for at least 2 calls to take.", - takeTimes.size() >= 5); - for (int i = 0; i < 5; i++) { + for (int i = 0; i < count; i++) { assertThat( "Observations collected in takeTimes should increase by roughly 10ms every interval", times.next(), greaterThan(testStart + TimeUnit.MILLISECONDS.toNanos(i * 10))); From 877830e06c875f9ae1e0d84104c6733cf935babd Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Mon, 22 Jan 2024 17:18:06 -0900 Subject: [PATCH 067/109] HBASE-28271 Infinite waiting on lock acquisition by snapshot can result in unresponsive master (#5603) Signed-off-by: Hui Ruan Signed-off-by: Duo Zhang --- .../hbase/master/locking/LockManager.java | 9 - .../master/snapshot/TakeSnapshotHandler.java | 38 ++++- .../TestSnapshotProcedureWithLockTimeout.java | 159 ++++++++++++++++++ 3 files changed, 190 insertions(+), 16 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedureWithLockTimeout.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java index 7fab616449c2..eae9c7dcc1b7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java @@ -108,15 +108,6 @@ public MasterLock(final RegionInfo[] regionInfos, final String description) { this.description = description; } - /** - * Acquire the lock, waiting indefinitely until the lock is released or the thread is - * interrupted. - * @throws InterruptedException If current thread is interrupted while waiting for the lock - */ - public boolean acquire() throws InterruptedException { - return tryAcquire(0); - } - /** * Acquire the lock within a wait time. * @param timeoutMs The maximum time (in milliseconds) to wait for the lock, 0 to wait diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java index 58ecaca09ec6..f746adf0b89e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java @@ -17,12 +17,16 @@ */ package org.apache.hadoop.hbase.master.snapshot; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_RPC_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY; + import java.io.IOException; import java.util.List; import java.util.concurrent.CancellationException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -65,6 +69,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements SnapshotSentinel, ForeignExceptionSnare { private static final Logger LOG = LoggerFactory.getLogger(TakeSnapshotHandler.class); + public static final String HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT = + "hbase.snapshot.master.lock.acquire.timeout"; private volatile boolean finished; @@ -85,6 +91,13 @@ public abstract class TakeSnapshotHandler extends EventHandler protected final TableName snapshotTable; protected final SnapshotManifest snapshotManifest; protected final SnapshotManager snapshotManager; + /** + * Snapshot creation requires table lock. If any region of the table is in transition, table lock + * cannot be acquired by LockProcedure and hence snapshot creation could hang for potentially very + * long time. This timeout will ensure snapshot creation fails-fast by waiting for only given + * timeout. + */ + private final long lockAcquireTimeoutMs; protected TableDescriptor htd; @@ -129,6 +142,8 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, true); this.snapshotManifest = SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status); + this.lockAcquireTimeoutMs = conf.getLong(HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT, + conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT)); } private TableDescriptor loadTableDescriptor() throws IOException { @@ -147,12 +162,16 @@ private TableDescriptor loadTableDescriptor() throws IOException { public TakeSnapshotHandler prepare() throws Exception { super.prepare(); // after this, you should ensure to release this lock in case of exceptions - this.tableLock.acquire(); - try { - this.htd = loadTableDescriptor(); // check that .tableinfo is present - } catch (Exception e) { - this.tableLock.release(); - throw e; + if (this.tableLock.tryAcquire(this.lockAcquireTimeoutMs)) { + try { + this.htd = loadTableDescriptor(); // check that .tableinfo is present + } catch (Exception e) { + this.tableLock.release(); + throw e; + } + } else { + LOG.error("Master lock could not be acquired in {} ms", lockAcquireTimeoutMs); + throw new DoNotRetryIOException("Master lock could not be acquired"); } return this; } @@ -176,7 +195,12 @@ public void process() { tableLockToRelease = master.getLockManager().createMasterLock(snapshotTable, LockType.SHARED, this.getClass().getName() + ": take snapshot " + snapshot.getName()); tableLock.release(); - tableLockToRelease.acquire(); + boolean isTableLockAcquired = tableLockToRelease.tryAcquire(this.lockAcquireTimeoutMs); + if (!isTableLockAcquired) { + LOG.error("Could not acquire shared lock on table {} in {} ms", snapshotTable, + lockAcquireTimeoutMs); + throw new IOException("Could not acquire shared lock on table " + snapshotTable); + } } // If regions move after this meta scan, the region specific snapshot should fail, triggering // an external exception that gets captured here. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedureWithLockTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedureWithLockTimeout.java new file mode 100644 index 000000000000..f780d68acf49 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedureWithLockTimeout.java @@ -0,0 +1,159 @@ +/* + * 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.procedure; + +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.SnapshotDescription; +import org.apache.hadoop.hbase.client.SnapshotType; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RegionSplitter; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; + +/** + * Snapshot creation with master lock timeout test. + */ +@Category({ MasterTests.class, MediumTests.class }) +public class TestSnapshotProcedureWithLockTimeout { + + private static final Logger LOG = + LoggerFactory.getLogger(TestSnapshotProcedureWithLockTimeout.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSnapshotProcedureWithLockTimeout.class); + + private static HBaseTestingUtil TEST_UTIL; + private HMaster master; + private TableName TABLE_NAME; + private byte[] CF; + private String SNAPSHOT_NAME; + + @Before + public void setup() throws Exception { + TEST_UTIL = new HBaseTestingUtil(); + Configuration config = TEST_UTIL.getConfiguration(); + config.setInt("hbase.snapshot.remote.verify.threshold", 1); + config.setLong(TakeSnapshotHandler.HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT, 1L); + TEST_UTIL.startMiniCluster(3); + master = TEST_UTIL.getHBaseCluster().getMaster(); + TABLE_NAME = TableName.valueOf(Bytes.toBytes("TestSnapshotProcedureWithLockTimeout")); + CF = Bytes.toBytes("cf"); + SNAPSHOT_NAME = "SnapshotProcLockTimeout"; + final byte[][] splitKeys = new RegionSplitter.HexStringSplit().split(10); + Table table = TEST_UTIL.createTable(TABLE_NAME, CF, splitKeys); + TEST_UTIL.loadTable(table, CF, false); + } + + @After + public void teardown() throws Exception { + if (this.master != null) { + ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(master.getMasterProcedureExecutor(), + false); + } + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testTakeZkCoordinatedSnapshot() { + for (int i = 0; i < 10; i++) { + try { + // Verify that snapshot creation is not possible because lock could not be + // acquired on time. This can be flaky behavior because even though we provide 1ms + // as lock timeout, it could still be fast enough and eventually lead to successful + // snapshot creation. If that happens, retry again. + testTakeZkCoordinatedSnapshot(i); + break; + } catch (Exception e) { + LOG.error("Error because of faster lock acquisition. retrying....", e); + } + assertNotEquals("Retries exhausted", 9, i); + } + } + + private void testTakeZkCoordinatedSnapshot(int i) throws Exception { + SnapshotDescription snapshotOnSameTable = + new SnapshotDescription(SNAPSHOT_NAME + i, TABLE_NAME, SnapshotType.SKIPFLUSH); + SnapshotProtos.SnapshotDescription snapshotOnSameTableProto = + ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotOnSameTable); + Thread second = new Thread("zk-snapshot") { + @Override + public void run() { + try { + master.getSnapshotManager().takeSnapshot(snapshotOnSameTableProto); + } catch (IOException e) { + LOG.error("zk snapshot failed", e); + fail("zk snapshot failed"); + } + } + }; + second.start(); + + Thread.sleep(5000); + boolean snapshotCreated = false; + try { + SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotOnSameTableProto, TABLE_NAME, + CF); + snapshotCreated = true; + } catch (AssertionError e) { + LOG.error("Assertion error..", e); + if ( + e.getMessage() != null && e.getMessage().contains("target snapshot directory") + && e.getMessage().contains("doesn't exist.") + ) { + LOG.debug("Expected behaviour - snapshot could not be created"); + } else { + throw new IOException(e); + } + } + + if (snapshotCreated) { + throw new IOException("Snapshot created successfully"); + } + + // ensure all scheduled procedures are successfully completed + TEST_UTIL.waitFor(4000, 400, + () -> master.getMasterProcedureExecutor().getProcedures().stream() + .filter(masterProcedureEnvProcedure -> masterProcedureEnvProcedure.getState() + == ProcedureProtos.ProcedureState.SUCCESS) + .count() == master.getMasterProcedureExecutor().getProcedures().size()); + } +} From b8cea19d4a65e625f497eb6082de5cfcd23144c8 Mon Sep 17 00:00:00 2001 From: Nick Dimiduk Date: Tue, 23 Jan 2024 14:21:29 +0100 Subject: [PATCH 068/109] HBASE-28325 Enable infra automation to comment on a Jira when a new PR is posted (#5645) Signed-off-by: Bryan Beaudreault Signed-off-by: Duo Zhang --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index a991e8528d78..3b7cf932e3b6 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -38,4 +38,4 @@ notifications: commits: commits@hbase.apache.org issues: issues@hbase.apache.org pullrequests: issues@hbase.apache.org - jira_options: link + jira_options: link comment From 3a91c5b6981023a11b272f76a502891c3728f9b2 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Tue, 23 Jan 2024 11:05:11 -0500 Subject: [PATCH 069/109] HBASE-28302 Add tracking of fs read times in ScanMetrics and slow logs (#5622) Signed-off-by: Nick Dimiduk --- .../hadoop/hbase/client/OnlineLogRecord.java | 29 +++-- .../client/metrics/ServerSideScanMetrics.java | 4 + .../hbase/shaded/protobuf/ProtobufUtil.java | 1 + .../hbase/client/TestOnlineLogRecord.java | 24 ++-- .../hadoop/hbase/io/MetricsIOSource.java | 5 + .../hadoop/hbase/io/MetricsIOSourceImpl.java | 8 ++ .../protobuf/server/region/TooSlowLog.proto | 2 + .../org/apache/hadoop/hbase/io/MetricsIO.java | 4 + .../apache/hadoop/hbase/io/hfile/HFile.java | 7 +- .../hadoop/hbase/io/hfile/HFileBlock.java | 5 +- .../org/apache/hadoop/hbase/ipc/RpcCall.java | 4 + .../apache/hadoop/hbase/ipc/RpcServer.java | 14 ++- .../apache/hadoop/hbase/ipc/ServerCall.java | 11 ++ .../hbase/namequeues/RpcLogDetails.java | 9 +- .../namequeues/impl/SlowLogQueueService.java | 4 +- .../hbase/regionserver/RSRpcServices.java | 3 + ...stServerSideScanMetricsFromClientSide.java | 32 ++++- .../namequeues/TestNamedQueueRecorder.java | 18 ++- .../hbase/namequeues/TestRpcLogDetails.java | 12 +- .../hbase/namequeues/TestTooLargeLog.java | 2 +- .../region/TestRegionProcedureStore.java | 10 ++ .../hadoop/hbase/thrift2/ThriftUtilities.java | 1 + .../thrift2/generated/TOnlineLogRecord.java | 111 +++++++++++++++++- .../apache/hadoop/hbase/thrift2/hbase.thrift | 1 + 24 files changed, 275 insertions(+), 46 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java index d9fd51e80a95..26979129cf18 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java @@ -81,6 +81,7 @@ final public class OnlineLogRecord extends LogEntry { private final int queueTime; private final long responseSize; private final long blockBytesScanned; + private final long fsReadTime; private final String clientAddress; private final String serverClass; private final String methodName; @@ -120,6 +121,10 @@ public long getBlockBytesScanned() { return blockBytesScanned; } + public long getFsReadTime() { + return fsReadTime; + } + public String getClientAddress() { return clientAddress; } @@ -178,16 +183,18 @@ public Map getConnectionAttributes() { } OnlineLogRecord(final long startTime, final int processingTime, final int queueTime, - final long responseSize, final long blockBytesScanned, final String clientAddress, - final String serverClass, final String methodName, final String callDetails, final String param, - final String regionName, final String userName, final int multiGetsCount, - final int multiMutationsCount, final int multiServiceCalls, final Scan scan, - final Map requestAttributes, final Map connectionAttributes) { + final long responseSize, final long blockBytesScanned, final long fsReadTime, + final String clientAddress, final String serverClass, final String methodName, + final String callDetails, final String param, final String regionName, final String userName, + final int multiGetsCount, final int multiMutationsCount, final int multiServiceCalls, + final Scan scan, final Map requestAttributes, + final Map connectionAttributes) { this.startTime = startTime; this.processingTime = processingTime; this.queueTime = queueTime; this.responseSize = responseSize; this.blockBytesScanned = blockBytesScanned; + this.fsReadTime = fsReadTime; this.clientAddress = clientAddress; this.serverClass = serverClass; this.methodName = methodName; @@ -209,6 +216,7 @@ public static class OnlineLogRecordBuilder { private int queueTime; private long responseSize; private long blockBytesScanned; + private long fsReadTime; private String clientAddress; private String serverClass; private String methodName; @@ -251,6 +259,11 @@ public OnlineLogRecordBuilder setBlockBytesScanned(long blockBytesScanned) { return this; } + public OnlineLogRecordBuilder setFsReadTime(long fsReadTime) { + this.fsReadTime = fsReadTime; + return this; + } + public OnlineLogRecordBuilder setClientAddress(String clientAddress) { this.clientAddress = clientAddress; return this; @@ -319,9 +332,9 @@ public OnlineLogRecordBuilder setRequestAttributes(Map requestAt public OnlineLogRecord build() { return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize, - blockBytesScanned, clientAddress, serverClass, methodName, callDetails, param, regionName, - userName, multiGetsCount, multiMutationsCount, multiServiceCalls, scan, requestAttributes, - connectionAttributes); + blockBytesScanned, fsReadTime, clientAddress, serverClass, methodName, callDetails, param, + regionName, userName, multiGetsCount, multiMutationsCount, multiServiceCalls, scan, + requestAttributes, connectionAttributes); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java index cf730501be0a..ff83584ccb44 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java @@ -50,6 +50,8 @@ protected AtomicLong createCounter(String counterName) { public static final String BLOCK_BYTES_SCANNED_KEY_METRIC_NAME = "BLOCK_BYTES_SCANNED"; + public static final String FS_READ_TIME_METRIC_NAME = "FS_READ_TIME"; + /** * number of rows filtered during scan RPC */ @@ -65,6 +67,8 @@ protected AtomicLong createCounter(String counterName) { public final AtomicLong countOfBlockBytesScanned = createCounter(BLOCK_BYTES_SCANNED_KEY_METRIC_NAME); + public final AtomicLong fsReadTime = createCounter(FS_READ_TIME_METRIC_NAME); + public void setCounter(String counterName, long value) { AtomicLong c = this.counters.get(counterName); if (c != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 598ad932e679..e50b54e8eb02 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -3462,6 +3462,7 @@ private static LogEntry getSlowLogRecord(final TooSlowLog.SlowLogPayload slowLog .setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName()) .setResponseSize(slowLogPayload.getResponseSize()) .setBlockBytesScanned(slowLogPayload.getBlockBytesScanned()) + .setFsReadTime(slowLogPayload.getFsReadTime()) .setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime()) .setUserName(slowLogPayload.getUserName()) .setRequestAttributes(convertNameBytesPairsToMap(slowLogPayload.getRequestAttributeList())) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java index fe753973ae20..a16993d56591 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOnlineLogRecord.java @@ -46,15 +46,15 @@ public void itSerializesScan() { scan.withStopRow(Bytes.toBytes(456)); String expectedOutput = "{\n" + " \"startTime\": 1,\n" + " \"processingTime\": 2,\n" + " \"queueTime\": 3,\n" + " \"responseSize\": 4,\n" + " \"blockBytesScanned\": 5,\n" - + " \"multiGetsCount\": 6,\n" + " \"multiMutationsCount\": 7,\n" + " \"scan\": {\n" - + " \"startRow\": \"\\\\x00\\\\x00\\\\x00{\",\n" + + " \"fsReadTime\": 6,\n" + " \"multiGetsCount\": 6,\n" + " \"multiMutationsCount\": 7,\n" + + " \"scan\": {\n" + " \"startRow\": \"\\\\x00\\\\x00\\\\x00{\",\n" + " \"stopRow\": \"\\\\x00\\\\x00\\\\x01\\\\xC8\",\n" + " \"batch\": -1,\n" + " \"cacheBlocks\": true,\n" + " \"totalColumns\": 0,\n" + " \"maxResultSize\": -1,\n" + " \"families\": {},\n" + " \"caching\": -1,\n" + " \"maxVersions\": 1,\n" + " \"timeRange\": [\n" + " 0,\n" + " 9223372036854775807\n" + " ]\n" + " }\n" + "}"; - OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null, - 6, 7, 0, scan, Collections.emptyMap(), Collections.emptyMap()); + OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null, + null, 6, 7, 0, scan, Collections.emptyMap(), Collections.emptyMap()); String actualOutput = o.toJsonPrettyPrint(); System.out.println(actualOutput); Assert.assertEquals(actualOutput, expectedOutput); @@ -67,8 +67,8 @@ public void itSerializesRequestAttributes() { Set expectedOutputs = ImmutableSet. builder().add("requestAttributes").add("\"r\": \"1\"") .add("\"2\": \"\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\"").build(); - OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null, - 6, 7, 0, null, requestAttributes, Collections.emptyMap()); + OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null, + null, 6, 7, 0, null, requestAttributes, Collections.emptyMap()); String actualOutput = o.toJsonPrettyPrint(); System.out.println(actualOutput); expectedOutputs.forEach(expected -> Assert.assertTrue(actualOutput.contains(expected))); @@ -76,8 +76,8 @@ ImmutableSet. builder().add("requestAttributes").add("\"r\": \"1\"") @Test public void itOmitsEmptyRequestAttributes() { - OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null, - 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap()); + OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null, + null, 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap()); String actualOutput = o.toJsonPrettyPrint(); System.out.println(actualOutput); Assert.assertFalse(actualOutput.contains("requestAttributes")); @@ -90,8 +90,8 @@ public void itSerializesConnectionAttributes() { Set expectedOutputs = ImmutableSet. builder().add("connectionAttributes").add("\"c\": \"1\"") .add("\"2\": \"\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\"").build(); - OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null, - 6, 7, 0, null, Collections.emptyMap(), connectionAttributes); + OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null, + null, 6, 7, 0, null, Collections.emptyMap(), connectionAttributes); String actualOutput = o.toJsonPrettyPrint(); System.out.println(actualOutput); expectedOutputs.forEach(expected -> Assert.assertTrue(actualOutput.contains(expected))); @@ -99,8 +99,8 @@ ImmutableSet. builder().add("connectionAttributes").add("\"c\": \"1\"") @Test public void itOmitsEmptyConnectionAttributes() { - OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, null, null, null, null, null, null, null, - 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap()); + OnlineLogRecord o = new OnlineLogRecord(1, 2, 3, 4, 5, 6, null, null, null, null, null, null, + null, 6, 7, 0, null, Collections.emptyMap(), Collections.emptyMap()); String actualOutput = o.toJsonPrettyPrint(); System.out.println(actualOutput); Assert.assertFalse(actualOutput.contains("connectionAttributes")); diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java index af7e87483d1d..5b4fc4b2c69a 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSource.java @@ -47,6 +47,9 @@ public interface MetricsIOSource extends BaseSource { String FS_PREAD_TIME_HISTO_KEY = "fsPReadTime"; String FS_WRITE_HISTO_KEY = "fsWriteTime"; + String SLOW_FS_READS_KEY = "fsSlowReadsCount"; + String SLOW_FS_READS_DESC = "Number of HFile reads which were slower than a configured threshold"; + String CHECKSUM_FAILURES_KEY = "fsChecksumFailureCount"; String FS_READ_TIME_HISTO_DESC = @@ -76,4 +79,6 @@ public interface MetricsIOSource extends BaseSource { * @param t time it took, in milliseconds */ void updateFsWriteTime(long t); + + void incrSlowFsRead(); } diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java index 6ef5d180cd5e..5aca9a3d84b9 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/io/MetricsIOSourceImpl.java @@ -22,6 +22,7 @@ import org.apache.hadoop.metrics2.MetricsCollector; import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.lib.Interns; +import org.apache.hadoop.metrics2.lib.MutableFastCounter; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private @@ -32,6 +33,7 @@ public class MetricsIOSourceImpl extends BaseSourceImpl implements MetricsIOSour private final MetricHistogram fsReadTimeHisto; private final MetricHistogram fsPReadTimeHisto; private final MetricHistogram fsWriteTimeHisto; + private final MutableFastCounter fsSlowReads; public MetricsIOSourceImpl(MetricsIOWrapper wrapper) { this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, wrapper); @@ -49,6 +51,7 @@ public MetricsIOSourceImpl(String metricsName, String metricsDescription, String getMetricsRegistry().newTimeHistogram(FS_PREAD_TIME_HISTO_KEY, FS_PREAD_TIME_HISTO_DESC); fsWriteTimeHisto = getMetricsRegistry().newTimeHistogram(FS_WRITE_HISTO_KEY, FS_WRITE_TIME_HISTO_DESC); + fsSlowReads = getMetricsRegistry().newCounter(SLOW_FS_READS_KEY, SLOW_FS_READS_DESC, 0L); } @Override @@ -66,6 +69,11 @@ public void updateFsWriteTime(long t) { fsWriteTimeHisto.add(t); } + @Override + public void incrSlowFsRead() { + fsSlowReads.incr(); + } + @Override public void getMetrics(MetricsCollector metricsCollector, boolean all) { MetricsRecordBuilder mrb = metricsCollector.addRecord(metricsName); diff --git a/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto b/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto index 4c275948b277..80b984e999cd 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/region/TooSlowLog.proto @@ -53,6 +53,8 @@ message SlowLogPayload { repeated NameBytesPair connection_attribute = 18; repeated NameBytesPair request_attribute = 19; + optional int64 fs_read_time = 20; + // SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large. // Majority of times, slow logs are also large logs and hence, ALL is combination of // both diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java index 58e6f7d01b71..4d2437d418bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java @@ -75,4 +75,8 @@ public void updateFsPreadTime(long t) { public void updateFsWriteTime(long t) { source.updateFsWriteTime(t); } + + public void incrSlowFsRead() { + source.incrSlowFsRead(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 207c99866511..84fe9387d6e9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType; +import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.ShipperListener; import org.apache.hadoop.hbase.util.BloomFilterWriter; @@ -187,12 +188,16 @@ public static final long getChecksumFailuresCount() { return CHECKSUM_FAILURES.sum(); } - public static final void updateReadLatency(long latencyMillis, boolean pread) { + public static final void updateReadLatency(long latencyMillis, boolean pread, boolean tooSlow) { + RpcServer.getCurrentCall().ifPresent(call -> call.updateFsReadTime(latencyMillis)); if (pread) { MetricsIO.getInstance().updateFsPreadTime(latencyMillis); } else { MetricsIO.getInstance().updateFsReadTime(latencyMillis); } + if (tooSlow) { + MetricsIO.getInstance().incrSlowFsRead(); + } } public static final void updateWriteLatency(long latencyMillis) { 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 a3ead34730fb..47c20b691b4a 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 @@ -1826,8 +1826,9 @@ protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset, int sizeWithoutChecksum = curBlock.getInt(Header.ON_DISK_DATA_SIZE_WITH_HEADER_INDEX); curBlock.limit(sizeWithoutChecksum); long duration = EnvironmentEdgeManager.currentTime() - startTime; + boolean tooSlow = this.readWarnTime >= 0 && duration > this.readWarnTime; if (updateMetrics) { - HFile.updateReadLatency(duration, pread); + HFile.updateReadLatency(duration, pread, tooSlow); } // The onDiskBlock will become the headerAndDataBuffer for this block. // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already @@ -1839,7 +1840,7 @@ protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset, hFileBlock.sanityCheckUncompressed(); } LOG.trace("Read {} in {} ms", hFileBlock, duration); - if (!LOG.isTraceEnabled() && this.readWarnTime >= 0 && duration > this.readWarnTime) { + if (!LOG.isTraceEnabled() && tooSlow) { LOG.warn("Read Block Slow: read {} cost {} ms, threshold = {} ms", hFileBlock, duration, this.readWarnTime); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java index 260d6e1a9803..2d06aa7c47af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java @@ -132,4 +132,8 @@ public interface RpcCall extends RpcCallContext { /** Returns A short string format of this call without possibly lengthy params */ String toShortString(); + + void updateFsReadTime(long latencyMillis); + + long getFsReadTime(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index d3ec4ff8c73e..0876a1fd55f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -444,14 +444,17 @@ public Pair call(RpcCall call, MonitoredRPCHandler status) int totalTime = (int) (endTime - receiveTime); if (LOG.isTraceEnabled()) { LOG.trace( - "{}, response: {}, receiveTime: {}, queueTime: {}, processingTime: {}, totalTime: {}", + "{}, response: {}, receiveTime: {}, queueTime: {}, processingTime: {}, " + + "totalTime: {}, fsReadTime: {}", CurCall.get().toString(), TextFormat.shortDebugString(result), - CurCall.get().getReceiveTime(), qTime, processingTime, totalTime); + CurCall.get().getReceiveTime(), qTime, processingTime, totalTime, + CurCall.get().getFsReadTime()); } // Use the raw request call size for now. long requestSize = call.getSize(); long responseSize = result.getSerializedSize(); long responseBlockSize = call.getBlockBytesScanned(); + long fsReadTime = call.getFsReadTime(); if (call.isClientCellBlockSupported()) { // Include the payload size in HBaseRpcController responseSize += call.getResponseCellSize(); @@ -472,13 +475,13 @@ public Pair call(RpcCall call, MonitoredRPCHandler status) // note that large responses will often also be slow. logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")", tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize, - responseBlockSize, userName); + responseBlockSize, fsReadTime, userName); if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) { // send logs to ring buffer owned by slowLogRecorder final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName(); this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(), - responseSize, responseBlockSize, className, tooSlow, tooLarge)); + responseSize, responseBlockSize, fsReadTime, className, tooSlow, tooLarge)); } } return new Pair<>(result, controller.cellScanner()); @@ -522,7 +525,7 @@ public Pair call(RpcCall call, MonitoredRPCHandler status) */ void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow, String clientAddress, long startTime, int processingTime, int qTime, long responseSize, - long blockBytesScanned, String userName) { + long blockBytesScanned, long fsReadTime, String userName) { final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName(); // base information that is reported regardless of type of call Map responseInfo = new HashMap<>(); @@ -531,6 +534,7 @@ void logResponse(Message param, String methodName, String call, boolean tooLarge responseInfo.put("queuetimems", qTime); responseInfo.put("responsesize", responseSize); responseInfo.put("blockbytesscanned", blockBytesScanned); + responseInfo.put("fsreadtime", fsReadTime); responseInfo.put("client", clientAddress); responseInfo.put("class", className); responseInfo.put("method", methodName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index ed688977b963..a2c578fd6664 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -99,6 +99,7 @@ public abstract class ServerCall implements RpcCa private long responseCellSize = 0; private long responseBlockSize = 0; + private long fsReadTimeMillis = 0; // cumulative size of serialized exceptions private long exceptionSize = 0; private final boolean retryImmediatelySupported; @@ -567,4 +568,14 @@ public int getRemotePort() { public synchronized BufferChain getResponse() { return response; } + + @Override + public void updateFsReadTime(long latencyMillis) { + fsReadTimeMillis += latencyMillis; + } + + @Override + public long getFsReadTime() { + return fsReadTimeMillis; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java index 235d82302d64..263fff66a738 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java @@ -42,6 +42,7 @@ public class RpcLogDetails extends NamedQueuePayload { private final String clientAddress; private final long responseSize; private final long blockBytesScanned; + private final long fsReadTime; private final String className; private final boolean isSlowLog; private final boolean isLargeLog; @@ -49,12 +50,14 @@ public class RpcLogDetails extends NamedQueuePayload { private final Map requestAttributes; public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize, - long blockBytesScanned, String className, boolean isSlowLog, boolean isLargeLog) { + long blockBytesScanned, long fsReadTime, String className, boolean isSlowLog, + boolean isLargeLog) { super(SLOW_LOG_EVENT); this.rpcCall = rpcCall; this.clientAddress = clientAddress; this.responseSize = responseSize; this.blockBytesScanned = blockBytesScanned; + this.fsReadTime = fsReadTime; this.className = className; this.isSlowLog = isSlowLog; this.isLargeLog = isLargeLog; @@ -92,6 +95,10 @@ public long getBlockBytesScanned() { return blockBytesScanned; } + public long getFsReadTime() { + return fsReadTime; + } + public String getClassName() { return className; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java index fb29b8563ef7..ea4e286bf43f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java @@ -124,6 +124,7 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) { final String clientAddress = rpcLogDetails.getClientAddress(); final long responseSize = rpcLogDetails.getResponseSize(); final long blockBytesScanned = rpcLogDetails.getBlockBytesScanned(); + final long fsReadTime = rpcLogDetails.getFsReadTime(); final String className = rpcLogDetails.getClassName(); final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails); if (type == null) { @@ -168,7 +169,8 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) { .setProcessingTime(processingTime).setQueueTime(qTime) .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY) .setResponseSize(responseSize).setBlockBytesScanned(blockBytesScanned) - .setServerClass(className).setStartTime(startTime).setType(type).setUserName(userName) + .setFsReadTime(fsReadTime).setServerClass(className).setStartTime(startTime).setType(type) + .setUserName(userName) .addAllRequestAttribute(buildNameBytesPairs(rpcLogDetails.getRequestAttributes())) .addAllConnectionAttribute(buildNameBytesPairs(rpcLogDetails.getConnectionAttributes())); if (slowLogParams != null && slowLogParams.getScan() != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 0fe6f6476a6c..05d7c2e56055 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -3454,6 +3454,9 @@ private void scan(HBaseRpcController controller, ScanRequest request, RegionScan // from block size progress before writing into the response scannerContext.getMetrics().countOfBlockBytesScanned .set(scannerContext.getBlockSizeProgress()); + if (rpcCall != null) { + scannerContext.getMetrics().fsReadTime.set(rpcCall.getFsReadTime()); + } Map metrics = scannerContext.getMetrics().getMetricsMap(); ScanMetrics.Builder metricBuilder = ScanMetrics.newBuilder(); NameInt64Pair.Builder pairBuilder = NameInt64Pair.newBuilder(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java index b80cd207683c..6ac87ce03026 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -195,6 +196,18 @@ public void testRowsSeenMetric() throws Exception { } } + @Test + public void testFsReadTimeMetric() throws Exception { + // write some new puts and flush, as an easy way to ensure the read blocks are not cached + // so that we go into the fs write code path + List puts = createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE); + TABLE.put(puts); + TEST_UTIL.flush(TABLE_NAME); + Scan scan = new Scan(); + scan.setScanMetricsEnabled(true); + testMetric(scan, ServerSideScanMetrics.FS_READ_TIME_METRIC_NAME, 0, CompareOperator.GREATER); + } + private void testRowsSeenMetric(Scan baseScan) throws Exception { Scan scan; scan = new Scan(baseScan); @@ -333,6 +346,11 @@ private void testRowsFilteredMetric(Scan baseScan, Filter filter, int expectedNu * @throws Exception on unexpected failure */ private void testMetric(Scan scan, String metricKey, long expectedValue) throws Exception { + testMetric(scan, metricKey, expectedValue, CompareOperator.EQUAL); + } + + private void testMetric(Scan scan, String metricKey, long expectedValue, + CompareOperator compareOperator) throws Exception { assertTrue("Scan should be configured to record metrics", scan.isScanMetricsEnabled()); ResultScanner scanner = TABLE.getScanner(scan); // Iterate through all the results @@ -341,11 +359,17 @@ private void testMetric(Scan scan, String metricKey, long expectedValue) throws } scanner.close(); ScanMetrics metrics = scanner.getScanMetrics(); - assertTrue("Metrics are null", metrics != null); + assertNotNull("Metrics are null", metrics); assertTrue("Metric : " + metricKey + " does not exist", metrics.hasCounter(metricKey)); final long actualMetricValue = metrics.getCounter(metricKey).get(); - assertEquals( - "Metric: " + metricKey + " Expected: " + expectedValue + " Actual: " + actualMetricValue, - expectedValue, actualMetricValue); + if (compareOperator == CompareOperator.EQUAL) { + assertEquals( + "Metric: " + metricKey + " Expected: " + expectedValue + " Actual: " + actualMetricValue, + expectedValue, actualMetricValue); + } else { + assertTrue( + "Metric: " + metricKey + " Expected: > " + expectedValue + " Actual: " + actualMetricValue, + actualMetricValue > expectedValue); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java index 35a1757115c9..00953353187e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java @@ -667,13 +667,13 @@ public void testOnlineSlowLogConnectionAttributes() throws Exception { static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className, int forcedParamIndex) { RpcCall rpcCall = getRpcCall(userName, forcedParamIndex); - return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true, + return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className, true, true); } static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) { RpcCall rpcCall = getRpcCall(userName); - return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true, + return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className, true, true); } @@ -685,8 +685,8 @@ private static RpcLogDetails getRpcLogDetailsOfScan() { private RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className, boolean isSlowLog, boolean isLargeLog) { RpcCall rpcCall = getRpcCall(userName); - return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, isSlowLog, - isLargeLog); + return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, 0, className, + isSlowLog, isLargeLog); } private static RpcCall getRpcCall(String userName) { @@ -859,6 +859,16 @@ public long getResponseExceptionSize() { @Override public void incrementResponseExceptionSize(long exceptionSize) { } + + @Override + public void updateFsReadTime(long latencyMillis) { + + } + + @Override + public long getFsReadTime() { + return 0; + } }; return rpcCall; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java index 8a93f2d0ff54..67d8a2579097 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java @@ -80,7 +80,7 @@ public void itDeepCopiesRpcLogDetailsParams() throws IOException { ProtobufUtil.mergeFrom(messageBuilder, cis, buffer.capacity()); Message message = messageBuilder.build(); RpcLogDetails rpcLogDetails = - new RpcLogDetails(getRpcCall(message), message, null, 0L, 0L, null, true, false); + new RpcLogDetails(getRpcCall(message), message, null, 0L, 0L, 0, null, true, false); // log's scan should be equal ClientProtos.Scan logScan = ((ClientProtos.ScanRequest) rpcLogDetails.getParam()).getScan(); @@ -258,6 +258,16 @@ public long getResponseExceptionSize() { @Override public void incrementResponseExceptionSize(long exceptionSize) { } + + @Override + public void updateFsReadTime(long latencyMillis) { + + } + + @Override + public long getFsReadTime() { + return 0; + } }; return rpcCall; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java index da3d97547645..fdc3e288bfed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestTooLargeLog.java @@ -118,6 +118,6 @@ public void testLogLargeBlockBytesScanned() throws IOException { record.getBlockBytesScanned() >= 100); assertTrue("expected " + record.getResponseSize() + " to be < 100", record.getResponseSize() < 100); - + assertTrue("expected " + record.getFsReadTime() + " to be > 0", record.getFsReadTime() > 0); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java index 305f0e29e952..d069c2560a55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java @@ -320,6 +320,16 @@ public long getResponseExceptionSize() { @Override public void incrementResponseExceptionSize(long exceptionSize) { } + + @Override + public void updateFsReadTime(long latencyMillis) { + + } + + @Override + public long getFsReadTime() { + return 0; + } }; } } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java index 76bc96df05c5..a02f944e12a7 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java @@ -1647,6 +1647,7 @@ public static LogQueryFilter getSlowLogQueryFromThrift(TLogQueryFilter tLogQuery tOnlineLogRecord.setRegionName(slowLogRecord.getRegionName()); tOnlineLogRecord.setResponseSize(slowLogRecord.getResponseSize()); tOnlineLogRecord.setBlockBytesScanned(slowLogRecord.getBlockBytesScanned()); + tOnlineLogRecord.setFsReadTime(slowLogRecord.getFsReadTime()); tOnlineLogRecord.setServerClass(slowLogRecord.getServerClass()); tOnlineLogRecord.setStartTime(slowLogRecord.getStartTime()); tOnlineLogRecord.setUserName(slowLogRecord.getUserName()); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java index 672b8b96d551..c3d6cba7ad20 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TOnlineLogRecord.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.client.OnlineLogRecord */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2023-02-04") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2024-01-12") public class TOnlineLogRecord implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOnlineLogRecord"); @@ -30,6 +30,7 @@ public class TOnlineLogRecord implements org.apache.thrift.TBase byName = new java.util.HashMap(); @@ -112,6 +115,8 @@ public static _Fields findByThriftId(int fieldId) { return REGION_NAME; case 15: // BLOCK_BYTES_SCANNED return BLOCK_BYTES_SCANNED; + case 16: // FS_READ_TIME + return FS_READ_TIME; default: return null; } @@ -161,8 +166,9 @@ public java.lang.String getFieldName() { private static final int __MULTIMUTATIONSCOUNT_ISSET_ID = 5; private static final int __MULTISERVICECALLS_ISSET_ID = 6; private static final int __BLOCKBYTESSCANNED_ISSET_ID = 7; - private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.BLOCK_BYTES_SCANNED}; + private static final int __FSREADTIME_ISSET_ID = 8; + private short __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.REGION_NAME,_Fields.BLOCK_BYTES_SCANNED,_Fields.FS_READ_TIME}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -196,6 +202,8 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.BLOCK_BYTES_SCANNED, new org.apache.thrift.meta_data.FieldMetaData("blockBytesScanned", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.FS_READ_TIME, new org.apache.thrift.meta_data.FieldMetaData("fsReadTime", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOnlineLogRecord.class, metaDataMap); } @@ -275,6 +283,7 @@ public TOnlineLogRecord(TOnlineLogRecord other) { this.regionName = other.regionName; } this.blockBytesScanned = other.blockBytesScanned; + this.fsReadTime = other.fsReadTime; } public TOnlineLogRecord deepCopy() { @@ -306,6 +315,8 @@ public void clear() { this.regionName = null; setBlockBytesScannedIsSet(false); this.blockBytesScanned = 0; + setFsReadTimeIsSet(false); + this.fsReadTime = 0; } public long getStartTime() { @@ -667,6 +678,29 @@ public void setBlockBytesScannedIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __BLOCKBYTESSCANNED_ISSET_ID, value); } + public long getFsReadTime() { + return this.fsReadTime; + } + + public TOnlineLogRecord setFsReadTime(long fsReadTime) { + this.fsReadTime = fsReadTime; + setFsReadTimeIsSet(true); + return this; + } + + public void unsetFsReadTime() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FSREADTIME_ISSET_ID); + } + + /** Returns true if field fsReadTime is set (has been assigned a value) and false otherwise */ + public boolean isSetFsReadTime() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FSREADTIME_ISSET_ID); + } + + public void setFsReadTimeIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FSREADTIME_ISSET_ID, value); + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case START_TIME: @@ -789,6 +823,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case FS_READ_TIME: + if (value == null) { + unsetFsReadTime(); + } else { + setFsReadTime((java.lang.Long)value); + } + break; + } } @@ -840,6 +882,9 @@ public java.lang.Object getFieldValue(_Fields field) { case BLOCK_BYTES_SCANNED: return getBlockBytesScanned(); + case FS_READ_TIME: + return getFsReadTime(); + } throw new java.lang.IllegalStateException(); } @@ -881,6 +926,8 @@ public boolean isSet(_Fields field) { return isSetRegionName(); case BLOCK_BYTES_SCANNED: return isSetBlockBytesScanned(); + case FS_READ_TIME: + return isSetFsReadTime(); } throw new java.lang.IllegalStateException(); } @@ -1033,6 +1080,15 @@ public boolean equals(TOnlineLogRecord that) { return false; } + boolean this_present_fsReadTime = true && this.isSetFsReadTime(); + boolean that_present_fsReadTime = true && that.isSetFsReadTime(); + if (this_present_fsReadTime || that_present_fsReadTime) { + if (!(this_present_fsReadTime && that_present_fsReadTime)) + return false; + if (this.fsReadTime != that.fsReadTime) + return false; + } + return true; } @@ -1086,6 +1142,10 @@ public int hashCode() { if (isSetBlockBytesScanned()) hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(blockBytesScanned); + hashCode = hashCode * 8191 + ((isSetFsReadTime()) ? 131071 : 524287); + if (isSetFsReadTime()) + hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fsReadTime); + return hashCode; } @@ -1247,6 +1307,16 @@ public int compareTo(TOnlineLogRecord other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetFsReadTime(), other.isSetFsReadTime()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFsReadTime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fsReadTime, other.fsReadTime); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1359,6 +1429,12 @@ public java.lang.String toString() { sb.append(this.blockBytesScanned); first = false; } + if (isSetFsReadTime()) { + if (!first) sb.append(", "); + sb.append("fsReadTime:"); + sb.append(this.fsReadTime); + first = false; + } sb.append(")"); return sb.toString(); } @@ -1549,6 +1625,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TOnlineLogRecord st org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 16: // FS_READ_TIME + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.fsReadTime = iprot.readI64(); + struct.setFsReadTimeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1648,6 +1732,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TOnlineLogRecord s oprot.writeI64(struct.blockBytesScanned); oprot.writeFieldEnd(); } + if (struct.isSetFsReadTime()) { + oprot.writeFieldBegin(FS_READ_TIME_FIELD_DESC); + oprot.writeI64(struct.fsReadTime); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1685,13 +1774,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TOnlineLogRecord st if (struct.isSetBlockBytesScanned()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetFsReadTime()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetRegionName()) { oprot.writeString(struct.regionName); } if (struct.isSetBlockBytesScanned()) { oprot.writeI64(struct.blockBytesScanned); } + if (struct.isSetFsReadTime()) { + oprot.writeI64(struct.fsReadTime); + } } @Override @@ -1723,7 +1818,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TOnlineLogRecord str struct.setMultiMutationsCountIsSet(true); struct.multiServiceCalls = iprot.readI32(); struct.setMultiServiceCallsIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(2); + java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { struct.regionName = iprot.readString(); struct.setRegionNameIsSet(true); @@ -1732,6 +1827,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TOnlineLogRecord str struct.blockBytesScanned = iprot.readI64(); struct.setBlockBytesScannedIsSet(true); } + if (incoming.get(2)) { + struct.fsReadTime = iprot.readI64(); + struct.setFsReadTimeIsSet(true); + } } } diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift index a32f266cf313..ed3fdf32b973 100644 --- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift +++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift @@ -499,6 +499,7 @@ struct TOnlineLogRecord { 13: required i32 multiServiceCalls 14: optional string regionName 15: optional i64 blockBytesScanned + 16: optional i64 fsReadTime } // From 0c48e5a901f2ad0ef6bab689eb32e6bbd5a31c1c Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 24 Jan 2024 22:00:38 +0800 Subject: [PATCH 070/109] HBASE-28326 All nightly jobs are failing (#5646) Use downloads.a.o instead of a.o/dist Signed-off-by: Xin Sun Signed-off-by: Nihal Jain --- dev-support/Jenkinsfile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index bb79ea8928ab..7d2ef32df6eb 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -91,7 +91,7 @@ pipeline { rm -rf "${YETUS_DIR}" "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \ --working-dir "${WORKSPACE}/downloads-yetus" \ - --keys 'https://www.apache.org/dist/yetus/KEYS' \ + --keys 'https://downloads.apache.org/yetus/KEYS' \ --verify-tar-gz \ "${WORKSPACE}/yetus-${YETUS_RELEASE}-bin.tar.gz" \ "yetus/${YETUS_RELEASE}/apache-yetus-${YETUS_RELEASE}-bin.tar.gz" @@ -138,7 +138,7 @@ pipeline { echo "Ensure we have a copy of Hadoop ${HADOOP2_VERSION}" "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \ --working-dir "${WORKSPACE}/downloads-hadoop-2" \ - --keys 'http://www.apache.org/dist/hadoop/common/KEYS' \ + --keys 'https://downloads.apache.org/hadoop/common/KEYS' \ --verify-tar-gz \ "${WORKSPACE}/hadoop-${HADOOP2_VERSION}-bin.tar.gz" \ "hadoop/common/hadoop-${HADOOP2_VERSION}/hadoop-${HADOOP2_VERSION}.tar.gz" @@ -166,7 +166,7 @@ pipeline { echo "Ensure we have a copy of Hadoop ${HADOOP3_VERSION}" "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \ --working-dir "${WORKSPACE}/downloads-hadoop-3" \ - --keys 'http://www.apache.org/dist/hadoop/common/KEYS' \ + --keys 'https://downloads.apache.org/hadoop/common/KEYS' \ --verify-tar-gz \ "${WORKSPACE}/hadoop-${HADOOP3_VERSION}-bin.tar.gz" \ "hadoop/common/hadoop-${HADOOP3_VERSION}/hadoop-${HADOOP3_VERSION}.tar.gz" From 354a3a2774d203f9299b3368c1ef06a5e4234808 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Wed, 24 Jan 2024 22:02:00 +0800 Subject: [PATCH 071/109] HBASE-28322 Attach the design doc in HBASE-26220 to our code base (#5640) Signed-off-by: Yi Mei Signed-off-by: Xin Sun --- ...vers to sync the list for bootstrap node.pdf | Bin 0 -> 43316 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 dev-support/design-docs/HBASE-26220 Use P2P communicate between region servers to sync the list for bootstrap node.pdf diff --git a/dev-support/design-docs/HBASE-26220 Use P2P communicate between region servers to sync the list for bootstrap node.pdf b/dev-support/design-docs/HBASE-26220 Use P2P communicate between region servers to sync the list for bootstrap node.pdf new file mode 100644 index 0000000000000000000000000000000000000000..7fcd1f6b9cd991cf0c7d3dc63d471e535ad64d9f GIT binary patch literal 43316 zcmd3NV~}RevhK8P`)y5Q+O};Q)3$Bfwmogzwrx+lr|rJ|eP^H8`|KU}#`$?8-iTFi z#Zy_Cm07iFWz|C}CnQ2mPs0RFI)8Su3r!E81K8-BLvwM_Dw#Q28Ue_~1o#z&sTo)p z80Y}X4n_bu204I%jg^&?wV8pQqY*&g$kD~f$Qod8WMXDx4RA2BcQ&$j065wJ9Ner8 z0FI_c0829mM}V=7JwV^a#?isiUe6X_ZDVLe0Zl7sZ)50WU}O&3>Pr<_j{Z;%+JH~%%X`lz7ho%*f2GGjb*jwpY{u9Lf zPmqY2rQ=s8v?7*Y8Vea2e02^@D`jMD;%EwBW?*LL;Q@T9Wu#{X?V5R^y&g@{4Cj?o zok|c&6rWGShuG8Jbes9}6Mz8%bc1;O=_ki+HEmq7!a?bD?+|$nQnLLip6&B8t=O$Y!K=9N z_IA6S^YQ2I*d|-9<tkR5}>!$s*BcyJyp zKbK-@q#8p_l6tuQ`s#;JC z4cmVJw?$>{58^lm>I?Unu~e;*NqCcRqqnq6 z8H3-_EFA!8>EerQx%YEz+d*&^n6kC3Bp|$9m>AiVm_<{_bcxVnVnZqt?>nJ95A+Qv z>J*aftmF8E-V9<=o%W9M4Lsh=X-s|kAXc94s+s$c9Ua^0k)L2Jm$pWQ<;#3rsEWBN z(17Aj#Y1l&%6`F`##srtLL(zi(w#*xCg{K=j1;TkWa@YUiMpPiaYJ__lR zNylP2cIMd!aP<)!y|)0DUCt}@<3%Qbl07}elQAW$b=f7FeTe4?7h_7Uw1!}D+LT9$ zC*vexlFBI0F4rUrlvX%NS@*=koJ)hu5+? z=6aJtzM78S9gbcHcPPy>GX3!%%&GQ+i{7d!#3{{Q^y;BZ)Rx!{(KwepzMR$kRnQ`N z^(U!p{0x`gcd5tC%bV_V{#WoxLo!hO3s;9p7#Cd0j!(V?HG?IgxSaVB z&bzQGWiJbvcy`-`l(28!KbE*^z_~{Vtf2oum6%`#GEwk;{wU5&>x!2K>ER`8;1f15 zb_i8ZP#=epgG|sEU`8#@jB_D9k2BtDNQ7oM`=ECsTVAI$`O%SvRJHmO3Q)2EQ@?Jf zBl5I1#Az*nA9#b1?^67}I_M0C!FXjZ=(vgBz-8qTWBfDHkhBogFE7p(r%OU=pa$LW zH2Q`gQvynV8dPd>J9i9feKkL<5S+~lAMF=xO=Xr#5aN1Eo%lG5G{gqi1vMxwYR22g0`eT`*o5iY^#_z zBkJAcc^xJQRiZ;Gwl>-7)ylIV1>Q)j0ME&CySpAJ4%F4(gg%AZ@}Eo95>rX&DN1}% zW;XFIb0NMjtyELz0q`BSf8u^Cuhm>Xu%z+#_3H#5_1?i#JVvCbIVSi?{9W_h`^?{( z%aZsz=MAc+PC%V#ZyZ3l1d9X_0 zcks6ihlT!Z!w1HbI>zi>kjvk# zaEWAd=;bDe`6gEC7zZc6yoDDVx%$l#n*^lBU@M8mDko2^Z0RS06lC>0vtaLvi!2S( zTlZc=2Ex;%BCZzewabdhuV5*#aGDC>Y<=GnBzyOsT!Xrco<#KBbFRZJ&8tqN_q+l! zaXNsX=Mjx;#(?Q|zP;s&P8V;?h32or4uepi)F7`98UFRk1wF&-Ye) zEBEMHYSZGDQFcZ})rBCBIjv3JoiKPe_~O0N?<8B?c%td>Y)?0a)d5eFKlqp>j7=lf zR@sB2TE?VVcE%L+1s~R!WM|P3FRQ@Ja2LVj7!E4zv8*GuV7@~a*$F>Y;B(z{a$@HG z9<)P-IVf;C-nA`v$yRS1%StBC`yP}kGVVQZSW`-P=Z{dGz#p~(1wr2oC9rXig(c2r zDtXb=I=}oKcLKMpAQHR%hoLb^$rIgjNH}4!R@`D{2~7)4fpAA*<1$-ZpjtqLmXOuz z`5826K260eI`T0(a*}&R{~OYq`lyI#Uu?wn<-2;L6HY5(;oGa7DN@;cYD?9h>K;(R zGQz|3yr*l=wP;Qk)k0xEKfmZfWy-qh#+Z~l&gCh|lCR8PUReo8liKdgzk?6^m` zuhKyJcWQ&6kA?`$;a? z+p#2`Q;$mWP*fXKyE#NIsKO0MFHLJYOi~-xp%2jMd-bSaWfi|XnH2P+-Of1 zc&5esa#{G((_ zq>yP&wXx9WG=tP*;kZXBhMHMoNQ5nyV!G`Ub%HJXwPHh(#c(+*=R?UbzFgDP5L;s! zH+@Pr>8M(z%4+1m-tTU|PY6_08KPtFg87D5ycOV%O#E-qHfFZbcpXJ&ASy@a1SFxc zP#%FeAW*;eYikhpJ+~W-yr-FO_E4ZfL(`h`DMLMYirANLPQNo9uiY?@dRExt_O1#H z2y<|g3EodV;?NJb5_@3L_&ZKlzT27CQKzE|GL3+YWE8U?XpIXVC=nJZhAN1d-)_Z- zM!VZPhryLu*H|K?MkP&Tat>vk8&^&wI}08PU6JM8Rbjz_?@IoV6ZVHFWQ}43NX4;2 zdRB_S7lKZOVJa0Ci%PS9`DGQl-8^6; zyXVBjF8RCJhCSy7Ile3vw;Owz?2+`(oR&HCLvB+~fXO$q*bwMm60g##KW^!xN8c49 z=d~5x2O$!g(R#V(1CPy!*fb0rj}aK44%DF#4sJN+9AknSZ(vGh zu@$E?K-2^}i$TC=WoqdhdS-yA|L9@B_=w&$)2cz`Q9VcngMtZJ(LyuMd1%ZMIU{h- zR(k!+T>|$Kd|*V7$KNX^ml47(+;en4qR+*VCeMd;=EmF&1wpd7`x_df!ummsk+^Y_ zC~8*q9f#(LIAaQ(VTTsCf%$ki3q2L zv-S?!M(lRH-@GT{V1^T>n~Xr$9=22cmR`&yE>_3kO6!&%@+24%wz=B1@$%cw#pJ7O zW1z7!`_27)S|QI~FPEJ0ZznWcVc_^R4HTqAa7SKliUr?tE4k9#bADySM)A;EzR6$1 zgZNE7ZfFUwo7^r`Yy`GZ#&iQ6khhM&8sV(zW}EBD+kNXk_8J%zBkF0gOmdO<;%f;( zm_KCOuraR&OI!G4k_xnfH^ymJGor;6bu0S%w9ZvTDEYS$8vT`F+gL(>dax!LwA{lM zhBq9#2Oem~>D(vQAUF6%G^kS zrBWh4dhr=pXo;*&g$rQ?H~bN>vzlvN0gU8OB`Rr;Kv(0{ET?xC)ZIWeLyP$PdWCHL z5~|E+Pa&?_%s>0}{WT$!#o0Go!&zQ)|!y;`_TaG_^8{uE~Z;0@2Lg1^E+mwOmO% z=#^_CSs6Xt5B(tx<$<4C#a-sd+)M+9FFt{#O$%T-DhU`HVzF)43%{8cg`b^Vj+O+I z>WSi+N}a`;e!5(Yot5%JX)^SNn@ zR)MoV`~rbRm(&c`J9xq$?%$YJhL5WbQXUf|s)iEBDT~}Iw|=@awT1d3>Taq$LFgpwJ@IYhIak^~*M0wrd&CNhGbssJc`S zL-Cd!a>JpE8+eWEvT#u3Tunckn*(=M0Qs9lkM+yFqm@KFC5t_FAqDwR;T7!yX6{jZ zWHPf;mFkqRG5I%3n#Qc;-rk+&AM=|KqJxBS2Z{cU>1^wnh(r?S~ffV$Q;(AVvy4}M%kzBOM$-ox*)$Ym#dp}aJ)=T%?&tPBI{KLiD zi?~2sGGb2kiDr$hzb7FktUBSf(7g1oAR7NTNUrG$#I)$hc@i;}#7&szh_XBmd%C=7 zi8WeekwQ+y>_xhTI$5^N_?`7UDCA`QKvO#$+0E;1KO|y0gc|f6N}}}MsPBzvLKGZ> z5t#JRNW!dWX6ce-qt;FpL^T)@b7idMl}=xzL*V4qOp0~e#=SM2>+w`ujxZZ9JAHbJ zJ-caiC86q1y@Sjw$7xnE*#e^MTSJvvsM0{42Dm-x$*FtDW~ta7xw!(rr+DmxIJ%7pvlUj>#HT^6dHs3jp}jJ7a}VA(t~~X`(D@B*NiIPlz_NzSHbv z^nl1yT($Iv%t>+EIg4PGI8U)}&aE)^aINT&kO4Cq1z2}~Qlo)5vM}j&4=|+GF5!k8 z{7A0Mqr9U17zFX-`}e-~-neHVi$yEnYT1W=u1V(Yb$0u$lw0bAQwwpD%M}e3D0ENS zLQ%pl`tnM7BsddiLFz|uam6Z`KYjv8?Jf^I5#qf`V)>XKFFzaXY4Yp>^uZn#adO3F z!?LeewFx#L>vxR=Zl2p3$iUGh(FWgJDnF-b09N1`yl4716Qoe1KflKia3Mw0CQjr+=b_en(X z>tgpd@hvpQTh%bwRYYk@F=3hpDD#ts=TNCqg3ioj6rCSxSLc_u3IvCroOyB-4Flp? zq^mvBDh{+x4SNs3S*V+W>x!Cjh!=~YmR(AA0XUfV)TDLOxo_DYeOqO@NGdzmYhBT4 ziVPQc;PhLPqZFm76Aa}YV9B6nE&Uz+tr(c3rJ<)@e7V%H6O-0E2-CJJnb!=!WrqnEfX6~uOD)G|RaRwYCwAgE%M2Uz-roquSHTUUw$_zqW8IJ=k*0i6# z)o#7&GSnE$V*BNd`5fd&kqdXJw95wunhmbzaB$_OA1(GWi2{aQ^Xb2w_Y?-O4GF+6 z_2HDU;<<()NaVKTpd9AOMz-*l98&j~|WD%Fo{JsYPUJVSLA6$RHAhT!= z?Q~|5DIdsfm3|YEnPZ$GC5SY>0-1~%fhWlQ7;Kpc31f}ZOvGiIwYLah8v+@Ql-v;X zH9-yfok~+}gQ(3Oga~_Hs7sq$Yjb*G3w1i!rj;yh9}^@1)zj=^ixbVvC=;tv%UZ%? ze291f3r+%)@O(ewX`tzvZ;yl*swE`g5@JsY$0wdUc01v`>72YAJqimkkDw@N^ zxkE4-?nKY)^?Q@~Orh-ufD}firj=--4?+avSM4nbAc`eRy>m^tB^`GeT)9v^QFPUi ze>O1iMYW|u(^bdf_em=ZC>-SUG=xNrv0~BkX(bfBtI2%fNmhEqE~>jB;W$ zzxJb?&>=%rMC;?a2yW@yY8*uF=JbLrX)|$_9V3#_jG-J{5HT{nmbHb7`*fo~=vJx? zG8=UT-)p6+S4|W|@09Xu<>QsPKT@K#u1dRO<94pOUb$TNV`mg((WCe23E8DBKtOA` zei?*N5mc#L4?*`t$jrdprK* zWS-#j0pY~#?ViBL&HXvX=j~!W=Tjm3F?%Dgmk(Xp&v?dW zpRdxRjp1m;+0)?ou|92%q&Zcmsn6?C%5_GH$@{~*L!SAgETiT+s?6zlxdi1#=>3H> z@@Vd^?Anm5);VLnta^NC8gv76;@A`Mm5QGyQJYiU2ZzFG*~~go2Xv))9gOODhUME0 zT#-tdv)ikPH?J4*s6>Lo+H2CQ8QK&yDYyFBzBW$Zd+O|Y`9mF>59H&`cj>G~i4%h4 zB{o!X2o1JwpA|om3JcR4%LyZPBt6$0A3n%K??qHTFpb46RR)ePg@bR11SPdw z8QvRQvUR%wq+?54Houx%c|NTlpbpncEyu4p_`4)EtMUnSoJ4|k^J%@gy>@M8B)Sf_ zZGOdc%lQWzlo+IY`8S0q>Mp;&nQ;*dFnM2Kcqjykh*x#0wU?J(=t=l^sj{3Ntvh=B zLPHUt5|?KN2}C@PJAF~rtOVs^(CFGOs6?w0xh`CG__RA}T5`IbI84`Q4wlkll2GR) z(|R-DCBtJQL+zy8;JaG%1O!w!A=E6qn=tF-+j{pEKtOL&C>sL@MjQ3$DA$p8zo_{> zLmlh=aVHN_ECWN*b1ykc$w);xHcmb^V*D%%Ye;&AqRpp1;ejXrgb$-$_2GEf>T5DN zm$s~usRXv+I8VMl6u$_(wr&m^94Zfn`a?<=tsvQ2PuTCK6N_y6%K^-(p{O zP-}w)6S2n-Y50c%&n7Q&(t)JMrjej2YN?UTS1n#igs7R9mqQ6`7fI-_oT>LZC_f)w zemz=Gs|Ejt`M9_fmNoV!g#>}%$(&<&kASpBgnU+xS3dJaYwTCVJ*}24K(O{&wnN$g zy$KFjwId8}Ms@CvAgyX;889ic2)1t#h#w*?8p8 za{MRy}Wg+>?eW!Bhaw-O_$~nEY8GvA>Ws5!1YLmlo)#x@2}$2Yt?e!TuxT3o zvNowilFIewdL9SFT1HgC!KVX>X%3IopSO^C*k5PaC-y^BcyZ%=W{8-7Z-?H!#9*d^=F5s>m?)wz>RY6U#Yf zR08pBySL^MWZiEuvu3X{Db!0M|JZ|wkdMQh)MXuN1s_bwKIIZhGa4I=d1U;Ipll;_ znm~mc^xiZ+?mF4^)p*JX)p-eM@HB?s3kk~iVpzVYk0K^x^RfMw;SVX} zVLxIhL7m2%63xs>G$TgYVqrd_lRM7_3*M?oRtVRMW2BlC4*GBB9ZiK3@Ydx1SssXE z50;5q;px;~-3ZSif#~vjJ}l6XS*1ZMtOTi_VH%O39NwSUF0CToo4>@}syCI*K5;Np z-+Gq)U1g3@B93}_Bsv!^=a@m`q{f>#3beJw@0(s(3Y}oOnFDP;Mla!5%}O4=!GIv^4C$*$~-a5)*eEUofI6m=opqJ(QOJJeEJ<;gcD6EZ8A( zb@eK4f3ef3tvOcTbBs;wImpUg34M$N_{=oOJ?8KM(ru6&R>`3lcE1+I`@(ykWzXvU1; z$KJZzjNBS|&m{JQ?BI~ZXx}eQmYutbm<}EXS*t^nSyPYrCy}IbO#u!tBdDk~h#r8v z(!Vp)z=_H=T4Zj{v?4k@-5(l?J(Q_7+0KUc+b+V)kIgldzJ1q?QoNL3Kb~3r19y4+V)Ur0SB2xU6^7q z{vrF7+5uH$td?A`Ka1Bkr`&cxX=LMvKq6Nas>$qtbtZ;kFrT8H(&Is}GFuyh^v$!Y zCrYiH_O+a@BK>&=e(|y*Qc+>a5|i@CIp^~<#akCzs#7vb>$U05({I(* zt~GRfGkhJe$5J?_WlfQdmMgMnncs)-lyS|7fL^^lDU{zd#;{s}1;x6RnQta&mmn31 zc?7IODwI8VensOa?Lm;T0!@TL$TD6H!u2hwz($#M*|)gNkTVtugi}r3m#^C2Ls{XS zI@wuGe}7DFODyuo8wU>+jGoI?GxEGVzUS4bFN3cPL>*~uIXl{?<29m)l!J*;VH-Z2 z>P`mB`$NRNSm74qPo-WtN5*h4?Wo?Ta18ONYH^}jc54Sw+JefL+0s>DFs~xvRg-yC zVEPgyMh3CgGa86FX4#n$?nun9ErRrdmM55mVqeC>)rfMBdYr<@lEn8K+(7bcDVHng zRgX>4#fg6DB^cJk`F(k~xN|s?W(g^@M6{9$`xtq$`65mBT_6l|Ql31Kl2MnDM$C~9 zD0jV_hjKv6i>BTppv_)WMqxv({e<~wzYaWzlAssm>TKU!HkXGtBKt{E{9%qzXJ?TD zE-ha0n078#!Kn1~moBF;U;DGS|H}{%XpqD2E({-_KegJbF$MQAQ=Qh60H%2hM(hp! zlz21=CLLHkCEnG_wfH~v*nODKp+U93Vi6Ggu8DV3a{uHVx<1s?eOCx^PGreokvS%~ zr}8rv!ou#7q@+-|y>H2$E(l7XODskgxIGc;y;N{&4}d4TvK0PBuDxB%&8*T0>jb2> zP$w+19~SQ0KAv*(`b3^+Xu}QXvn^33y5M@^jS#n)9K!Hc-Uvv!d`Ayi3r5eRa^Rnc zG;3a6lw1_6E<1*Q232Xr7ZT5F`W@pduY}j37+etmHg=9xuUF_Cx;lFoadH!L3m3PC zZ#dsfArYN6vZyQFYsMVKqdGFX7PpB?DbdjRi}ojr$cwlxH_`jv-ZYwBGKv^jHL-9t z-$G}(N65&PHPS03$*5wN&*YDw{i9LzOsq%-xFKY~AyNSv#U3>fHF>*5KlY7?W9#8R`;CBq!+sYgf4;RBw2=Apl_Ou|x16rD$mw%F#6Owu-9L123L!AII$(x-^5GYo# zstU2j>S;bSofoQVkRRZI!GPb|5%8~SdwP+YEDd4VdIaL4^ z`oTp}jnZ0xbf{7#U!)Y&@;lg44|yiU%He~%Kq3?Rb)I$2FPNLP7XvAZg)}GzA2Erg z&D(ajlB#lf@HMqa`ABiRDy4{Yv$j2>00u}MyoOZx7%#}qcM}DqMb1TvtbD-Ssoicc zZIz(G|ETTH&S(s-9Y*FG9Rh-5!{XbT`a`g5eUvOau@N#spDc%CitY$rA528Ff`Z@r ziY~9nrhT1CoYxy9JhU+$Gf>{q1FMVN`9tCh-`ml}GQiZHKub;-GX1WM<2YTS5v?C~ z?)X0E&AY4U!n4e+Ccnd(X~I#fv^+^k(ywXH6Ds7S9@F~q8ZGa1v-Gfuw=egc zG1je&r(b_j-7d0HJMZO@aF?4TgFv`S0i(*rZ;pqq?y|>MDmd|`P3v{T?bGdivFj&i zVJDOcP#Zoh5@o7|Gw)5OVRO|W9};HcriCoFIsgHG~P|!!LzwwdjEyrZ3ey_oH~>xJyS){x7LaCf9#a~ zsixa^B_rK2fBKT{-OB{~l}Y z$VsWp8sstI>ofTQUuQ5c7G0jXCrKv(3Nm)lWhS0ctSf2kf1lNnVGL5TCR&A@J95xL zY@O|al&?oeHh5zy3IrAlAK~ZtiCJF8rkiJxjv3(m=a3`3jPHDxvloEVlw$*W*)jJ6 z)2fUaxoo)j69;@zSfkw&NjyRz$YAjl8^vaZ=UOLvUyqe*7{7#n(Vt4Z>SMNJ+#%Jy6 zuz?J~(e$tv9hS(XiE=6F{NWj}HALW>ghSXGJtHVX_R3`L z>^M|du}+pU2f0EPj8g0wPy@?-_Akj7#H8B19YxmCqOnF8E{96^97uE2<}ph=v|MN< z74J6aU9indyH@lHA}u>U2|28@%|Z0$G{pl%_=qmtp-k*|&mZVst?sm&6XF|H!mzy% za>+!=D;I0klf4+-$l>daV$T`E?JDp4N6r^LsQvr!6j(NQX&YXm!LHCBsai>w=zoQ*kHcUr)&?-NS`574ZG zrkhY$nHKYCFK2T|Kj>?J2lY&fcos$B-y?mSw62w(Bmj=ojlWRUk`LXM zX=YNVESY08RM{3Ue9zPdFtC6RJ*?YcR2+#-C^fXT`3yexmLk6kzx|j*#?lK5eLT~B zzAntMkbURHi_*{h&(s+E|1LGgz{v1FFZ*e&N8_@>f2GEL|9#m{2q@wk*6`FV%@Kk# zD-ilhi1Y`6n#v=a!32}0sInsCCO=oORl|mIqRwtHAzYGcR!t&y7ju z=byL8km3GE-LW`}UnRQ3!@BTyRH}7Da~5X9!-s_1cH&C*M%+9fXHU(Ut)I4@Z_|F} z`ul4h_g9&sVS4(k?@XgE`?1P562`PG0c!&bk zXG;S;KMdds6&-#!rDzY&aZ9(feqh%EuemzcH&FRDCU?ilRF&@s;UOsNt%>aoTBd@v z6F7VVLa9;D!}h-EyuW*>Eu*Y-=;TJG@mRrCX38_Y2?n9 z5Bw=Q+{^9wP}Zauz$Fo5B*aE9^EtN?M9EOCtluxrT!3x{PPK@XV!1;GtV3^ zCbQ9oW2Y;NjAhG$I+;WFMD2@xTQV~4nA#TS4F2|r<}rkuB0hGGQ{=lJ-oUOz5j)#* zHz^Hazmqoy*W*YY=t{m(syq-7a?X0-oQnv*k(4eP;fn}@&t5N-R}V?{;v4skEFPcc z2cPn>S>Xd89=|`)5y5ff%x^bZPCSCBjq_BSuxUX+4$`V!sr>94geKiB2^AAcv%-ad zZ$XTi3b|kl7#2zuM_^+xxe})FWWU3RX^6+pcI=0=4s;|9v?5W-I$vm4x^*>6K>l9s zH+sZ2o8YNA^Eu1)KC0NRhtt!sV6{gb$g-}lO9_S~`7naPCl!#GyX6`;kIB>$P?teX z@rH4(+y)xGWrC8>^MT8FB0YV=@yX{u>#~E0VH^-b*VwV3Oh^nnbcKo(6fpcuXGHq* zB*MHj0uPk?pLsxGP_24BlU-x4)N+swBvo38R~l5aPnDF$@Wk;$$}U!Fic!+%E51A9 z?m9z@Y>gFJ=m9 z6>E}E3uh#tZ*RYQRfhyyKl-<9xmhmFJRC**6kKB1DJ`jkH5_Ec3h+}Gvr7JI{xkc> z*dAd)AGxrTu0RnH3q;2$vLv<4L**~VNp1Dh@V!gNaK|M?zLN84d$t4>KdC+FIwTnr zgTr`9O?+9N?MI1V)w zR;*2-wu(MPg7Wjw%y3ZOMcf0En|MsPSyP=;_V|C6p^Xx4^2hh^vpvUS3b0+Witi+d zEx3b(o~n#%u-V;0q5=7ql2U9fjC@W%h!0Kx8cIxU_zaa!%aXx?_U|k9uchZu?%`1l zMDcX?3o8S%x}oy)^MC^U_=^@el@zwiCo1vDnqeC+E|9+K?#(7CY8$&btZqG$A#jt) z&xa?3(1V$j*n`qvkLVw__N&OW(Ahd*d~37_;v(HBaut}bMVIJp_vs*?1#bs=$KrIm z!p|h~qH^i6n(|!BZOU9fWGPoHJqYE;3;)jIpHMTb2j3bRJ_OFR{fE_b2Pb@!JdA!W z@=tQqH;{2GehAGb zIh_2092YbY6VTork zR5IsE%Q;xW1HCNkt$Atm>!WcnTdEnwa=25wbe=0Xk-HMd+kD8!Q_Gx;O`0dnsYS9{ zA)wc6h-j(<>S+*zU~LL@v(S>35M0Q^!>YVUw@?IG|47**13FToN^~<1bh;s|&`V1=c9fgkS@=!69zext9bKQ% zR;Ti$)fS}y@>MTEj=?*85ev@--u~hSL_=j{0x#~VdHTiYs)iZmWR5uAuNW(C@Vbp^ z5wJ3@DT2}r>+nz5BUE$y(VM1qxubd|8=X3Q(U;6N<<;AtUOkQMWIj2Knm;zRTVt$W zDhRP_nw2)sh_m&(tQ=!!g7Yu;NbY3)#c8_h%>~`zc<>X30+&4f7=)Udm*NMwg{$g{ z&@s|StxKdAk`44`q!n-F+T%g0E{Bav6oo-szu6nGJF-vCRst{-t$4?OB!7MUVr+FX zigPKMr{^`5J@SrcXcM#uK=3!kvmGs@<~sL|SajjFW_0s8%1z>$Y%nEBgc7Ld6{NzR zR-Z2ACcx-Hdp{f{|6UqFY*yWHWcDhPbJHSp>!8@N#;`GydI{d{_<2XaI@k8(my#Lc?@ge6lVO@!t+*3U zw*BFHxowrY4}=2d$&&xvUorf1f2HJRYXqQ`(=++vP$(EV*f`l67&!pG2qRx)4g*Ca zM}X$vJPukVBUeWNt+>?}P~aa=!GAo(wV`Q+T^&Ug9sgpGd^Hti1pLM6Q21hR(2Cet zJN{KC!USOYue!f#rHu^D^aN~N0h)AQl`QlO09JN}FEN5&Xd`P!2f$x+4+UsiMMrxl z14lVM`@bvcz9=St7XkEtHTz5cKh9EFzW)9<(dGY7gZ?`0{SSk_xGw(}gV_GRH|VcD z`hOV2@P9SvZ;$-n2nyQ&^fLXw@R>|$ zoXi+qtzE3`jI8aM{<|Hmnz{yn^(&gb=uLEA@$~;jv9K}Iu+V=+-v0*6Oi#nW$jtT? z#DB|WHnDOswAMGVV)%;Ie`x*Ry|d6W(y%czvVK|dH`Y$iNuQa?&W*{;%<|u{jBM;* zSUR?^3G+9W-rmBDiGhXD(2~Ilp!r|p@ZWO(X_Xzjjj5rPi90L3vD3e!SlC!;n3)+E zzNXOM63tkgtqt_-ZQZ^oTmL4InVpV?fsT#+YoPyzTCg&(>RDJYF&nYd{X3S4nU#i> zm686-?|);h=&V@S3>;bMOzGVI9m~l0WfU7D%U96$BcoMo{`~SuVkEeLV2PIKk&N$ zYI}9w?!4+cqG=t48zm(n@k4|>0EWs%S|#Ks;^$*Y!W)cg`4*1D5kAfq&PgK%8KIZv z$3KK;x6nI_g@%TPhcu*T>$mYLqr8pNSNh4?{pjJnWwK@RqOi@;==fOnOR;3C>=K^u zTaO<=;ACd4jqnch;#+Qt-epG~v1nwg{u~OWuQkaxbxy~L7XIMvQC8y3@WUeFp~OV( zBZ6-5GY|Srt}bWS{t-^&PZU7#y_Jhhr|Vb;m*R&K6>sEWMhT5NsvG&!)b=;n>Rskx z9X6Yx!D?!&4umriIO%<>WWxI<&W{wKi??cXhJeFiT$-CoS9aA=0NCJDAzlu5 z%u$L?fikVV7Hx@LLkq)pA~FhD0bX@B8@@(5j`k%PAYz}>ER!$P!slCyuLzi1L9sJ~ zI1^7~aVAhaLS0hCuI6|{qSJ>0`q6^axg$8yOW>#VhBOi&FaYW(YWU8Vi#mqn!A~&I z^+Y^F-~(!|hrCxTRV_cm=431~7N%8UWZ(CUL>RnG9kS)>4F~I2pNlWj$CmmHMUSE) z=81^C0!*zOXR8&Xni(-zuX2|zeL3B~u_F$^a+mPgZ*$tajoa}my8q%361gmFEu4u$ zhoiy3#1EwwpGTqg(>NHtX1m?Uv{n)i>P=0uz)1|qzU!MY;V=f_qk+6giGe%e(FE9c z_Rbnr5B+MHUWH^v40Jt$iGB^F^OOjBAMmA4P{#q%DKgp(}TgWhlwlOhmxH99Q~HfmLaB@M3!kr0y=^k zs)k=G$`_RnORm3Y)7|-Jv}&1*bWFq@B=*7$TrQrL zayoijyH7!2UDn+{HstRF^kpASB3_fEC>xVLdngv# zD{ZN@f^Q44NZYVXt`IAXdh2hWCN9B}*es^|*f&G#&5VmfQ7cZ? zdxHJs8RPUFeq=PGP4&LjuVBT-XoMe@r|KKAV;&c=3Q`B_iNy@j!3%-}i=R}>_p8sd zt7T|IHDU#GbvR=sFA;Hr_S3;0@2nQk7DXfoW5dq>d{A2j8N#uaqphiNn1 zGldnhD+s%YIb>4Re-oTzmqoeqfby6zqHpHeCWwt-QHZfC0E+;4uX%#1*f*l>D{#G} zsmyLNKF#*`p!@XJMw1%sfMB@C7jQ?;Rvj~~VzcRWBKD}3wSDtB;^UbbssR=@BN-B; zTiEBu!$jz-iSK}9jm4NpFkm0D#oO{B9Uqp<2n*56p&Guy=AuQIUnaZ|<#EK+q!;qJ z-1_AHaDjMu#e@~K)6%e|{$|GcME6ug)DQCAYykIHf1xq5qxy-K9?=QyGQ+SVT)#WZzr^#s!&Rss=fc8K|G9 zf)l7J_+sb#)*VI-Y%+|^uKIXyB>Fhtf{?>5y1mTTyDkf1@JtRNE1-OZYcAeeY&*S+ z_??^!d_6E_4zE=4jSkJ{bxv2(N9Gevby?gNV*`dDp7T!A)+xuAdh`1m_;T}W;P<#f zlR_Up_5Ff}8RNH2oj0S?A~qc*}+6HNzz5MFORL z@vYaKu1H$|`Wp`pRR(b*1%)=Qcb3zQ>SWCwv!~=0aRP?$Oi|9+o|;DxOP0>$m(@wp#A zQOEk3N8Ytd-B=1T&qG}PJQ8|Jk?514zX^*aMiQ&;T$$U$xTRJQ%f?T-rm&6-AHdwe zJ(IDE%O@$~!Of6Q4(dkIcyDk$B1|A%3rC?+qnTUctv#alrLJl3Jc+-$hR!nlKE#x=FR}?Vu;@e{#2M?0$yD#9wdb<=9hIr{}N!oQ21C$j?FJZC>Es#{WBz$F`O)@K_YFR>hl^IO@F zaWa8s{=VRSq}5g`(m=PHkD!pulvj_fjLhIC%SD;C^f78qU#W&Cesm~jtyT4Y67Brg zIrv@a6sT2Tr4io;ATbhUCc&SiFPixztaWEbn;>vpLM zCJlaqNJ1JLNR>Yt9j_N0RTB09>nJSZi0gxu(K58i#U;jiyt$1*>CRvehf%f$ z_SY@7APLMMFRG7cnu{fMs={vS({u6ogPSPAL7UKf&u+CFrX;vyP~dMo)x#I|9Oo>H z=JXkn;GINgB=WhSZOenfOrje5la=~hES75YlEcM6hZ1S9CM%I9uP(t3&dri% zBbMDxRYWz!ABkz7#ga0rE;29g9kO7<1sDXB;DopLiv%Ev6Dw3^uXIEWF#6j1yPv*T z3&CtgN;SS1G&)swsZsCeDvwlWFjP7nlm@r0@iDEbZFDBVW@ESKqUi@{R6^;1f#0q- zoPsxxy|KipS1{Wu8X7!qmRnn^?)o1EnrT?BC5ppU_9coqBWMC&X|A8PzQsN|BGXzE zY}C|QnDMD8A7`QIjVhO`h-fTFoB!6pHdn4{Y(HpR(fXPi8J`sO&`!I8j5Da&OyP%0}98ljCNIXLG8$(;Q!9 zM9!PVH1_`kg+O}0Q!+WaeD$BeNG2zpqK4&Jx;2q#Eh=InA`iwfMIHz6bg$y>OWJ8H zHcyRUgxv=tv2Uj|_ASBgk4D)5-_)Lj3oy~zGO9sKN8o}$8zf5-jjgnqmF&)zxW=)P zmTXDq>CG{e_KcKP z-lwxHL$x{vHwXcmp=dzhr6`n>IxDk8gQ-=_$A3C5)YIB!1*MYbkjRi$wRxC^jfQCS z-#n>3eT<#OqtmBl*lg{aI5XYvjPx1onN|tNFfNt#V<)a!WjLb@MNPZmS(ykW9>z9A zBZFIE9L~&q{MVk|-G_z7Kx-1aY%r^U@~35Gl4n#5Wavgbm~<9*AA*W*)v6(}$dFad ztERN4TP}!2)Yz(RbPwILYF=G)R&d(W+iwcA4!OAz`)VfXi<5%Fpp7wd!>El(a>JO3 z4ckj$IYT-tZecg+e%p zfeM9fiJb5@p+LxK>laS83$aahSDAHSu@J#~mT-(9&c(0+!!CjCXHFJEn6Dx(!_b0Z z1BP7~-oT&(KxCy640AC&h~XeB5pslJTO_OwED(Hnu8-}62p%|!A&o(RFk%UY5f~<8 zxDCUD7<9ZI%bAN|8HQaLj`5PD;Ay*~9C`P&-Nd6!vo9*+$thZLQWH;Zn$W1l!$)ax z?O9s2zFM`O67#_s1 z3&T+iGJPLWrGI7mi6f(HFi`qeM3w#poAnn&D}4{C_vm}b+FNavRb|^aktoR!;rtBY z31kSnyR4nQ+4g-QN>m(28YOCnkP8E#T*z(9?-_0vylvI9!tL~vO_4-+?Z8s{Hnd`( z$jRGy=eHq(VI+oT4D&GPkUD_W0cgRn8pB!)tr$?s5LFBj`U-~EFdTqV3`q*~969Ho6P@E{s@$p$5YU43jb3hCxSn(cHFaVLM)A2fU&H)@*|>c>Fj# ztUz*hI2j*|0vKTs@7o_Kga;!J#%VIX=3Y!Ph~Ih#QVinP-;5N4_~q9i#UOss5~LW! zr_Dx+L44w5q!`3Uj718eoj&kfUO~8W#2gYCXr-58FI|egbSd`Ir68hzh(GvVWW4>l zt*8il?7n28uqfQJowV#EEu%@x!=z<8X}OxTTtix_Nz3`9B|%z(q$NaJlB8t^VGl=I zNOJQT$*QE6w7f!E){~Znq$N&T@<~e`X^D`^WIK(vomI}GIv#Br$R-`e{Rg1-vC=5^ zXcT2WiaKEz;(iQio=Re+NUm1d7h-YlrlJ}x-J@^W+=0XB^LXO(I1JCjhZsa0pyyGH zp2v$k&o(p>YcNd4up7fs3~3BHtj!()-Nw<1Sc0Jj!(vbtRXTYln&8KaJ$hwu-g<{J27wl*8JVKX3!7R z(_86nkb?uUIuqa4_I*yco!rwF-w__@B6mYbM5!ZH5GVN<_ko2x-3x*Wiz^^VpT@YX zEjS)ev$n;%hqn_8J8NtB`{1GQ7r}Ol%qk+g;XggH-d4Zj_{G5m5#yMj48 zCa*g0FnrN}};Ba4vP!XUqCEW|H3uCkrXOm>&o$PMxc^w!Jd z?s8NP%Q&b(z_RWeDh58F3ttZCB+KBAUgs5e3#-#QfL93yu}Ub& zs}Mf~d6gjX^D5&{S1HLjmIbA2H)iEE;)-fN%I*EVxm_8_xAe;I%PMyS@ zuI5HM6*S1cL)kZe+y@n=-@_PZ7@sSN16&TcrHa?FxQKJFnwK))N9OPE#4dBAtFY3KIupO)83QXS9#}k(72%v>MHc2+XNtU6_lpL$lXaOQ6|-%zL6BIqc;RC3*3GKb z{w>4^{qu?>RW5jn%yYNnA^;n)Y>bqbKIxErHV~g;D><~a8 z?2rV3P&6~W(pSda@STzdRDtOf=s}2u5JxQHn1F)x3Fr>p5Q{nXfFi^aNFbI% zF=9`|e?l4bKrDw6#0n@)ABSEXdqYpeN+?I{0~P6iKo!Tn&422}(Fc^q<77Ri>n`1o;Mmz^< z5r@N&^iepM;|LgvI1+{-j)Jq&U&CmQW8iGWu~45r0^?vf;&_e|;9SH87?J)88et^j zM8qR-9*jbI5{yA?g0YC_bDRw0(tl&msS&5Z1jGvvzl5pKh&T-Ca#> z%tBlOvk@;v{1h&OIfzT)V#Leg62vPwUI}v%uY!4qSHt}DAy~$-1r{P+1B(!^g~e&M zkG%x(dXCHC()7o$0xm;b2}==gfXfkYge%e?!78{C@g}$mb8bfb2yTI^5pRWMh_}Ht zh^r9~!tHP^;vH}u;;-QP^oMXK$Gc!T;u=_qcsJaDrS~9y2=~H`i1)#&^at>3xC!xo zxEb*`93OyN5g&xx5ZA(L#D@^yhlk;I#7E!`#7E&*h>yXY>A%9`a98>-@C2+uTnBd} zKFRSZxCilRxEJvm_%-5sj?cpVh#TNHh^_Dd;zq>xpbZ{G+yrY8H^al}cVP=WlKwMn z<@g*tir5a1A#Q`m5w~;P0qYQV!jtKD;J5G;;_o=_f~OI8!!wA#hxLei;911yVMF== z?1fguKfuQHpWp?KFG3sQORy>ZHoOd*5nq8Vh_Av{#Mj_C#C_0?_&RJu+|Th1*pB!| z*opWi{5Jg-yoLBCybZrY{1fa#JOI1Xe}s29{u%Zlz6;MIz6W~||HARF@CU^A;RVDG z;KlSC@FB;8@N#-Td<3r`ehjZ7e!}q(yoUHG>_hwvUPt^K-blX=hdF)$e?QFsUO7{|ZEpAo--cM<;q?;#%N_)qw2`c*gq?<0N-A0U2* z_zHXvA0qw$2N6%gM~J65cEBg;mmvj*5YzA};(vAeFR#=8{vWQ>|MqX|^e=u!r+@Jy zoqqUdbo%E%(&?Z58#?_^r%qpRhED(FXLR}}T&I8XBc1*+*XbX3(di#^o&GV`=^uB| z=^y<_ryt}x{UF!r2mb?g`n&%h>GZe%tJD9h)Bodi`hU8e{-3MUf6h+-uTKB3PXB40 z{?h*sbULtiL1CxSpndzIV<3tEf+z?&JV2*|%vUTrP-I!gv`j%pT0u(I$+!$)QahU+ zqE1JtL=}z3ix~JNet|<2Dcch_8TC9VcjX;9m65$H??@|py&{92eU_w05-_F5#VgA@ zYl@7gVY3AzJ6pt*q?e>$ig$rAW;Pqn5ji#pC?`gPiGkjL#({PkM(n`T z81H6AOPG?4zZCD1#Ccb(+6PwpuDr8e#;(osj(u%3nT*J>fO29sn$XfQ>Wz$dP2jLD z3YDl=L_tB`@$6rkcP1fVvzbrx&gPb+lcdwUOK0*fox!`U3-63yWS{ZQXq_VJk$3$s z!#nB%#=FB|;Ynjx-i>%5ds*I*U$ezxHi21(GHx|n&44L08jjho;tWOMI5G2UG+?NdCHmWTYKGBn`%*i?%R4Ra>)yaqPKEL$v+&ZKi%aF7&}Y%$u5rk}j^|N1~(z<3V?ojl2z zU}LQ46~&4NY9Pqy5i8j3PN&@l4h7u|kHcjHr(JcR+rUJg*RbNi$X2UPXVJOY*?Nm& zH9Jg}Uxs&_PfQ?kLT;Yql4C$!&mQ`!cz~*AdB?tXxLgi9xMY-ZkIUl#7v`Xw1KOZr zU8}0%X|imQee7(5RZ-0@v-y|d-Jq@5k%*TkITLJp>WzB69S`hl$mkJd+U4=MoZ!); zj0ZeE7dF7*VV~G)gNE7)h00*F%d#p5>^7UtsOs%jkHz}S@NU$!Z#38UZ}9GRd%bQa zc=hOl2E2YZc-<~9x{;l{+wo_xXL+~VjW))+^_S&6KOW>s{?QpzEx2kLTzFt#1N(xI zm-&2wfX@Q~Jvz(bK#mVE6+l;)O;>g}T@IHMPm}cyIn2&B*$pmRz-IepcsH5&qF+=L z;YrSf1?QH@VlsK~01wMDX=TA+C=~QV$b|DV7K#P|Qz3N6(c{<7#!28Ydpy`|LmoTZ z;>3&OxPSB+-v2v?#llyP9z9|_$(dkdj4O}Xj|cc!mRTzchojMO5TYgsLw+0*!@3^7-{SWh3?4(Aoo)4){hp}D^ULsVwKA;&Wo2=m(Jw%9C|ARgG)tc6-x zG@6$e4MUy};FQ67SyrtqFR!2=FA4=#$b;g7qCCJ<0rc+thAzGQ8XEe9`aQw>wWp7&y>EQ5~up!2`7?WYm5H3JQvf3Sv-Xg#suk zDlUMcg1jQ=%f6V`&W_|nB6ynF95VN0XFGD#NKRq!7vkOF;48=AL1jFd&w#zp<#0P3 z`FKD+%W`OC#l@wi#f4DnfMTdDttbXemBJv%em9z*pPQeLr&+DJ)+% zpLV+4T>IA7_u)zYhZ=k+pDv%vRfGp>AjqiBBAgYydzbZq-Y%5!{=NH_LGQBS-cS$x z+jmx@C|*>Ar`c?ATMawg7k3rq_sYxvWq9}bxb_`0W&lreCfM}!1U;S}c%TM?jOy$G zm6g@imE}W9#{$0l@--62B2KKSdSh>J$iV1^sqY;_Mz|8#Juj zqr5y&QSNk=IEUlO72zKK^5TKT#lH;i;joh}=+maw^CV{?it-tZ27`U@fIciMsFe*K zJZ#wD{xB>EgJI0Da|gq)!2^cDRKTWaXZI=V(U<6QZuD5OYKYN7NCW zG2zJ<&rTgBB@@<6ApIuXJz@Q;^~9ZeZNi%IsTU?JCVlHufAEo~?jm#UBI{FQShy>7 z*IlXclp`z2L*!hW_VCI-_o8wwpmr_ zI3&K?Y_)RY&}Orl6JI6K`k>>imM{+1GxiLR?DemI;nNWVTSY+}f8}nE+ojTuYe*uI z+rQxQYj#c?zCSgJ93&s@+_q-b#5YfNy!UnLn-u#i7Eh%M+htwln8W^fa)Q%&>X0Pg5 zTHdSN<Ss_g{7BDIn2(L@6?F}yg`|DU7WSg7CAUNF!ss;fJ!eO9?aonE7n~R$+$1CGpIHGD}z~sIOC7z8%BH<$Cggm)A z)!>u9`oYOG#yHd%HfcA?Lk_hj%~q4iG-#aFWHb&Mr)y6iPTE*n^0Kt7G7PX>Bcrrh zm_D9#o6Up91$7LqCWf-ga3f`(^dt@bkSIArX0u0+Cv#b1MAFBENpAxa%K^8^glJ-U zP=bQA4`Cnn59^`8M%|CEbR4gwV^{hNrIHh0Cw)evj$Kk^IiQ+M3|U_1B~M>-vo7LO zgDB>0G-CW+`XIP5*fChs2b1TDx|Q?>;|t!GW=kNv=u0(SbSg z9OG2SMXou%=D<>Vscwn!a_dUnJ@Oj$W$%0RfbM|tFIImiyNp}blSq8CnROcy-zA-_ zlROLc$!M&iR8K&!>M3s+tHHY{$ZF_Uhi!aZY!+5vQ&^^LAybXX2%Z`N_DvP90Sc=D zyaucey?7IoGi)>}N<>CB&F5sM6G+p1Xacqr_8}O$ym}0YghRFWE-!Pt?Ko|9vD|{V zLv@#z^|q<;Sgx!aH|MRjOWGCH%T}FDkOF-3%iy?B2rSKlFzuERFmbPP5k*nCCp4k< z23#3_cIYdUk;`hFuQK*RhCQ{-^H7W+b~GyBoh*vUfkJpKgb=OSeOQQF%Ei zpJi$^jkU}%O|x8Xzua-7eW(3X|EGate$y`Fa}F9%gKCZ%Qgy#eAH!Ze2nxn}9ASUR zpej1uD?z_A81ySaKk84#9~8_XwVgh?X@rf~+KG1yYXLBpHxX(wWGAX2J5dd6qH=9! zSm=2Rg_liMvV&fO-knPNBu%z0HFPqaOP5iRZl`%@dEK^=PrD{m!Y2|;7xVdsMo7(( zj;2F4JChZJm6jd}3#w*qWrFM+>;p}tX+dTt=DVWtN|d19y?a&Qtm7*ePDxzD*cP-b z%BL!+C;yTAj;_1+ifivD+Z^Bh@vRfXp4jv7q|o~H1FNU*zIyMc7tZjM&sgeH(q=BpW<$5D1$a2@@trVa$N5 zg*9X_6uLq|F;p0o3e7RI$?HRFBBHX{7Lnupnpizv!j$j667~aoRZR`9$wzSTkG!D1 zV6Rg5Cdyb~gWpXuyUlgx6=t!{Ho>+eAdGfjq|SCub1yb8b*?b4a^4tt%xsV%0v9@? z$!rm2g7+nCOxgFQI|%zbEN0RR8{-na+v%g=qcf9w%8}_Y`yl%hnx@4%dVPw?e*73w!Ao_ZQA7fdxU;_>+aM)KVwqEtc9rbIE2Wlq{G2A zwwPU<&Af#R*6v#ZaUaG8odm;O!6i0OqtvU1pC#@ zq>=@zf9FI`D1;I9o(@OWsZqlp`EKlE5~~O z(wtQ}YaCBFo;MvZ{UxC29bQY3U(lCIrN-^3UIdgo)nRbi9gbHlR;R_`v{=n3d`Smm zBxzZT7O};ebdd~G&sjzC786WVHc1;}&o)_|t1eS-Q$_XvM#Q{a#JmK&s+W4RBIaEk zvF{|kz)J2yIq%bE+457O7Cuwd&JeaHW*MV+@qTDR52*>~!cN>)wm5-Ms$h2{(6p@>cq8$8#gDzkN4R7Tt3E zr4G`fuDWUO!}qm~sBzQ3Ka*NADRttHFW%mEkXfU{QNCQLm2#kn3fs^aJI4FQ)}A3E z+1^sJL_t=JVx3|X)vQ&H7A{P@HY0Rw-7;tfs_C?b3tLH;OeO*ebPFZT#B9bjEgM~P@~BLvJY2tdxUuTVSeE#0?Y7AgHn0WEZec`>D>p_(d4bFd3L>#cloi+* zmm|#-`3#V^CUe&q6GnZLLV;# zxo~~567Juf9WI;&lzZ{j_}_ znU!asJ-lzf#A6rqo%P`Ad)@oq`||PThZmiDcJoE4yIG6V9RO0Jv>p4>LUWk%{w`}| zo!XpPIKIo;9vUGsS>;QIqA=8nTuCaKN!V} z%&6lxHt_3k)LZ!VIW2RVSghzfP&REc#CLpFQg+=&ChK|=Ly2RFTB^=aX6l>O8-&&B z%hC(F-Rd#bs7Q@uJRPadG`6b$F#W^)4~t$jiDuCv7!7(!6wz=|bh2zhTG5$g^!!wpQ7ys#dCd_gePuT`6fXN>zQUaZG({Xw<}p&7xIM z5&l|~t`At9mj`8#nBjqF}u>Ce6UZ9f&TK-4c@cz>szkcXl^7ng(iR zjx8rgDAIMa6$K;Vq2}?<39j+JnbMq`E9^Jf?-TB|+!tI&9;NGSf3i5h34Yb7`bD-? z*Ve6yyGeE3s#GfwF@TGy5D1BSHEul{;t@iKKkVT{=ix)=;X~(%M-+nUjhQPZ#--v5 zO`XKH9I)q`_z>V|Ady)?H3g1EYzGHd$_2!u6Jxn~*s1nB)ZQLB&U86-Iqhtb6gThb zpL+h&BdK?Oy@3qg^H&jKzk2eMNf&>%;*pOj?RoU%@5v=^eoDq|Joswy+B+Uj z9ld==>Wfu7nO*h(>hFm-zpRi$d`WvGOa?33e6Xn@D=40<1b5xvI+4uHG)zyz+=J9} z!PFc04xpFkaDnD3)gR7Ld4H(fKvTH|qW;?g{cTpDzsm}A=qCg^lWaOeihGt0UYhJJ z1Y||W-$4|0KCjpZb)xc8p|g>ASw; zLZ*Kn#hH@JMvIuXff)zJ?Qm6yLP&2|Yk0#z4HBhB_NHrg!YJ7Zqp%61=hHtT%eZ)a z&0B)RH_Q`2iu)Wo<^UA&HH7drq^SiR7HZ`EV&wf|?MZ$;X_qZ37=jB7X$1!3^Bw<_s}2q z?CH=++dCeo6Mq;&H+2k0PIqDVU4uO*kldt?^7aTEsmyl`7<%b?8-^Mt2rGnl1bKyOTw-k?_uk|2r^ z$>5YEOrcQg(5BTH3?NaF;Fv0)7zAo0Ahy%KNo%P@)=I6?Zt0*TN@pu9&sZvx2->Kv zvLLt970J*~N%{ZO-udRVCR@kM-o)tzp#xnw=Eku{$JI=NtE*YSxzB80wpELPtv3m8L^Kk$Lz+Nf&$F-g<>(=xO0S(jfBJ=J#=G~$vZ5cu9g<3hO3^xz5 z81A9>3Txyw$~xf*S!btKi=|YeI1i~}GMh^!1xdv;+B%vf(REZ5_MrxQ@m^)))NF6D zQ~P$h4$P!yn-o#nNzbI)q&IL^T63HYk)&yvkr=mQgDu2}wP-s=R>G%VMm?Xd zou?ADosNGlA~j1b5^71hZj+5IBR=+|zNy;V!Ill~b7IQhC3&a`ebAa}?PZS4&ySud zn~y86;;tx0Jxk%V%lTU{rB9-i9srs?z&+0TRuh(WweVKVP@%&otdXDC~Akd@O^Y+ zcBxjpCO10#%w5T0ltOXckrmXQ3Y7&x(TkMoWknSDV#k)rtPR0((1u`H5uHxTu0B$y z4Iyb$6W91;oX?M@h>=8$BaO|*dBzr_WK=p$D-$=Zxcy*8rv5z}OZ>43(P?9KS>_T= ziE2K`P4kcctIc4qVoy@5R*HOtv-(s>AAAngs1iW{t}~h8jbpf3Ne-#PcJJOgq)JJa zX=GWIj7pR_V_SVlmT4r*jcIRF7-Lnk#fibe(#N+tkj&A@93)+g{BC2XRwZ2~rzQgB zgc&o$cK?e4-TvaK6b|?`;xZiUmXj^)@0+3*`o8owSRg>$$$Gz)I8~=J;0XjoQ5Bs= zk1-%V>Dg*|!6JA(-T;l{ByA%cBRolegVdm(ppLUmc1-k4_Kx>Y2;Ahkm#V&yAlO4j zy(?=sy0T;LVl$1eg|2u6&F1Xu1N`wZ8!$O*d&t@@Iw!v5s!P`P#X0dqvWDwGe@hO@ zvGRr1%4NaIFK&%93DDjy;7%&&x@b6GaO!CjI_s=9b&mG%e|_TsomPhm(K3;_O5AI$ zRH3{KY!wupR+vg|AiZBDL!RE8+PZ6hYWupENX|QdA%Ue|-2Qs%9r_BnnEYl>>ao9m zm|DB#B{K1Msc%#JNd*aPBF0~(KGpnHu>iwtQV(*wdRb^R=$cbZge#;+A$3Xl`f-9e)~)+K}$_KxyF}!!Np{@$1yfsT;@@ zJ0EB|r|0#l8>Q_Q`}D0B??`oYJR^{smrc6PWoADAL#V6PV;_1U7j3bq-Dn~9-oc6C z3zdt*qMpApp>UKr=HbNS(q-13J7}9#oG@ky?AVVt+5Ht5AKR2$P+?=~oPr896I(Me zmj8KEPFyR;`f4U-<;k;<$hVvwJUcSRI4O8>aDo0Z%Tnud!wuHE%}-j}t%oh2TUA`! zBQ~qkX0zI?CcQmCqkgwRXJ_v(NM60(?eY6U9>#HIuVQ!7#RkFSfoLul3$NE|wJ4#i zY=pA15$bf)LUGIey6pQ*y6muW|Ez+0YC68V*A&T{m)DXfT8G6l7kOiD%!M;ZMChjPj~%0DK(%9L?Dc zUsIMaW{F^y;4i%OKmFjEawHt53RA^}!Xi=3FX$yy1qTaf$>-$Mg=_PM6pRrXO6 zOz7fh+#ZPg_3=Uy2PpL0#j>7pn2vhDT)Z@JV}J(o-R9y@K|aZs+!E7&+9Qxqu|5=X z3tYJ;Z1$=q44J5jzqwFyMEeDj#rb&_vr&o$b3y?{)`)o11hQgJ9Gg>1RA zSm?My#&(!pv(R4_<@<|KJ{zO_2`BadiQAdc#46jf)6m}eT*f|@%~oH}soEA;)9^zH z_G~k=HJe@MGW)YCdhdrA9A~|N^Nc+%VcX+qy8wHMEPIfy2|no@l_6SQmf1PXD~QK? zRcLQ@WcGAjPLEsk@JXe^wKG2HIrHR~uA2MQn30qEr7jvZYsS^z-1*4&E2Qn#^-s1w zRMm&P+t70Pij%*2G4+po$vf&Lw@etcu(obS%rhlX`N;ISzn?biwQDRl-FnS=Bg)I? zbnCZe$>RMB7kvTz|BwW6JAZEXLDHw=`b*}h5+6MHf?MoBb;qYwPQ z0@1hAg_|PUbE@Zb5kgA@A|SnmWcIHPCyiV`DjDVYP1dFVIIAU3Wwl~TbF7~(`PDwE4U`JE!BBMZP8wAPmelMIpV5RKr*ji|HD7n(_OUW*nXaqn39IL zwLxr9URFeRduC^^LhPpu5zkhZSRa=TTV)ea8+-Xpr*~#o4rf*>kvOrHwcZ)0S<9Su zTIQ5*3Q@Lt^x&RqQ^ZXo?vZZV?4IXtaSQIBFfif~12JUX1_NJ44B2JGkd^MpTtr7Or%I9k*H*M6+jV8Zucn5mw9e!Vj_uRVawV$2& z;RXJfdBL6#X9N0l!@GLzXGSPrD=q5Yy$IENvs70=H{ObFybay>HoJj6ZqEJV8NVCrlgc)?=+*#E@*;IH?GU9C*=6F%thr@s2fO`C;h(e-P(IWm3d z$w!#pEJVH8it=qDakgXjP14TSCGE9lj&=*1w^p;QVi*~!4ATn+#i-B9mBj=WGckrt zC=wwZicCj!O~>AhxFntyi9iVv^#)Nf7~(n6if#t-J!*mo5m6XKLpNivf-q2+9gVSw zaniwZ@f0Z}>!{HXGJ#^)LAJ0zHBPo910a_wNrftBn`(^2=(iA%bfds$W_lC{eE4x3 zQuaiodie4A)#@SjROh3fYFicWS^lZPd^VUkUfP%~V0}NoiEnJ`X>PQNc%xN%wrh#* zIU0@JLS&0xc`wqtlBr-KM_q;V=*VHGUKjnRUTzfDZ5E!McJ})9r{o#y(Hc2BeOL^N z1E3pJ(sjx1db7UBXZ9Brnv066%)MQefxbm&6*ZZgie{T<6*ZTxGOsAS&;4uvlV(@9 z?1SI}{_O#y9``-fZL4obx4pjo-QIM)-%Y7?lMqubHl`ohho0G&JbE$3Jc3bSPuQF2 zUQ|&fR&_s19M*lj(wMkVnUz>#T4{RO^u77}gsrl|LPWJBufkInb$TZk&Ml;c!4gZ2 zDCcrwb~;VilXa*oFq8!aW8oBaO4mI(?cEx-WU}|MnG?^K(EMl~Tahv* z>Q~x|lqa$^BaeOb!(QUf(>4~kqxV;~BqPQ7%F8amdF}MPq@^Iq-iM9EOXC~jQWbNt zm^Fo_>wzqBocq@Co>hErDHN+Ht=e5h*H)1#_L)R-4!g96zv$!1_vV&x+pL7E@)ACE zO7eE;_UmX^SEHjkCpS0vOKm#sMIj6K$M~yxIxly`OpJdW_p)^s?v(M@{&hY3oPLVM z-sRVY3+#m}B+J=@<6c7bv~9CD>>RpqSg$$n%^>A< zH!NG4)9StCjT>)xYNV?7qzNB>AjqPia)3()nyS;uLXjr zYYncT#pum?Yu@bI%-*a?=5yiBd&5XnW761cY%z-3n*>HaV~pO+!-NyreTHLPM2*>x zgN*FGNjBCjCpm_Vy>St-QT_o97Z+POaNP_dpAVw8jmgc(a?Oemv=317Dcw!b1zAg4 z$uS~^Nevl61j5WbHg5=EB-zV)NtxO#8AAj|R$sGC;3t)aUhBg^G3?b@LA(mSppuf=h zo+K%LV)KHH7i^fHOnvj)opWf#xZ9UJ^Vs4g&q&)l{(0Mo+g@3iI+{B08?t8CxSRI9 z`o;_VtF)2n!@?2Nx_pNH8 zewl>j8kx!#zH4h>r^$SElC`hxcx}07pki;Q)1wJUm&f!AGXJ#}e|Nr5BbF{W&fex) zkbP&_R#l=l9m0RyK}Au@zKlye+FQ33{By8QCzQ#QP`xqd=(F#ozFqLejqCp!-r!p{ z@rI`!y?*wsWVz?L{UnDNo+0#_4G#t8T=e`~2liZxI%f#V(udl+dSqAfN&^+m`Q{39 zty${j>=m3q#~MaE#{_54Y0`B4RA+N=cld4TPmcF}pE^Eu9`*dq_bJyc?r=EaX9}s_ z&$N@=gXWogxckyx=6YIZ9^yPJIKeR9Jj48{?sNAKacyPeE*~35n}v;|E&KS;mVHdfh-5RLNQ>->_DKZapVkb=JehA@X|0aAngZgFR_>c6lYc&-A^^{y-BhGB4Y$^}u8O%w%YB2huvltDyLM2IMw5GCdsG0Ov& zhZ>Wxjfkk%x!2{!g$G`jT=B8wNer40d4}*5n3;RdS5-p~n&0oua{BAFy3YB|xBkDc zI)o18Ds3h=SGYx)smU?F5*eoIkaq>6%CZ2UkA5tiaF3_?q0v#Pq1p4J&+ zS7g5*ouB@Gc8RzocAOkfS}2Z9BaxUGXYMr@&zftk4HmI1K9f}?w3Gar?UGQkcYCl@ z7TwT~I8-bo!g_=O<(y#FBs_!KN8orHe&PkH^1gs2HrPigbq&SM1~$?a3_+2vEwbAe zm>E=oE8OD5?fr#!IiSv)22WEZZWI#kE?z59E=v2vSs-h>~y z@4}llGM(d4c32_2j}my0SMDm7G@do&v{7$tPIagF6q{BrT~REP z4Nf<+DvptY>+qb2)b%cQ`y7}<$KwOQB}`^Q_f1S5apYkPna*uPM;0ODS;$rt*CL$q z*zQev2Hm~Lg(y@#M4_tKenyQ!{OPM8O78|y`Y%Au1F!*#OsSm@)fMQ;Wa?$S6IG() zSVhS~;dtqWV^MqNWWlaGQaIAV!c?tLHQ2&ROD|mNxt1*|k|&+TOZ;y_3FY@#XEAk;AUIX!maJ=W9CW7B9X! z`YZL~8P}~jHH+@N3wkGUU(lUbjvDYA-V99>+(s?$UapOHMWrlN)~MCGjnysM2=_AW zVt0ykjdqiELj6Z9a&~oN-FemL)m>hnpDx zG^24v*cNT7fMUSdXKMS^dnOG2({1ZkWAX`JYn#tD$dtXu^i z{lb^V>{YCMC6LAmAA|zLiEyu<@NNRmS!Cios;U5HssLuH0A{MFdDYx)9x(e%zQSxX zCs0enucrrDZNeTk(~MWlDt2@;%Z}{VotYq;%~YXrQ6&@4h4H7iX=e~vwtby-SXE8e z@I|^?LgLU3=KzO8NOwy}cXxM#ARsCr-JMD+2uMn&bP7l#Al>zD?)QGw`?=rm+t>MH z?=`b#X00`|<~rx>YkqQc+!0pO&E4A2shsTW?HJ0u?R@4ZZLP&i^5Y|~<@}g3h!@Pn z=RZv{;ZD@>Sbt&HbDO&9MKSn%5tj!XxwewDH_dw#hk19|Y5I6i=XAhvPwQ|1x+QyI z3{$XOxTcwJa3t@toh;*pZft*(&s-s0*bX@PHnrgQdUtjL#W|eH;Ib!*QK?RyRV7qa z(dqNywUYyP-PqFi94+c+3O%(Bjp;$LDTK6|Y9~)OetdU<-)w?gcK7bIB~{r`bE*qj z$#z^9TRJn;M@oF2yH=eHIWY7b%_euLAi{Tf3PgvNp-{o<;-4jyFn1i=%T zP?`YN9o^L%a{l6c-ODgN8lNa}HjK{Q_~6^;=$OjQI{v-}*eZ@K#(+2{tmj9@J|2ke z9LG#MB#!s{6dzX1f!oJI;oab^`xf+b)0ZCr%Ow`Zgy)4^Ql1ypQs4HHo8i-u` z1i-hX?5$%Tn!p0%86t)cd^ZI%&-4Y`-;l8iWA_ig+c|&2h$f^?;FtUwVtMaoOeSQ6FLytWnk`-I!p9MA__aYZ$y$If0 z5M-^e9gA;}y=urZia2*J88dcUB$+hkzV7z#CYB8fPSD`-N#jg2^_`wxm#rwwQqXu(Uzr3!zRLkLU`q`d3>EHDVFuwyH@` zVP3-Xx{4a@akKp+(L@`j?i>5HWUd0nqXOM-T~?iMk+WTpsgwfNoDO~zRT#9a)~p2t zqSm!2vnwHaFOiRN2pb_`a^d+h&Cz0EjiYhnW2m{O_>6`*RoV@jakp7;y|LGHHkc7D z$?l)*C|v*j#2E%Akv+pZgD$x*&i_?3X{UDo~q{joc5|HI&?n#wHWvQ3pnUq9QjOu=BXk-}Qh%PASPgvRrNcGJGJ z<-kMA$ncKgf;)x)n(o|EW+yUbVqUG-Epo}kBO%)4m5a~V_!?UlE9MHYx2>a8sV~$6 zka+|pm#7GI>1urQT$j*wh4if1pat*}V5hA0sl1KY;c=n3OIC$LVzqbWn2OmCUpw$S zGNM=*v8+C?HV&+gz#;5dGNRc(R*{3Kk%fmL^b-MdS)qBTo-R; ztl}&bip7OdB|)O#ux5m@eAA_csLA+_Hvrl+L+It2Ca-zOLc)+Itv z_K|g6ooWh3k+l^NolF?e*x?yZ-5tvi$mS<`8)sT39xj2Kf_?et?MB3$Xj~6xa&oV{ zSR0*vxToyHf`fHx>1WdJ8t7J9$UGHJXwr(tCK-c>Va9r|>6q>ID}?tlo*RB!ygL8Z zKBUI6GT0R5>i{D3Jg&x)Kl_UL6@&54Ia8%n63Ob#gBinI2Bb}-78aEfdScK`ib0Vt zqM0D*B2D@-;&$2T&ZFn)U2UKAwXe5LezI6tr2#K!AAi$HeTg{~u^lZ(d8tH06Lo{d z=TtU(-*mjsAleDW-^V`Wqq808ud|($VCA1x=65H8mHr&8{cVbz?s%~@Ybrw-9egwfggvE7y51II|u_Vk$1(|&DTvpb} z!50$rw4+ox;o6QgwA56_FGBHk_B3Q9#M`k@8Mq&o#YDUaCC8IOF^CrzGqx^wPE{E= z3O^U`qr?yFL#0IPcBQ4-C`~YE%_O_IyF1J zPURIB+KOo=4)R*54*T$&9zLELryXv>RGgv%3(k)&m~8%g0>c{KG!fXAvxVn5ms60! z9UP}q5C%H+?9Ky{N?b0S?S8N^Vwa7SWq4mGQxb-Nm>{*{#QwnqXs1+k`q{<-MX0t> zoH~g9kQc|>d=bw(TUO;srHpb|^EPA{actbOH-ZDv_E2E|t-Ym_L*56Q z_-}i`Pn zh%NG=pwmraO*ucxI`$(w=gJz%An4Y8BE2LL{={^*QfTdVhivVQ0QK;(8KHT7!6O69 zU0Di+Q@V@v9jSS*_)9LWSC5&uGblLiY3H3|A4Ex%q|%;)W%n()_sxLMhX7?Mcu4NE zHkJ?@Pn-21hCIkPCOEthi7$GEVv`*+oepzM*xv7LWSP|Z1Z^~ZZ^-J4)paw^x5_U0 zOplP8BjK?-gMF91Df=DgL{A1h`MG|O4877{9@r0d+dj>;LfAJ&N_l7Une>Z3P6|d*u zY9vA-mHgC#lNnK2f(}{`jmr^R6rr~ExO1jj?}L=2bZ0n&w61BM=}KPQu2`c>*aYrw zgkfwt$AG3o<+}7nx1-F$8`GDJ#AEd=VJs6{mrooD!D<;r?;=?_X4Ic1EQh}di^#1I zMAjQg#gO3TY~@!Ctl>hX9u#G#Q+Nr78`KmnX2kp;6nn5Ve3-P1qv*bD#>YUYDH1Jc4gu7l87f_iy4@haoHNOc;loo z2!a6?cmxj~sV|A+eK1jQ#l!9iIb{$<@5b^MxI;yK+E~DSvlV&?qHxt-Y@g*%W6?gC zP=4)mUU+pjP`}%nF-E~ZxA4iU*HxLL!CijO?t22IQtF37$8T{>41sPJ#hG#QYK;bg z8(%yZZsFsQl8%Sgu}0nNo<8C@Shvm>Aq9@Tsxm8KukD;wHC40ihsC2y7A0G=wDi<7 za?PZq*Duryapn0*ULY@rj|_JVpK!dL1gRDfnXs9C3Sg?&%965(i#}0)o7E+(?4QUd zsW9-^f&ZN>?#SSwTNQpRJwdsrbs+@BmpU<&v&!6O9m6%OH%WZDhD>8Vw6bW1@|2ru z$69Q9Z2r6H&uRZ!(!xbJ@ngdUI6McHua?yxj?4z@4iyV7x=CYTIE(S1W~NSm%E4iw z_=GHy8H|T%bSi`3Lhtt{bMjIARphkP(g?u=a5IHnFQhAc3eNg<7YuKuHt+=#TC`oZ zyA-1weXBirVuwkhhr32!Eco&Wc`b|mtZVdWI;EuaJEj~^ZBAnUndYW%|6%blY`X1H z^t%h{Kq}#stWJ()$<9PVZ3vkz1c$*ClZRS0;qoJA;yV{C51EOe+Voquo1;%gE#dng zEt85>aYNT0T=ipPzN9nQJl{;7YGL8_>iNN28}qER=E6iq0L2QlT{eMR{AOFLCl&v^ zv0WKmMjd8cxt}t-#Gder(hOYyk;g#10U0!DS<1rpn<6-1ufw~j&_1do~B<(8w z-OB0bQg&!vmv6E3gj94#o%$GT#B%v*OT}_IX|?1z_8cM(p4C6gi|>z*vME#_e6LgZ zr1*X5%;uA+^S$&O+R@&Qn5x&5bfFmaFL7*%HR5(%8b`}(ZFhYdA<~(711cLRJ)-tI;`(+;U1Pbc1 zx~`4bkJ<6U@t^Tlh0LmT`B?eqFUQ51-ui^Oh`c(aS)~w6YMM8pOMSHP9JT$C#h^k% zUHt*|Dy=pZq|34{X^@4+d?vZ$3WY6#Kj4wnBjF>mE-AlLyE5w6V^F4H*+=13b=cwa z8l#jm4vM143+iv0xDrhFuD7fBEqjlz z3|iR;cN*7%&|RVQRz&#CjVb7%bEp-MBq-v~kuC_WuFW*aQQ$>NazuIJ<#I;XdDKyf z-msEUhP0xEDJ9~wIh(x!cSr<7S$7doh>yBSMzg-1gDM}uOYN#x{Ct0C(`NtvV_ET< zR=Sq?DyK|I5~mh#6xJGhlP(ExLT9OscoK36iJoCmu}Sc7Hrl&a+8^F$xV{q(LLYys zRqsN_9GlFFH1B*F47W?QZL>W@|0RkhwdI+JNTBv>VzZFh+EncMdb{5JYvn~hyY>s3 z#{rxd<(h>vX(zgt9BKH`*hBAqPDYq&7C+kp|4+>C%!hGt@YnIDP7|dcI;j?sWOP9gn^JOSpi0VZs z+Agt;HbM1VnnENY)2m8{;$(4Wd8TCMk72xnA4;DR@*pKMlsaZ^eo}rb&5~y+gASbH zsuL+aUi=MYpV&|!_!fmOR@8nKVx+OzjABH)sD>Vqt#gR5Pr;Rxg4!uo5j?RY06(h4 ziA;A8!B#UAF)KpkBJAlJVf}%|{%f>Q-}1nuE*+xgMQC)M(UxxGj0KZ!>o`7(zxTHL z5h-vo#p(5tQcm;r!O5gIR~BfhqHXl^0%gV4#TIa?bX2K|wtq}Tf#Gr{kxFVXhDjuK zFQ2mpr{}o5zqO_;CB#l8k&W@q^_f01Xs}5QU-l1hoS_P z`)o9>V$h-7v2Q*qOePnp{v$#t3toUDE5Erv>U>6P~i?r?Jd8FS&b8OJY_`r4epQf?%a0Hi{%!q&QmW zhn9;T0{#VPOAQ)^DeV?YM;aYqOr#0R*CEB*z1t*aJ+)cnR#sZFy9Ad*4+zyX(Pe`X z>?CDyob3CBt{39)SW95cDM+FTm~X54$PQNK9SVV~Ju9ejnv(F?l?Tl3W?m0P8* zscgM0H4BXfO({+BW>?0DzSw~xd|DCRuW1onSHZ4*SS45Kg(@ml%4(t_8A6Soi~@a1 zDfFq*G3lct(VCovcxBDtXRuytai4=qiX7L@BRYyf;7w{ApRe}EkC5w_qZ*jj5+v1? zk)0^wOZ{*=w?y`KPZFoH^dSB+zYo z_z7q;bEr6IYOYMI3jKuJ3b+XGtN6;l#_+@OEB+6bO+@Rc%SS%q-=c)n?_LT$Q_eND zp%Em9MlRb3mOe|HKVoOdYF-JGR>Nni7nvXaZNTHT1|wI8a#;Ti}3`9hh?7; z7eU{Yc+nq`A9ft5X{-r{X;JXlF5t7pv$t!Xa~odfQoo7$Y#2#Z%&dA-ji#bXh-GYC z`R0Uq=IDtzqV3)ru3_JHv|yC@!+5d-r-akaK+f7HZ=K-;-}KZg z*hNSEQ{u!^g!J{RlNcY4u^S9D^lq_h)&01NpxiaNhkJ(S38Rm!bW|{G4wXWkm+%81 z(G?MS-|=1U@ZS=L#eYzZQ5RHuPLAI_|E6TH^Bch6U@oewDoI*tMO&H&j>8(c9 zyATGf@W|RP0S#?K7s|+_R)NW03oX0Hk4yxuqc0$Qz1>4m!LS>+s`UzH9T=c|d6MbVp+h=-pBHM~ZxIIJa=Nb}b>s~PHSd>`$hO)0_> zpB;pv1ya8tVcyf5Z^c&aH9vK`5>2bgaeq&G8owqU;y!i~|FSPPf$c~wZ~>hVDrqoI zC_TVfQrt>oFiyZmEzP#-R34werK6pXv{ryrkHeJveU$Mcy+?AMcqFc?*YqR$Amr41 zWNV#pq<$3h@byT+GWjU=g`{&)z2GJGLB49m%t`S=anyL4`$*eQnQ^iv{bpr@ZoyLF z*I87>R@QOZbgbs} zB+~J5q+Fn&D@)K0wfZbfkDSF$4Y-Af)-iGqwIg|ut4!?Zd^9rbHO#4FI;enGZ8SyEB z$wO)Cr|e{hWIecbaA-Ag@=^3RKI1xL(om~0G-da(`wOt_pmm^9YrQ=qaD~ z&N}@ANWVXFoebaIh}GAtFv5bp8KtHNmow^kJ2T9f##x^TU(*WQ(3}K%-Cg1B3pMU7 z@j$ybzV>7ru`)W}8To!dgpLL#I=X9Nx!DlMGoN=hd?m8TKDikVNR~m0+@Z~$eD!W+ ze|jOwLujrGCU~`c`F+D-aeVG%wC4G(h~zN`){J9Zn(t&&P1FE?eG;qh?YIN;J4GA9 zYeVLDY$4i5sUjM>H^dK>1u<7q+yVwVQhV?vX>&zM<`BY%$@Cs9yr6%ZLNvh0^G-Fu zwd~yRax?c_$?dbxZ#vw~rtQX0teYB;H zUUvV%zl&Hce%EyBv8r#&ah7wKc@TNR044pYqA0C0r$9O`6pG$2L2Ujqf>`ERBubMZ zu_YmzXvQ3?%Ib7MV-+Lw!(kQI#r*Z8V8yXYZW~lQ@fA0DLPe9($+S(yFyI?vF|WyI zYa^Np*Ya}L(p)Dp+!E=MZJ7+5&Jp0n44(v>07LaeU*?H+4zFbiXT;<2rmw&OI&1kh z4>NY;cg-u1r<@;jDx6fGFllTCpXhI_^RVt4U*?TY8olILd)B7Y8`^ z^vHP*N*B6s&sS$=x~^-5k8cs~tQHqknzG`2=C1~msff5Ub|!ppiJI+f9FfN^6Pn^a zwWuh`#nq2zDP6Ymv>ZOTh9g!Kl4+tiLb{>~l=<9Vd{wa@_kgY#v}hN7%9Ur(_sYD~ z)`c`lF-1iMpPa*Hom+VrQ6Ys?Pi9Y^!LvZ^QDd;zqvHAj9-30CtmQrjQ+n}&#UqQNhuDQZ4mS~a=K`q{XZazS|n_~%l{b(z_D z5@$U>d3OAS23S=@mOE28O_dB=ap-c)2z=uh7jC=*`@DCYZ?^O9X>gYAx9~cmc1`F_ zDY>oK>z>QKasND6SfZ<5tL6$`<5vZ=FwAFIurPA6D< ze}e3X)7;pfcp}%2j?Ly8#Opowhsxy&>?6)8>V-*Js-F@JBqoi8Z{oW-o2$o+^==#Mr!&=6W9i6lPv;Oc+6>0Jjq*PvkmHo*Hq);sNNe}bM`tGO4u(e$it??9^&faz4qGoX^qu=a=z}|H#RevFYucO~&X+`e< zwEz4vv)e(p-VT!Ddw7<{p3I-pxz6Ak$S@Xq7W)Ekv*q*YNr@uUVZm{MIF<}DS+(x$ zZj5GuCV92&>}t%(s89G4TEW$rwNaPw^mye4v&<*MbYrX5>?7vi=*ns2$6Y6IECcYr zH&fw0H)@&?DSQ9%R>HB;HOj5-IHp5(nq;e13Zp53*@PpOUCQChs7!r=(RiOTHm-td z?@-YVF}~68`%G~sN#bmPD`=~{jf*~usUy1Rbs7G?PRs5A+s@8TM@Ad=&(hgV zx}zpm$z#bOlUGa6uEaQ$%u%Hk3}ShU`Lt7E%VP_yB9CH|Rg35>#iSf6Bq<~^AahA7 zI$_lh^VvDLJy+5WReF?|6*mnp4$cq>(Sf{4F5}Or;HPn>NFcXscr}OeT?aR#n!f{W zV58Npiqtf5%Q~!t-wY13X?wAU!)Chd^RAAH48K$pJQu_>&nKv#Z0cdlO5U#(Fi6g1yjqS6Bn*si#saBs zl&Ht@z42n{`dE*jR7nl1Q+T5{KNjv7HFkSwf_KBpE7^z7*M)ysk(jxT>f^Dq%D+$5 zp;Vtx1t;)sKYe;DzWnrJ@nhF9%F(CcgZj$qsL!)!1Q@+f_c`B9N!{4OMM0Kqk!-)J`9K5-IGI*PM^Oz=^HG~PHv7a|4YPEGPVb(d!m0`ETS&f#&(M8K#85Pr7H-aPl>vk z0>C>E9~jEc3FErwQQbFiFJj|_K-l>p++d!2IG(w^28bKV4gkEkxp)DTnz56NxwWO0 z8wdsgq?}+b05B&D0I{r1MI9{d%s~J^?|w95f06gtczC!0P#74-3u5EqC?SB71D@0!rDY`)jhz70pc!DlwVU@}Vl{79z<5~)3&(r5 zpsKm0wX2(pH;7Tx%+bW035`S9#mwBr+QAaU_#ZVusOIkEWM^)F??y1d5aZCirw{7F zpkRG~H>V4Q@~}gooLt=c_na_*NXW&-#SR9%13W+hY8wp3&dUY3^bh%O*{?f5nGXb@ z*M7AI;^g~7Ss%m&u}7|J?m9<$-{KkpLkF zod3K0*9ZXV@48?R?=LHU*M0jPTEAu0MS7hy1rgIDZ-Z>rNkFLjr2Q3VvCn4+3HXxZ$@pFA%6-VFQX` z|7Umlz`_FJ{+|YckO3V)c)@=}7IGhTUQW=jUV(KWp$US5?j~ob@MlK8{>Ev0rOuz~NsrLsH&YI>8IwbXp841g z`U*TiSE(&ah&nr}c2AY=vWB!$cZAIMsI-~spw>or>}&H#HP$9p^7TC3ll}baHveu&#Da_{e9h<@H)IBFG)pD#&u~4 z6L;0vIPzEyGb%cC)B>Co6zcvoEZI^L*U?T1iU@_s{88&rGcl+dz)Gm0)!#D3B{D0a z^-jU9*Iks0K5Ch*=>L=#VkaF3SlOY=yX~p3gb^-7f(Z7@M|CxjD6TshT z=03jwx$ys;kR%;U9nJ3Z$FH=*Ci9;pY-SAPLm){3*he5S+!HF%IMf}VIamWK=Agf3 z#UDA+%f;LR4alf4G$675eSvtOP%bFQ0`yCE-_ZR9a`;;Y;RKfWf62h$`)T>ldfc2a z=)dc6LI7^!KidOfX8$b%L!tkY0c1lcKyCb|E|~jYV}WpS-!K1v*5iT#+3mk%z)l5F z7XMj~8^ZbDy4?2-{#QM22ozX?|J3CM67(N-xVabuJCh3 Date: Thu, 25 Jan 2024 17:06:55 +0800 Subject: [PATCH 072/109] HBASE-28329 Output hbase logs to working directory in client integration test (#5648) Signed-off-by: Nihal Jain --- dev-support/hbase_nightly_pseudo-distributed-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/hbase_nightly_pseudo-distributed-test.sh b/dev-support/hbase_nightly_pseudo-distributed-test.sh index ffe630865925..8290f06110f3 100755 --- a/dev-support/hbase_nightly_pseudo-distributed-test.sh +++ b/dev-support/hbase_nightly_pseudo-distributed-test.sh @@ -324,7 +324,7 @@ redirect_and_run "${working_dir}/hadoop_cluster_smoke" \ "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -ls -R / echo "Starting up HBase" -HBASE_CONF_DIR="${working_dir}/hbase-conf/" "${component_install}/bin/start-hbase.sh" +HBASE_CONF_DIR="${working_dir}/hbase-conf/" HBASE_LOG_DIR="${working_dir}" "${component_install}/bin/start-hbase.sh" sleep_time=2 until "${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive >"${working_dir}/waiting_hbase_startup.log" 2>&1 < Date: Thu, 25 Jan 2024 17:07:30 +0800 Subject: [PATCH 073/109] HBASE-28255 Correcting spelling errors or annotations with non-standard spelling (#5577) Co-authored-by: mazhengxuan Co-authored-by: Duo Zhang Signed-off-by: Duo Zhang --- .../org/apache/hadoop/hbase/Abortable.java | 2 +- .../hadoop/hbase/ipc/AbstractRpcClient.java | 2 +- .../example/ZooKeeperScanPolicyObserver.java | 2 +- .../AbstractProcedureScheduler.java | 2 +- .../replication/AbstractPeerProcedure.java | 2 +- .../hbase/regionserver/AbstractMemStore.java | 2 +- .../regionserver/AbstractMultiFileWriter.java | 2 +- .../hbase/regionserver/wal/AbstractFSWAL.java | 54 +++++++++---------- .../wal/AbstractProtobufWALReader.java | 12 ++--- 9 files changed, 40 insertions(+), 40 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java index b9736d573454..b0a5a86d50bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java @@ -37,7 +37,7 @@ public interface Abortable { void abort(String why, Throwable e); /** - * It just call another abort method and the Throwable parameter is null. + * It just calls another abort method and the Throwable parameter is null. * @param why Why we're aborting. * @see Abortable#abort(String, Throwable) */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index fcded9f5b69d..5926539d0679 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -209,7 +209,7 @@ private void cleanupIdleConnections() { for (T conn : connections.values()) { // Remove connection if it has not been chosen by anyone for more than maxIdleTime, and the // connection itself has already shutdown. The latter check is because we may still - // have some pending calls on connection so we should not shutdown the connection outside. + // have some pending calls on connection, so we should not shut down the connection outside. // The connection itself will disconnect if there is no pending call for maxIdleTime. if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) { if (LOG.isTraceEnabled()) { diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java index 53288be872ea..fa7ccf737365 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java @@ -41,7 +41,7 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This is an example showing how a RegionObserver could configured via ZooKeeper in order to + * This is an example showing how a RegionObserver could be configured via ZooKeeper in order to * control a Region compaction, flush, and scan policy. This also demonstrated the use of shared * {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} state. See * {@link RegionCoprocessorEnvironment#getSharedData()}. diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java index 4d1d5c1ccd9a..61f73544b1bb 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java @@ -236,7 +236,7 @@ public long getNullPollCalls() { // ========================================================================== /** - * Wake up all of the given events. Note that we first take scheduler lock and then wakeInternal() + * Wake up all the given events. Note that we first take scheduler lock and then wakeInternal() * synchronizes on the event. Access should remain package-private. Use ProcedureEvent class to * wake/suspend events. * @param events the list of events to wake diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java index cd4cca0f9186..b3b01f675c7f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java @@ -52,7 +52,7 @@ public abstract class AbstractPeerProcedure extends AbstractPeerNoLockPr // The sleep interval when waiting table to be enabled or disabled. protected static final int SLEEP_INTERVAL_MS = 1000; - // used to keep compatible with old client where we can only returns after updateStorage. + // used to keep compatible with old client where we can only return after updateStorage. protected ProcedurePrepareLatch latch; protected AbstractPeerProcedure() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java index 5cd3a92e5b69..62ff6f9a92fd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java @@ -157,7 +157,7 @@ protected void doAdd(MutableSegment currentActive, Cell cell, MemStoreSizing mem Cell toAdd = maybeCloneWithAllocator(currentActive, cell, false); boolean mslabUsed = (toAdd != cell); // This cell data is backed by the same byte[] where we read request in RPC(See - // HBASE-15180). By default MSLAB is ON and we might have copied cell to MSLAB area. If + // HBASE-15180). By default, MSLAB is ON and we might have copied cell to MSLAB area. If // not we must do below deep copy. Or else we will keep referring to the bigger chunk of // memory and prevent it from getting GCed. // Copy to MSLAB would not have happened if diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java index a02b05f66ba3..6370d6a79ccb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java @@ -64,7 +64,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) { * Commit all writers. *

* Notice that here we use the same maxSeqId for all output files since we haven't - * find an easy to find enough sequence ids for different output files in some corner cases. See + * found an easy to find enough sequence ids for different output files in some corner cases. See * comments in HBASE-15400 for more details. */ public List commitWriters(long maxSeqId, boolean majorCompaction) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 1a5b5384b01f..ef25068512f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -259,8 +259,8 @@ public abstract class AbstractFSWAL implements WAL { protected final long blocksize; /* - * If more than this many logs, force flush of oldest region to oldest edit goes to disk. If too - * many and we crash, then will take forever replaying. Keep the number of logs tidy. + * If more than this many logs, force flush of oldest region to the oldest edit goes to disk. If + * too many and we crash, then will take forever replaying. Keep the number of logs tidy. */ protected final int maxLogs; @@ -336,7 +336,7 @@ private static final class WALProps { /** * The log file size. Notice that the size may not be accurate if we do asynchronous close in - * sub classes. + * subclasses. */ private final long logSize; @@ -346,7 +346,7 @@ private static final class WALProps { private final long rollTimeNs; /** - * If we do asynchronous close in sub classes, it is possible that when adding WALProps to the + * If we do asynchronous close in subclasses, it is possible that when adding WALProps to the * rolled map, the file is not closed yet, so in cleanOldLogs we should not archive this file, * for safety. */ @@ -404,9 +404,9 @@ private static final class WALProps { protected Supplier hasConsumerTask; private static final int MAX_EPOCH = 0x3FFFFFFF; - // the lowest bit is waitingRoll, which means new writer is created and we are waiting for old + // the lowest bit is waitingRoll, which means new writer is created, and we are waiting for old // writer to be closed. - // the second lowest bit is writerBroken which means the current writer is broken and rollWriter + // the second-lowest bit is writerBroken which means the current writer is broken and rollWriter // is needed. // all other bits are the epoch number of the current writer, this is used to detect whether the // writer is still the one when you issue the sync. @@ -807,7 +807,7 @@ public int getNumLogFiles() { * If the number of un-archived WAL files ('live' WALs) is greater than maximum allowed, check the * first (oldest) WAL, and return those regions which should be flushed so that it can be * let-go/'archived'. - * @return stores of regions (encodedRegionNames) to flush in order to archive oldest WAL file. + * @return stores of regions (encodedRegionNames) to flush in order to archive the oldest WAL file */ Map> findRegionsToForceFlush() throws IOException { Map> regions = null; @@ -861,7 +861,7 @@ private synchronized void cleanOldLogs() { long now = System.nanoTime(); boolean mayLogTooOld = nextLogTooOldNs <= now; ArrayList regionsBlockingWal = null; - // For each log file, look at its Map of regions to highest sequence id; if all sequence ids + // For each log file, look at its Map of regions to the highest sequence id; if all sequence ids // are older than what is currently in memory, the WAL can be GC'd. for (Map.Entry e : this.walFile2Props.entrySet()) { if (!e.getValue().closed) { @@ -1084,7 +1084,7 @@ private Map> rollWriterInternal(boolean force) throws IOExc try { Path oldPath = getOldPath(); Path newPath = getNewPath(); - // Any exception from here on is catastrophic, non-recoverable so we currently abort. + // Any exception from here on is catastrophic, non-recoverable, so we currently abort. W nextWriter = this.createWriterInstance(fs, newPath); if (remoteFs != null) { // create a remote wal if necessary @@ -1107,7 +1107,7 @@ private Map> rollWriterInternal(boolean force) throws IOExc regionsToFlush = findRegionsToForceFlush(); } } catch (CommonFSUtils.StreamLacksCapabilityException exception) { - // If the underlying FileSystem can't do what we ask, treat as IO failure so + // If the underlying FileSystem can't do what we ask, treat as IO failure, so // we'll abort. throw new IOException( "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.", @@ -1195,9 +1195,9 @@ public Void call() throws Exception { throw new IOException(e.getCause()); } } finally { - // in shutdown we may call cleanOldLogs so shutdown this executor in the end. - // In sync replication implementation, we may shutdown a WAL without shutting down the whole - // region server, if we shutdown this executor earlier we may get reject execution exception + // in shutdown, we may call cleanOldLogs so shutdown this executor in the end. + // In sync replication implementation, we may shut down a WAL without shutting down the whole + // region server, if we shut down this executor earlier we may get reject execution exception // and abort the region server logArchiveExecutor.shutdown(); } @@ -1467,8 +1467,8 @@ private static int epoch(int epochAndState) { // return whether we have successfully set readyForRolling to true. private boolean trySetReadyForRolling() { // Check without holding lock first. Usually we will just return here. - // waitingRoll is volatile and unacedEntries is only accessed inside event loop so it is safe to - // check them outside the consumeLock. + // waitingRoll is volatile and unacedEntries is only accessed inside event loop, so it is safe + // to check them outside the consumeLock. if (!waitingRoll(epochAndState) || !unackedAppends.isEmpty()) { return false; } @@ -1532,13 +1532,13 @@ private void syncCompleted(long epochWhenSync, W writer, long processedTxid, lon // changed, i.e, we have already rolled the writer, or the writer is already broken, we should // just skip here, to avoid mess up the state or accidentally release some WAL entries and // cause data corruption. - // The syncCompleted call is on the critical write path so we should try our best to make it + // The syncCompleted call is on the critical write path, so we should try our best to make it // fast. So here we do not hold consumeLock, for increasing performance. It is safe because // there are only 3 possible situations: // 1. For normal case, the only place where we change epochAndState is when rolling the writer. // Before rolling actually happen, we will only change the state to waitingRoll which is another // bit than writerBroken, and when we actually change the epoch, we can make sure that there is - // no out going sync request. So we will always pass the check here and there is no problem. + // no outgoing sync request. So we will always pass the check here and there is no problem. // 2. The writer is broken, but we have not called syncFailed yet. In this case, since // syncFailed and syncCompleted are executed in the same thread, we will just face the same // situation with #1. @@ -1706,7 +1706,7 @@ private void appendAndSync() throws IOException { for (Iterator iter = toWriteAppends.iterator(); iter.hasNext();) { FSWALEntry entry = iter.next(); /** - * For {@link FSHog},here may throws IOException,but for {@link AsyncFSWAL}, here would not + * For {@link FSHog},here may throw IOException,but for {@link AsyncFSWAL}, here would not * throw any IOException. */ boolean appended = appendEntry(writer, entry); @@ -1753,7 +1753,7 @@ private void appendAndSync() throws IOException { } if (writer.getLength() == fileLengthAtLastSync) { // we haven't written anything out, just advance the highestSyncedSequence since we may only - // stamped some region sequence id. + // stamp some region sequence id. if (unackedAppends.isEmpty()) { highestSyncedTxid.set(highestProcessedAppendTxid); finishSync(); @@ -1761,7 +1761,7 @@ private void appendAndSync() throws IOException { } return; } - // reach here means that we have some unsynced data but haven't reached the batch size yet + // reach here means that we have some unsynced data but haven't reached the batch size yet, // but we will not issue a sync directly here even if there are sync requests because we may // have some new data in the ringbuffer, so let's just return here and delay the decision of // whether to issue a sync in the caller method. @@ -1876,12 +1876,12 @@ private boolean shouldScheduleConsumer() { * have its region edit/sequence id assigned else it messes up our unification of mvcc and * sequenceid. On return key will have the region edit/sequence id filled in. *

- * NOTE: This append, at a time that is usually after this call returns, starts an mvcc + * NOTE: This appends, at a time that is usually after this call returns, starts a mvcc * transaction by calling 'begin' wherein which we assign this update a sequenceid. At assignment * time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must * 'complete' the transaction this mvcc transaction by calling * MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it - * in the finally of a try/finally block within which this append lives and any subsequent + * in the finally of a try/finally block within which this appends lives and any subsequent * operations like sync or update of memstore, etc. Get the WriteEntry to pass mvcc out of the * passed in WALKey walKey parameter. Be warned that the WriteEntry is not * immediately available on return from this method. It WILL be available subsequent to a sync of @@ -2034,14 +2034,14 @@ protected final void closeWriter(W writer, Path path) { * Notice that you need to clear the {@link #rollRequested} flag in this method, as the new writer * will begin to work before returning from this method. If we clear the flag after returning from * this call, we may miss a roll request. The implementation class should choose a proper place to - * clear the {@link #rollRequested} flag so we do not miss a roll request, typically before you + * clear the {@link #rollRequested} flag, so we do not miss a roll request, typically before you * start writing to the new writer. */ protected void doReplaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException { Preconditions.checkNotNull(nextWriter); waitForSafePoint(); /** - * For {@link FSHLog},here would shutdown {@link FSHLog.SyncRunner}. + * For {@link FSHLog},here would shut down {@link FSHLog.SyncRunner}. */ doCleanUpResources(); // we will call rollWriter in init method, where we want to create the first writer and @@ -2084,7 +2084,7 @@ protected void doReplaceWriter(Path oldPath, Path newPath, W nextWriter) throws protected void doShutdown() throws IOException { waitForSafePoint(); /** - * For {@link FSHLog},here would shutdown {@link FSHLog.SyncRunner}. + * For {@link FSHLog},here would shut down {@link FSHLog.SyncRunner}. */ doCleanUpResources(); if (this.writer != null) { @@ -2214,7 +2214,7 @@ public void checkLogLowReplication(long checkInterval) { // So here we need to skip the creation of remote writer and make it possible to write the region // close marker. // Setting markerEdit only to true is for transiting from A to S, where we need to give up writing - // any pending wal entries as they will be discarded. The remote cluster will replicated the + // any pending wal entries as they will be discarded. The remote cluster will replicate the // correct data back later. We still need to allow writing marker edits such as close region event // to allow closing a region. @Override @@ -2261,7 +2261,7 @@ private static void usage() { } /** - * Pass one or more log file names and it will either dump out a text version on + * Pass one or more log file names, and it will either dump out a text version on * stdout or split the specified log files. */ public static void main(String[] args) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufWALReader.java index f5e65e08c84c..5d51750ba5f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufWALReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufWALReader.java @@ -121,7 +121,7 @@ public abstract class AbstractProtobufWALReader * Get or create the input stream used by cell decoder. *

* For implementing replication, we may need to limit the bytes we can read, so here we provide a - * method so sub classes can wrap the original input stream. + * method so subclasses can wrap the original input stream. */ protected abstract InputStream getCellCodecInputStream(FSDataInputStream stream); @@ -366,7 +366,7 @@ protected final void readTrailer(FSDataInputStream stream, FileStatus stat) thro this.fileLength = stat.getLen(); this.walEditsStopOffset = this.fileLength; long currentPos = stream.getPos(); - // we will reset walEditsStopOffset if trailer if available + // we will reset walEditsStopOffset if trailer is available trailerPresent = setTrailerIfPresent(stream); if (currentPos != stream.getPos()) { // seek back @@ -509,18 +509,18 @@ protected final IOException extractHiddenEof(Exception ex) { * This is used to determine whether we have already reached the WALTrailer. As the size and magic * are at the end of the WAL file, it is possible that these two options are missing while * writing, so we will consider there is no trailer. And when we actually reach the WALTrailer, we - * will try to decode it as WALKey and we will fail but the error could be vary as it is parsing + * will try to decode it as WALKey and we will fail but the error could be varied as it is parsing * WALTrailer actually. * @return whether this is a WALTrailer and we should throw EOF to upper layer the file is done */ protected final boolean isWALTrailer(long startPosition) throws IOException { - // We have nothing in the WALTrailer PB message now so its size is just a int length size and a + // We have nothing in the WALTrailer PB message now so its size is just an int length size and a // magic at the end int trailerSize = PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT; if (fileLength - startPosition >= trailerSize) { // We still have more than trailerSize bytes before reaching the EOF so this is not a trailer. // We also test for == here because if this is a valid trailer, we can read it while opening - // the reader so we should not reach here + // the reader, so we should not reach here return false; } inputStream.seek(startPosition); @@ -548,7 +548,7 @@ protected final boolean isWALTrailer(long startPosition) throws IOException { return false; } } - // in fact we should not reach here, as this means the trailer bytes are all matched and + // in fact, we should not reach here, as this means the trailer bytes are all matched and // complete, then we should not call this method... return true; } From b87b05c847f00c292664d894c21f83c73d48460d Mon Sep 17 00:00:00 2001 From: Nick Dimiduk Date: Thu, 25 Jan 2024 10:16:27 +0100 Subject: [PATCH 074/109] Revert "HBASE-28325 Enable infra automation to comment on a Jira when a new PR is posted" --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 3b7cf932e3b6..a991e8528d78 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -38,4 +38,4 @@ notifications: commits: commits@hbase.apache.org issues: issues@hbase.apache.org pullrequests: issues@hbase.apache.org - jira_options: link comment + jira_options: link From ef1ddc94c727528a644e2d94be28e845ec72d7a4 Mon Sep 17 00:00:00 2001 From: eab148 <54775485+eab148@users.noreply.github.com> Date: Thu, 25 Jan 2024 08:17:25 -0500 Subject: [PATCH 075/109] HBASE-28327 Add remove(String key, Metric metric) method to MetricRegistry interface (#5647) Co-authored-by: Evie Boland Signed-off-by: Bryan Beaudreault --- .../hadoop/hbase/metrics/MetricRegistry.java | 8 ++++++++ .../metrics/impl/MetricRegistryImpl.java | 5 +++++ .../metrics/impl/TestMetricRegistryImpl.java | 19 +++++++++++++++++++ 3 files changed, 32 insertions(+) diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java index b70526e1c5a9..96f4b313d794 100644 --- a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java +++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java @@ -96,6 +96,14 @@ public interface MetricRegistry extends MetricSet { */ boolean remove(String name); + /** + * Removes the metric with the given name only if it is registered to the provided metric. + * @param name the name of the metric + * @param metric the metric expected to be registered to the given name + * @return true if the metric is removed. + */ + boolean remove(String name, Metric metric); + /** * Return the MetricRegistryInfo object for this registry. * @return MetricRegistryInfo describing the registry. diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java index 1c8927b15b3a..caea02740c60 100644 --- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java +++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java @@ -114,6 +114,11 @@ public boolean remove(String name) { return metrics.remove(name) != null; } + @Override + public boolean remove(String name, Metric metric) { + return metrics.remove(name, metric); + } + @Override public MetricRegistryInfo getMetricRegistryInfo() { return info; diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java index 56b3f0d6a9ee..8cae06361204 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.metrics.impl; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -141,4 +142,22 @@ public void testGetMetrics() { assertEquals(gauge, metrics.get("mygauge")); assertEquals(timer, metrics.get("mytimer")); } + + @Test + public void testRemove() { + CounterImpl counter1 = new CounterImpl(); + CounterImpl counter2 = new CounterImpl(); + registry.register("mycounter", counter1); + + boolean removed = registry.remove("mycounter", counter2); + Optional metric = registry.get("mycounter"); + assertFalse(removed); + assertTrue(metric.isPresent()); + assertEquals(metric.get(), counter1); + + removed = registry.remove("mycounter"); + metric = registry.get("mycounter"); + assertTrue(removed); + assertFalse(metric.isPresent()); + } } From 1c5b5ffcf5de8edeb28372f3c828dc0478e700ae Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Thu, 25 Jan 2024 09:30:47 -0500 Subject: [PATCH 076/109] HBASE-28315 Remove noisy WARN from trying to construct MetricsServlet (#5651) Signed-off-by: Duo Zhang --- .../apache/hadoop/hbase/http/HttpServer.java | 19 ++++++++++++------- .../hadoop/hbase/http/ServletConfig.java | 16 +++++++++++++--- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java index 6bbbc5608a0d..d5af8df1c7fd 100644 --- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java +++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java @@ -171,7 +171,9 @@ public class HttpServer implements FilterContainer { .put("jmx", new ServletConfig("jmx", "/jmx", "org.apache.hadoop.hbase.http.jmx.JMXJsonServlet")) .put("metrics", - new ServletConfig("metrics", "/metrics", "org.apache.hadoop.metrics.MetricsServlet")) + // MetricsServlet is deprecated in hadoop 2.8 and removed in 3.0. We shouldn't expect it, + // so pass false so that we don't create a noisy warn during instantiation. + new ServletConfig("metrics", "/metrics", "org.apache.hadoop.metrics.MetricsServlet", false)) .put("prometheus", new ServletConfig("prometheus", "/prometheus", "org.apache.hadoop.hbase.http.prometheus.PrometheusHadoopServlet")) .build(); @@ -846,16 +848,19 @@ protected void addDefaultServlets(ContextHandlerCollection contexts, Configurati /* register metrics servlets */ String[] enabledServlets = conf.getStrings(METRIC_SERVLETS_CONF_KEY, METRICS_SERVLETS_DEFAULT); for (String enabledServlet : enabledServlets) { - try { - ServletConfig servletConfig = METRIC_SERVLETS.get(enabledServlet); - if (servletConfig != null) { + ServletConfig servletConfig = METRIC_SERVLETS.get(enabledServlet); + if (servletConfig != null) { + try { Class clz = Class.forName(servletConfig.getClazz()); addPrivilegedServlet(servletConfig.getName(), servletConfig.getPathSpec(), clz.asSubclass(HttpServlet.class)); + } catch (Exception e) { + if (servletConfig.isExpected()) { + // metrics are not critical to read/write, so an exception here shouldn't be fatal + // if the class was expected we should warn though + LOG.warn("Couldn't register the servlet " + enabledServlet, e); + } } - } catch (Exception e) { - /* shouldn't be fatal, so warn the user about it */ - LOG.warn("Couldn't register the servlet " + enabledServlet, e); } } } diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServletConfig.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServletConfig.java index befe60957605..366dbfd9f228 100644 --- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServletConfig.java +++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServletConfig.java @@ -23,14 +23,20 @@ @InterfaceAudience.Private class ServletConfig { - private String name; - private String pathSpec; - private String clazz; + private final String name; + private final String pathSpec; + private final String clazz; + private final boolean expected; public ServletConfig(String name, String pathSpec, String clazz) { + this(name, pathSpec, clazz, true); + } + + public ServletConfig(String name, String pathSpec, String clazz, boolean expected) { this.name = name; this.pathSpec = pathSpec; this.clazz = clazz; + this.expected = expected; } public String getName() { @@ -44,4 +50,8 @@ public String getPathSpec() { public String getClazz() { return clazz; } + + public boolean isExpected() { + return expected; + } } From 73cb0dddf8bcd0768e2e8eb7cf9d1bc3eddc1ea1 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 26 Jan 2024 10:03:42 +0000 Subject: [PATCH 077/109] HBASE-28303 Interrupt cache prefetch thread when a heap usage threshold is reached (#5615) Signed-off-by: Tak Lon (Stephen) Wu Signed-off-by: Peter Somogyi --- .../hadoop/hbase/io/hfile/CacheConfig.java | 28 ++++++++++++++ .../hbase/io/hfile/HFilePreadReader.java | 24 ++++++++---- .../hadoop/hbase/io/hfile/TestPrefetch.java | 38 +++++++++++++++++++ .../bucket/TestBucketCachePersister.java | 16 ++++---- 4 files changed, 92 insertions(+), 14 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java index 4587eced6163..f89a6194cefb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java @@ -99,6 +99,12 @@ public class CacheConfig { public static final String BUCKETCACHE_PERSIST_INTERVAL_KEY = "hbase.bucketcache.persist.intervalinmillis"; + /** + * Configuration key to set the heap usage threshold limit once prefetch threads should be + * interrupted. + */ + public static final String PREFETCH_HEAP_USAGE_THRESHOLD = "hbase.rs.prefetchheapusage"; + // Defaults public static final boolean DEFAULT_CACHE_DATA_ON_READ = true; public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = false; @@ -111,6 +117,7 @@ public class CacheConfig { public static final boolean DEFAULT_CACHE_COMPACTED_BLOCKS_ON_WRITE = false; public static final boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true; public static final long DEFAULT_CACHE_COMPACTED_BLOCKS_ON_WRITE_THRESHOLD = Long.MAX_VALUE; + public static final double DEFAULT_PREFETCH_HEAP_USAGE_THRESHOLD = 1d; /** * Whether blocks should be cached on read (default is on if there is a cache but this can be @@ -157,6 +164,8 @@ public class CacheConfig { private final ByteBuffAllocator byteBuffAllocator; + private final double heapUsageThreshold; + /** * Create a cache configuration using the specified configuration object and defaults for family * level settings. Only use if no column family context. @@ -201,6 +210,8 @@ public CacheConfig(Configuration conf, ColumnFamilyDescriptor family, BlockCache this.cacheCompactedDataOnWrite = conf.getBoolean(CACHE_COMPACTED_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_COMPACTED_BLOCKS_ON_WRITE); this.cacheCompactedDataOnWriteThreshold = getCacheCompactedBlocksOnWriteThreshold(conf); + this.heapUsageThreshold = + conf.getDouble(PREFETCH_HEAP_USAGE_THRESHOLD, DEFAULT_PREFETCH_HEAP_USAGE_THRESHOLD); this.blockCache = blockCache; this.byteBuffAllocator = byteBuffAllocator; } @@ -222,6 +233,7 @@ public CacheConfig(CacheConfig cacheConf) { this.dropBehindCompaction = cacheConf.dropBehindCompaction; this.blockCache = cacheConf.blockCache; this.byteBuffAllocator = cacheConf.byteBuffAllocator; + this.heapUsageThreshold = cacheConf.heapUsageThreshold; } private CacheConfig() { @@ -237,6 +249,7 @@ private CacheConfig() { this.dropBehindCompaction = false; this.blockCache = null; this.byteBuffAllocator = ByteBuffAllocator.HEAP; + this.heapUsageThreshold = DEFAULT_PREFETCH_HEAP_USAGE_THRESHOLD; } /** @@ -386,6 +399,17 @@ public boolean shouldReadBlockFromCache(BlockType blockType) { return false; } + /** + * Checks if the current heap usage is below the threshold configured by + * "hbase.rs.prefetchheapusage" (0.8 by default). + */ + public boolean isHeapUsageBelowThreshold() { + double total = Runtime.getRuntime().maxMemory(); + double available = Runtime.getRuntime().freeMemory(); + double usedRatio = 1d - (available / total); + return heapUsageThreshold > usedRatio; + } + /** * If we make sure the block could not be cached, we will not acquire the lock otherwise we will * acquire lock @@ -413,6 +437,10 @@ public ByteBuffAllocator getByteBuffAllocator() { return this.byteBuffAllocator; } + public double getHeapUsageThreshold() { + return heapUsageThreshold; + } + private long getCacheCompactedBlocksOnWriteThreshold(Configuration conf) { long cacheCompactedBlocksOnWriteThreshold = conf.getLong(CACHE_COMPACTED_BLOCKS_ON_WRITE_THRESHOLD_KEY, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java index 92f6a8169f32..6063ffe68891 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java @@ -106,13 +106,23 @@ public void run() { HFileBlock block = prefetchStreamReader.readBlock(offset, onDiskSizeOfNextBlock, /* cacheBlock= */true, /* pread= */false, false, false, null, null, true); try { - if (!cacheConf.isInMemory() && !cache.blockFitsIntoTheCache(block).orElse(true)) { - LOG.warn( - "Interrupting prefetch for file {} because block {} of size {} " - + "doesn't fit in the available cache space.", - path, cacheKey, block.getOnDiskSizeWithHeader()); - interrupted = true; - break; + if (!cacheConf.isInMemory()) { + if (!cache.blockFitsIntoTheCache(block).orElse(true)) { + LOG.warn( + "Interrupting prefetch for file {} because block {} of size {} " + + "doesn't fit in the available cache space.", + path, cacheKey, block.getOnDiskSizeWithHeader()); + interrupted = true; + break; + } + if (!cacheConf.isHeapUsageBelowThreshold()) { + LOG.warn( + "Interrupting prefetch because heap usage is above the threshold: {} " + + "configured via {}", + cacheConf.getHeapUsageThreshold(), CacheConfig.PREFETCH_HEAP_USAGE_THRESHOLD); + interrupted = true; + break; + } } onDiskSizeOfNextBlock = block.getNextBlockOnDiskSize(); offset += block.getOnDiskSizeWithHeader(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java index 0b45a930dceb..85b9199638c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java @@ -42,6 +42,7 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -156,6 +157,43 @@ public void testPrefetchBlockCacheDisabled() throws Exception { poolExecutor.getCompletedTaskCount() + poolExecutor.getQueue().size()); } + @Test + public void testPrefetchHeapUsageAboveThreshold() throws Exception { + ColumnFamilyDescriptor columnFamilyDescriptor = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f")).setPrefetchBlocksOnOpen(true) + .setBlockCacheEnabled(true).build(); + HFileContext meta = new HFileContextBuilder().withBlockSize(DATA_BLOCK_SIZE).build(); + Configuration newConf = new Configuration(conf); + newConf.setDouble(CacheConfig.PREFETCH_HEAP_USAGE_THRESHOLD, 0.1); + CacheConfig cacheConfig = + new CacheConfig(newConf, columnFamilyDescriptor, blockCache, ByteBuffAllocator.HEAP); + Path storeFile = writeStoreFile("testPrefetchHeapUsageAboveThreshold", meta, cacheConfig); + MutableInt cachedCount = new MutableInt(0); + MutableInt unCachedCount = new MutableInt(0); + readStoreFile(storeFile, (r, o) -> { + HFileBlock block = null; + try { + block = r.readBlock(o, -1, false, true, false, true, null, null); + } catch (IOException e) { + fail(e.getMessage()); + } + return block; + }, (key, block) -> { + boolean isCached = blockCache.getBlock(key, true, false, true) != null; + if ( + block.getBlockType() == BlockType.DATA || block.getBlockType() == BlockType.ROOT_INDEX + || block.getBlockType() == BlockType.INTERMEDIATE_INDEX + ) { + if (isCached) { + cachedCount.increment(); + } else { + unCachedCount.increment(); + } + } + }, cacheConfig); + assertTrue(unCachedCount.compareTo(cachedCount) > 0); + } + @Test public void testPrefetch() throws Exception { TraceUtil.trace(() -> { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java index f6d3efa9015d..a39df7e14715 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java @@ -86,9 +86,10 @@ public Configuration setupBucketCacheConfig(long bucketCachePersistInterval) thr return conf; } - public BucketCache setupBucketCache(Configuration conf) throws IOException { - BucketCache bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, - constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen, + public BucketCache setupBucketCache(Configuration conf, String persistentCacheFile) + throws IOException { + BucketCache bucketCache = new BucketCache("file:" + testDir + "/" + persistentCacheFile, + capacitySize, constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen, testDir + "/bucket.persistence", 60 * 1000, conf); return bucketCache; } @@ -103,7 +104,7 @@ public void cleanupBucketCache(BucketCache bucketCache) throws IOException { public void testPrefetchPersistenceCrash() throws Exception { long bucketCachePersistInterval = 3000; Configuration conf = setupBucketCacheConfig(bucketCachePersistInterval); - BucketCache bucketCache = setupBucketCache(conf); + BucketCache bucketCache = setupBucketCache(conf, "testPrefetchPersistenceCrash"); CacheConfig cacheConf = new CacheConfig(conf, bucketCache); FileSystem fs = HFileSystem.get(conf); // Load Cache @@ -121,7 +122,7 @@ public void testPrefetchPersistenceCrash() throws Exception { public void testPrefetchPersistenceCrashNegative() throws Exception { long bucketCachePersistInterval = Long.MAX_VALUE; Configuration conf = setupBucketCacheConfig(bucketCachePersistInterval); - BucketCache bucketCache = setupBucketCache(conf); + BucketCache bucketCache = setupBucketCache(conf, "testPrefetchPersistenceCrashNegative"); CacheConfig cacheConf = new CacheConfig(conf, bucketCache); FileSystem fs = HFileSystem.get(conf); // Load Cache @@ -134,7 +135,7 @@ public void testPrefetchPersistenceCrashNegative() throws Exception { @Test public void testPrefetchListUponBlockEviction() throws Exception { Configuration conf = setupBucketCacheConfig(200); - BucketCache bucketCache = setupBucketCache(conf); + BucketCache bucketCache = setupBucketCache(conf, "testPrefetchListUponBlockEviction"); CacheConfig cacheConf = new CacheConfig(conf, bucketCache); FileSystem fs = HFileSystem.get(conf); // Load Blocks in cache @@ -156,7 +157,8 @@ public void testPrefetchListUponBlockEviction() throws Exception { @Test public void testPrefetchBlockEvictionWhilePrefetchRunning() throws Exception { Configuration conf = setupBucketCacheConfig(200); - BucketCache bucketCache = setupBucketCache(conf); + BucketCache bucketCache = + setupBucketCache(conf, "testPrefetchBlockEvictionWhilePrefetchRunning"); CacheConfig cacheConf = new CacheConfig(conf, bucketCache); FileSystem fs = HFileSystem.get(conf); // Load Blocks in cache From addb4e7fc3c0b6e486281033499b6de1f368d686 Mon Sep 17 00:00:00 2001 From: guluo Date: Fri, 26 Jan 2024 18:17:22 +0800 Subject: [PATCH 078/109] HBASE-28332 Type conversion is no need in method CompactionChecker.chore() (#5653) Signed-off-by: Duo Zhang --- .../apache/hadoop/hbase/regionserver/HRegionServer.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 a77fa0cd879e..dfb8e2a204fe 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 @@ -1684,14 +1684,13 @@ private static class CompactionChecker extends ScheduledChore { @Override protected void chore() { - for (Region r : this.instance.onlineRegions.values()) { + for (HRegion hr : this.instance.onlineRegions.values()) { // If region is read only or compaction is disabled at table level, there's no need to // iterate through region's stores - if (r == null || r.isReadOnly() || !r.getTableDescriptor().isCompactionEnabled()) { + if (hr == null || hr.isReadOnly() || !hr.getTableDescriptor().isCompactionEnabled()) { continue; } - HRegion hr = (HRegion) r; for (HStore s : hr.stores.values()) { try { long multiplier = s.getCompactionCheckMultiplier(); @@ -1719,7 +1718,7 @@ protected void chore() { } } } catch (IOException e) { - LOG.warn("Failed major compaction check on " + r, e); + LOG.warn("Failed major compaction check on " + hr, e); } } } From 9c79a7f267e974d36745ef5ec2adbbad871fd64c Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 26 Jan 2024 18:53:39 +0800 Subject: [PATCH 079/109] HBASE-28331 Client integration test fails after upgrading hadoop3 version to 3.3.x (#5652) Signed-off-by: Xin Sun --- dev-support/Jenkinsfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index 7d2ef32df6eb..9c03d97a3c44 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -152,7 +152,7 @@ pipeline { } stage ('hadoop 3 cache') { environment { - HADOOP3_VERSION="3.1.1" + HADOOP3_VERSION="3.3.5" } steps { // directory must be unique for each parallel stage, because jenkins runs them in the same workspace :( From 11458ec57a6f756510ea6f6aa316290ccb88b694 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 26 Jan 2024 21:36:29 +0800 Subject: [PATCH 080/109] HBASE-28333 Refactor TestClientTimeouts to make it more clear that what we want to test (#5655) Signed-off-by: Xin Sun --- .../hbase/client/TestClientTimeouts.java | 67 +++++++------------ 1 file changed, 26 insertions(+), 41 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index d358695c5f9b..9a92f4b1aa55 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -17,9 +17,10 @@ */ package org.apache.hadoop.hbase.client; -import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.IOException; import java.net.SocketAddress; import java.net.SocketTimeoutException; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException; import org.apache.hadoop.hbase.ipc.AbstractRpcClient; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -67,9 +67,6 @@ public class TestClientTimeouts { @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(SLAVES); - // Set the custom RPC client with random timeouts as the client - TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, - RandomTimeoutRpcClient.class.getName()); } @AfterClass @@ -77,51 +74,39 @@ public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } + private Connection createConnection() { + // Ensure the HBaseAdmin uses a new connection by changing Configuration. + Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); + // Set the custom RPC client with random timeouts as the client + conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, + RandomTimeoutRpcClient.class.getName()); + conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); + for (;;) { + try { + return ConnectionFactory.createConnection(conf); + } catch (IOException e) { + // since we randomly throw SocketTimeoutException, it is possible that we fail when creating + // the Connection, but this is not what we want to test here, so just ignore it and try + // again + } + } + } + /** * Test that a client that fails an RPC to the master retries properly and doesn't throw any * unexpected exceptions. */ @Test public void testAdminTimeout() throws Exception { - boolean lastFailed = false; - int initialInvocations = invokations.get(); - RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); - - try { - for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) { - lastFailed = false; - // Ensure the HBaseAdmin uses a new connection by changing Configuration. - Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); - conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - Admin admin = null; - Connection connection = null; - try { - connection = ConnectionFactory.createConnection(conf); - admin = connection.getAdmin(); - admin.balancerSwitch(false, false); - } catch (MasterRegistryFetchException ex) { - // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get - // a MasterRegistryFetchException. It's a bug if we get other exceptions. - lastFailed = true; - } finally { - if (admin != null) { - admin.close(); - if (admin.getConnection().isClosed()) { - rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); - } - } - if (connection != null) { - connection.close(); - } - } + try (Connection conn = createConnection(); Admin admin = conn.getAdmin()) { + int initialInvocations = invokations.get(); + boolean balanceEnabled = admin.isBalancerEnabled(); + for (int i = 0; i < 5; i++) { + assertEquals(balanceEnabled, admin.balancerSwitch(!balanceEnabled, false)); + balanceEnabled = !balanceEnabled; } // Ensure the RandomTimeoutRpcEngine is actually being used. - assertFalse(lastFailed); assertTrue(invokations.get() > initialInvocations); - } finally { - rpcClient.close(); } } From d82a892fb8d50d348a0d1e80b3d5e147d195f3e6 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Fri, 26 Jan 2024 23:50:03 +0800 Subject: [PATCH 081/109] HBASE-28325 Enable infra automation to add label on a Jira when a new PR is posted (#5656) Signed-off-by: Bryan Beaudreault --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index a991e8528d78..d1abb7131437 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -38,4 +38,4 @@ notifications: commits: commits@hbase.apache.org issues: issues@hbase.apache.org pullrequests: issues@hbase.apache.org - jira_options: link + jira_options: link label From 3b2e98192091368dc961eaef8e25c679f99aa8fd Mon Sep 17 00:00:00 2001 From: Rahul Kumar Date: Sun, 28 Jan 2024 00:51:49 +0530 Subject: [PATCH 082/109] HBASE-28151 Option to allow/disallow bypassing pre transit check for assing/unassign (#5493) Signed-off-by: Andrew Purtell Signed-off-by: Viraj Jasani Signed-off-by: Ravi Kishore Valeti --- .../apache/hadoop/hbase/client/HBaseHbck.java | 9 ++-- .../org/apache/hadoop/hbase/client/Hbck.java | 43 +++++++++++++------ .../shaded/protobuf/RequestConverter.java | 8 ++-- .../main/protobuf/server/master/Master.proto | 2 + .../hbase/master/MasterRpcServices.java | 9 ++-- .../master/assignment/AssignmentManager.java | 22 +++++++--- .../procedure/TruncateRegionProcedure.java | 4 +- .../apache/hadoop/hbase/client/TestHbck.java | 19 +++++++- .../master/assignment/TestRegionBypass.java | 2 +- 9 files changed, 81 insertions(+), 37 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java index 8df0504b2a9a..83b53ccba3c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java @@ -135,10 +135,11 @@ public Map setRegionStateInMeta( } @Override - public List assigns(List encodedRegionNames, boolean override) throws IOException { + public List assigns(List encodedRegionNames, boolean override, boolean force) + throws IOException { try { AssignsResponse response = this.hbck.assigns(rpcControllerFactory.newController(), - RequestConverter.toAssignRegionsRequest(encodedRegionNames, override)); + RequestConverter.toAssignRegionsRequest(encodedRegionNames, override, force)); return response.getPidList(); } catch (ServiceException se) { LOG.debug(toCommaDelimitedString(encodedRegionNames), se); @@ -147,11 +148,11 @@ public List assigns(List encodedRegionNames, boolean override) thr } @Override - public List unassigns(List encodedRegionNames, boolean override) + public List unassigns(List encodedRegionNames, boolean override, boolean force) throws IOException { try { UnassignsResponse response = this.hbck.unassigns(rpcControllerFactory.newController(), - RequestConverter.toUnassignRegionsRequest(encodedRegionNames, override)); + RequestConverter.toUnassignRegionsRequest(encodedRegionNames, override, force)); return response.getPidList(); } catch (ServiceException se) { LOG.debug(toCommaDelimitedString(encodedRegionNames), se); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java index b5ba25058838..6baa876f9387 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java @@ -62,19 +62,26 @@ public interface Hbck extends Abortable, Closeable { * good if many Regions to online -- and it will schedule the assigns even in the case where * Master is initializing (as long as the ProcedureExecutor is up). Does NOT call Coprocessor * hooks. - * @param override You need to add the override for case where a region has previously - * been bypassed. When a Procedure has been bypassed, a Procedure will - * have completed but no other Procedure will be able to make progress - * on the target entity (intentionally). This override flag will - * override this fencing mechanism. + * @param override You need to add override for unset of the procedure from + * RegionStateNode without byPassing preTransitCheck + * @param force You need to add force for case where a region has previously been + * bypassed. When a Procedure has been bypassed, a Procedure will have + * completed but no other Procedure will be able to make progress on the + * target entity (intentionally). Skips preTransitCheck only when + * selected along with override option * @param encodedRegionNames Region encoded names; e.g. 1588230740 is the hard-coded encoding for * hbase:meta region and de00010733901a05f5a2a3a382e27dd4 is an example * of what a random user-space encoded Region name looks like. */ - List assigns(List encodedRegionNames, boolean override) throws IOException; + List assigns(List encodedRegionNames, boolean override, boolean force) + throws IOException; + + default List assigns(List encodedRegionNames, boolean override) throws IOException { + return assigns(encodedRegionNames, override, true); + } default List assigns(List encodedRegionNames) throws IOException { - return assigns(encodedRegionNames, false); + return assigns(encodedRegionNames, false, false); } /** @@ -82,19 +89,27 @@ default List assigns(List encodedRegionNames) throws IOException { * at a time -- good if many Regions to offline -- and it will schedule the assigns even in the * case where Master is initializing (as long as the ProcedureExecutor is up). Does NOT call * Coprocessor hooks. - * @param override You need to add the override for case where a region has previously - * been bypassed. When a Procedure has been bypassed, a Procedure will - * have completed but no other Procedure will be able to make progress - * on the target entity (intentionally). This override flag will - * override this fencing mechanism. + * @param override You need to add override for unset of the procedure from + * RegionStateNode without byPassing preTransitCheck + * @param force You need to add force for case where a region has previously been + * bypassed. When a Procedure has been bypassed, a Procedure will have + * completed but no other Procedure will be able to make progress on the + * target entity (intentionally). Skips preTransitCheck only when + * selected along with override option * @param encodedRegionNames Region encoded names; e.g. 1588230740 is the hard-coded encoding for * hbase:meta region and de00010733901a05f5a2a3a382e27dd4 is an example * of what a random user-space encoded Region name looks like. */ - List unassigns(List encodedRegionNames, boolean override) throws IOException; + List unassigns(List encodedRegionNames, boolean override, boolean force) + throws IOException; + + default List unassigns(List encodedRegionNames, boolean override) + throws IOException { + return unassigns(encodedRegionNames, override, true); + } default List unassigns(List encodedRegionNames) throws IOException { - return unassigns(encodedRegionNames, false); + return unassigns(encodedRegionNames, false, true); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java index 377b46494633..ce12aaea0d24 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java @@ -1590,17 +1590,17 @@ private static List toProtoServerNames(List // HBCK2 public static MasterProtos.AssignsRequest toAssignRegionsRequest(List encodedRegionNames, - boolean override) { + boolean override, boolean force) { MasterProtos.AssignsRequest.Builder b = MasterProtos.AssignsRequest.newBuilder(); return b.addAllRegion(toEncodedRegionNameRegionSpecifiers(encodedRegionNames)) - .setOverride(override).build(); + .setOverride(override).setForce(force).build(); } public static MasterProtos.UnassignsRequest - toUnassignRegionsRequest(List encodedRegionNames, boolean override) { + toUnassignRegionsRequest(List encodedRegionNames, boolean override, boolean force) { MasterProtos.UnassignsRequest.Builder b = MasterProtos.UnassignsRequest.newBuilder(); return b.addAllRegion(toEncodedRegionNameRegionSpecifiers(encodedRegionNames)) - .setOverride(override).build(); + .setOverride(override).setForce(force).build(); } public static MasterProtos.ScheduleServerCrashProcedureRequest diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index b1e750f4d920..a8adaa27453f 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -1302,6 +1302,7 @@ message SetRegionStateInMetaResponse { message AssignsRequest { repeated RegionSpecifier region = 1; optional bool override = 2 [default = false]; + optional bool force = 3 [default = false]; } /** Like Admin's AssignRegionResponse except it can @@ -1317,6 +1318,7 @@ message AssignsResponse { message UnassignsRequest { repeated RegionSpecifier region = 1; optional bool override = 2 [default = false]; + optional bool force= 3 [default = false]; } /** Like Admin's UnassignRegionResponse except it can diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 6d330d6eb791..1da8e03d179e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -2714,6 +2714,7 @@ public MasterProtos.AssignsResponse assigns(RpcController controller, MasterProtos.AssignsResponse.Builder responseBuilder = MasterProtos.AssignsResponse.newBuilder(); final boolean override = request.getOverride(); + final boolean force = request.getForce(); LOG.info("{} assigns, override={}", server.getClientIdAuditPrefix(), override); for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) { final RegionInfo info = getRegionInfo(rs); @@ -2721,7 +2722,7 @@ public MasterProtos.AssignsResponse assigns(RpcController controller, LOG.info("Unknown region {}", rs); continue; } - responseBuilder.addPid(Optional.ofNullable(am.createOneAssignProcedure(info, override)) + responseBuilder.addPid(Optional.ofNullable(am.createOneAssignProcedure(info, override, force)) .map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID)); } return responseBuilder.build(); @@ -2741,6 +2742,7 @@ public MasterProtos.UnassignsResponse unassigns(RpcController controller, MasterProtos.UnassignsResponse.Builder responseBuilder = MasterProtos.UnassignsResponse.newBuilder(); final boolean override = request.getOverride(); + final boolean force = request.getForce(); LOG.info("{} unassigns, override={}", server.getClientIdAuditPrefix(), override); for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) { final RegionInfo info = getRegionInfo(rs); @@ -2748,8 +2750,9 @@ public MasterProtos.UnassignsResponse unassigns(RpcController controller, LOG.info("Unknown region {}", rs); continue; } - responseBuilder.addPid(Optional.ofNullable(am.createOneUnassignProcedure(info, override)) - .map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID)); + responseBuilder + .addPid(Optional.ofNullable(am.createOneUnassignProcedure(info, override, force)) + .map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID)); } return responseBuilder.build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 49cf29ee61d9..9cee9f87ce2f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -765,11 +765,14 @@ private void preTransitCheck(RegionStateNode regionNode, RegionState.State[] exp * @param override If false, check RegionState is appropriate for assign; if not throw exception. */ private TransitRegionStateProcedure createAssignProcedure(RegionInfo regionInfo, ServerName sn, - boolean override) throws IOException { + boolean override, boolean force) throws IOException { RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo); regionNode.lock(); try { if (override) { + if (!force) { + preTransitCheck(regionNode, STATES_EXPECTED_ON_ASSIGN); + } if (regionNode.getProcedure() != null) { regionNode.unsetProcedure(regionNode.getProcedure()); } @@ -787,7 +790,7 @@ private TransitRegionStateProcedure createAssignProcedure(RegionInfo regionInfo, /** * Create an assign TransitRegionStateProcedure. Does NO checking of RegionState. Presumes * appriopriate state ripe for assign. - * @see #createAssignProcedure(RegionInfo, ServerName, boolean) + * @see #createAssignProcedure(RegionInfo, ServerName, boolean, boolean) */ private TransitRegionStateProcedure createAssignProcedure(RegionStateNode regionNode, ServerName targetServer) { @@ -801,7 +804,7 @@ private TransitRegionStateProcedure createAssignProcedure(RegionStateNode region } public long assign(RegionInfo regionInfo, ServerName sn) throws IOException { - TransitRegionStateProcedure proc = createAssignProcedure(regionInfo, sn, false); + TransitRegionStateProcedure proc = createAssignProcedure(regionInfo, sn, false, false); ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc); return proc.getProcId(); } @@ -817,7 +820,7 @@ public long assign(RegionInfo regionInfo) throws IOException { */ public Future assignAsync(RegionInfo regionInfo, ServerName sn) throws IOException { return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), - createAssignProcedure(regionInfo, sn, false)); + createAssignProcedure(regionInfo, sn, false, false)); } /** @@ -961,10 +964,11 @@ static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure * method is called from HBCK2. * @return an assign or null */ - public TransitRegionStateProcedure createOneAssignProcedure(RegionInfo ri, boolean override) { + public TransitRegionStateProcedure createOneAssignProcedure(RegionInfo ri, boolean override, + boolean force) { TransitRegionStateProcedure trsp = null; try { - trsp = createAssignProcedure(ri, null, override); + trsp = createAssignProcedure(ri, null, override, force); } catch (IOException ioe) { LOG.info( "Failed {} assign, override={}" @@ -978,12 +982,16 @@ public TransitRegionStateProcedure createOneAssignProcedure(RegionInfo ri, boole * Create one TransitRegionStateProcedure to unassign a region. This method is called from HBCK2. * @return an unassign or null */ - public TransitRegionStateProcedure createOneUnassignProcedure(RegionInfo ri, boolean override) { + public TransitRegionStateProcedure createOneUnassignProcedure(RegionInfo ri, boolean override, + boolean force) { RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(ri); TransitRegionStateProcedure trsp = null; regionNode.lock(); try { if (override) { + if (!force) { + preTransitCheck(regionNode, STATES_EXPECTED_ON_UNASSIGN_OR_MOVE); + } if (regionNode.getProcedure() != null) { regionNode.unsetProcedure(regionNode.getProcedure()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index 9730391baf22..83722d6c1dca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java @@ -210,10 +210,10 @@ public TableOperationType getTableOperationType() { private TransitRegionStateProcedure createUnAssignProcedures(MasterProcedureEnv env) throws IOException { - return env.getAssignmentManager().createOneUnassignProcedure(getRegion(), true); + return env.getAssignmentManager().createOneUnassignProcedure(getRegion(), true, true); } private TransitRegionStateProcedure createAssignProcedures(MasterProcedureEnv env) { - return env.getAssignmentManager().createOneAssignProcedure(getRegion(), true); + return env.getAssignmentManager().createOneAssignProcedure(getRegion(), true, true); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java index 406b25fed4e0..360641e64b7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java @@ -250,11 +250,20 @@ public void testAssigns() throws Exception { for (long pid : pids) { assertEquals(Procedure.NO_PROC_ID, pid); } - // If we pass override, then we should be able to unassign EVEN THOUGH Regions already + // Rerun the unassign with override. Should fail for all Regions since they already + // unassigned; failed + // unassign will manifest as all pids being -1 (ever since HBASE-24885). + pids = hbck.unassigns( + regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()), true, false); + waitOnPids(pids); + for (long pid : pids) { + assertEquals(Procedure.NO_PROC_ID, pid); + } + // If we pass force, then we should be able to unassign EVEN THOUGH Regions already // unassigned.... makes for a mess but operator might want to do this at an extreme when // doing fixup of broke cluster. pids = hbck.unassigns( - regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()), true); + regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()), true, true); waitOnPids(pids); for (long pid : pids) { assertNotEquals(Procedure.NO_PROC_ID, pid); @@ -283,6 +292,12 @@ public void testAssigns() throws Exception { LOG.info("RS: {}", rs.toString()); assertTrue(rs.toString(), rs.isOpened()); } + // Rerun the assign with override. Should fail for all Regions since they already assigned + pids = hbck.assigns( + regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()), true, false); + for (long pid : pids) { + assertEquals(Procedure.NO_PROC_ID, pid); + } // What happens if crappy region list passed? pids = hbck.assigns( Arrays.stream(new String[] { "a", "some rubbish name" }).collect(Collectors.toList())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionBypass.java index 61520873240c..8295da82f49c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionBypass.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionBypass.java @@ -125,7 +125,7 @@ public void testBypass() throws IOException, InterruptedException { .getMasterProcedureExecutor().getActiveProcIds().isEmpty()); // Now assign with the override flag. for (RegionInfo ri : regions) { - TEST_UTIL.getHbck().assigns(Arrays. asList(ri.getEncodedName()), true); + TEST_UTIL.getHbck().assigns(Arrays. asList(ri.getEncodedName()), true, true); } TEST_UTIL.waitFor(60000, () -> TEST_UTIL.getHBaseCluster().getMaster() .getMasterProcedureExecutor().getActiveProcIds().isEmpty()); From 97574a265b31a4b8ecdbad1709c0226cb3212825 Mon Sep 17 00:00:00 2001 From: Charles Connell Date: Mon, 29 Jan 2024 08:04:06 -0500 Subject: [PATCH 083/109] HBASE-28317 Expose client TLS certificate on RpcCallContext (#5644) Signed-off-by: Duo Zhang Signed-off-by: Bryan Beaudreault --- .../hadoop/hbase/ipc/NettyRpcServer.java | 31 +++++++++++++++++-- .../hadoop/hbase/ipc/RpcCallContext.java | 8 +++++ .../apache/hadoop/hbase/ipc/ServerCall.java | 9 ++++++ .../hadoop/hbase/ipc/ServerRpcConnection.java | 2 ++ .../namequeues/TestNamedQueueRecorder.java | 6 ++++ .../hbase/namequeues/TestRpcLogDetails.java | 6 ++++ .../region/TestRegionProcedureStore.java | 6 ++++ 7 files changed, 65 insertions(+), 3 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index ceff84a90e11..c291338e40c2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -27,9 +27,12 @@ import java.io.InterruptedIOException; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.security.cert.Certificate; +import java.security.cert.X509Certificate; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import javax.net.ssl.SSLPeerUnverifiedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseServerBase; @@ -166,10 +169,10 @@ protected void initChannel(Channel ch) throws Exception { ChannelPipeline pipeline = ch.pipeline(); FixedLengthFrameDecoder preambleDecoder = new FixedLengthFrameDecoder(6); preambleDecoder.setSingleDecode(true); + NettyServerRpcConnection conn = createNettyServerRpcConnection(ch); if (conf.getBoolean(HBASE_SERVER_NETTY_TLS_ENABLED, false)) { - initSSL(pipeline, conf.getBoolean(HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, true)); + initSSL(pipeline, conn, conf.getBoolean(HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, true)); } - NettyServerRpcConnection conn = createNettyServerRpcConnection(ch); pipeline.addLast(NettyRpcServerPreambleHandler.DECODER_NAME, preambleDecoder) .addLast(new NettyRpcServerPreambleHandler(NettyRpcServer.this, conn)) // We need NettyRpcServerResponseEncoder here because NettyRpcServerPreambleHandler may @@ -378,7 +381,7 @@ public int getNumOpenConnections() { return allChannels.size(); } - private void initSSL(ChannelPipeline p, boolean supportPlaintext) + private void initSSL(ChannelPipeline p, NettyServerRpcConnection conn, boolean supportPlaintext) throws X509Exception, IOException { SslContext nettySslContext = getSslContext(); @@ -413,6 +416,28 @@ private void initSSL(ChannelPipeline p, boolean supportPlaintext) sslHandler.setWrapDataSize( conf.getInt(HBASE_SERVER_NETTY_TLS_WRAP_SIZE, DEFAULT_HBASE_SERVER_NETTY_TLS_WRAP_SIZE)); + sslHandler.handshakeFuture().addListener(future -> { + try { + Certificate[] certificates = sslHandler.engine().getSession().getPeerCertificates(); + if (certificates != null && certificates.length > 0) { + conn.clientCertificateChain = (X509Certificate[]) certificates; + } else if (sslHandler.engine().getNeedClientAuth()) { + LOG.error( + "Could not get peer certificate on TLS connection from {}, although one is required", + remoteAddress); + } + } catch (SSLPeerUnverifiedException e) { + if (sslHandler.engine().getNeedClientAuth()) { + LOG.error( + "Could not get peer certificate on TLS connection from {}, although one is required", + remoteAddress, e); + } + } catch (Exception e) { + LOG.error("Unexpected error getting peer certificate for TLS connection from {}", + remoteAddress, e); + } + }); + p.addLast("ssl", sslHandler); LOG.debug("SSL handler added for channel: {}", p.channel()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java index 4f299b4a85d2..43432324579b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.net.InetAddress; +import java.security.cert.X509Certificate; import java.util.Optional; import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; @@ -60,6 +61,13 @@ default Optional getRequestUserName() { return getRequestUser().map(User::getShortName); } + /** + * Returns the TLS certificate(s) that the client presented to this HBase server when making its + * connection. TLS is orthogonal to Kerberos, so this is unrelated to + * {@link RpcCallContext#getRequestUser()}. Both, one, or neither may be present. + */ + Optional getClientCertificateChain(); + /** Returns Address of remote client in this call */ InetAddress getRemoteAddress(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index a2c578fd6664..25d153c068aa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.InetAddress; import java.nio.ByteBuffer; +import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -95,6 +96,7 @@ public abstract class ServerCall implements RpcCa protected final User user; protected final InetAddress remoteAddress; + protected final X509Certificate[] clientCertificateChain; protected RpcCallback rpcCallback; private long responseCellSize = 0; @@ -135,9 +137,11 @@ public abstract class ServerCall implements RpcCa if (connection != null) { this.user = connection.user; this.retryImmediatelySupported = connection.retryImmediatelySupported; + this.clientCertificateChain = connection.clientCertificateChain; } else { this.user = null; this.retryImmediatelySupported = false; + this.clientCertificateChain = null; } this.remoteAddress = remoteAddress; this.timeout = timeout; @@ -499,6 +503,11 @@ public Optional getRequestUser() { return Optional.ofNullable(user); } + @Override + public Optional getClientCertificateChain() { + return Optional.ofNullable(clientCertificateChain); + } + @Override public InetAddress getRemoteAddress() { return remoteAddress; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java index 695f1e7050c4..4c32b2b6a5fa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java @@ -31,6 +31,7 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.security.GeneralSecurityException; +import java.security.cert.X509Certificate; import java.util.Collections; import java.util.Map; import java.util.Objects; @@ -133,6 +134,7 @@ abstract class ServerRpcConnection implements Closeable { protected User user = null; protected UserGroupInformation ugi = null; protected SaslServerAuthenticationProviders saslProviders = null; + protected X509Certificate[] clientCertificateChain = null; public ServerRpcConnection(RpcServer rpcServer) { this.rpcServer = rpcServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java index 00953353187e..124214e46af3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java @@ -22,6 +22,7 @@ import java.net.InetAddress; import java.security.PrivilegedAction; import java.security.PrivilegedExceptionAction; +import java.security.cert.X509Certificate; import java.util.Collections; import java.util.List; import java.util.Map; @@ -814,6 +815,11 @@ public Optional getRequestUser() { return getUser(userName); } + @Override + public Optional getClientCertificateChain() { + return Optional.empty(); + } + @Override public InetAddress getRemoteAddress() { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java index 67d8a2579097..1de0a0d31a33 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestRpcLogDetails.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.net.InetAddress; import java.nio.ByteBuffer; +import java.security.cert.X509Certificate; import java.util.Arrays; import java.util.Collections; import java.util.Map; @@ -213,6 +214,11 @@ public Optional getRequestUser() { return null; } + @Override + public Optional getClientCertificateChain() { + return Optional.empty(); + } + @Override public InetAddress getRemoteAddress() { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java index d069c2560a55..fdd5c7d5cf90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.security.cert.X509Certificate; import java.util.HashSet; import java.util.Map; import java.util.Optional; @@ -275,6 +276,11 @@ public Optional getRequestUser() { return Optional.empty(); } + @Override + public Optional getClientCertificateChain() { + return Optional.empty(); + } + @Override public InetAddress getRemoteAddress() { return null; From f3db223587685388f655dfe83d2f8cc745ea9443 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Mon, 29 Jan 2024 23:52:48 +0800 Subject: [PATCH 084/109] HBASE-28331 Client integration test fails after upgrading hadoop3 version to 3.3.x (#5657) Signed-off-by: Yi Mei --- dev-support/Jenkinsfile | 2 ++ dev-support/patch-hadoop3.sh | 24 ++++++++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100755 dev-support/patch-hadoop3.sh diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index 9c03d97a3c44..5ff411ee473c 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -726,6 +726,8 @@ pipeline { echo "Attempting to use run an instance on top of Hadoop 3." artifact=$(ls -1 "${WORKSPACE}"/hadoop-3*.tar.gz | head -n 1) tar --strip-components=1 -xzf "${artifact}" -C "hadoop-3" + # we need to patch some files otherwise minicluster will fail to start, see MAPREDUCE-7471 + ${BASEDIR}/dev-support/patch-hadoop3.sh hadoop-3 if ! "${BASEDIR}/dev-support/hbase_nightly_pseudo-distributed-test.sh" \ --single-process \ --working-dir output-integration/hadoop-3 \ diff --git a/dev-support/patch-hadoop3.sh b/dev-support/patch-hadoop3.sh new file mode 100755 index 000000000000..fed5a535bcdd --- /dev/null +++ b/dev-support/patch-hadoop3.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +## +# 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. +## + +hadoop_dir=$1 + +sed -i "s/HADOOP_TOOLS_DIR=\${HADOOP_TOOLS_DIR:-\"share\/hadoop\/tools\"}/HADOOP_TOOLS_DIR=\${HADOOP_TOOLS_DIR:-\"\$HADOOP_TOOLS_HOME\/share\/hadoop\/tools\"}/g" "$hadoop_dir/libexec/hadoop-functions.sh" +sed -i "/hadoop_add_classpath \"\${junitjar}\"/a mockitojar=\$(echo \"\${HADOOP_TOOLS_LIB_JARS_DIR}\"\/mockito-core-[0-9]*.jar)\nhadoop_add_classpath \"\${mockitojar}\"" "$hadoop_dir/bin/mapred" +curl https://repo1.maven.org/maven2/org/mockito/mockito-core/2.28.2/mockito-core-2.28.2.jar -o "$hadoop_dir/share/hadoop/tools/lib/mockito-core-2.28.2.jar" From f34e214ffc9d6f70ead8d8730e3d3199523ed419 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 30 Jan 2024 11:23:07 +0800 Subject: [PATCH 085/109] HBASE-28331 Addendum remove stale hadoop tarballs in client integration test --- dev-support/Jenkinsfile | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index 5ff411ee473c..a767775b36b4 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -661,6 +661,9 @@ pipeline { rm -rf "hadoop-3" && mkdir "hadoop-3" rm -rf ".m2-for-repo" && mkdir ".m2-for-repo" rm -rf ".m2-for-src" && mkdir ".m2-for-src" + # remove old hadoop tarballs in workspace + rm -rf hadoop-2*.tar.gz + rm -rf hadoop-3*.tar.gz echo "(x) {color:red}-1 source release artifact{color}\n-- Something went wrong with this stage, [check relevant console output|${BUILD_URL}/console]." >output-srctarball/commentfile echo "(x) {color:red}-1 client integration test{color}\n-- Something went wrong with this stage, [check relevant console output|${BUILD_URL}/console]." >output-integration/commentfile ''' From c4e332a93b563d6a8a832fd46b552f68a3e76490 Mon Sep 17 00:00:00 2001 From: Charles Connell Date: Tue, 30 Jan 2024 14:20:34 -0500 Subject: [PATCH 086/109] HBASE-28336 Correctly cast array of Certificates to array of X509Certificates (#5658) Signed-off-by: Bryan Beaudreault --- .../hadoop/hbase/ipc/NettyRpcServer.java | 50 ++++++++------- .../hadoop/hbase/ipc/TestNettyRpcServer.java | 63 +++++++++++++++++++ 2 files changed, 92 insertions(+), 21 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index c291338e40c2..629b3468cbe5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -416,33 +416,41 @@ private void initSSL(ChannelPipeline p, NettyServerRpcConnection conn, boolean s sslHandler.setWrapDataSize( conf.getInt(HBASE_SERVER_NETTY_TLS_WRAP_SIZE, DEFAULT_HBASE_SERVER_NETTY_TLS_WRAP_SIZE)); - sslHandler.handshakeFuture().addListener(future -> { - try { - Certificate[] certificates = sslHandler.engine().getSession().getPeerCertificates(); - if (certificates != null && certificates.length > 0) { - conn.clientCertificateChain = (X509Certificate[]) certificates; - } else if (sslHandler.engine().getNeedClientAuth()) { - LOG.error( - "Could not get peer certificate on TLS connection from {}, although one is required", - remoteAddress); - } - } catch (SSLPeerUnverifiedException e) { - if (sslHandler.engine().getNeedClientAuth()) { - LOG.error( - "Could not get peer certificate on TLS connection from {}, although one is required", - remoteAddress, e); - } - } catch (Exception e) { - LOG.error("Unexpected error getting peer certificate for TLS connection from {}", - remoteAddress, e); - } - }); + sslHandler.handshakeFuture() + .addListener(future -> sslHandshakeCompleteHandler(conn, sslHandler, remoteAddress)); p.addLast("ssl", sslHandler); LOG.debug("SSL handler added for channel: {}", p.channel()); } } + static void sslHandshakeCompleteHandler(NettyServerRpcConnection conn, SslHandler sslHandler, + SocketAddress remoteAddress) { + try { + Certificate[] certificates = sslHandler.engine().getSession().getPeerCertificates(); + if (certificates != null && certificates.length > 0) { + X509Certificate[] x509Certificates = new X509Certificate[certificates.length]; + for (int i = 0; i < x509Certificates.length; i++) { + x509Certificates[i] = (X509Certificate) certificates[i]; + } + conn.clientCertificateChain = x509Certificates; + } else if (sslHandler.engine().getNeedClientAuth()) { + LOG.debug( + "Could not get peer certificate on TLS connection from {}, although one is required", + remoteAddress); + } + } catch (SSLPeerUnverifiedException e) { + if (sslHandler.engine().getNeedClientAuth()) { + LOG.debug( + "Could not get peer certificate on TLS connection from {}, although one is required", + remoteAddress, e); + } + } catch (Exception e) { + LOG.debug("Unexpected error getting peer certificate for TLS connection from {}", + remoteAddress, e); + } + } + SslContext getSslContext() throws X509Exception, IOException { SslContext result = sslContextForServer.get(); if (result == null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java index 6af67e2190d3..2d5b95028f6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java @@ -17,11 +17,25 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.io.ByteArrayInputStream; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.nio.charset.StandardCharsets; +import java.security.cert.Certificate; +import java.security.cert.CertificateException; +import java.security.cert.CertificateFactory; +import java.security.cert.X509Certificate; import java.util.Arrays; import java.util.Collection; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; @@ -47,6 +61,8 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler; + @Category({ RPCTests.class, MediumTests.class }) @RunWith(Parameterized.class) public class TestNettyRpcServer { @@ -122,4 +138,51 @@ protected void doTest(TableName tableName) throws Exception { } } + private static final String CERTIFICATE = "-----BEGIN CERTIFICATE-----\n" + + "MIIEITCCAwmgAwIBAgIUaLL8vLOhWLCLXVHEJqXJhfmsTB8wDQYJKoZIhvcNAQEL\n" + + "BQAwgawxCzAJBgNVBAYTAlVTMRYwFAYDVQQIDA1NYXNzYWNodXNldHRzMRIwEAYD\n" + + "VQQHDAlDYW1icmlkZ2UxGDAWBgNVBAoMD25ldHR5IHRlc3QgY2FzZTEYMBYGA1UE\n" + + "CwwPbmV0dHkgdGVzdCBjYXNlMRgwFgYDVQQDDA9uZXR0eSB0ZXN0IGNhc2UxIzAh\n" + + "BgkqhkiG9w0BCQEWFGNjb25uZWxsQGh1YnNwb3QuY29tMB4XDTI0MDEyMTE5MzMy\n" + + "MFoXDTI1MDEyMDE5MzMyMFowgawxCzAJBgNVBAYTAlVTMRYwFAYDVQQIDA1NYXNz\n" + + "YWNodXNldHRzMRIwEAYDVQQHDAlDYW1icmlkZ2UxGDAWBgNVBAoMD25ldHR5IHRl\n" + + "c3QgY2FzZTEYMBYGA1UECwwPbmV0dHkgdGVzdCBjYXNlMRgwFgYDVQQDDA9uZXR0\n" + + "eSB0ZXN0IGNhc2UxIzAhBgkqhkiG9w0BCQEWFGNjb25uZWxsQGh1YnNwb3QuY29t\n" + + "MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAy+qzEZpQMjVdLj0siUcG\n" + + "y8LIHOW4S+tgHIKFkF865qWq6FVGbROe2Z0f5W6yIamZkdxzptT0iv+8S5okNNeW\n" + + "2NbsN/HNJIRtWfxku1Jh1gBqSkAYIjXyq7+20hIaJTzzxqike9M/Lc14EGb33Ja/\n" + + "kDPRV3UtiM3Ntf3eALXKbrWptkbgQngCaTgtfg8IkMAEpP270wZ9fW0lDHv3NPPt\n" + + "Zt0QSJzWSqWfu+l4ayvcUQYyNJesx9YmTHSJu69lvT4QApoX8FEiHfNCJ28R50CS\n" + + "aIgOpCWUvkH7rqx0p9q393uJRS/S6RlLbU30xUN1fNrVmP/XAapfy+R0PSgiUi8o\n" + + "EQIDAQABozkwNzAWBgNVHRIEDzANggt3d3cuZm9vLmNvbTAdBgNVHQ4EFgQUl4FD\n" + + "Y8jJ/JHJR68YqPsGUjUJuwgwDQYJKoZIhvcNAQELBQADggEBADVzivYz2M0qsWUc\n" + + "jXjCHymwTIr+7ud10um53FbYEAfKWsIY8Pp35fKpFzUwc5wVdCnLU86K/YMKRzNB\n" + + "zL2Auow3PJFRvXecOv7dWxNlNneLDcwbVrdNRu6nQXmZUgyz0oUKuJbF+JGtI+7W\n" + + "kRw7yhBfki+UCSQWeDqvaWzgmA4Us0N8NFq3euAs4xFbMMPMQWrT9Z7DGchCeRiB\n" + + "dkQBvh88vbR3v2Saq14W4Wt5rj2++vXWGQSeAQL6nGbOwc3ohW6isNNV0eGQQTmS\n" + + "khS2d/JDZq2XL5RGexf3CA6YYzWiTr9YZHNjuobvLH7mVnA2c8n6Zty/UhfnuK1x\n" + "JbkleFk=\n" + + "-----END CERTIFICATE-----"; + + @Test + public void testHandshakeCompleteHandler() + throws SSLPeerUnverifiedException, CertificateException { + NettyServerRpcConnection conn = mock(NettyServerRpcConnection.class); + SslHandler sslHandler = mock(SslHandler.class); + SocketAddress remoteAddress = new InetSocketAddress("localhost", 5555); + SSLEngine engine = mock(SSLEngine.class); + SSLSession session = mock(SSLSession.class); + CertificateFactory certificateFactory = CertificateFactory.getInstance("X.509"); + X509Certificate x509Certificate = (X509Certificate) certificateFactory + .generateCertificate(new ByteArrayInputStream(CERTIFICATE.getBytes(StandardCharsets.UTF_8))); + Certificate[] certificates = new Certificate[] { x509Certificate }; + + when(sslHandler.engine()).thenReturn(engine); + when(engine.getSession()).thenReturn(session); + when(session.getPeerCertificates()).thenReturn(certificates); + + NettyRpcServer.sslHandshakeCompleteHandler(conn, sslHandler, remoteAddress); + + assertArrayEquals(certificates, conn.clientCertificateChain); + } + } From f253453d92fc20e6ce18a06f5db9e6b795889707 Mon Sep 17 00:00:00 2001 From: Nirdosh Yadav Date: Mon, 5 Feb 2024 18:46:43 +0530 Subject: [PATCH 087/109] HBASE-28334 Add comment around erasure coding policy in DEFAULT_VALUE map (#5662) Signed-off-by: Bryan Beaudreault --- .../org/apache/hadoop/hbase/client/TableDescriptorBuilder.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index 7cab90c7e378..fcdbe4e4ae64 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -234,6 +234,7 @@ public class TableDescriptorBuilder { DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); // use the enum name DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION)); DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY)); + // Setting ERASURE_CODING_POLICY to NULL so that it is not considered as metadata DEFAULT_VALUES.put(ERASURE_CODING_POLICY, String.valueOf(DEFAULT_ERASURE_CODING_POLICY)); DEFAULT_VALUES.keySet().stream().map(s -> new Bytes(Bytes.toBytes(s))) .forEach(RESERVED_KEYWORDS::add); From 256f10be78f0e1811b9e2374d0e5960eae9d8d20 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Mon, 5 Feb 2024 08:26:31 -0500 Subject: [PATCH 088/109] HBASE-28338 Bounded leak of FSDataInputStream buffers from checksum switching (#5660) Signed-off-by: Duo Zhang --- .../hbase/io/FSDataInputStreamWrapper.java | 46 ++++--------------- .../io/TestFSDataInputStreamWrapper.java | 28 +++++------ 2 files changed, 23 insertions(+), 51 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java index cb9dc84b94be..33eace47d632 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java @@ -19,17 +19,13 @@ import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.io.Closeables; @@ -40,8 +36,6 @@ */ @InterfaceAudience.Private public class FSDataInputStreamWrapper implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(FSDataInputStreamWrapper.class); - private static final boolean isLogTraceEnabled = LOG.isTraceEnabled(); private final HFileSystem hfs; private final Path path; @@ -94,9 +88,6 @@ private static class ReadStatistics { long totalZeroCopyBytesRead; } - private Boolean instanceOfCanUnbuffer = null; - private CanUnbuffer unbuffer = null; - protected Path readerPath; public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException { @@ -314,41 +305,22 @@ public HFileSystem getHfs() { * stream, the current socket will be closed and a new socket will be opened to serve the * requests. */ - @SuppressWarnings({ "rawtypes" }) public void unbuffer() { + // todo: it may make sense to always unbuffer both streams. we'd need to carefully + // research the usages to know if that is safe. for now just do the current. FSDataInputStream stream = this.getStream(this.shouldUseHBaseChecksum()); if (stream != null) { - InputStream wrappedStream = stream.getWrappedStream(); - // CanUnbuffer interface was added as part of HDFS-7694 and the fix is available in Hadoop - // 2.6.4+ and 2.7.1+ versions only so check whether the stream object implements the - // CanUnbuffer interface or not and based on that call the unbuffer api. - final Class streamClass = wrappedStream.getClass(); - if (this.instanceOfCanUnbuffer == null) { - // To ensure we compute whether the stream is instance of CanUnbuffer only once. - this.instanceOfCanUnbuffer = false; - if (wrappedStream instanceof CanUnbuffer) { - this.unbuffer = (CanUnbuffer) wrappedStream; - this.instanceOfCanUnbuffer = true; - } - } - if (this.instanceOfCanUnbuffer) { - try { - this.unbuffer.unbuffer(); - } catch (UnsupportedOperationException e) { - if (isLogTraceEnabled) { - LOG.trace("Failed to invoke 'unbuffer' method in class " + streamClass - + " . So there may be the stream does not support unbuffering.", e); - } - } - } else { - if (isLogTraceEnabled) { - LOG.trace("Failed to find 'unbuffer' method in class " + streamClass); - } - } + stream.unbuffer(); } } public Path getReaderPath() { return readerPath; } + + // For tests + void setShouldUseHBaseChecksum() { + useHBaseChecksumConfigured = true; + useHBaseChecksum = true; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java index bb476670f10f..77aa00ef91f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.io; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -31,6 +32,7 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.HasEnhancedByteBufferAccess; import org.apache.hadoop.fs.ReadOption; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.ByteBufferPool; @@ -48,22 +50,22 @@ public class TestFSDataInputStreamWrapper { @Test public void testUnbuffer() throws Exception { InputStream pc = new ParentClass(); - FSDataInputStreamWrapper fsdisw1 = new FSDataInputStreamWrapper(new FSDataInputStream(pc)); + InputStream noChecksumPc = new ParentClass(); + FSDataInputStreamWrapper fsdisw1 = + new FSDataInputStreamWrapper(new FSDataInputStream(pc), new FSDataInputStream(noChecksumPc)); fsdisw1.unbuffer(); - // parent class should be true + // should have called main stream unbuffer, but not no-checksum assertTrue(((ParentClass) pc).getIsCallUnbuffer()); + assertFalse(((ParentClass) noChecksumPc).getIsCallUnbuffer()); + // switch to checksums and call unbuffer again. should unbuffer the nochecksum stream now + fsdisw1.setShouldUseHBaseChecksum(); + fsdisw1.unbuffer(); + assertTrue(((ParentClass) noChecksumPc).getIsCallUnbuffer()); fsdisw1.close(); - - InputStream cc1 = new ChildClass1(); - FSDataInputStreamWrapper fsdisw2 = new FSDataInputStreamWrapper(new FSDataInputStream(cc1)); - fsdisw2.unbuffer(); - // child1 class should be true - assertTrue(((ChildClass1) cc1).getIsCallUnbuffer()); - fsdisw2.close(); } private class ParentClass extends FSInputStream implements ByteBufferReadable, CanSetDropBehind, - CanSetReadahead, HasEnhancedByteBufferAccess, CanUnbuffer { + CanSetReadahead, HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities { public boolean isCallUnbuffer = false; @@ -122,12 +124,10 @@ public long getPos() throws IOException { public boolean seekToNewSource(long paramLong) throws IOException { return false; } - } - private class ChildClass1 extends ParentClass { @Override - public void unbuffer() { - isCallUnbuffer = true; + public boolean hasCapability(String s) { + return s.equals(StreamCapabilities.UNBUFFER); } } } From 16de74c194aa5db028f78d926b82b84c4df8b257 Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Tue, 6 Feb 2024 10:08:12 +0800 Subject: [PATCH 089/109] HBASE-25051 DIGEST based auth broken for rpc based ConnectionRegistry (#5631) Signed-off-by: Bryan Beaudreault --- .../AbstractRpcBasedConnectionRegistry.java | 61 +++---- .../hadoop/hbase/client/ClusterIdFetcher.java | 134 ++++++++++++++ .../hbase/client/ConnectionFactory.java | 2 +- .../client/ConnectionOverAsyncConnection.java | 2 +- .../client/ConnectionRegistryFactory.java | 5 +- .../ConnectionRegistryRpcStubHolder.java | 168 ++++++++++++++++++ .../hadoop/hbase/client/ConnectionUtils.java | 10 ++ .../hadoop/hbase/client/MasterRegistry.java | 8 +- .../hbase/client/RpcConnectionRegistry.java | 7 +- .../hbase/client/TableOverAsyncTable.java | 2 +- .../hbase/client/ZKConnectionRegistry.java | 4 +- .../hadoop/hbase/ipc/AbstractRpcClient.java | 2 +- .../hbase/ipc/BlockingRpcConnection.java | 103 +++-------- .../org/apache/hadoop/hbase/ipc/Call.java | 5 + .../hadoop/hbase/ipc/NettyRpcConnection.java | 26 ++- .../hbase/ipc/NettyRpcDuplexHandler.java | 88 +-------- .../apache/hadoop/hbase/ipc/RpcClient.java | 2 + .../hadoop/hbase/ipc/RpcConnection.java | 120 ++++++++++++- .../BuiltInSaslAuthenticationProvider.java | 6 + .../DigestSaslAuthenticationProvider.java | 5 +- .../GssSaslAuthenticationProvider.java | 5 +- .../SimpleSaslAuthenticationProvider.java | 6 +- .../client/DoNothingConnectionRegistry.java | 5 +- .../client/TestAsyncAdminRpcPriority.java | 7 +- .../client/TestAsyncConnectionTracing.java | 7 +- .../TestAsyncMetaRegionLocatorFailFast.java | 11 +- .../client/TestAsyncRegionLocatorTracing.java | 6 +- .../client/TestAsyncTableRpcPriority.java | 6 +- .../hbase/client/TestAsyncTableTracing.java | 60 +++---- .../client/TestConnectionRegistryLeak.java | 5 +- .../TestRpcBasedRegistryHedgedReads.java | 14 +- .../hbase/ipc/TestTLSHandshadeFailure.java | 7 +- .../hadoop/hbase/util/ConcurrentMapUtils.java | 8 - .../hbase/util/IOExceptionSupplier.java | 30 ++++ .../src/main/protobuf/server/Registry.proto | 19 +- .../client/ClusterConnectionFactory.java | 2 +- .../hadoop/hbase/ipc/NettyRpcServer.java | 9 +- .../ipc/NettyRpcServerPreambleHandler.java | 18 +- .../hadoop/hbase/ipc/ServerRpcConnection.java | 79 +++++--- .../hbase/ipc/SimpleServerRpcConnection.java | 19 +- .../apache/hadoop/hbase/wal/WALSplitUtil.java | 2 +- .../TestZooKeeperTableArchiveClient.java | 13 +- .../client/AbstractTestRegionLocator.java | 3 +- .../hbase/client/DummyConnectionRegistry.java | 57 ------ .../TestAsyncAdminWithRegionReplicas.java | 3 +- .../client/TestAsyncMetaRegionLocator.java | 4 +- .../client/TestAsyncNonMetaRegionLocator.java | 4 +- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 2 +- .../hbase/client/TestAsyncRegionLocator.java | 2 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 2 +- .../client/TestAsyncTableUseMetaReplicas.java | 4 +- .../hbase/client/TestBootstrapNodeUpdate.java | 3 +- ...talogReplicaLoadBalanceSimpleSelector.java | 3 +- .../hbase/client/TestMasterRegistry.java | 54 +++--- .../client/TestMetaRegionLocationCache.java | 4 +- .../client/TestRpcConnectionRegistry.java | 57 +++++- .../client/TestZKConnectionRegistry.java | 6 +- .../hadoop/hbase/ipc/AbstractTestIPC.java | 83 ++++++++- .../hadoop/hbase/ipc/TestBlockingIPC.java | 4 +- .../apache/hadoop/hbase/ipc/TestNettyIPC.java | 5 +- .../hadoop/hbase/ipc/TestNettyTlsIPC.java | 23 ++- .../regionserver/TestWALEntrySinkFilter.java | 13 +- .../token/TestGenerateDelegationToken.java | 80 ++++++--- 63 files changed, 1031 insertions(+), 483 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterIdFetcher.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryRpcStubHolder.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/util/IOExceptionSupplier.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index 4e97dcab24dd..62c6951b4535 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java @@ -33,22 +33,17 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; @@ -79,30 +74,21 @@ abstract class AbstractRpcBasedConnectionRegistry implements ConnectionRegistry private final int hedgedReadFanOut; - // Configured list of end points to probe the meta information from. - private volatile ImmutableMap addr2Stub; - // RPC client used to talk to the masters. - private final RpcClient rpcClient; + private final ConnectionRegistryRpcStubHolder rpcStubHolder; private final RpcControllerFactory rpcControllerFactory; - private final int rpcTimeoutMs; private final RegistryEndpointsRefresher registryEndpointRefresher; - protected AbstractRpcBasedConnectionRegistry(Configuration conf, + protected AbstractRpcBasedConnectionRegistry(Configuration conf, User user, String hedgedReqsFanoutConfigName, String initialRefreshDelaySecsConfigName, String refreshIntervalSecsConfigName, String minRefreshIntervalSecsConfigName) throws IOException { this.hedgedReadFanOut = Math.max(1, conf.getInt(hedgedReqsFanoutConfigName, HEDGED_REQS_FANOUT_DEFAULT)); - rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, - conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch - // this through the master registry... - // This is a problem as we will use the cluster id to determine the authentication method - rpcClient = RpcClientFactory.createClient(conf, null); rpcControllerFactory = RpcControllerFactory.instantiate(conf); - populateStubs(getBootstrapNodes(conf)); + rpcStubHolder = new ConnectionRegistryRpcStubHolder(conf, user, rpcControllerFactory, + getBootstrapNodes(conf)); // could return null here is refresh interval is less than zero registryEndpointRefresher = RegistryEndpointsRefresher.create(conf, initialRefreshDelaySecsConfigName, @@ -114,19 +100,7 @@ protected AbstractRpcBasedConnectionRegistry(Configuration conf, protected abstract CompletableFuture> fetchEndpoints(); private void refreshStubs() throws IOException { - populateStubs(FutureUtils.get(fetchEndpoints())); - } - - private void populateStubs(Set addrs) throws IOException { - Preconditions.checkNotNull(addrs); - ImmutableMap.Builder builder = - ImmutableMap.builderWithExpectedSize(addrs.size()); - User user = User.getCurrent(); - for (ServerName masterAddr : addrs) { - builder.put(masterAddr, - ClientMetaService.newStub(rpcClient.createRpcChannel(masterAddr, user, rpcTimeoutMs))); - } - addr2Stub = builder.build(); + rpcStubHolder.refreshStubs(() -> FutureUtils.get(fetchEndpoints())); } /** @@ -211,20 +185,25 @@ private void groupCall(CompletableFuture future, Set CompletableFuture call(Callable callable, Predicate isValidResp, String debug) { - ImmutableMap addr2StubRef = addr2Stub; - Set servers = addr2StubRef.keySet(); - List stubs = new ArrayList<>(addr2StubRef.values()); - Collections.shuffle(stubs, ThreadLocalRandom.current()); CompletableFuture future = new CompletableFuture<>(); - groupCall(future, servers, stubs, 0, callable, isValidResp, debug, - new ConcurrentLinkedQueue<>()); + FutureUtils.addListener(rpcStubHolder.getStubs(), (addr2Stub, error) -> { + if (error != null) { + future.completeExceptionally(error); + return; + } + Set servers = addr2Stub.keySet(); + List stubs = new ArrayList<>(addr2Stub.values()); + Collections.shuffle(stubs, ThreadLocalRandom.current()); + groupCall(future, servers, stubs, 0, callable, isValidResp, debug, + new ConcurrentLinkedQueue<>()); + }); return future; } @RestrictedApi(explanation = "Should only be called in tests", link = "", allowedOnPath = ".*/src/test/.*") - Set getParsedServers() { - return addr2Stub.keySet(); + Set getParsedServers() throws IOException { + return FutureUtils.get(rpcStubHolder.getStubs()).keySet(); } /** @@ -277,8 +256,8 @@ public void close() { if (registryEndpointRefresher != null) { registryEndpointRefresher.stop(); } - if (rpcClient != null) { - rpcClient.close(); + if (rpcStubHolder != null) { + rpcStubHolder.close(); } }, getClass().getSimpleName() + ".close"); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterIdFetcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterIdFetcher.java new file mode 100644 index 000000000000..277629681ec6 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterIdFetcher.java @@ -0,0 +1,134 @@ +/* + * 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.client; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.FutureUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse; + +/** + * Fetch cluster id through special preamble header. + *

+ * An instance of this class should only be used once, like: + * + *

+ * new ClusterIdFetcher().fetchClusterId()
+ * 
+ * + * Calling the fetchClusterId multiple times will lead unexpected behavior. + *

+ * See HBASE-25051 for more details. + */ +@InterfaceAudience.Private +class ClusterIdFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(ClusterIdFetcher.class); + + private final List bootstrapServers; + + private final User user; + + private final RpcClient rpcClient; + + private final RpcControllerFactory rpcControllerFactory; + + private final CompletableFuture future; + + ClusterIdFetcher(Configuration conf, User user, RpcControllerFactory rpcControllerFactory, + Set bootstrapServers) { + this.user = user; + // use null cluster id here as we do not know the cluster id yet, we will fetch it through this + // rpc client + this.rpcClient = RpcClientFactory.createClient(conf, null); + this.rpcControllerFactory = rpcControllerFactory; + this.bootstrapServers = new ArrayList(bootstrapServers); + // shuffle the bootstrap servers so we will not always fetch from the same one + Collections.shuffle(this.bootstrapServers); + future = new CompletableFuture(); + } + + /** + * Try get cluster id from the server with the given {@code index} in {@link #bootstrapServers}. + */ + private void getClusterId(int index) { + ServerName server = bootstrapServers.get(index); + LOG.debug("Going to request {} for getting cluster id", server); + // user and rpcTimeout are both not important here, as we will not actually send any rpc calls + // out, only a preamble connection header, but if we pass null as user, there will be NPE in + // some code paths... + RpcChannel channel = rpcClient.createRpcChannel(server, user, 0); + ConnectionRegistryService.Interface stub = ConnectionRegistryService.newStub(channel); + HBaseRpcController controller = rpcControllerFactory.newController(); + stub.getConnectionRegistry(controller, GetConnectionRegistryRequest.getDefaultInstance(), + new RpcCallback() { + + @Override + public void run(GetConnectionRegistryResponse resp) { + if (!controller.failed()) { + LOG.debug("Got connection registry info: {}", resp); + future.complete(resp.getClusterId()); + return; + } + if (ConnectionUtils.isUnexpectedPreambleHeaderException(controller.getFailed())) { + // this means we have connected to an old server where it does not support passing + // cluster id through preamble connnection header, so we fallback to use null + // cluster id, which is the old behavior + LOG.debug("Failed to get connection registry info, should be an old server," + + " fallback to use null cluster id", controller.getFailed()); + future.complete(null); + } else { + LOG.debug("Failed to get connection registry info", controller.getFailed()); + if (index == bootstrapServers.size() - 1) { + future.completeExceptionally(controller.getFailed()); + } else { + // try next bootstrap server + getClusterId(index + 1); + } + } + } + }); + + } + + CompletableFuture fetchClusterId() { + getClusterId(0); + // close the rpc client after we finish the request + FutureUtils.addListener(future, (r, e) -> rpcClient.close()); + return future; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index ac70091dcf65..716fb4863fe8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -338,7 +338,7 @@ public static CompletableFuture createAsyncConnection(Configura final User user, Map connectionAttributes) { return TraceUtil.tracedFuture(() -> { CompletableFuture future = new CompletableFuture<>(); - ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf); + ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf, user); addListener(registry.getClusterId(), (clusterId, error) -> { if (error != null) { registry.close(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java index 51368fc23c15..30c348e6d1f1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java @@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.log.HBaseMarkers; -import org.apache.hadoop.hbase.util.ConcurrentMapUtils.IOExceptionSupplier; import org.apache.hadoop.hbase.util.FutureUtils; +import org.apache.hadoop.hbase.util.IOExceptionSupplier; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java index f198c3c22002..415d46397b8f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -33,10 +34,10 @@ private ConnectionRegistryFactory() { } /** Returns The connection registry implementation to use. */ - static ConnectionRegistry getRegistry(Configuration conf) { + static ConnectionRegistry getRegistry(Configuration conf, User user) { Class clazz = conf.getClass(CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, RpcConnectionRegistry.class, ConnectionRegistry.class); - return ReflectionUtils.newInstance(clazz, conf); + return ReflectionUtils.newInstance(clazz, conf, user); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryRpcStubHolder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryRpcStubHolder.java new file mode 100644 index 000000000000..3dbcfbe8e6bf --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryRpcStubHolder.java @@ -0,0 +1,168 @@ +/* + * 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.client; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.FutureUtils; +import org.apache.hadoop.hbase.util.IOExceptionSupplier; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService; + +/** + * A class for creating {@link RpcClient} and related stubs used by + * {@link AbstractRpcBasedConnectionRegistry}. We need to connect to bootstrap nodes to get the + * cluster id first, before creating the final {@link RpcClient} and related stubs. + *

+ * See HBASE-25051 for more details. + */ +@InterfaceAudience.Private +class ConnectionRegistryRpcStubHolder implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(ConnectionRegistryRpcStubHolder.class); + + private final Configuration conf; + + // used for getting cluster id + private final Configuration noAuthConf; + + private final User user; + + private final RpcControllerFactory rpcControllerFactory; + + private final Set bootstrapNodes; + + private final int rpcTimeoutMs; + + private volatile ImmutableMap addr2Stub; + + private volatile RpcClient rpcClient; + + private CompletableFuture> addr2StubFuture; + + ConnectionRegistryRpcStubHolder(Configuration conf, User user, + RpcControllerFactory rpcControllerFactory, Set bootstrapNodes) { + this.conf = conf; + if (User.isHBaseSecurityEnabled(conf)) { + this.noAuthConf = new Configuration(conf); + this.noAuthConf.set(User.HBASE_SECURITY_CONF_KEY, "simple"); + } else { + this.noAuthConf = conf; + } + this.user = user; + this.rpcControllerFactory = rpcControllerFactory; + this.bootstrapNodes = Collections.unmodifiableSet(bootstrapNodes); + this.rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, + conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + } + + private ImmutableMap createStubs(RpcClient rpcClient, + Collection addrs) { + LOG.debug("Going to use new servers to create stubs: {}", addrs); + Preconditions.checkNotNull(addrs); + ImmutableMap.Builder builder = + ImmutableMap.builderWithExpectedSize(addrs.size()); + for (ServerName masterAddr : addrs) { + builder.put(masterAddr, + ClientMetaService.newStub(rpcClient.createRpcChannel(masterAddr, user, rpcTimeoutMs))); + } + return builder.build(); + } + + private CompletableFuture> + fetchClusterIdAndCreateStubs() { + CompletableFuture> future = + new CompletableFuture<>(); + addr2StubFuture = future; + FutureUtils.addListener( + new ClusterIdFetcher(noAuthConf, user, rpcControllerFactory, bootstrapNodes).fetchClusterId(), + (clusterId, error) -> { + synchronized (ConnectionRegistryRpcStubHolder.this) { + if (error != null) { + addr2StubFuture.completeExceptionally(error); + } else { + RpcClient c = RpcClientFactory.createClient(conf, clusterId); + ImmutableMap m = + createStubs(c, bootstrapNodes); + rpcClient = c; + addr2Stub = m; + addr2StubFuture.complete(m); + } + addr2StubFuture = null; + } + }); + // here we must use the local variable future instead of addr2StubFuture, as the above listener + // could be executed directly in the same thread(if the future completes quick enough), since + // the synchronized lock is reentrant, it could set addr2StubFuture to null in the end, so when + // arriving here the addr2StubFuture could be null. + return future; + } + + CompletableFuture> getStubs() { + ImmutableMap s = this.addr2Stub; + if (s != null) { + return CompletableFuture.completedFuture(s); + } + synchronized (this) { + s = this.addr2Stub; + if (s != null) { + return CompletableFuture.completedFuture(s); + } + if (addr2StubFuture != null) { + return addr2StubFuture; + } + return fetchClusterIdAndCreateStubs(); + } + } + + void refreshStubs(IOExceptionSupplier> fetchEndpoints) throws IOException { + // There is no actual call yet so we have not initialize the rpc client and related stubs yet, + // give up refreshing + if (addr2Stub == null) { + LOG.debug("Skip refreshing stubs as we have not initialized rpc client yet"); + return; + } + LOG.debug("Going to refresh stubs"); + assert rpcClient != null; + addr2Stub = createStubs(rpcClient, fetchEndpoints.get()); + } + + @Override + public void close() { + if (rpcClient != null) { + rpcClient.close(); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index 4827708a02e3..d073fef929fd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.hadoop.hbase.ipc.FatalConnectionException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.util.Bytes; @@ -663,4 +664,13 @@ static void setCoprocessorError(RpcController controller, Throwable error) { controller.setFailed(error.toString()); } } + + static boolean isUnexpectedPreambleHeaderException(IOException e) { + if (!(e instanceof RemoteException)) { + return false; + } + RemoteException re = (RemoteException) e; + return FatalConnectionException.class.getName().equals(re.getClassName()) + && re.getMessage().startsWith("Expected HEADER="); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java index b6f81c30f0bd..364180fe1414 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java @@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.DNS.ServerType; import org.apache.yetus.audience.InterfaceAudience; @@ -105,9 +106,10 @@ public static Set parseMasterAddrs(Configuration conf) throws Unknow private final String connectionString; - MasterRegistry(Configuration conf) throws IOException { - super(conf, MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, MASTER_REGISTRY_INITIAL_REFRESH_DELAY_SECS, - MASTER_REGISTRY_PERIODIC_REFRESH_INTERVAL_SECS, MASTER_REGISTRY_MIN_SECS_BETWEEN_REFRESHES); + MasterRegistry(Configuration conf, User user) throws IOException { + super(conf, user, MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, + MASTER_REGISTRY_INITIAL_REFRESH_DELAY_SECS, MASTER_REGISTRY_PERIODIC_REFRESH_INTERVAL_SECS, + MASTER_REGISTRY_MIN_SECS_BETWEEN_REFRESHES); connectionString = getConnectionString(conf); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcConnectionRegistry.java index 2c320d3a9d1d..c3ed560923ff 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcConnectionRegistry.java @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; @@ -75,9 +76,9 @@ public class RpcConnectionRegistry extends AbstractRpcBasedConnectionRegistry { private final String connectionString; - RpcConnectionRegistry(Configuration conf) throws IOException { - super(conf, HEDGED_REQS_FANOUT_KEY, INITIAL_REFRESH_DELAY_SECS, PERIODIC_REFRESH_INTERVAL_SECS, - MIN_SECS_BETWEEN_REFRESHES); + RpcConnectionRegistry(Configuration conf, User user) throws IOException { + super(conf, user, HEDGED_REQS_FANOUT_KEY, INITIAL_REFRESH_DELAY_SECS, + PERIODIC_REFRESH_INTERVAL_SECS, MIN_SECS_BETWEEN_REFRESHES); connectionString = buildConnectionString(conf); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java index 0a7dabd476ce..8c61e8b584f9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java @@ -55,9 +55,9 @@ import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ConcurrentMapUtils.IOExceptionSupplier; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FutureUtils; +import org.apache.hadoop.hbase.util.IOExceptionSupplier; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index 1634b13ec7e8..0e13f0b83c91 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; @@ -60,7 +61,8 @@ class ZKConnectionRegistry implements ConnectionRegistry { private final ZNodePaths znodePaths; - ZKConnectionRegistry(Configuration conf) { + // User not used, but for rpc based registry we need it + ZKConnectionRegistry(Configuration conf, User user) { this.znodePaths = new ZNodePaths(conf); this.zk = new ReadOnlyZKClient(conf); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index 5926539d0679..7972cc08acd2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -454,7 +454,7 @@ public void run(Call call) { } } - private static Address createAddr(ServerName sn) { + static Address createAddr(ServerName sn) { return Address.fromParts(sn.getHostname(), sn.getPort()); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java index f30b77c64fe9..0478000a2375 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.ipc; import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader; -import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited; import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; import static org.apache.hadoop.hbase.ipc.IPCUtil.write; @@ -43,7 +42,6 @@ import java.util.concurrent.ThreadLocalRandom; import javax.security.sasl.SaslException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; @@ -64,19 +62,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; -import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; /** * Thread that reads responses and notifies callers. Each connection owns a socket connected to a @@ -218,7 +211,7 @@ public void cleanup(IOException e) { BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException { super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor, - rpcClient.metrics, rpcClient.connectionAttributes); + rpcClient.cellBlockBuilder, rpcClient.metrics, rpcClient.connectionAttributes); this.rpcClient = rpcClient; this.connectionHeaderPreamble = getConnectionHeaderPreamble(); ConnectionHeader header = getConnectionHeader(); @@ -435,6 +428,15 @@ public Object run() throws IOException, InterruptedException { }); } + private void getConnectionRegistry(OutputStream outStream) throws IOException { + outStream.write(RpcClient.REGISTRY_PREAMBLE_HEADER); + } + + private void createStreams(InputStream inStream, OutputStream outStream) { + this.in = new DataInputStream(new BufferedInputStream(inStream)); + this.out = new DataOutputStream(new BufferedOutputStream(outStream)); + } + private void setupIOstreams() throws IOException { if (socket != null) { // The connection is already available. Perfect. @@ -462,6 +464,11 @@ private void setupIOstreams() throws IOException { InputStream inStream = NetUtils.getInputStream(socket); // This creates a socket with a write timeout. This timeout cannot be changed. OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO); + if (connectionRegistryCall != null) { + getConnectionRegistry(outStream); + createStreams(inStream, outStream); + break; + } // Write out the preamble -- MAGIC, version, and auth to use. writeConnectionHeaderPreamble(outStream); if (useSasl) { @@ -494,13 +501,11 @@ public Boolean run() throws IOException { // reconnecting because regionserver may change its sasl config after restart. } } - this.in = new DataInputStream(new BufferedInputStream(inStream)); - this.out = new DataOutputStream(new BufferedOutputStream(outStream)); + createStreams(inStream, outStream); // Now write out the connection header writeConnectionHeader(); // process the response from server for connection header if necessary processResponseForConnectionHeader(); - break; } } catch (Throwable t) { @@ -611,7 +616,9 @@ private void writeRequest(Call call) throws IOException { cellBlockMeta = null; } RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta); - + if (call.isConnectionRegistryCall()) { + connectionRegistryCall = call; + } setupIOstreams(); // Now we're going to write the call. We take the lock, then check that the connection @@ -646,77 +653,13 @@ private void writeRequest(Call call) throws IOException { * Receive a response. Because only one receiver, so no synchronization on in. */ private void readResponse() { - Call call = null; - boolean expectedCall = false; try { - // See HBaseServer.Call.setResponse for where we write out the response. - // Total size of the response. Unused. But have to read it in anyways. - int totalSize = in.readInt(); - - // Read the header - ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); - int id = responseHeader.getCallId(); - if (LOG.isTraceEnabled()) { - LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader) - + ", totalSize: " + totalSize + " bytes"); - } - RemoteException remoteExc; - if (responseHeader.hasException()) { - ExceptionResponse exceptionResponse = responseHeader.getException(); - remoteExc = IPCUtil.createRemoteException(exceptionResponse); - if (IPCUtil.isFatalConnectionException(exceptionResponse)) { - // Here we will cleanup all calls so do not need to fall back, just return. - synchronized (this) { - closeConn(remoteExc); - } - return; - } - } else { - remoteExc = null; - } - - call = calls.remove(id); // call.done have to be set before leaving this method - expectedCall = (call != null && !call.isDone()); - if (!expectedCall) { - // So we got a response for which we have no corresponding 'call' here on the client-side. - // We probably timed out waiting, cleaned up all references, and now the server decides - // to return a response. There is nothing we can do w/ the response at this stage. Clean - // out the wire of the response so its out of the way and we can get other responses on - // this connection. - int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader); - int whatIsLeftToRead = totalSize - readSoFar; - LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead - + " bytes"); - IOUtils.skipFully(in, whatIsLeftToRead); - if (call != null) { - call.callStats.setResponseSizeBytes(totalSize); + readResponse(in, calls, remoteExc -> { + synchronized (this) { + closeConn(remoteExc); } - return; - } - call.callStats.setResponseSizeBytes(totalSize); - if (remoteExc != null) { - call.setException(remoteExc); - return; - } - Message value = null; - if (call.responseDefaultType != null) { - Message.Builder builder = call.responseDefaultType.newBuilderForType(); - ProtobufUtil.mergeDelimitedFrom(builder, in); - value = builder.build(); - } - CellScanner cellBlockScanner = null; - if (responseHeader.hasCellBlockMeta()) { - int size = responseHeader.getCellBlockMeta().getLength(); - byte[] cellBlock = new byte[size]; - IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); - cellBlockScanner = - this.rpcClient.cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock); - } - call.setResponse(value, cellBlockScanner); + }); } catch (IOException e) { - if (expectedCall) { - call.setException(e); - } if (e instanceof SocketTimeoutException) { // Clean up open calls but don't treat this as a fatal condition, // since we expect certain responses to not make it by the specified diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java index 669fc73a3bfa..d175ea0b6e90 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java @@ -34,6 +34,7 @@ import org.apache.hbase.thirdparty.io.netty.util.Timeout; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService; /** A call waiting for a value. */ @InterfaceAudience.Private @@ -156,4 +157,8 @@ public synchronized boolean isDone() { public long getStartTime() { return this.callStats.getStartTime(); } + + public boolean isConnectionRegistryCall() { + return md.getService().equals(ConnectionRegistryService.getDescriptor()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java index 408ea347e7a3..a0f8f10d1cf9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -104,7 +104,7 @@ class NettyRpcConnection extends RpcConnection { NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException { super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor, - rpcClient.metrics, rpcClient.connectionAttributes); + rpcClient.cellBlockBuilder, rpcClient.metrics, rpcClient.connectionAttributes); this.rpcClient = rpcClient; this.eventLoop = rpcClient.group.next(); byte[] connectionHeaderPreamble = getConnectionHeaderPreamble(); @@ -274,6 +274,12 @@ public void operationComplete(Future future) throws Exception { }); } + private void getConnectionRegistry(Channel ch) throws IOException { + established(ch); + NettyFutureUtils.safeWriteAndFlush(ch, + Unpooled.directBuffer(6).writeBytes(RpcClient.REGISTRY_PREAMBLE_HEADER)); + } + private void connect() throws UnknownHostException { assert eventLoop.inEventLoop(); LOG.trace("Connecting to {}", remoteId.getAddress()); @@ -303,12 +309,16 @@ protected void initChannel(Channel ch) throws Exception { .addListener(new ChannelFutureListener() { private void succeed(Channel ch) throws IOException { - ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate()); + if (connectionRegistryCall != null) { + getConnectionRegistry(ch); + return; + } + NettyFutureUtils.safeWriteAndFlush(ch, connectionHeaderPreamble.retainedDuplicate()); if (useSasl) { saslNegotiate(ch); } else { // send the connection header to server - ch.write(connectionHeaderWithLength.retainedDuplicate()); + NettyFutureUtils.safeWrite(ch, connectionHeaderWithLength.retainedDuplicate()); established(ch); } } @@ -317,6 +327,9 @@ private void fail(Channel ch, Throwable error) { IOException ex = toIOE(error); LOG.warn("Exception encountered while connecting to the server " + remoteId.getAddress(), ex); + if (connectionRegistryCall != null) { + connectionRegistryCall.setException(ex); + } failInit(ch, ex); rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), error); } @@ -346,6 +359,13 @@ public void operationComplete(ChannelFuture future) throws Exception { private void sendRequest0(Call call, HBaseRpcController hrc) throws IOException { assert eventLoop.inEventLoop(); + if (call.isConnectionRegistryCall()) { + connectionRegistryCall = call; + // For get connection registry call, we will send a special preamble header to get the + // response, instead of sending a real rpc call. See HBASE-25051 + connect(); + return; + } if (reloginInProgress) { throw new IOException(RpcConnectionConstants.RELOGIN_IS_IN_PROGRESS); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java index ad8c51568a32..44772ae2dbf9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java @@ -18,21 +18,16 @@ package org.apache.hadoop.hbase.ipc; import io.opentelemetry.context.Scope; -import java.io.EOFException; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.protobuf.Message; -import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufInputStream; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream; @@ -44,9 +39,7 @@ import org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseCombiner; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; /** * The netty rpc handler. @@ -127,88 +120,15 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) } } - private void finishCall(ResponseHeader responseHeader, ByteBufInputStream in, Call call) - throws IOException { - Message value; - if (call.responseDefaultType != null) { - Message.Builder builder = call.responseDefaultType.newBuilderForType(); - if (!builder.mergeDelimitedFrom(in)) { - // The javadoc of mergeDelimitedFrom says returning false means the stream reaches EOF - // before reading any bytes out, so here we need to manually finish create the EOFException - // and finish the call - call.setException(new EOFException("EOF while reading response with type: " - + call.responseDefaultType.getClass().getName())); - return; - } - value = builder.build(); - } else { - value = null; - } - CellScanner cellBlockScanner; - if (responseHeader.hasCellBlockMeta()) { - int size = responseHeader.getCellBlockMeta().getLength(); - // Maybe we could read directly from the ByteBuf. - // The problem here is that we do not know when to release it. - byte[] cellBlock = new byte[size]; - in.readFully(cellBlock); - cellBlockScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock); - } else { - cellBlockScanner = null; - } - call.setResponse(value, cellBlockScanner); - } - private void readResponse(ChannelHandlerContext ctx, ByteBuf buf) throws IOException { - int totalSize = buf.readInt(); - ByteBufInputStream in = new ByteBufInputStream(buf); - ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); - int id = responseHeader.getCallId(); - if (LOG.isTraceEnabled()) { - LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader) - + ", totalSize: " + totalSize + " bytes"); - } - RemoteException remoteExc; - if (responseHeader.hasException()) { - ExceptionResponse exceptionResponse = responseHeader.getException(); - remoteExc = IPCUtil.createRemoteException(exceptionResponse); - if (IPCUtil.isFatalConnectionException(exceptionResponse)) { - // Here we will cleanup all calls so do not need to fall back, just return. - exceptionCaught(ctx, remoteExc); - return; - } - } else { - remoteExc = null; - } - Call call = id2Call.remove(id); - if (call == null) { - // So we got a response for which we have no corresponding 'call' here on the client-side. - // We probably timed out waiting, cleaned up all references, and now the server decides - // to return a response. There is nothing we can do w/ the response at this stage. Clean - // out the wire of the response so its out of the way and we can get other responses on - // this connection. - if (LOG.isDebugEnabled()) { - int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); - int whatIsLeftToRead = totalSize - readSoFar; - LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead - + " bytes"); - } - return; - } - call.callStats.setResponseSizeBytes(totalSize); - if (remoteExc != null) { - call.setException(remoteExc); - return; - } try { - finishCall(responseHeader, in, call); + conn.readResponse(new ByteBufInputStream(buf), id2Call, + remoteExc -> exceptionCaught(ctx, remoteExc)); } catch (IOException e) { - // As the call has been removed from id2Call map, if we hit an exception here, the - // exceptionCaught method can not help us finish the call, so here we need to catch the - // exception and finish it - // And in netty, the decoding the frame based, when reaching here we have already read a full + // In netty, the decoding the frame based, when reaching here we have already read a full // frame, so hitting exception here does not mean the stream decoding is broken, thus we do // not need to throw the exception out and close the connection. - call.setException(e); + LOG.warn("failed to process response", e); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index 045216e88811..369430e337ae 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -54,6 +54,8 @@ public interface RpcClient extends Closeable { // The client in 0.99+ does not ping the server. int PING_CALL_ID = -1; + byte[] REGISTRY_PREAMBLE_HEADER = new byte[] { 'R', 'e', 'g', 'i', 's', 't' }; + /** * Creates a "channel" that can be used by a blocking protobuf service. Useful setting up protobuf * blocking stubs. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java index dbe6ed1648df..65f936d6fc38 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java @@ -17,12 +17,17 @@ */ package org.apache.hadoop.hbase.ipc; +import java.io.DataInput; +import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.codec.Codec; @@ -34,12 +39,15 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; @@ -48,6 +56,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; /** @@ -72,6 +82,8 @@ abstract class RpcConnection { protected final CompressionCodec compressor; + protected final CellBlockBuilder cellBlockBuilder; + protected final MetricsConnection metrics; private final Map connectionAttributes; @@ -90,10 +102,12 @@ abstract class RpcConnection { protected RpcConnection(Configuration conf, HashedWheelTimer timeoutTimer, ConnectionId remoteId, String clusterId, boolean isSecurityEnabled, Codec codec, CompressionCodec compressor, - MetricsConnection metrics, Map connectionAttributes) throws IOException { + CellBlockBuilder cellBlockBuilder, MetricsConnection metrics, + Map connectionAttributes) throws IOException { this.timeoutTimer = timeoutTimer; this.codec = codec; this.compressor = compressor; + this.cellBlockBuilder = cellBlockBuilder; this.conf = conf; this.metrics = metrics; this.connectionAttributes = connectionAttributes; @@ -150,14 +164,13 @@ protected byte[] getConnectionHeaderPreamble() { // Assemble the preamble up in a buffer first and then send it. Writing individual elements, // they are getting sent across piecemeal according to wireshark and then server is messing // up the reading on occasion (the passed in stream is not buffered yet). - - // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE int rpcHeaderLen = HConstants.RPC_HEADER.length; + // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE byte[] preamble = new byte[rpcHeaderLen + 2]; System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen); preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION; synchronized (this) { - preamble[rpcHeaderLen + 1] = provider.getSaslAuthMethod().getCode(); + preamble[preamble.length - 1] = provider.getSaslAuthMethod().getCode(); } return preamble; } @@ -238,4 +251,103 @@ public void setLastTouched(long lastTouched) { * Does the clean up work after the connection is removed from the connection pool */ public abstract void cleanupConnection(); + + protected Call connectionRegistryCall; + + private void finishCall(ResponseHeader responseHeader, T in, + Call call) throws IOException { + Message value; + if (call.responseDefaultType != null) { + Message.Builder builder = call.responseDefaultType.newBuilderForType(); + if (!builder.mergeDelimitedFrom(in)) { + // The javadoc of mergeDelimitedFrom says returning false means the stream reaches EOF + // before reading any bytes out, so here we need to manually finish create the EOFException + // and finish the call + call.setException(new EOFException("EOF while reading response with type: " + + call.responseDefaultType.getClass().getName())); + return; + } + value = builder.build(); + } else { + value = null; + } + CellScanner cellBlockScanner; + if (responseHeader.hasCellBlockMeta()) { + int size = responseHeader.getCellBlockMeta().getLength(); + // Maybe we could read directly from the ByteBuf. + // The problem here is that we do not know when to release it. + byte[] cellBlock = new byte[size]; + in.readFully(cellBlock); + cellBlockScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock); + } else { + cellBlockScanner = null; + } + call.setResponse(value, cellBlockScanner); + } + + void readResponse(T in, Map id2Call, + Consumer fatalConnectionErrorConsumer) throws IOException { + int totalSize = in.readInt(); + ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); + int id = responseHeader.getCallId(); + if (LOG.isTraceEnabled()) { + LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader) + + ", totalSize: " + totalSize + " bytes"); + } + RemoteException remoteExc; + if (responseHeader.hasException()) { + ExceptionResponse exceptionResponse = responseHeader.getException(); + remoteExc = IPCUtil.createRemoteException(exceptionResponse); + if (IPCUtil.isFatalConnectionException(exceptionResponse)) { + // Here we will cleanup all calls so do not need to fall back, just return. + fatalConnectionErrorConsumer.accept(remoteExc); + if (connectionRegistryCall != null) { + connectionRegistryCall.setException(remoteExc); + connectionRegistryCall = null; + } + return; + } + } else { + remoteExc = null; + } + if (id < 0) { + if (connectionRegistryCall != null) { + LOG.debug("process connection registry call"); + finishCall(responseHeader, in, connectionRegistryCall); + connectionRegistryCall = null; + return; + } + } + Call call = id2Call.remove(id); + if (call == null) { + // So we got a response for which we have no corresponding 'call' here on the client-side. + // We probably timed out waiting, cleaned up all references, and now the server decides + // to return a response. There is nothing we can do w/ the response at this stage. Clean + // out the wire of the response so its out of the way and we can get other responses on + // this connection. + if (LOG.isDebugEnabled()) { + int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); + int whatIsLeftToRead = totalSize - readSoFar; + LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead + + " bytes"); + } + return; + } + call.callStats.setResponseSizeBytes(totalSize); + if (remoteExc != null) { + call.setException(remoteExc); + return; + } + try { + finishCall(responseHeader, in, call); + } catch (IOException e) { + // As the call has been removed from id2Call map, if we hit an exception here, the + // exceptionCaught method can not help us finish the call, so here we need to catch the + // exception and finish it + call.setException(e); + // throw the exception out, the upper layer should determine whether this is a critical + // problem + throw e; + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/BuiltInSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/BuiltInSaslAuthenticationProvider.java index 712d4035448b..b573a5ee771e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/BuiltInSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/BuiltInSaslAuthenticationProvider.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; +import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.yetus.audience.InterfaceAudience; /** @@ -35,4 +36,9 @@ public abstract class BuiltInSaslAuthenticationProvider implements SaslAuthentic public String getTokenKind() { return AUTH_TOKEN_TYPE; } + + protected static SaslAuthMethod createSaslAuthMethod(AuthMethod authMethod) { + return new SaslAuthMethod(authMethod.name(), authMethod.code, authMethod.mechanismName, + authMethod.authenticationMethod); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/DigestSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/DigestSaslAuthenticationProvider.java index d71c07d1575a..f22a06474aef 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/DigestSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/DigestSaslAuthenticationProvider.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.yetus.audience.InterfaceAudience; /** @@ -26,8 +26,7 @@ @InterfaceAudience.Private public class DigestSaslAuthenticationProvider extends BuiltInSaslAuthenticationProvider { - public static final SaslAuthMethod SASL_AUTH_METHOD = - new SaslAuthMethod("DIGEST", (byte) 82, "DIGEST-MD5", AuthenticationMethod.TOKEN); + public static final SaslAuthMethod SASL_AUTH_METHOD = createSaslAuthMethod(AuthMethod.DIGEST); @Override public SaslAuthMethod getSaslAuthMethod() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslAuthenticationProvider.java index 7dea40f2657a..df6fce859b7f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslAuthenticationProvider.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.yetus.audience.InterfaceAudience; /** @@ -26,8 +26,7 @@ @InterfaceAudience.Private public class GssSaslAuthenticationProvider extends BuiltInSaslAuthenticationProvider { - public static final SaslAuthMethod SASL_AUTH_METHOD = - new SaslAuthMethod("KERBEROS", (byte) 81, "GSSAPI", AuthenticationMethod.KERBEROS); + public static final SaslAuthMethod SASL_AUTH_METHOD = createSaslAuthMethod(AuthMethod.KERBEROS); @Override public SaslAuthMethod getSaslAuthMethod() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/SimpleSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/SimpleSaslAuthenticationProvider.java index 01b1f452685a..9d79b648c6e4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/SimpleSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/SimpleSaslAuthenticationProvider.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; -import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.yetus.audience.InterfaceAudience; /** @@ -25,8 +25,8 @@ */ @InterfaceAudience.Private public class SimpleSaslAuthenticationProvider extends BuiltInSaslAuthenticationProvider { - public static final SaslAuthMethod SASL_AUTH_METHOD = - new SaslAuthMethod("SIMPLE", (byte) 80, "", AuthenticationMethod.SIMPLE); + + public static final SaslAuthMethod SASL_AUTH_METHOD = createSaslAuthMethod(AuthMethod.SIMPLE); @Override public SaslAuthMethod getSaslAuthMethod() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 3b792a5bd15f..30d69d4b3f9e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java @@ -21,15 +21,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; /** * Registry that does nothing. Otherwise, default Registry wants zookeeper up and running. */ @InterfaceAudience.Private -class DoNothingConnectionRegistry implements ConnectionRegistry { +public class DoNothingConnectionRegistry implements ConnectionRegistry { - public DoNothingConnectionRegistry(Configuration conf) { + public DoNothingConnectionRegistry(Configuration conf, User user) { } @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 4bf425ed562e..f65c7ccb6e75 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -141,9 +142,9 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(adminStub).stopServer(any(HBaseRpcController.class), any(StopServerRequest.class), any()); - - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null, - UserProvider.instantiate(CONF).getCurrent()) { + User user = UserProvider.instantiate(CONF).getCurrent(); + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, + user) { @Override CompletableFuture getMasterStub() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index ff4a92ae394d..e56fffbb2642 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -63,15 +64,15 @@ public class TestAsyncConnectionTracing { @Before public void setUp() throws IOException { - ConnectionRegistry registry = new DoNothingConnectionRegistry(CONF) { + User user = UserProvider.instantiate(CONF).getCurrent(); + ConnectionRegistry registry = new DoNothingConnectionRegistry(CONF, user) { @Override public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = new AsyncConnectionImpl(CONF, registry, "test", null, - UserProvider.instantiate(CONF).getCurrent()); + conn = new AsyncConnectionImpl(CONF, registry, "test", null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java index b306500c8b13..6380f1f6fb0f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java @@ -24,6 +24,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FutureUtils; @@ -45,8 +47,8 @@ public class TestAsyncMetaRegionLocatorFailFast { private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry { - public FaultyConnectionRegistry(Configuration conf) { - super(conf); + public FaultyConnectionRegistry(Configuration conf, User user) { + super(conf, user); } @Override @@ -56,8 +58,9 @@ public CompletableFuture getMetaRegionLocations() { } @BeforeClass - public static void setUp() { - LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF)); + public static void setUp() throws IOException { + LOCATOR = new AsyncMetaRegionLocator( + new FaultyConnectionRegistry(CONF, UserProvider.instantiate(CONF).getCurrent())); } @Test(expected = DoNotRetryIOException.class) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 335894303c08..a7df92999d08 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -92,13 +93,14 @@ public void setUp() throws IOException { ServerName.valueOf("127.0.0.2", 12345, EnvironmentEdgeManager.currentTime())), new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(metaRegionInfo, 2), ServerName.valueOf("127.0.0.3", 12345, EnvironmentEdgeManager.currentTime()))); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF) { + User user = UserProvider.instantiate(CONF).getCurrent(); + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user) { @Override public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(locs); } - }, "test", null, UserProvider.instantiate(CONF).getCurrent()); + }, "test", null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index e57967ae7211..cb5431c35d3e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -160,8 +161,9 @@ public Void answer(InvocationOnMock invocation) throws Throwable { return null; } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null, - UserProvider.instantiate(CONF).getCurrent()) { + User user = UserProvider.instantiate(CONF).getCurrent(); + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, + user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index f9b86221af1e..2cecc974b6ef 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java @@ -209,37 +209,37 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); final User user = UserProvider.instantiate(CONF).getCurrent(); - conn = - new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null, user) { - - @Override - AsyncRegionLocator getLocator() { - AsyncRegionLocator locator = mock(AsyncRegionLocator.class); - Answer> answer = - new Answer>() { - - @Override - public CompletableFuture answer(InvocationOnMock invocation) - throws Throwable { - TableName tableName = invocation.getArgument(0); - RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); - ServerName serverName = ServerName.valueOf("rs", 16010, 12345); - HRegionLocation loc = new HRegionLocation(info, serverName); - return CompletableFuture.completedFuture(loc); - } - }; - doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), - any(RegionLocateType.class), anyLong()); - doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), - anyInt(), any(RegionLocateType.class), anyLong()); - return locator; - } + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, + user) { - @Override - ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException { - return stub; - } - }; + @Override + AsyncRegionLocator getLocator() { + AsyncRegionLocator locator = mock(AsyncRegionLocator.class); + Answer> answer = + new Answer>() { + + @Override + public CompletableFuture answer(InvocationOnMock invocation) + throws Throwable { + TableName tableName = invocation.getArgument(0); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + ServerName serverName = ServerName.valueOf("rs", 16010, 12345); + HRegionLocation loc = new HRegionLocation(info, serverName); + return CompletableFuture.completedFuture(loc); + } + }; + doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), + any(RegionLocateType.class), anyLong()); + doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), + anyInt(), any(RegionLocateType.class), anyLong()); + return locator; + } + + @Override + ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException { + return stub; + } + }; table = conn.getTable(TableName.valueOf("table"), ForkJoinPool.commonPool()); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java index a2df7e932395..bd2ca9867f34 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FutureUtils; @@ -49,8 +50,8 @@ public static final class ConnectionRegistryForTest extends DoNothingConnectionR private boolean closed = false; - public ConnectionRegistryForTest(Configuration conf) { - super(conf); + public ConnectionRegistryForTest(Configuration conf, User user) { + super(conf, user); CREATED.add(this); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java index 54b351f00a3b..08c56fe95868 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java @@ -58,7 +58,9 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse; @Category({ ClientTests.class, SmallTests.class }) public class TestRpcBasedRegistryHedgedReads { @@ -132,6 +134,12 @@ public static final class RpcChannelImpl implements RpcChannel { @Override public void callMethod(MethodDescriptor method, RpcController controller, Message request, Message responsePrototype, RpcCallback done) { + if (method.getService().equals(ConnectionRegistryService.getDescriptor())) { + // this is for setting up the rpc client + done.run( + GetConnectionRegistryResponse.newBuilder().setClusterId(RESP.getClusterId()).build()); + return; + } if (!method.getName().equals("GetClusterId")) { // On RPC failures, MasterRegistry internally runs getMasters() RPC to keep the master list // fresh. We do not want to intercept those RPCs here and double count. @@ -155,9 +163,9 @@ public void callMethod(MethodDescriptor method, RpcController controller, Messag private AbstractRpcBasedConnectionRegistry createRegistry(int hedged) throws IOException { Configuration conf = UTIL.getConfiguration(); conf.setInt(HEDGED_REQS_FANOUT_CONFIG_NAME, hedged); - return new AbstractRpcBasedConnectionRegistry(conf, HEDGED_REQS_FANOUT_CONFIG_NAME, - INITIAL_DELAY_SECS_CONFIG_NAME, REFRESH_INTERVAL_SECS_CONFIG_NAME, - MIN_REFRESH_INTERVAL_SECS_CONFIG_NAME) { + return new AbstractRpcBasedConnectionRegistry(conf, User.getCurrent(), + HEDGED_REQS_FANOUT_CONFIG_NAME, INITIAL_DELAY_SECS_CONFIG_NAME, + REFRESH_INTERVAL_SECS_CONFIG_NAME, MIN_REFRESH_INTERVAL_SECS_CONFIG_NAME) { @Override protected Set getBootstrapNodes(Configuration conf) throws IOException { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java index 10948358ff92..8aadce85651d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java @@ -56,6 +56,8 @@ import org.apache.hbase.thirdparty.com.google.common.io.Closeables; import org.apache.hbase.thirdparty.io.netty.handler.ssl.NotSslRecordException; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService; + /** * A simple UT to make sure that we do not leak the SslExceptions to netty's TailContext, where it * will generate a confusing WARN message. @@ -149,11 +151,12 @@ public Void answer(InvocationOnMock invocation) throws Throwable { Address.fromParts("127.0.0.1", server.getLocalPort())); NettyRpcConnection conn = client.createConnection(id); BlockingRpcCallback done = new BlockingRpcCallback<>(); - Call call = - new Call(1, null, null, null, null, 0, 0, Collections.emptyMap(), done, new CallStats()); + Call call = new Call(1, ClientMetaService.getDescriptor().getMethods().get(0), null, null, null, + 0, 0, Collections.emptyMap(), done, new CallStats()); HBaseRpcController hrc = new HBaseRpcControllerImpl(); conn.sendRequest(call, hrc); done.get(); + call.error.printStackTrace(); assertThat(call.error, instanceOf(NotSslRecordException.class)); Waiter.waitFor(conf, 5000, () -> msg.get() != null); verify(mockAppender).append(any()); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ConcurrentMapUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ConcurrentMapUtils.java index cf8130e624e2..9c5ebe8519f4 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ConcurrentMapUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ConcurrentMapUtils.java @@ -38,14 +38,6 @@ public static V computeIfAbsent(ConcurrentMap map, K key, Supplier< }); } - /** - * A supplier that throws IOException when get. - */ - @FunctionalInterface - public interface IOExceptionSupplier { - V get() throws IOException; - } - /** * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the * value already exists. So here we copy the implementation of diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IOExceptionSupplier.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IOExceptionSupplier.java new file mode 100644 index 000000000000..11771a47c083 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IOExceptionSupplier.java @@ -0,0 +1,30 @@ +/* + * 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.util; + +import java.io.IOException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A supplier that throws IOException when get. + */ +@InterfaceAudience.Private +@FunctionalInterface +public interface IOExceptionSupplier { + V get() throws IOException; +} diff --git a/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto b/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto index 8dd0d1abdf3d..f55b892413b2 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto @@ -105,4 +105,21 @@ service ClientMetaService { * Get nodes which could be used as ClientMetaService */ rpc GetBootstrapNodes(GetBootstrapNodesRequest) returns (GetBootstrapNodesResponse); -} \ No newline at end of file +} + +message GetConnectionRegistryRequest { +} + +/** + * For returning connection registry information to client, like cluster id + */ +message GetConnectionRegistryResponse { + required string cluster_id = 1; +} + +/** + * Just a fake rpc service for getting connection registry information + */ +service ConnectionRegistryService { + rpc GetConnectionRegistry(GetConnectionRegistryRequest) returns(GetConnectionRegistryResponse); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java index 579da46af1c1..7225f92b7ff9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java @@ -64,7 +64,7 @@ private static AsyncClusterConnection createAsyncClusterConnection(Configuration */ public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, SocketAddress localAddress, User user) throws IOException { - return createAsyncClusterConnection(conf, ConnectionRegistryFactory.getRegistry(conf), + return createAsyncClusterConnection(conf, ConnectionRegistryFactory.getRegistry(conf, user), localAddress, user); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index 629b3468cbe5..1d93fbd0f668 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -63,7 +63,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.WriteBufferWaterMark; import org.apache.hbase.thirdparty.io.netty.channel.group.ChannelGroup; import org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroup; -import org.apache.hbase.thirdparty.io.netty.handler.codec.FixedLengthFrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.ssl.OptionalSslHandler; import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.hbase.thirdparty.io.netty.handler.ssl.SslHandler; @@ -167,13 +166,15 @@ protected void initChannel(Channel ch) throws Exception { ch.config().setWriteBufferWaterMark(writeBufferWaterMark); ch.config().setAllocator(channelAllocator); ChannelPipeline pipeline = ch.pipeline(); - FixedLengthFrameDecoder preambleDecoder = new FixedLengthFrameDecoder(6); - preambleDecoder.setSingleDecode(true); + NettyServerRpcConnection conn = createNettyServerRpcConnection(ch); + if (conf.getBoolean(HBASE_SERVER_NETTY_TLS_ENABLED, false)) { initSSL(pipeline, conn, conf.getBoolean(HBASE_SERVER_NETTY_TLS_SUPPORTPLAINTEXT, true)); } - pipeline.addLast(NettyRpcServerPreambleHandler.DECODER_NAME, preambleDecoder) + pipeline + .addLast(NettyRpcServerPreambleHandler.DECODER_NAME, + NettyRpcServerPreambleHandler.createDecoder()) .addLast(new NettyRpcServerPreambleHandler(NettyRpcServer.this, conn)) // We need NettyRpcServerResponseEncoder here because NettyRpcServerPreambleHandler may // send RpcResponse to client. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java index 02e1b5858117..5aa77e0e8ace 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.ipc.ServerRpcConnection.PreambleResponse; import org.apache.hadoop.hbase.util.NettyFutureUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -25,6 +26,7 @@ import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext; import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline; import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; +import org.apache.hbase.thirdparty.io.netty.handler.codec.FixedLengthFrameDecoder; import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder; /** @@ -45,6 +47,12 @@ public NettyRpcServerPreambleHandler(NettyRpcServer rpcServer, NettyServerRpcCon this.conn = conn; } + static FixedLengthFrameDecoder createDecoder() { + FixedLengthFrameDecoder preambleDecoder = new FixedLengthFrameDecoder(6); + preambleDecoder.setSingleDecode(true); + return preambleDecoder; + } + @Override protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception { if (processPreambleError) { @@ -57,11 +65,19 @@ protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Excep ByteBuffer buf = ByteBuffer.allocate(msg.readableBytes()); msg.readBytes(buf); buf.flip(); - if (!conn.processPreamble(buf)) { + PreambleResponse resp = conn.processPreamble(buf); + if (resp == PreambleResponse.CLOSE) { processPreambleError = true; conn.close(); return; } + if (resp == PreambleResponse.CONTINUE) { + // we use a single decode decoder, so here we need to replace it with a new one so it will + // decode a new preamble header again + ctx.pipeline().replace(DECODER_NAME, DECODER_NAME, createDecoder()); + return; + } + // resp == PreambleResponse.SUCCEED ChannelPipeline p = ctx.pipeline(); if (conn.useSasl) { LengthFieldBasedFrameDecoder decoder = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java index 4c32b2b6a5fa..be97ad582c37 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java @@ -41,6 +41,7 @@ import org.apache.commons.crypto.random.CryptoRandomFactory; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.ConnectionRegistryEndpoint; import org.apache.hadoop.hbase.client.VersionInfoUtil; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.io.ByteBufferOutputStream; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.security.provider.SaslServerAuthenticationProviders; import org.apache.hadoop.hbase.security.provider.SimpleSaslServerAuthenticationProvider; import org.apache.hadoop.hbase.trace.TraceUtil; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.IntWritable; @@ -87,6 +89,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo; /** Reads calls from a connection and queues them for handling. */ @@ -688,10 +691,32 @@ private void doBadPreambleHandling(String msg) throws IOException { } private void doBadPreambleHandling(String msg, Exception e) throws IOException { - RpcServer.LOG.warn(msg); + RpcServer.LOG.warn(msg, e); doRespond(getErrorResponse(msg, e)); } + private boolean doConnectionRegistryResponse() throws IOException { + if (!(rpcServer.server instanceof ConnectionRegistryEndpoint)) { + // should be in tests or some scenarios where we should not reach here + return false; + } + // on backup masters, this request may be blocked since we need to fetch it from filesystem, + // but since it is just backup master, it is not a critical problem + String clusterId = ((ConnectionRegistryEndpoint) rpcServer.server).getClusterId(); + RpcServer.LOG.debug("Response connection registry, clusterId = '{}'", clusterId); + if (clusterId == null) { + // should be in tests or some scenarios where we should not reach here + return false; + } + GetConnectionRegistryResponse resp = + GetConnectionRegistryResponse.newBuilder().setClusterId(clusterId).build(); + ResponseHeader header = ResponseHeader.newBuilder().setCallId(-1).build(); + ByteBuffer buf = ServerCall.createHeaderAndMessageBytes(resp, header, 0, null); + BufferChain bufChain = new BufferChain(buf); + doRespond(() -> bufChain); + return true; + } + protected final void callCleanupIfNeeded() { if (callCleanup != null) { callCleanup.run(); @@ -699,30 +724,42 @@ protected final void callCleanupIfNeeded() { } } - protected final boolean processPreamble(ByteBuffer preambleBuffer) throws IOException { - assert preambleBuffer.remaining() == 6; - for (int i = 0; i < RPC_HEADER.length; i++) { - if (RPC_HEADER[i] != preambleBuffer.get()) { - doBadPreambleHandling( - "Expected HEADER=" + Bytes.toStringBinary(RPC_HEADER) + " but received HEADER=" - + Bytes.toStringBinary(preambleBuffer.array(), 0, RPC_HEADER.length) + " from " - + toString()); - return false; - } - } - int version = preambleBuffer.get() & 0xFF; - byte authbyte = preambleBuffer.get(); + protected enum PreambleResponse { + SUCCEED, // successfully processed the rpc preamble header + CONTINUE, // the preamble header is for other purpose, wait for the rpc preamble header + CLOSE // close the rpc connection + } + protected final PreambleResponse processPreamble(ByteBuffer preambleBuffer) throws IOException { + assert preambleBuffer.remaining() == 6; + if ( + ByteBufferUtils.equals(preambleBuffer, preambleBuffer.position(), 6, + RpcClient.REGISTRY_PREAMBLE_HEADER, 0, 6) && doConnectionRegistryResponse() + ) { + return PreambleResponse.CLOSE; + } + if (!ByteBufferUtils.equals(preambleBuffer, preambleBuffer.position(), 4, RPC_HEADER, 0, 4)) { + doBadPreambleHandling( + "Expected HEADER=" + Bytes.toStringBinary(RPC_HEADER) + " but received HEADER=" + + Bytes.toStringBinary( + ByteBufferUtils.toBytes(preambleBuffer, preambleBuffer.position(), RPC_HEADER.length), + 0, RPC_HEADER.length) + + " from " + toString()); + return PreambleResponse.CLOSE; + } + int version = preambleBuffer.get(preambleBuffer.position() + 4) & 0xFF; + byte authByte = preambleBuffer.get(preambleBuffer.position() + 5); if (version != RpcServer.CURRENT_VERSION) { - String msg = getFatalConnectionString(version, authbyte); + String msg = getFatalConnectionString(version, authByte); doBadPreambleHandling(msg, new WrongVersionException(msg)); - return false; + return PreambleResponse.CLOSE; } - this.provider = this.saslProviders.selectProvider(authbyte); + + this.provider = this.saslProviders.selectProvider(authByte); if (this.provider == null) { - String msg = getFatalConnectionString(version, authbyte); + String msg = getFatalConnectionString(version, authByte); doBadPreambleHandling(msg, new BadAuthException(msg)); - return false; + return PreambleResponse.CLOSE; } // TODO this is a wart while simple auth'n doesn't go through sasl. if (this.rpcServer.isSecurityEnabled && isSimpleAuthentication()) { @@ -732,7 +769,7 @@ protected final boolean processPreamble(ByteBuffer preambleBuffer) throws IOExce } else { AccessDeniedException ae = new AccessDeniedException("Authentication is required"); doRespond(getErrorResponse(ae.getMessage(), ae)); - return false; + return PreambleResponse.CLOSE; } } if (!this.rpcServer.isSecurityEnabled && !isSimpleAuthentication()) { @@ -745,7 +782,7 @@ protected final boolean processPreamble(ByteBuffer preambleBuffer) throws IOExce skipInitialSaslHandshake = true; } useSasl = !(provider instanceof SimpleSaslServerAuthenticationProvider); - return true; + return PreambleResponse.SUCCEED; } boolean isSimpleAuthentication() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java index ac705d7a26fa..9e90a7a31339 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleServerRpcConnection.java @@ -137,12 +137,21 @@ private int readPreamble() throws IOException { return count; } preambleBuffer.flip(); - if (!processPreamble(preambleBuffer)) { - return -1; + PreambleResponse resp = processPreamble(preambleBuffer); + switch (resp) { + case SUCCEED: + preambleBuffer = null; // do not need it anymore + connectionPreambleRead = true; + return count; + case CONTINUE: + // wait for the next preamble header + preambleBuffer.reset(); + return count; + case CLOSE: + return -1; + default: + throw new IllegalArgumentException("Unknown preamble response: " + resp); } - preambleBuffer = null; // do not need it anymore - connectionPreambleRead = true; - return count; } private int read4Bytes() throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java index d001ea755b74..cbfde9c7e172 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java @@ -50,9 +50,9 @@ import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; -import org.apache.hadoop.hbase.util.ConcurrentMapUtils.IOExceptionSupplier; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.IOExceptionSupplier; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index a94c214a3250..509d74e0335c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -40,7 +40,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.DummyConnectionRegistry; +import org.apache.hadoop.hbase.client.ConnectionRegistry; +import org.apache.hadoop.hbase.client.DoNothingConnectionRegistry; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.DirScanPool; @@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -92,9 +94,10 @@ public class TestZooKeeperTableArchiveClient { private static RegionServerServices rss; private static DirScanPool POOL; - public static final class MockRegistry extends DummyConnectionRegistry { + public static final class MockRegistry extends DoNothingConnectionRegistry { - public MockRegistry(Configuration conf) { + public MockRegistry(Configuration conf, User user) { + super(conf, user); } @Override @@ -110,8 +113,8 @@ public CompletableFuture getClusterId() { public static void setupCluster() throws Exception { setupConf(UTIL.getConfiguration()); UTIL.startMiniZKCluster(); - UTIL.getConfiguration().setClass(MockRegistry.REGISTRY_IMPL_CONF_KEY, MockRegistry.class, - DummyConnectionRegistry.class); + UTIL.getConfiguration().setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + MockRegistry.class, ConnectionRegistry.class); CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration()); archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION); // make hfile archiving node so we can archive files diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 8b38db974d7c..0ff105743e0c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.After; @@ -59,7 +60,7 @@ protected static void startClusterAndCreateTable() throws Exception { UTIL.getAdmin().createTable(td, SPLIT_KEYS); UTIL.waitTableAvailable(TABLE_NAME); try (ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration())) { + ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java deleted file mode 100644 index cc2e9493d039..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.client; - -import java.util.concurrent.CompletableFuture; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; - -/** - * Can be overridden in UT if you only want to implement part of the methods in - * {@link ConnectionRegistry}. - */ -public class DummyConnectionRegistry implements ConnectionRegistry { - - public static final String REGISTRY_IMPL_CONF_KEY = - HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY; - - @Override - public CompletableFuture getMetaRegionLocations() { - return null; - } - - @Override - public CompletableFuture getClusterId() { - return null; - } - - @Override - public CompletableFuture getActiveMaster() { - return null; - } - - @Override - public String getConnectionString() { - return null; - } - - @Override - public void close() { - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index 4dd4d4550777..da400f29c0c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -55,7 +56,7 @@ public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); try (ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) { + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index ad9bf551c033..90d2cb51e8cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator; import org.apache.hadoop.hbase.client.trace.StringTraceRenderer; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.trace.OpenTelemetryClassRule; @@ -106,7 +107,8 @@ protected void before() throws Throwable { testUtil = miniClusterRule.getTestingUtility(); HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 3); testUtil.waitUntilNoRegionsInTransition(); - registry = ConnectionRegistryFactory.getRegistry(testUtil.getConfiguration()); + registry = + ConnectionRegistryFactory.getRegistry(testUtil.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(testUtil, registry); admin.balancerSwitch(false).get(); locator = new AsyncMetaRegionLocator(registry); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index f3b231d5bde8..a6d0ab81f912 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -128,7 +128,7 @@ public void setUpBeforeTest() throws InterruptedException, ExecutionException, I // Enable meta replica LoadBalance mode for this connection. c.set(RegionLocator.LOCATOR_META_REPLICAS_MODE, metaReplicaMode.toString()); ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); conn = new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), null, User.getCurrent()); locator = new AsyncNonMetaRegionLocator(conn); @@ -147,7 +147,7 @@ public void tearDownAfterTest() throws IOException { } @Parameterized.Parameters - public static Collection parameters() { + public static Collection paramAbstractTestRegionLocatoreters() { return Arrays .asList(new Object[][] { { CatalogReplicaMode.NONE }, { CatalogReplicaMode.LOAD_BALANCE } }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 2dd08b36b30d..50c9ab9f5657 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -125,7 +125,7 @@ public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); TEST_UTIL.getAdmin().balancerSwitch(false, true); ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), null, User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index ee0963e1f8b3..bacd7bb32d70 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java @@ -100,7 +100,7 @@ public static void setUp() throws Exception { TEST_UTIL.createTable(TABLE_NAME, FAMILY); TEST_UTIL.waitTableAvailable(TABLE_NAME); ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), null, User.getCurrent()); LOCATOR = CONN.getLocator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index ab43ec545d93..3c8327145f32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -73,7 +73,7 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.createTable(TABLE_NAME, FAMILY); TEST_UTIL.waitTableAvailable(TABLE_NAME); ConnectionRegistry registry = - ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), null, User.getCurrent()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index 1a7ac8819e49..0de59a4c32bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -94,7 +95,8 @@ public static void setUp() throws Exception { FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); - try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) { + try (ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(conf, User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java index d5b0ee18e594..549575f4f404 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBootstrapNodeUpdate.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; +import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.AfterClass; @@ -65,7 +66,7 @@ public static void setUpBeforeClass() throws Exception { conf.setLong(RpcConnectionRegistry.PERIODIC_REFRESH_INTERVAL_SECS, 1); conf.setLong(RpcConnectionRegistry.MIN_SECS_BETWEEN_REFRESHES, 1); UTIL.startMiniCluster(3); - REGISTRY = new RpcConnectionRegistry(conf); + REGISTRY = new RpcConnectionRegistry(conf, UserProvider.instantiate(conf).getCurrent()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 0e2feae841cf..5c78e53f7e60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -77,7 +77,8 @@ public static void setUp() throws Exception { () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() >= numOfMetaReplica); - registry = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), null, User.getCurrent()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index c9238bc99978..d79603cea3cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; @@ -85,45 +85,39 @@ private static String generateDummyMastersList(int size) { * Makes sure the master registry parses the master end points in the configuration correctly. */ @Test - public void testMasterAddressParsing() throws IOException { + public void testMasterAddressParsing() throws Exception { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); int numMasters = 10; conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters)); - try (MasterRegistry registry = new MasterRegistry(conf)) { - List parsedMasters = new ArrayList<>(registry.getParsedServers()); - // Half of them would be without a port, duplicates are removed. - assertEquals(numMasters / 2 + 1, parsedMasters.size()); - // Sort in the increasing order of port numbers. - Collections.sort(parsedMasters, Comparator.comparingInt(ServerName::getPort)); - for (int i = 0; i < parsedMasters.size(); i++) { - ServerName sn = parsedMasters.get(i); - assertEquals("localhost", sn.getHostname()); - if (i == parsedMasters.size() - 1) { - // Last entry should be the one with default port. - assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort()); - } else { - assertEquals(1000 + (2 * i), sn.getPort()); - } + List parsedMasters = new ArrayList<>(MasterRegistry.parseMasterAddrs(conf)); + // Half of them would be without a port, duplicates are removed. + assertEquals(numMasters / 2 + 1, parsedMasters.size()); + // Sort in the increasing order of port numbers. + Collections.sort(parsedMasters, Comparator.comparingInt(ServerName::getPort)); + for (int i = 0; i < parsedMasters.size(); i++) { + ServerName sn = parsedMasters.get(i); + assertEquals("localhost", sn.getHostname()); + if (i == parsedMasters.size() - 1) { + // Last entry should be the one with default port. + assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort()); + } else { + assertEquals(1000 + (2 * i), sn.getPort()); } } } @Test - public void testMasterPortDefaults() throws IOException { + public void testMasterPortDefaults() throws Exception { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.set(HConstants.MASTER_ADDRS_KEY, "localhost"); - try (MasterRegistry registry = new MasterRegistry(conf)) { - List parsedMasters = new ArrayList<>(registry.getParsedServers()); - ServerName sn = parsedMasters.get(0); - assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort()); - } + List parsedMasters = new ArrayList<>(MasterRegistry.parseMasterAddrs(conf)); + ServerName sn = parsedMasters.get(0); + assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort()); final int CUSTOM_MASTER_PORT = 9999; conf.setInt(HConstants.MASTER_PORT, CUSTOM_MASTER_PORT); - try (MasterRegistry registry = new MasterRegistry(conf)) { - List parsedMasters = new ArrayList<>(registry.getParsedServers()); - ServerName sn = parsedMasters.get(0); - assertEquals(CUSTOM_MASTER_PORT, sn.getPort()); - } + parsedMasters = new ArrayList<>(MasterRegistry.parseMasterAddrs(conf)); + sn = parsedMasters.get(0); + assertEquals(CUSTOM_MASTER_PORT, sn.getPort()); } @Test @@ -133,7 +127,7 @@ public void testRegistryRPCs() throws Exception { final int size = activeMaster.getMetaLocations().size(); for (int numHedgedReqs = 1; numHedgedReqs <= size; numHedgedReqs++) { conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, numHedgedReqs); - try (MasterRegistry registry = new MasterRegistry(conf)) { + try (MasterRegistry registry = new MasterRegistry(conf, User.getCurrent())) { // Add wait on all replicas being assigned before proceeding w/ test. Failed on occasion // because not all replicas had made it up before test started. RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -166,7 +160,7 @@ public void testDynamicMasterConfigurationRefresh() throws Exception { conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4); // Do not limit the number of refreshes during the test run. conf.setLong(MasterRegistry.MASTER_REGISTRY_MIN_SECS_BETWEEN_REFRESHES, 0); - try (MasterRegistry registry = new MasterRegistry(conf)) { + try (MasterRegistry registry = new MasterRegistry(conf, User.getCurrent())) { final Set masters = registry.getParsedServers(); assertTrue(masters.contains(badServer)); // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index d78832d9a8a0..beb054eaf366 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -63,7 +64,8 @@ public class TestMetaRegionLocationCache { public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); - REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + REGISTRY = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY); TEST_UTIL.getAdmin().balancerSwitch(false, true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index 9c26bccbbb31..d33cc943355c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -17,22 +17,29 @@ */ package org.apache.hadoop.hbase.client; -import static org.hamcrest.CoreMatchers.hasItems; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasItems; +import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; import org.apache.hadoop.hbase.regionserver.RSRpcServices; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.After; @@ -43,6 +50,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.io.Closeables; @Category({ MediumTests.class, ClientTests.class }) @@ -74,7 +82,7 @@ public static void tearDownAfterClass() throws Exception { @Before public void setUp() throws IOException { - registry = new RpcConnectionRegistry(UTIL.getConfiguration()); + registry = new RpcConnectionRegistry(UTIL.getConfiguration(), User.getCurrent()); } @After @@ -94,9 +102,20 @@ private void setMaxNodeCount(int count) { @Test public void testRegistryRPCs() throws Exception { HMaster activeMaster = UTIL.getHBaseCluster().getMaster(); - // sleep 3 seconds, since our initial delay is 1 second, we should have refreshed the endpoints - Thread.sleep(3000); - assertThat(registry.getParsedServers(), + // should only contains the active master + Set initialParsedServers = registry.getParsedServers(); + assertThat(initialParsedServers, hasSize(1)); + // no start code in configuration + assertThat(initialParsedServers, + hasItem(ServerName.valueOf(activeMaster.getServerName().getHostname(), + activeMaster.getServerName().getPort(), -1))); + // Since our initial delay is 1 second, finally we should have refreshed the endpoints + UTIL.waitFor(5000, () -> registry.getParsedServers() + .contains(activeMaster.getServerManager().getOnlineServersList().get(0))); + Set parsedServers = registry.getParsedServers(); + assertThat(parsedServers, + hasSize(activeMaster.getServerManager().getOnlineServersList().size())); + assertThat(parsedServers, hasItems(activeMaster.getServerManager().getOnlineServersList().toArray(new ServerName[0]))); // Add wait on all replicas being assigned before proceeding w/ test. Failed on occasion @@ -116,4 +135,32 @@ public void testRegistryRPCs() throws Exception { setMaxNodeCount(1); UTIL.waitFor(10000, () -> registry.getParsedServers().size() == 1); } + + /** + * Make sure that we can create the RpcClient when there are broken servers in the bootstrap nodes + */ + @Test + public void testBrokenBootstrapNodes() throws Exception { + Configuration conf = new Configuration(UTIL.getConfiguration()); + String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY)); + HMaster activeMaster = UTIL.getHBaseCluster().getMaster(); + String clusterId = activeMaster.getClusterId(); + // Add a non-working master + ServerName badServer = ServerName.valueOf("localhost", 1234, -1); + conf.set(RpcConnectionRegistry.BOOTSTRAP_NODES, badServer.toShortString()); + // only a bad server, the request should fail + try (RpcConnectionRegistry reg = new RpcConnectionRegistry(conf, User.getCurrent())) { + assertThrows(IOException.class, () -> reg.getParsedServers()); + } + + conf.set(RpcConnectionRegistry.BOOTSTRAP_NODES, + badServer.toShortString() + ", " + currentMasterAddrs); + // we will choose bootstrap node randomly so here we need to test it multiple times to make sure + // that we can skip the broken node + for (int i = 0; i < 10; i++) { + try (RpcConnectionRegistry reg = new RpcConnectionRegistry(conf, User.getCurrent())) { + assertEquals(clusterId, reg.getClusterId().get()); + } + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 1cbb36196684..6d585245e959 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -65,7 +65,7 @@ public class TestZKConnectionRegistry { public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); - REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration()); + REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration(), null); } @AfterClass @@ -99,7 +99,7 @@ public void testIndependentZKConnections() throws IOException { try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) { Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); otherConf.set(HConstants.ZOOKEEPER_QUORUM, MiniZooKeeperCluster.HOST); - try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) { + try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf, null)) { ReadOnlyZKClient zk2 = otherRegistry.getZKClient(); assertNotSame("Using a different configuration / quorum should result in different " + "backing zk connection.", zk1, zk2); @@ -116,7 +116,7 @@ public void testIndependentZKConnections() throws IOException { public void testNoMetaAvailable() throws InterruptedException { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.set("zookeeper.znode.metaserver", "whatever"); - try (ZKConnectionRegistry registry = new ZKConnectionRegistry(conf)) { + try (ZKConnectionRegistry registry = new ZKConnectionRegistry(conf, null)) { try { registry.getMetaRegionLocations().get(); fail("Should have failed since we set an incorrect meta znode prefix"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index a93f54d4d9d1..e4427c1690c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -34,6 +34,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -42,8 +43,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.times; import io.opentelemetry.api.common.AttributeKey; @@ -63,12 +66,18 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseServerBase; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MatcherPredicate; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.util.StringUtils; import org.hamcrest.Matcher; import org.junit.Rule; @@ -78,6 +87,8 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; @@ -88,6 +99,9 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse; /** * Some basic ipc tests. @@ -105,9 +119,14 @@ public abstract class AbstractTestIPC { CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, SimpleRpcServer.class.getName()); } - protected abstract RpcServer createRpcServer(final String name, - final List services, final InetSocketAddress bindAddress, - Configuration conf, RpcScheduler scheduler) throws IOException; + protected abstract RpcServer createRpcServer(Server server, String name, + List services, InetSocketAddress bindAddress, Configuration conf, + RpcScheduler scheduler) throws IOException; + + private RpcServer createRpcServer(String name, List services, + InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { + return createRpcServer(null, name, services, bindAddress, conf, scheduler); + } protected abstract AbstractRpcClient createRpcClientNoCodec(Configuration conf); @@ -568,4 +587,62 @@ public void testBadPreambleHeader() throws IOException, ServiceException { rpcServer.stop(); } } + + /** + * Testcase for getting connection registry information through connection preamble header, see + * HBASE-25051 for more details. + */ + @Test + public void testGetConnectionRegistry() throws IOException, ServiceException { + Configuration clientConf = new Configuration(CONF); + String clusterId = "test_cluster_id"; + HBaseServerBase server = mock(HBaseServerBase.class); + when(server.getClusterId()).thenReturn(clusterId); + // do not need any services + RpcServer rpcServer = createRpcServer(server, "testRpcServer", Collections.emptyList(), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); + try (AbstractRpcClient client = createRpcClient(clientConf)) { + rpcServer.start(); + InetSocketAddress addr = rpcServer.getListenerAddress(); + BlockingRpcChannel channel = + client.createBlockingRpcChannel(ServerName.valueOf(addr.getHostName(), addr.getPort(), + EnvironmentEdgeManager.currentTime()), User.getCurrent(), 0); + ConnectionRegistryService.BlockingInterface stub = + ConnectionRegistryService.newBlockingStub(channel); + GetConnectionRegistryResponse resp = + stub.getConnectionRegistry(null, GetConnectionRegistryRequest.getDefaultInstance()); + assertEquals(clusterId, resp.getClusterId()); + } + } + + /** + * Test server does not support getting connection registry information through connection + * preamble header, i.e, a new client connecting to an old server. We simulate this by using a + * Server without implementing the ConnectionRegistryEndpoint interface. + */ + @Test + public void testGetConnectionRegistryError() throws IOException, ServiceException { + Configuration clientConf = new Configuration(CONF); + // do not need any services + RpcServer rpcServer = createRpcServer("testRpcServer", Collections.emptyList(), + new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1)); + try (AbstractRpcClient client = createRpcClient(clientConf)) { + rpcServer.start(); + InetSocketAddress addr = rpcServer.getListenerAddress(); + RpcChannel channel = client.createRpcChannel(ServerName.valueOf(addr.getHostName(), + addr.getPort(), EnvironmentEdgeManager.currentTime()), User.getCurrent(), 0); + ConnectionRegistryService.Interface stub = ConnectionRegistryService.newStub(channel); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + BlockingRpcCallback done = new BlockingRpcCallback<>(); + stub.getConnectionRegistry(pcrc, GetConnectionRegistryRequest.getDefaultInstance(), done); + // should have failed so no response + assertNull(done.get()); + assertTrue(pcrc.failed()); + // should be a FatalConnectionException + assertThat(pcrc.getFailed(), instanceOf(RemoteException.class)); + assertEquals(FatalConnectionException.class.getName(), + ((RemoteException) pcrc.getFailed()).getClassName()); + assertThat(pcrc.getFailed().getMessage(), startsWith("Expected HEADER=")); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java index 9544e8c35458..e60cc879fd4f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java @@ -40,10 +40,10 @@ public class TestBlockingIPC extends AbstractTestIPC { HBaseClassTestRule.forClass(TestBlockingIPC.class); @Override - protected RpcServer createRpcServer(String name, + protected RpcServer createRpcServer(Server server, String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return RpcServerFactory.createRpcServer(null, name, services, bindAddress, conf, scheduler); + return RpcServerFactory.createRpcServer(server, name, services, bindAddress, conf, scheduler); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java index 6feab5f2cac8..a1b60e2cfa45 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RPCTests; @@ -103,10 +104,10 @@ private void setConf(Configuration conf) { } @Override - protected RpcServer createRpcServer(String name, + protected RpcServer createRpcServer(Server server, String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new NettyRpcServer(null, name, services, bindAddress, conf, scheduler, true); + return new NettyRpcServer(server, name, services, bindAddress, conf, scheduler, true); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java index 4c654123e130..1cbf6be26c65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyTlsIPC.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HBaseServerBase; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.client.ConnectionRegistryEndpoint; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.io.crypto.tls.KeyStoreFileType; import org.apache.hadoop.hbase.io.crypto.tls.X509KeyType; @@ -65,8 +67,6 @@ public class TestNettyTlsIPC extends AbstractTestIPC { private static NettyEventLoopGroupConfig EVENT_LOOP_GROUP_CONFIG; - private static HBaseServerBase SERVER; - @Parameterized.Parameter(0) public X509KeyType caKeyType; @@ -115,8 +115,6 @@ public static void setUpBeforeClass() throws IOException { PROVIDER = new X509TestContextProvider(CONF, dir); EVENT_LOOP_GROUP_CONFIG = NettyEventLoopGroupConfig.setup(CONF, TestNettyTlsIPC.class.getSimpleName()); - SERVER = mock(HBaseServerBase.class); - when(SERVER.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); } @AfterClass @@ -147,9 +145,16 @@ public void tearDown() { } @Override - protected RpcServer createRpcServer(String name, List services, - InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new NettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler, true); + protected RpcServer createRpcServer(Server server, String name, + List services, InetSocketAddress bindAddress, Configuration conf, + RpcScheduler scheduler) throws IOException { + HBaseServerBase mockServer = mock(HBaseServerBase.class); + when(mockServer.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); + if (server instanceof ConnectionRegistryEndpoint) { + String clusterId = ((ConnectionRegistryEndpoint) server).getClusterId(); + when(mockServer.getClusterId()).thenReturn(clusterId); + } + return new NettyRpcServer(mockServer, name, services, bindAddress, conf, scheduler, true); } @Override @@ -184,7 +189,9 @@ protected boolean isTcpNoDelay() { protected RpcServer createTestFailingRpcServer(String name, List services, InetSocketAddress bindAddress, Configuration conf, RpcScheduler scheduler) throws IOException { - return new FailingNettyRpcServer(SERVER, name, services, bindAddress, conf, scheduler); + HBaseServerBase mockServer = mock(HBaseServerBase.class); + when(mockServer.getEventLoopGroupConfig()).thenReturn(EVENT_LOOP_GROUP_CONFIG); + return new FailingNettyRpcServer(mockServer, name, services, bindAddress, conf, scheduler); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index 93fa22c00fd3..d6c7a0250015 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -34,15 +34,17 @@ import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer; import org.apache.hadoop.hbase.client.AsyncClusterConnection; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.ClusterConnectionFactory; +import org.apache.hadoop.hbase.client.ConnectionRegistry; +import org.apache.hadoop.hbase.client.DoNothingConnectionRegistry; import org.apache.hadoop.hbase.client.DummyAsyncClusterConnection; import org.apache.hadoop.hbase.client.DummyAsyncTable; -import org.apache.hadoop.hbase.client.DummyConnectionRegistry; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ReplicationTests; @@ -84,8 +86,8 @@ public class TestWALEntrySinkFilter { public void testWALEntryFilter() throws IOException { Configuration conf = HBaseConfiguration.create(); // Make it so our filter is instantiated on construction of ReplicationSink. - conf.setClass(DummyConnectionRegistry.REGISTRY_IMPL_CONF_KEY, DevNullConnectionRegistry.class, - DummyConnectionRegistry.class); + conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + DevNullConnectionRegistry.class, ConnectionRegistry.class); conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY, IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class); conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL, @@ -166,9 +168,10 @@ public boolean filter(TableName table, long writeTime) { } } - public static class DevNullConnectionRegistry extends DummyConnectionRegistry { + public static class DevNullConnectionRegistry extends DoNothingConnectionRegistry { - public DevNullConnectionRegistry(Configuration conf) { + public DevNullConnectionRegistry(Configuration conf, User user) { + super(conf, user); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 16ac215acaff..f132eb6964b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -21,22 +21,26 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.Arrays; import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.AsyncConnection; +import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.NettyRpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.security.AccessDeniedException; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.token.Token; @@ -51,11 +55,9 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.AuthenticationService; import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.WhoAmIRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.WhoAmIResponse; @@ -92,24 +94,58 @@ public void setUpBeforeMethod() { rpcClientImpl); } - @Test - public void test() throws Exception { - try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Table table = conn.getTable(TableName.META_TABLE_NAME)) { - CoprocessorRpcChannel rpcChannel = table.coprocessorService(HConstants.EMPTY_START_ROW); - AuthenticationProtos.AuthenticationService.BlockingInterface service = - AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel); - WhoAmIResponse response = service.whoAmI(null, WhoAmIRequest.getDefaultInstance()); + private void testToken() throws Exception { + try (AsyncConnection conn = + ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) { + AsyncTable table = conn.getTable(TableName.META_TABLE_NAME); + WhoAmIResponse response = + table. coprocessorService( + AuthenticationService::newStub, + (s, c, r) -> s.whoAmI(c, WhoAmIRequest.getDefaultInstance(), r), + HConstants.EMPTY_START_ROW).get(); assertEquals(USERNAME, response.getUsername()); assertEquals(AuthenticationMethod.TOKEN.name(), response.getAuthMethod()); - try { - service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance()); - } catch (ServiceException e) { - IOException ioe = ProtobufUtil.getRemoteException(e); - assertThat(ioe, instanceOf(AccessDeniedException.class)); - assertThat(ioe.getMessage(), - containsString("Token generation only allowed for Kerberos authenticated clients")); - } + IOException ioe = + assertThrows(IOException.class, + () -> FutureUtils.get(table. coprocessorService(AuthenticationService::newStub, + (s, c, r) -> s.getAuthenticationToken(c, + GetAuthenticationTokenRequest.getDefaultInstance(), r), + HConstants.EMPTY_START_ROW))); + assertThat(ioe, instanceOf(AccessDeniedException.class)); + assertThat(ioe.getMessage(), + containsString("Token generation only allowed for Kerberos authenticated clients")); } + + } + + /** + * Confirm that we will use delegation token first if token and kerberos tickets are both present + */ + @Test + public void testTokenFirst() throws Exception { + testToken(); + } + + /** + * Confirm that we can connect to cluster successfully when there is only token present, i.e, no + * kerberos ticket + */ + @Test + public void testOnlyToken() throws Exception { + User user = + User.createUserForTesting(TEST_UTIL.getConfiguration(), "no_krb_user", new String[0]); + for (Token token : User.getCurrent().getUGI().getCredentials() + .getAllTokens()) { + user.getUGI().addToken(token); + } + user.getUGI().doAs(new PrivilegedExceptionAction() { + + @Override + public Void run() throws Exception { + testToken(); + return null; + } + }); } } From afbf2adba25c24ce6b895865005367563efc7cb4 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Tue, 6 Feb 2024 15:28:05 -0500 Subject: [PATCH 090/109] HBASE-27687 Enhance quotas to consume blockBytesScanned rather than response size (#5654) Signed-off-by: Bryan Beaudreault --- .../hbase/quotas/DefaultOperationQuota.java | 34 ++- .../hbase/quotas/ExceedOperationQuota.java | 6 +- .../hadoop/hbase/quotas/OperationQuota.java | 10 + .../quotas/RegionServerRpcQuotaManager.java | 20 +- .../hadoop/hbase/regionserver/HRegion.java | 10 + .../hadoop/hbase/regionserver/Region.java | 6 + .../quotas/TestBlockBytesScannedQuota.java | 233 ++++++++++++++++++ .../hbase/quotas/ThrottleQuotaTestUtil.java | 63 +++++ 8 files changed, 366 insertions(+), 16 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestBlockBytesScannedQuota.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java index ddf804243ed8..4b89e18a8021 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java @@ -22,6 +22,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.ipc.RpcCall; +import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -49,9 +51,15 @@ public class DefaultOperationQuota implements OperationQuota { protected long readDiff = 0; protected long writeCapacityUnitDiff = 0; protected long readCapacityUnitDiff = 0; + private boolean useResultSizeBytes; + private long blockSizeBytes; - public DefaultOperationQuota(final Configuration conf, final QuotaLimiter... limiters) { + public DefaultOperationQuota(final Configuration conf, final int blockSizeBytes, + final QuotaLimiter... limiters) { this(conf, Arrays.asList(limiters)); + this.useResultSizeBytes = + conf.getBoolean(OperationQuota.USE_RESULT_SIZE_BYTES, USE_RESULT_SIZE_BYTES_DEFAULT); + this.blockSizeBytes = blockSizeBytes; } /** @@ -94,8 +102,17 @@ public void checkQuota(int numWrites, int numReads, int numScans) throws RpcThro public void close() { // Adjust the quota consumed for the specified operation writeDiff = operationSize[OperationType.MUTATE.ordinal()] - writeConsumed; - readDiff = operationSize[OperationType.GET.ordinal()] - + operationSize[OperationType.SCAN.ordinal()] - readConsumed; + + long resultSize = + operationSize[OperationType.GET.ordinal()] + operationSize[OperationType.SCAN.ordinal()]; + if (useResultSizeBytes) { + readDiff = resultSize - readConsumed; + } else { + long blockBytesScanned = + RpcServer.getCurrentCall().map(RpcCall::getBlockBytesScanned).orElse(0L); + readDiff = Math.max(blockBytesScanned, resultSize) - readConsumed; + } + writeCapacityUnitDiff = calculateWriteCapacityUnitDiff(operationSize[OperationType.MUTATE.ordinal()], writeConsumed); readCapacityUnitDiff = calculateReadCapacityUnitDiff( @@ -140,8 +157,15 @@ public void addMutation(final Mutation mutation) { */ protected void updateEstimateConsumeQuota(int numWrites, int numReads, int numScans) { writeConsumed = estimateConsume(OperationType.MUTATE, numWrites, 100); - readConsumed = estimateConsume(OperationType.GET, numReads, 100); - readConsumed += estimateConsume(OperationType.SCAN, numScans, 1000); + + if (useResultSizeBytes) { + readConsumed = estimateConsume(OperationType.GET, numReads, 100); + readConsumed += estimateConsume(OperationType.SCAN, numScans, 1000); + } else { + // assume 1 block required for reads. this is probably a low estimate, which is okay + readConsumed = numReads > 0 ? blockSizeBytes : 0; + readConsumed += numScans > 0 ? blockSizeBytes : 0; + } writeCapacityUnitConsumed = calculateWriteCapacityUnit(writeConsumed); readCapacityUnitConsumed = calculateReadCapacityUnit(readConsumed); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ExceedOperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ExceedOperationQuota.java index 1b7200f5f22f..1788e550f22a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ExceedOperationQuota.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/ExceedOperationQuota.java @@ -40,9 +40,9 @@ public class ExceedOperationQuota extends DefaultOperationQuota { private static final Logger LOG = LoggerFactory.getLogger(ExceedOperationQuota.class); private QuotaLimiter regionServerLimiter; - public ExceedOperationQuota(final Configuration conf, QuotaLimiter regionServerLimiter, - final QuotaLimiter... limiters) { - super(conf, limiters); + public ExceedOperationQuota(final Configuration conf, int blockSizeBytes, + QuotaLimiter regionServerLimiter, final QuotaLimiter... limiters) { + super(conf, blockSizeBytes, limiters); this.regionServerLimiter = regionServerLimiter; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java index aaae64b6184a..e18d3eb34953 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java @@ -35,6 +35,16 @@ public enum OperationType { SCAN } + /** + * If false, the default, then IO based throttles will consume read availability based on the + * block bytes scanned by the given request. If true then IO based throttles will use result size + * rather than block bytes scanned. Using block bytes scanned should be preferable to using result + * size, because otherwise access patterns like heavily filtered scans may be able to produce a + * significant and effectively un-throttled workload. + */ + String USE_RESULT_SIZE_BYTES = "hbase.quota.use.result.size.bytes"; + boolean USE_RESULT_SIZE_BYTES_DEFAULT = false; + /** * Checks if it is possible to execute the specified operation. The quota will be estimated based * on the number of operations to perform and the average size accumulated during time. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java index 4b09c0308f9e..de76303e27ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Optional; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.ipc.RpcScheduler; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.regionserver.Region; @@ -113,7 +114,8 @@ QuotaCache getQuotaCache() { * @param table the table where the operation will be executed * @return the OperationQuota */ - public OperationQuota getQuota(final UserGroupInformation ugi, final TableName table) { + public OperationQuota getQuota(final UserGroupInformation ugi, final TableName table, + final int blockSizeBytes) { if (isQuotaEnabled() && !table.isSystemTable() && isRpcThrottleEnabled()) { UserQuotaState userQuotaState = quotaCache.getUserQuotaState(ugi); QuotaLimiter userLimiter = userQuotaState.getTableLimiter(table); @@ -123,7 +125,8 @@ public OperationQuota getQuota(final UserGroupInformation ugi, final TableName t LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" + userLimiter); } if (!useNoop) { - return new DefaultOperationQuota(this.rsServices.getConfiguration(), userLimiter); + return new DefaultOperationQuota(this.rsServices.getConfiguration(), blockSizeBytes, + userLimiter); } } else { QuotaLimiter nsLimiter = quotaCache.getNamespaceLimiter(table.getNamespaceAsString()); @@ -139,11 +142,11 @@ public OperationQuota getQuota(final UserGroupInformation ugi, final TableName t } if (!useNoop) { if (exceedThrottleQuotaEnabled) { - return new ExceedOperationQuota(this.rsServices.getConfiguration(), rsLimiter, - userLimiter, tableLimiter, nsLimiter); + return new ExceedOperationQuota(this.rsServices.getConfiguration(), blockSizeBytes, + rsLimiter, userLimiter, tableLimiter, nsLimiter); } else { - return new DefaultOperationQuota(this.rsServices.getConfiguration(), userLimiter, - tableLimiter, nsLimiter, rsLimiter); + return new DefaultOperationQuota(this.rsServices.getConfiguration(), blockSizeBytes, + userLimiter, tableLimiter, nsLimiter, rsLimiter); } } } @@ -213,9 +216,10 @@ private OperationQuota checkQuota(final Region region, final int numWrites, fina } else { ugi = User.getCurrent().getUGI(); } - TableName table = region.getTableDescriptor().getTableName(); + TableDescriptor tableDescriptor = region.getTableDescriptor(); + TableName table = tableDescriptor.getTableName(); - OperationQuota quota = getQuota(ugi, table); + OperationQuota quota = getQuota(ugi, table, region.getMinBlockSizeBytes()); try { quota.checkQuota(numWrites, numReads, numScans); } catch (RpcThrottlingException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 0dc96747dd36..ae4045b1216b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -458,6 +458,8 @@ public MetricsTableRequests getMetricsTableRequests() { private final CellComparator cellComparator; + private final int minBlockSizeBytes; + /** * @return The smallest mvcc readPoint across all the scanners in this region. Writes older than * this readPoint, are included in every read operation. @@ -916,6 +918,9 @@ public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration co .remove(getRegionInfo().getEncodedName()); } } + + minBlockSizeBytes = Arrays.stream(this.htableDescriptor.getColumnFamilies()) + .mapToInt(ColumnFamilyDescriptor::getBlocksize).min().orElse(HConstants.DEFAULT_BLOCKSIZE); } private void setHTableSpecificConf() { @@ -2047,6 +2052,11 @@ public Configuration getReadOnlyConfiguration() { return new ReadOnlyConfiguration(this.conf); } + @Override + public int getMinBlockSizeBytes() { + return minBlockSizeBytes; + } + private ThreadPoolExecutor getStoreOpenAndCloseThreadPool(final String threadNamePrefix) { int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount()); int maxThreads = Math.min(numStores, conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java index 6a897a5b9f36..42069e58092e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java @@ -571,4 +571,10 @@ void requestCompaction(byte[] family, String why, int priority, boolean major, * if you try to set a configuration. */ Configuration getReadOnlyConfiguration(); + + /** + * The minimum block size configuration from all relevant column families. This is used when + * estimating quota consumption. + */ + int getMinBlockSizeBytes(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestBlockBytesScannedQuota.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestBlockBytesScannedQuota.java new file mode 100644 index 000000000000..e27ba123381c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestBlockBytesScannedQuota.java @@ -0,0 +1,233 @@ +/* + * 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.quotas; + +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.doGets; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.doMultiGets; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.doPuts; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.doScans; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.triggerUserCacheRefresh; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.waitMinuteQuota; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +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.security.User; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestBlockBytesScannedQuota { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBlockBytesScannedQuota.class); + + private final static Logger LOG = LoggerFactory.getLogger(TestBlockBytesScannedQuota.class); + + private static final int REFRESH_TIME = 5000; + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final byte[] FAMILY = Bytes.toBytes("cf"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + + private static final TableName TABLE_NAME = TableName.valueOf("BlockBytesScannedQuotaTest"); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // client should fail fast + TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 10); + TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + + // quotas enabled, using block bytes scanned + TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true); + TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, REFRESH_TIME); + + // don't cache blocks to make IO predictable + TEST_UTIL.getConfiguration().setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f); + + TEST_UTIL.startMiniCluster(1); + TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME); + TEST_UTIL.createTable(TABLE_NAME, FAMILY); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + QuotaCache.TEST_FORCE_REFRESH = true; + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + EnvironmentEdgeManager.reset(); + TEST_UTIL.deleteTable(TABLE_NAME); + TEST_UTIL.shutdownMiniCluster(); + } + + @After + public void tearDown() throws Exception { + ThrottleQuotaTestUtil.clearQuotaCache(TEST_UTIL); + } + + @Test + public void testBBSGet() throws Exception { + final Admin admin = TEST_UTIL.getAdmin(); + final String userName = User.getCurrent().getShortName(); + int blockSize = admin.getDescriptor(TABLE_NAME).getColumnFamily(FAMILY).getBlocksize(); + Table table = admin.getConnection().getTable(TABLE_NAME); + + doPuts(10_000, FAMILY, QUALIFIER, table); + TEST_UTIL.flush(TABLE_NAME); + + // Add ~10 block/min limit + admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.READ_SIZE, + Math.round(10.1 * blockSize), TimeUnit.MINUTES)); + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + + // should execute at max 10 requests + testTraffic(() -> doGets(20, FAMILY, QUALIFIER, table), 10, 1); + + // wait a minute and you should get another 10 requests executed + waitMinuteQuota(); + testTraffic(() -> doGets(20, FAMILY, QUALIFIER, table), 10, 1); + + // Remove all the limits + admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)); + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + testTraffic(() -> doGets(100, FAMILY, QUALIFIER, table), 100, 0); + testTraffic(() -> doGets(100, FAMILY, QUALIFIER, table), 100, 0); + } + + @Test + public void testBBSScan() throws Exception { + final Admin admin = TEST_UTIL.getAdmin(); + final String userName = User.getCurrent().getShortName(); + int blockSize = admin.getDescriptor(TABLE_NAME).getColumnFamily(FAMILY).getBlocksize(); + Table table = admin.getConnection().getTable(TABLE_NAME); + + doPuts(10_000, FAMILY, QUALIFIER, table); + TEST_UTIL.flush(TABLE_NAME); + + // Add 1 block/min limit. + // This should only allow 1 scan per minute, because we estimate 1 block per scan + admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_SIZE, blockSize, + TimeUnit.MINUTES)); + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + waitMinuteQuota(); + + // should execute 1 request + testTraffic(() -> doScans(5, table), 1, 0); + + // Remove all the limits + admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)); + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + testTraffic(() -> doScans(100, table), 100, 0); + testTraffic(() -> doScans(100, table), 100, 0); + + // Add ~3 block/min limit. This should support >1 scans + admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_SIZE, + Math.round(3.1 * blockSize), TimeUnit.MINUTES)); + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + + // should execute some requests, but not all + testTraffic(() -> doScans(100, table), 100, 90); + + // Remove all the limits + admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)); + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + testTraffic(() -> doScans(100, table), 100, 0); + testTraffic(() -> doScans(100, table), 100, 0); + } + + @Test + public void testBBSMultiGet() throws Exception { + final Admin admin = TEST_UTIL.getAdmin(); + final String userName = User.getCurrent().getShortName(); + int blockSize = admin.getDescriptor(TABLE_NAME).getColumnFamily(FAMILY).getBlocksize(); + Table table = admin.getConnection().getTable(TABLE_NAME); + int rowCount = 10_000; + + doPuts(rowCount, FAMILY, QUALIFIER, table); + TEST_UTIL.flush(TABLE_NAME); + + // Add 1 block/min limit. + // This should only allow 1 multiget per minute, because we estimate 1 block per multiget + admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_SIZE, blockSize, + TimeUnit.MINUTES)); + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + waitMinuteQuota(); + + // should execute 1 request + testTraffic(() -> doMultiGets(10, 10, rowCount, FAMILY, QUALIFIER, table), 1, 1); + + // Remove all the limits + admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)); + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + testTraffic(() -> doMultiGets(100, 10, rowCount, FAMILY, QUALIFIER, table), 100, 0); + testTraffic(() -> doMultiGets(100, 10, rowCount, FAMILY, QUALIFIER, table), 100, 0); + + // Add ~100 block/min limit + admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_SIZE, + Math.round(100.1 * blockSize), TimeUnit.MINUTES)); + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + + // should execute approximately 10 batches of 10 requests + testTraffic(() -> doMultiGets(20, 10, rowCount, FAMILY, QUALIFIER, table), 10, 1); + + // wait a minute and you should get another ~10 batches of 10 requests + waitMinuteQuota(); + testTraffic(() -> doMultiGets(20, 10, rowCount, FAMILY, QUALIFIER, table), 10, 1); + + // Remove all the limits + admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)); + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + testTraffic(() -> doMultiGets(100, 10, rowCount, FAMILY, QUALIFIER, table), 100, 0); + testTraffic(() -> doMultiGets(100, 10, rowCount, FAMILY, QUALIFIER, table), 100, 0); + } + + private void testTraffic(Callable trafficCallable, long expectedSuccess, long marginOfError) + throws Exception { + TEST_UTIL.waitFor(90_000, () -> { + long actualSuccess; + try { + actualSuccess = trafficCallable.call(); + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("Traffic test yielded {} successful requests. Expected {} +/- {}", actualSuccess, + expectedSuccess, marginOfError); + boolean success = (actualSuccess >= expectedSuccess - marginOfError) + && (actualSuccess <= expectedSuccess + marginOfError); + if (!success) { + triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + waitMinuteQuota(); + Thread.sleep(15_000L); + } + return success; + }); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/ThrottleQuotaTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/ThrottleQuotaTestUtil.java index de6f5653ad2c..bc2d0ae0713e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/ThrottleQuotaTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/ThrottleQuotaTestUtil.java @@ -18,12 +18,17 @@ package org.apache.hadoop.hbase.quotas; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Objects; +import java.util.Random; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -105,6 +110,64 @@ static long doGets(int maxOps, final Table... tables) { return count; } + static long doGets(int maxOps, byte[] family, byte[] qualifier, final Table... tables) { + int count = 0; + try { + while (count < maxOps) { + Get get = new Get(Bytes.toBytes("row-" + count)); + get.addColumn(family, qualifier); + for (final Table table : tables) { + table.get(get); + } + count += tables.length; + } + } catch (IOException e) { + LOG.error("get failed after nRetries=" + count, e); + } + return count; + } + + static long doMultiGets(int maxOps, int batchSize, int rowCount, byte[] family, byte[] qualifier, + final Table... tables) { + int opCount = 0; + Random random = new Random(); + try { + while (opCount < maxOps) { + List gets = new ArrayList<>(batchSize); + while (gets.size() < batchSize) { + Get get = new Get(Bytes.toBytes("row-" + random.nextInt(rowCount))); + get.addColumn(family, qualifier); + gets.add(get); + } + for (final Table table : tables) { + table.get(gets); + } + opCount += tables.length; + } + } catch (IOException e) { + LOG.error("multiget failed after nRetries=" + opCount, e); + } + return opCount; + } + + static long doScans(int maxOps, Table table) { + int count = 0; + int caching = 100; + try { + Scan scan = new Scan(); + scan.setCaching(caching); + scan.setCacheBlocks(false); + ResultScanner scanner = table.getScanner(scan); + while (count < (maxOps * caching)) { + scanner.next(); + count += 1; + } + } catch (IOException e) { + LOG.error("scan failed after nRetries=" + count, e); + } + return count / caching; + } + static void triggerUserCacheRefresh(HBaseTestingUtil testUtil, boolean bypass, TableName... tables) throws Exception { triggerCacheRefresh(testUtil, bypass, true, false, false, false, false, tables); From 36562937468bf4a3b77724ab6c2fdd1e34de16a3 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Wed, 7 Feb 2024 13:06:11 +0100 Subject: [PATCH 091/109] HBASE-28345 Close HBase connection on exit from HBase Shell (#5665) Signed-off-by: Wellington Chevreuil Signed-off-by: Balazs Meszaros --- hbase-shell/src/main/ruby/shell.rb | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index 414ab9d2bd51..39fbd2ccba16 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -108,6 +108,11 @@ class Shell # exit the interactive shell and save that this # happend via a call to exit def exit(ret = 0) + # Non-deamon Netty threadpool in ZK ClientCnxnSocketNetty cannot be shut down otherwise + begin + hbase.shutdown + rescue Exception + end @exit_code = ret IRB.irb_exit(IRB.CurrentContext.irb, ret) end From 275d928a7d48aebc2172d32294bac40cc9460d11 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Wed, 7 Feb 2024 09:23:17 -0500 Subject: [PATCH 092/109] HBASE-27800: Add support for default user quotas (#5666) Signed-off-by: Bryan Beaudreault --- .../hadoop/hbase/quotas/QuotaCache.java | 3 +- .../apache/hadoop/hbase/quotas/QuotaUtil.java | 61 +++++++- .../hadoop/hbase/quotas/TestDefaultQuota.java | 138 ++++++++++++++++++ 3 files changed, 200 insertions(+), 2 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestDefaultQuota.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java index 0a57b9fd8f8f..67b2aecc5448 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java @@ -137,7 +137,8 @@ public QuotaLimiter getUserLimiter(final UserGroupInformation ugi, final TableNa * @return the quota info associated to specified user */ public UserQuotaState getUserQuotaState(final UserGroupInformation ugi) { - return computeIfAbsent(userQuotaCache, getQuotaUserName(ugi), UserQuotaState::new, + return computeIfAbsent(userQuotaCache, getQuotaUserName(ugi), + () -> QuotaUtil.buildDefaultUserQuotaState(rsServices.getConfiguration()), this::triggerCacheRefresh); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java index f9fa1a95c0cc..2e51a8f75610 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -49,7 +50,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeUnit; +import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaScope; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle; @@ -73,6 +76,26 @@ public class QuotaUtil extends QuotaTableUtil { // the default one write capacity unit is 1024 bytes (1KB) public static final long DEFAULT_WRITE_CAPACITY_UNIT = 1024; + /* + * The below defaults, if configured, will be applied to otherwise unthrottled users. For example, + * set `hbase.quota.default.user.machine.read.size` to `1048576` in your hbase-site.xml to ensure + * that any given user may not query more than 1mb per second from any given machine, unless + * explicitly permitted by a persisted quota. All of these defaults use TimeUnit.SECONDS and + * QuotaScope.MACHINE. + */ + public static final String QUOTA_DEFAULT_USER_MACHINE_READ_NUM = + "hbase.quota.default.user.machine.read.num"; + public static final String QUOTA_DEFAULT_USER_MACHINE_READ_SIZE = + "hbase.quota.default.user.machine.read.size"; + public static final String QUOTA_DEFAULT_USER_MACHINE_REQUEST_NUM = + "hbase.quota.default.user.machine.request.num"; + public static final String QUOTA_DEFAULT_USER_MACHINE_REQUEST_SIZE = + "hbase.quota.default.user.machine.request.size"; + public static final String QUOTA_DEFAULT_USER_MACHINE_WRITE_NUM = + "hbase.quota.default.user.machine.write.num"; + public static final String QUOTA_DEFAULT_USER_MACHINE_WRITE_SIZE = + "hbase.quota.default.user.machine.write.size"; + /** Table descriptor for Quota internal table */ public static final TableDescriptor QUOTA_TABLE_DESC = TableDescriptorBuilder.newBuilder(QUOTA_TABLE_NAME) @@ -284,10 +307,14 @@ public static Map fetchUserQuotas(final Connection conne assert isUserRowKey(key); String user = getUserFromRowKey(key); + if (results[i].isEmpty()) { + userQuotas.put(user, buildDefaultUserQuotaState(connection.getConfiguration())); + continue; + } + final UserQuotaState quotaInfo = new UserQuotaState(nowTs); userQuotas.put(user, quotaInfo); - if (results[i].isEmpty()) continue; assert Bytes.equals(key, results[i].getRow()); try { @@ -321,6 +348,38 @@ public void visitUserQuotas(String userName, Quotas quotas) { return userQuotas; } + protected static UserQuotaState buildDefaultUserQuotaState(Configuration conf) { + QuotaProtos.Throttle.Builder throttleBuilder = QuotaProtos.Throttle.newBuilder(); + + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_READ_NUM) + .ifPresent(throttleBuilder::setReadNum); + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_READ_SIZE) + .ifPresent(throttleBuilder::setReadSize); + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_REQUEST_NUM) + .ifPresent(throttleBuilder::setReqNum); + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_REQUEST_SIZE) + .ifPresent(throttleBuilder::setReqSize); + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_WRITE_NUM) + .ifPresent(throttleBuilder::setWriteNum); + buildDefaultTimedQuota(conf, QUOTA_DEFAULT_USER_MACHINE_WRITE_SIZE) + .ifPresent(throttleBuilder::setWriteSize); + + UserQuotaState state = new UserQuotaState(); + QuotaProtos.Quotas defaultQuotas = + QuotaProtos.Quotas.newBuilder().setThrottle(throttleBuilder.build()).build(); + state.setQuotas(defaultQuotas); + return state; + } + + private static Optional buildDefaultTimedQuota(Configuration conf, String key) { + int defaultSoftLimit = conf.getInt(key, -1); + if (defaultSoftLimit == -1) { + return Optional.empty(); + } + return Optional.of(ProtobufUtil.toTimedQuota(defaultSoftLimit, + java.util.concurrent.TimeUnit.SECONDS, org.apache.hadoop.hbase.quotas.QuotaScope.MACHINE)); + } + public static Map fetchTableQuotas(final Connection connection, final List gets, Map tableMachineFactors) throws IOException { return fetchGlobalQuotas("table", connection, gets, new KeyFromRow() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestDefaultQuota.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestDefaultQuota.java new file mode 100644 index 000000000000..9a2200731f60 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestDefaultQuota.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.quotas; + +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.triggerUserCacheRefresh; +import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.waitMinuteQuota; + +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +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.security.User; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.After; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestDefaultQuota { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDefaultQuota.class); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final TableName TABLE_NAME = TableName.valueOf(UUID.randomUUID().toString()); + private static final int REFRESH_TIME = 5000; + private static final byte[] FAMILY = Bytes.toBytes("cf"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + + @After + public void tearDown() throws Exception { + ThrottleQuotaTestUtil.clearQuotaCache(TEST_UTIL); + EnvironmentEdgeManager.reset(); + TEST_UTIL.deleteTable(TABLE_NAME); + TEST_UTIL.shutdownMiniCluster(); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // quotas enabled, using block bytes scanned + TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true); + TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, REFRESH_TIME); + TEST_UTIL.getConfiguration().setInt(QuotaUtil.QUOTA_DEFAULT_USER_MACHINE_READ_NUM, 1); + + // don't cache blocks to make IO predictable + TEST_UTIL.getConfiguration().setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f); + + TEST_UTIL.startMiniCluster(1); + TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME); + TEST_UTIL.createTable(TABLE_NAME, FAMILY); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + QuotaCache.TEST_FORCE_REFRESH = true; + + try (Admin admin = TEST_UTIL.getAdmin()) { + ThrottleQuotaTestUtil.doPuts(1_000, FAMILY, QUALIFIER, + admin.getConnection().getTable(TABLE_NAME)); + } + TEST_UTIL.flush(TABLE_NAME); + } + + @Test + public void testDefaultUserReadNum() throws Exception { + // Should have a strict throttle by default + TEST_UTIL.waitFor(60_000, () -> runGetsTest(100) < 100); + + // Add big quota and should be effectively unlimited + configureLenientThrottle(); + refreshQuotas(); + // Should run without error + TEST_UTIL.waitFor(60_000, () -> runGetsTest(100) == 100); + + // Remove all the limits, and should revert to strict default + unsetQuota(); + TEST_UTIL.waitFor(60_000, () -> runGetsTest(100) < 100); + } + + private void configureLenientThrottle() throws IOException { + try (Admin admin = TEST_UTIL.getAdmin()) { + admin.setQuota(QuotaSettingsFactory.throttleUser(getUserName(), ThrottleType.READ_NUMBER, + 100_000, TimeUnit.SECONDS)); + } + } + + private static String getUserName() throws IOException { + return User.getCurrent().getShortName(); + } + + private void refreshQuotas() throws Exception { + triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + waitMinuteQuota(); + } + + private void unsetQuota() throws Exception { + try (Admin admin = TEST_UTIL.getAdmin()) { + admin.setQuota(QuotaSettingsFactory.unthrottleUser(getUserName())); + } + refreshQuotas(); + } + + private long runGetsTest(int attempts) throws Exception { + refreshQuotas(); + try (Table table = getTable()) { + return ThrottleQuotaTestUtil.doGets(attempts, FAMILY, QUALIFIER, table); + } + } + + private Table getTable() throws IOException { + TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 100); + TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + return TEST_UTIL.getConnection().getTableBuilder(TABLE_NAME, null).setOperationTimeout(250) + .build(); + } + +} From e85557a34d763c37baa36e94800689947e69b930 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andor=20Moln=C3=A1r?= Date: Thu, 8 Feb 2024 15:50:52 +0100 Subject: [PATCH 093/109] HBASE-28337 Positive connection test in TestShadeSaslAuthenticationProvider runs with Kerberos instead of Shade authentication (#5659) Signed-off-by: Wellington Chevreuil --- .../NettyHBaseSaslRpcClientHandler.java | 6 +++ .../TestShadeSaslAuthenticationProvider.java | 43 +++++-------------- 2 files changed, 17 insertions(+), 32 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java index 48e631c76299..cc71355d4297 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java @@ -145,6 +145,12 @@ public byte[] run() throws Exception { // Mechanisms which have multiple steps will not return true on `SaslClient#isComplete()` // until the handshake has fully completed. Mechanisms which only send a single buffer may // return true on `isComplete()` after that initial response is calculated. + + // HBASE-28337 We still want to check if the SaslClient completed the handshake, because + // there are certain mechs like PLAIN which doesn't have a server response after the + // initial authentication request. We cannot remove this tryComplete(), otherwise mechs + // like PLAIN will fail with call timeout. + tryComplete(ctx); } catch (Exception e) { // the exception thrown by handlerAdded will not be passed to the exceptionCaught below // because netty will remove a handler if handlerAdded throws an exception. diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java index a479310691b1..26a8943096a9 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java @@ -27,8 +27,6 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; -import java.io.PrintWriter; -import java.io.StringWriter; import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -69,10 +67,8 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.minikdc.MiniKdc; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -84,8 +80,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.base.Throwables; - @Category({ MediumTests.class, SecurityTests.class }) public class TestShadeSaslAuthenticationProvider { private static final Logger LOG = @@ -212,21 +206,23 @@ public String run() throws Exception { @Test public void testPositiveAuthentication() throws Exception { final Configuration clientConf = new Configuration(CONF); - try (Connection conn = ConnectionFactory.createConnection(clientConf)) { + try (Connection conn1 = ConnectionFactory.createConnection(clientConf)) { UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", new String[0]); - user1.addToken(ShadeClientTokenUtil.obtainToken(conn, "user1", USER1_PASSWORD)); + user1.addToken(ShadeClientTokenUtil.obtainToken(conn1, "user1", USER1_PASSWORD)); user1.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table t = conn.getTable(tableName)) { - Result r = t.get(new Get(Bytes.toBytes("r1"))); - assertNotNull(r); - assertFalse("Should have read a non-empty Result", r.isEmpty()); - final Cell cell = r.getColumnLatestCell(Bytes.toBytes("f1"), Bytes.toBytes("q1")); - assertTrue("Unexpected value", CellUtil.matchingValue(cell, Bytes.toBytes("1"))); + try (Connection conn = ConnectionFactory.createConnection(clientConf)) { + try (Table t = conn.getTable(tableName)) { + Result r = t.get(new Get(Bytes.toBytes("r1"))); + assertNotNull(r); + assertFalse("Should have read a non-empty Result", r.isEmpty()); + final Cell cell = r.getColumnLatestCell(Bytes.toBytes("f1"), Bytes.toBytes("q1")); + assertTrue("Unexpected value", CellUtil.matchingValue(cell, Bytes.toBytes("1"))); - return null; + return null; + } } } }); @@ -268,7 +264,6 @@ public Void run() throws Exception { } catch (Exception e) { LOG.info("Caught exception in negative Master connectivity test", e); assertEquals("Found unexpected exception", pair.getSecond(), e.getClass()); - validateRootCause(Throwables.getRootCause(e)); } return null; } @@ -287,7 +282,6 @@ public Void run() throws Exception { } catch (Exception e) { LOG.info("Caught exception in negative RegionServer connectivity test", e); assertEquals("Found unexpected exception", pair.getSecond(), e.getClass()); - validateRootCause(Throwables.getRootCause(e)); } return null; } @@ -301,19 +295,4 @@ public Void run() throws Exception { } }); } - - void validateRootCause(Throwable rootCause) { - LOG.info("Root cause was", rootCause); - if (rootCause instanceof RemoteException) { - RemoteException re = (RemoteException) rootCause; - IOException actualException = re.unwrapRemoteException(); - assertEquals(InvalidToken.class, actualException.getClass()); - } else { - StringWriter writer = new StringWriter(); - rootCause.printStackTrace(new PrintWriter(writer)); - String text = writer.toString(); - assertTrue("Message did not contain expected text", - text.contains(InvalidToken.class.getName())); - } - } } From 98eb3e01b352684de3c647a6fda6208a657c4607 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Thu, 8 Feb 2024 16:16:47 -0500 Subject: [PATCH 094/109] HBASE-28349 Count atomic operations against read quotas (#5668) Signed-off-by: Bryan Beaudreault --- .../hadoop/hbase/quotas/OperationQuota.java | 3 +- .../apache/hadoop/hbase/quotas/QuotaUtil.java | 26 ++ .../quotas/RegionServerRpcQuotaManager.java | 16 +- .../hbase/regionserver/RSRpcServices.java | 9 +- .../hbase/quotas/TestAtomicReadQuota.java | 237 ++++++++++++++++++ 5 files changed, 283 insertions(+), 8 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestAtomicReadQuota.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java index e18d3eb34953..ffc3cd50825c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/OperationQuota.java @@ -32,7 +32,8 @@ public interface OperationQuota { public enum OperationType { MUTATE, GET, - SCAN + SCAN, + CHECK_AND_MUTATE } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java index 2e51a8f75610..44357c88d2dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java @@ -51,6 +51,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeUnit; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaScope; @@ -177,6 +178,31 @@ public static void deleteRegionServerQuota(final Connection connection, final St deleteQuotas(connection, getRegionServerRowKey(regionServer)); } + public static OperationQuota.OperationType getQuotaOperationType(ClientProtos.Action action, + boolean hasCondition) { + if (action.hasMutation()) { + return getQuotaOperationType(action.getMutation(), hasCondition); + } + return OperationQuota.OperationType.GET; + } + + public static OperationQuota.OperationType + getQuotaOperationType(ClientProtos.MutateRequest mutateRequest) { + return getQuotaOperationType(mutateRequest.getMutation(), mutateRequest.hasCondition()); + } + + private static OperationQuota.OperationType + getQuotaOperationType(ClientProtos.MutationProto mutationProto, boolean hasCondition) { + ClientProtos.MutationProto.MutationType mutationType = mutationProto.getMutateType(); + if ( + hasCondition || mutationType == ClientProtos.MutationProto.MutationType.APPEND + || mutationType == ClientProtos.MutationProto.MutationType.INCREMENT + ) { + return OperationQuota.OperationType.CHECK_AND_MUTATE; + } + return OperationQuota.OperationType.MUTATE; + } + protected static void switchExceedThrottleQuota(final Connection connection, boolean exceedThrottleQuotaEnabled) throws IOException { if (exceedThrottleQuotaEnabled) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java index de76303e27ac..3c72c662887b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java @@ -171,6 +171,8 @@ public OperationQuota checkQuota(final Region region, final OperationQuota.Opera return checkQuota(region, 0, 1, 0); case MUTATE: return checkQuota(region, 1, 0, 0); + case CHECK_AND_MUTATE: + return checkQuota(region, 1, 1, 0); } throw new RuntimeException("Invalid operation type: " + type); } @@ -178,18 +180,24 @@ public OperationQuota checkQuota(final Region region, final OperationQuota.Opera /** * Check the quota for the current (rpc-context) user. Returns the OperationQuota used to get the * available quota and to report the data/usage of the operation. - * @param region the region where the operation will be performed - * @param actions the "multi" actions to perform + * @param region the region where the operation will be performed + * @param actions the "multi" actions to perform + * @param hasCondition whether the RegionAction has a condition * @return the OperationQuota * @throws RpcThrottlingException if the operation cannot be executed due to quota exceeded. */ - public OperationQuota checkQuota(final Region region, final List actions) - throws IOException, RpcThrottlingException { + public OperationQuota checkQuota(final Region region, final List actions, + boolean hasCondition) throws IOException, RpcThrottlingException { int numWrites = 0; int numReads = 0; for (final ClientProtos.Action action : actions) { if (action.hasMutation()) { numWrites++; + OperationQuota.OperationType operationType = + QuotaUtil.getQuotaOperationType(action, hasCondition); + if (operationType == OperationQuota.OperationType.CHECK_AND_MUTATE) { + numReads++; + } } else if (action.hasGet()) { numReads++; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 05d7c2e56055..0538b9706e89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -2679,7 +2679,8 @@ public MultiResponse multi(final RpcController rpcc, final MultiRequest request) try { region = getRegion(regionSpecifier); - quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList()); + quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList(), + regionAction.hasCondition()); } catch (IOException e) { failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e); return responseBuilder.build(); @@ -2741,7 +2742,8 @@ public MultiResponse multi(final RpcController rpcc, final MultiRequest request) try { region = getRegion(regionSpecifier); - quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList()); + quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList(), + regionAction.hasCondition()); } catch (IOException e) { failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e); continue; // For this region it's a failure. @@ -2924,7 +2926,8 @@ public MutateResponse mutate(final RpcController rpcc, final MutateRequest reque server.getMemStoreFlusher().reclaimMemStoreMemory(); } long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE; - quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE); + OperationQuota.OperationType operationType = QuotaUtil.getQuotaOperationType(request); + quota = getRpcQuotaManager().checkQuota(region, operationType); ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestAtomicReadQuota.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestAtomicReadQuota.java new file mode 100644 index 000000000000..9b654ac8e6d0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestAtomicReadQuota.java @@ -0,0 +1,237 @@ +/* + * 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.quotas; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +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.CheckAndMutate; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestAtomicReadQuota { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAtomicReadQuota.class); + private static final Logger LOG = LoggerFactory.getLogger(TestAtomicReadQuota.class); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final TableName TABLE_NAME = TableName.valueOf(UUID.randomUUID().toString()); + private static final byte[] FAMILY = Bytes.toBytes("cf"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + + @AfterClass + public static void tearDown() throws Exception { + ThrottleQuotaTestUtil.clearQuotaCache(TEST_UTIL); + EnvironmentEdgeManager.reset(); + TEST_UTIL.deleteTable(TABLE_NAME); + TEST_UTIL.shutdownMiniCluster(); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true); + TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 1000); + TEST_UTIL.startMiniCluster(1); + TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME); + TEST_UTIL.createTable(TABLE_NAME, FAMILY); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + QuotaCache.TEST_FORCE_REFRESH = true; + } + + @Test + public void testIncrementCountedAgainstReadCapacity() throws Exception { + setupQuota(); + + Increment inc = new Increment(Bytes.toBytes(UUID.randomUUID().toString())); + inc.addColumn(FAMILY, QUALIFIER, 1); + testThrottle(table -> table.increment(inc)); + } + + @Test + public void testConditionalRowMutationsCountedAgainstReadCapacity() throws Exception { + setupQuota(); + + byte[] row = Bytes.toBytes(UUID.randomUUID().toString()); + Increment inc = new Increment(row); + inc.addColumn(FAMILY, Bytes.toBytes("doot"), 1); + Put put = new Put(row); + put.addColumn(FAMILY, Bytes.toBytes("doot"), Bytes.toBytes("v")); + + RowMutations rowMutations = new RowMutations(row); + rowMutations.add(inc); + rowMutations.add(put); + testThrottle(table -> table.mutateRow(rowMutations)); + } + + @Test + public void testNonConditionalRowMutationsOmittedFromReadCapacity() throws Exception { + setupQuota(); + + byte[] row = Bytes.toBytes(UUID.randomUUID().toString()); + Put put = new Put(row); + put.addColumn(FAMILY, Bytes.toBytes("doot"), Bytes.toBytes("v")); + + RowMutations rowMutations = new RowMutations(row); + rowMutations.add(put); + try (Table table = getTable()) { + for (int i = 0; i < 100; i++) { + table.mutateRow(rowMutations); + } + } + } + + @Test + public void testNonAtomicPutOmittedFromReadCapacity() throws Exception { + setupQuota(); + + byte[] row = Bytes.toBytes(UUID.randomUUID().toString()); + Put put = new Put(row); + put.addColumn(FAMILY, Bytes.toBytes("doot"), Bytes.toBytes("v")); + try (Table table = getTable()) { + for (int i = 0; i < 100; i++) { + table.put(put); + } + } + } + + @Test + public void testNonAtomicMultiPutOmittedFromReadCapacity() throws Exception { + setupQuota(); + + Put put1 = new Put(Bytes.toBytes(UUID.randomUUID().toString())); + put1.addColumn(FAMILY, Bytes.toBytes("doot"), Bytes.toBytes("v")); + Put put2 = new Put(Bytes.toBytes(UUID.randomUUID().toString())); + put2.addColumn(FAMILY, Bytes.toBytes("doot"), Bytes.toBytes("v")); + + Increment inc = new Increment(Bytes.toBytes(UUID.randomUUID().toString())); + inc.addColumn(FAMILY, Bytes.toBytes("doot"), 1); + + List puts = new ArrayList<>(2); + puts.add(put1); + puts.add(put2); + + try (Table table = getTable()) { + for (int i = 0; i < 100; i++) { + table.put(puts); + } + } + } + + @Test + public void testCheckAndMutateCountedAgainstReadCapacity() throws Exception { + setupQuota(); + + byte[] row = Bytes.toBytes(UUID.randomUUID().toString()); + byte[] value = Bytes.toBytes("v"); + Put put = new Put(row); + put.addColumn(FAMILY, Bytes.toBytes("doot"), value); + CheckAndMutate checkAndMutate = + CheckAndMutate.newBuilder(row).ifEquals(FAMILY, QUALIFIER, value).build(put); + + testThrottle(table -> table.checkAndMutate(checkAndMutate)); + } + + @Test + public void testAtomicBatchCountedAgainstReadCapacity() throws Exception { + setupQuota(); + + byte[] row = Bytes.toBytes(UUID.randomUUID().toString()); + Increment inc = new Increment(row); + inc.addColumn(FAMILY, Bytes.toBytes("doot"), 1); + + List incs = new ArrayList<>(2); + incs.add(inc); + incs.add(inc); + + testThrottle(table -> { + Object[] results = new Object[] {}; + table.batch(incs, results); + return results; + }); + } + + private void setupQuota() throws Exception { + try (Admin admin = TEST_UTIL.getAdmin()) { + admin.setQuota(QuotaSettingsFactory.throttleUser(User.getCurrent().getShortName(), + ThrottleType.READ_NUMBER, 1, TimeUnit.MINUTES)); + } + ThrottleQuotaTestUtil.triggerUserCacheRefresh(TEST_UTIL, false, TABLE_NAME); + } + + private void cleanupQuota() throws Exception { + try (Admin admin = TEST_UTIL.getAdmin()) { + admin.setQuota(QuotaSettingsFactory.unthrottleUser(User.getCurrent().getShortName())); + } + ThrottleQuotaTestUtil.triggerUserCacheRefresh(TEST_UTIL, true, TABLE_NAME); + } + + private void testThrottle(ThrowingFunction request) throws Exception { + try (Table table = getTable()) { + // we have a read quota configured, so this should fail + TEST_UTIL.waitFor(60_000, () -> { + try { + request.run(table); + return false; + } catch (Exception e) { + boolean success = e.getCause() instanceof RpcThrottlingException; + if (!success) { + LOG.error("Unexpected exception", e); + } + return success; + } + }); + } finally { + cleanupQuota(); + } + } + + private Table getTable() throws IOException { + TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 100); + TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + return TEST_UTIL.getConnection().getTableBuilder(TABLE_NAME, null).setOperationTimeout(250) + .build(); + } + + @FunctionalInterface + private interface ThrowingFunction { + O run(I input) throws Exception; + } + +} From b3ffc4acb64815eb5dfbbb02889ef830e119e37a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 10 Feb 2024 11:04:06 +0800 Subject: [PATCH 095/109] HBASE-28355 Bump cryptography in /dev-support/git-jira-release-audit (#5663) Bumps [cryptography](https://github.com/pyca/cryptography) from 41.0.6 to 42.0.0. - [Changelog](https://github.com/pyca/cryptography/blob/main/CHANGELOG.rst) - [Commits](https://github.com/pyca/cryptography/compare/41.0.6...42.0.0) --- updated-dependencies: - dependency-name: cryptography dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: Duo Zhang --- dev-support/git-jira-release-audit/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/git-jira-release-audit/requirements.txt b/dev-support/git-jira-release-audit/requirements.txt index 99fb2e2d0de9..e9fc0361d8d8 100644 --- a/dev-support/git-jira-release-audit/requirements.txt +++ b/dev-support/git-jira-release-audit/requirements.txt @@ -19,7 +19,7 @@ blessed==1.17.0 certifi==2023.7.22 cffi==1.13.2 chardet==3.0.4 -cryptography==41.0.6 +cryptography==42.0.0 defusedxml==0.6.0 enlighten==1.4.0 gitdb2==2.0.6 From 3ccf65579b682cd288eb37f8de634d9920392fdc Mon Sep 17 00:00:00 2001 From: Rajeshbabu Chintaguntla Date: Mon, 12 Feb 2024 09:19:24 +0530 Subject: [PATCH 096/109] HBASE-28341 [JDK17] Fix Failure TestLdapHttpServer (#5672) Co-authored-by: Rajeshbabu Chintaguntla --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a32e73fb3377..b8ac1067cf4a 100644 --- a/pom.xml +++ b/pom.xml @@ -990,7 +990,9 @@ --add-exports java.base/sun.net.util=ALL-UNNAMED - --add-opens java.base/jdk.internal.util.random=ALL-UNNAMED + --add-opens java.base/jdk.internal.util.random=ALL-UNNAMED + --add-opens java.base/sun.security.x509=ALL-UNNAMED + --add-opens java.base/sun.security.util=ALL-UNNAMED ${hbase-surefire.argLine} @{jacocoArgLine} 1.5.1 From 3e281bd207213e6618d599a4a55db1b79dc36787 Mon Sep 17 00:00:00 2001 From: Monani Mihir Date: Mon, 12 Feb 2024 09:14:06 -0800 Subject: [PATCH 097/109] HBASE-28204 : Region Canary can take lot more time If any region (except the first region) starts with delete markers (#5675) Signed-off-by: David Manning Signed-off-by: Viraj Jasani --- .../apache/hadoop/hbase/tool/CanaryTool.java | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java index d5676263c820..a3caf1b24c74 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java @@ -510,19 +510,44 @@ public Void call() { private Void readColumnFamily(Table table, ColumnFamilyDescriptor column) { byte[] startKey = null; - Get get = null; Scan scan = null; ResultScanner rs = null; StopWatch stopWatch = new StopWatch(); startKey = region.getStartKey(); // Can't do a get on empty start row so do a Scan of first element if any instead. if (startKey.length > 0) { - get = new Get(startKey); + Get get = new Get(startKey); get.setCacheBlocks(false); get.setFilter(new FirstKeyOnlyFilter()); get.addFamily(column.getName()); + // Converting get object to scan to enable RAW SCAN. + // This will work for all the regions of the HBase tables except first region of the table. + scan = new Scan(get); + scan.setRaw(rawScanEnabled); } else { scan = new Scan(); + // In case of first region of the HBase Table, we do not have start-key for the region. + // For Region Canary, we only need to scan a single row/cell in the region to make sure that + // region is accessible. + // + // When HBase table has more than 1 empty regions at start of the row-key space, Canary will + // create multiple scan object to find first available row in the table by scanning all the + // regions in sequence until it can find first available row. + // + // This could result in multiple millions of scans based on the size of table and number of + // empty regions in sequence. In test environment, A table with no data and 1100 empty + // regions, Single canary run was creating close to half million to 1 million scans to + // successfully do canary run for the table. + // + // Since First region of the table doesn't have any start key, We should set End Key as + // stop row and set inclusive=false to limit scan to single region only. + // + // TODO : In future, we can streamline Canary behaviour for all the regions by doing scan + // with startRow inclusive and stopRow exclusive instead of different behaviour for First + // Region of the table and rest of the region of the table. This way implementation is + // simplified. As of now this change has been kept minimal to avoid any unnecessary + // perf impact. + scan.withStopRow(region.getEndKey(), false); LOG.debug("rawScan {} for {}", rawScanEnabled, region.getTable()); scan.setRaw(rawScanEnabled); scan.setCaching(1); @@ -536,12 +561,8 @@ private Void readColumnFamily(Table table, ColumnFamilyDescriptor column) { column.getNameAsString(), Bytes.toStringBinary(startKey)); try { stopWatch.start(); - if (startKey.length > 0) { - table.get(get); - } else { - rs = table.getScanner(scan); - rs.next(); - } + rs = table.getScanner(scan); + rs.next(); stopWatch.stop(); this.readWriteLatency.add(stopWatch.getTime()); sink.publishReadTiming(serverName, region, column, stopWatch.getTime()); From e2ff15898410ee7e60e2dcd5eecedb5ed2fb525a Mon Sep 17 00:00:00 2001 From: Monani Mihir Date: Mon, 12 Feb 2024 17:16:32 -0800 Subject: [PATCH 098/109] HBASE-28357 MoveWithAck#isSuccessfulScan for Region movement should use Region End Key for limiting scan to one region only. (#5677) Signed-off-by: Viraj Jasani --- .../main/java/org/apache/hadoop/hbase/util/MoveWithAck.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MoveWithAck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MoveWithAck.java index ede1f8b71508..7143598679be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MoveWithAck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MoveWithAck.java @@ -110,8 +110,9 @@ private static String getTimeDiffInSec(long startTime) { * Tries to scan a row from passed region */ private void isSuccessfulScan(RegionInfo region) throws IOException { - Scan scan = new Scan().withStartRow(region.getStartKey()).setRaw(true).setOneRowLimit() - .setMaxResultSize(1L).setCaching(1).setFilter(new FirstKeyOnlyFilter()).setCacheBlocks(false); + Scan scan = new Scan().withStartRow(region.getStartKey()).withStopRow(region.getEndKey(), false) + .setRaw(true).setOneRowLimit().setMaxResultSize(1L).setCaching(1) + .setFilter(new FirstKeyOnlyFilter()).setCacheBlocks(false); try (Table table = conn.getTable(region.getTable()); ResultScanner scanner = table.getScanner(scan)) { scanner.next(); From a4002d69a118296d5ac19adf46b1301be6ab6cfe Mon Sep 17 00:00:00 2001 From: Monani Mihir Date: Mon, 12 Feb 2024 17:23:42 -0800 Subject: [PATCH 099/109] HBASE-28356 RegionServer Canary should use Scan just like Region Canary with option to enable Raw Scan (#5676) Signed-off-by: David Manning Signed-off-by: Viraj Jasani --- .../apache/hadoop/hbase/tool/CanaryTool.java | 42 +++++++++++++------ 1 file changed, 30 insertions(+), 12 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java index a3caf1b24c74..92dca7c24c92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java @@ -670,14 +670,16 @@ static class RegionServerTask implements Callable { private String serverName; private RegionInfo region; private RegionServerStdOutSink sink; + private Boolean rawScanEnabled; private AtomicLong successes; RegionServerTask(Connection connection, String serverName, RegionInfo region, - RegionServerStdOutSink sink, AtomicLong successes) { + RegionServerStdOutSink sink, Boolean rawScanEnabled, AtomicLong successes) { this.connection = connection; this.serverName = serverName; this.region = region; this.sink = sink; + this.rawScanEnabled = rawScanEnabled; this.successes = successes; } @@ -702,22 +704,35 @@ public Void call() { get = new Get(startKey); get.setCacheBlocks(false); get.setFilter(new FirstKeyOnlyFilter()); - stopWatch.start(); - table.get(get); - stopWatch.stop(); + // Converting get object to scan to enable RAW SCAN. + // This will work for all the regions of the HBase tables except first region. + scan = new Scan(get); + } else { scan = new Scan(); + // In case of first region of the HBase Table, we do not have start-key for the region. + // For Region Canary, we only need scan a single row/cell in the region to make sure that + // region is accessible. + // + // When HBase table has more than 1 empty regions at start of the row-key space, Canary + // will create multiple scan object to find first available row in the table by scanning + // all the regions in sequence until it can find first available row. + // + // Since First region of the table doesn't have any start key, We should set End Key as + // stop row and set inclusive=false to limit scan to first region only. + scan.withStopRow(region.getEndKey(), false); scan.setCacheBlocks(false); scan.setFilter(new FirstKeyOnlyFilter()); scan.setCaching(1); scan.setMaxResultSize(1L); scan.setOneRowLimit(); - stopWatch.start(); - ResultScanner s = table.getScanner(scan); - s.next(); - s.close(); - stopWatch.stop(); } + scan.setRaw(rawScanEnabled); + stopWatch.start(); + ResultScanner s = table.getScanner(scan); + s.next(); + s.close(); + stopWatch.stop(); successes.incrementAndGet(); sink.publishReadTiming(tableName.getNameAsString(), serverName, stopWatch.getTime()); } catch (TableNotFoundException tnfe) { @@ -1778,6 +1793,7 @@ private ZookeeperStdOutSink getSink() { * A monitor for regionserver mode */ private static class RegionServerMonitor extends Monitor { + private boolean rawScanEnabled; private boolean allRegions; public RegionServerMonitor(Connection connection, String[] monitorTargets, boolean useRegExp, @@ -1785,6 +1801,8 @@ public RegionServerMonitor(Connection connection, String[] monitorTargets, boole long allowedFailures) { super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError, allowedFailures); + Configuration conf = connection.getConfiguration(); + this.rawScanEnabled = conf.getBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, false); this.allRegions = allRegions; } @@ -1857,14 +1875,14 @@ private void monitorRegionServers(Map> rsAndRMap, } else if (this.allRegions) { for (RegionInfo region : entry.getValue()) { tasks.add(new RegionServerTask(this.connection, serverName, region, regionServerSink, - successes)); + this.rawScanEnabled, successes)); } } else { // random select a region if flag not set RegionInfo region = entry.getValue().get(ThreadLocalRandom.current().nextInt(entry.getValue().size())); - tasks.add( - new RegionServerTask(this.connection, serverName, region, regionServerSink, successes)); + tasks.add(new RegionServerTask(this.connection, serverName, region, regionServerSink, + this.rawScanEnabled, successes)); } } try { From 9656006778789850892403107be0a85074959765 Mon Sep 17 00:00:00 2001 From: Istvan Toth Date: Tue, 13 Feb 2024 11:21:01 +0100 Subject: [PATCH 100/109] HBASE-28353 Close HBase connection on implicit exit from HBase shell (#5673) Signed-off-by: Bryan Beaudreault Signed-off-by: Balazs Meszaros --- hbase-shell/src/main/ruby/shell.rb | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index 39fbd2ccba16..46b38dd96b89 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -108,11 +108,6 @@ class Shell # exit the interactive shell and save that this # happend via a call to exit def exit(ret = 0) - # Non-deamon Netty threadpool in ZK ClientCnxnSocketNetty cannot be shut down otherwise - begin - hbase.shutdown - rescue Exception - end @exit_code = ret IRB.irb_exit(IRB.CurrentContext.irb, ret) end @@ -323,6 +318,13 @@ def get_workspace hbase_receiver.send :define_singleton_method, :exit, lambda { |rc = 0| @shell.exit(rc) } + at_exit do + # Non-deamon Netty threadpool in ZK ClientCnxnSocketNetty cannot be shut down otherwise + begin + hbase.shutdown + rescue Exception + end + end ::IRB::WorkSpace.new(hbase_receiver.get_binding) end From d9257543a5eb3e93de225fb68c5a13afa78b6a76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andor=20Moln=C3=A1r?= Date: Thu, 15 Feb 2024 14:25:22 +0100 Subject: [PATCH 101/109] HBASE-28340. Use all Zk client properties that is found in HBase conf (#5669) Signed-off-by: Balazs Meszaros --- .../hadoop/hbase/zookeeper/ZKConfig.java | 54 ++++++++----------- .../hadoop/hbase/zookeeper/TestZKConfig.java | 25 ++++++++- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java index 12d81fee6586..87885e2b9fd5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map.Entry; import java.util.Properties; -import java.util.Set; import org.apache.commons.validator.routines.InetAddressValidator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; @@ -29,7 +28,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; /** * Utility methods for reading, and building the ZooKeeper configuration. The order and priority for @@ -42,12 +40,6 @@ public final class ZKConfig { private static final String VARIABLE_START = "${"; private static final String ZOOKEEPER_JAVA_PROPERTY_PREFIX = "zookeeper."; - /** Supported ZooKeeper client TLS properties */ - static final Set ZOOKEEPER_CLIENT_TLS_PROPERTIES = - ImmutableSet.of("client.secure", "clientCnxnSocket", "ssl.keyStore.location", - "ssl.keyStore.password", "ssl.keyStore.passwordPath", "ssl.trustStore.location", - "ssl.trustStore.password", "ssl.trustStore.passwordPath"); - private ZKConfig() { } @@ -62,16 +54,12 @@ public static Properties makeZKProps(Configuration conf) { } /** - * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from - * the HBase XML configs and generates the appropriate ZooKeeper properties. - * @param conf Configuration to read from. - * @return Properties holding mappings representing ZooKeeper config file. + * Directly map all the hbase.zookeeper.property.KEY properties. Synchronize on conf so no loading + * of configs while we iterate */ - private static Properties makeZKPropsFromHbaseConfig(Configuration conf) { + private static Properties extractZKPropsFromHBaseConfig(final Configuration conf) { Properties zkProperties = new Properties(); - // Directly map all of the hbase.zookeeper.property.KEY properties. - // Synchronize on conf so no loading of configs while we iterate synchronized (conf) { for (Entry entry : conf) { String key = entry.getKey(); @@ -87,6 +75,18 @@ private static Properties makeZKPropsFromHbaseConfig(Configuration conf) { } } + return zkProperties; + } + + /** + * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from + * the HBase XML configs and generates the appropriate ZooKeeper properties. + * @param conf Configuration to read from. + * @return Properties holding mappings representing ZooKeeper config file. + */ + private static Properties makeZKPropsFromHbaseConfig(Configuration conf) { + Properties zkProperties = extractZKPropsFromHBaseConfig(conf); + // If clientPort is not set, assign the default. if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) { zkProperties.put(HConstants.CLIENT_PORT_STR, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT); @@ -343,24 +343,12 @@ public static String getClientZKQuorumServersString(Configuration conf) { } private static void setZooKeeperClientSystemProperties(String prefix, Configuration conf) { - synchronized (conf) { - for (Entry entry : conf) { - String key = entry.getKey(); - if (!key.startsWith(prefix)) { - continue; - } - String zkKey = key.substring(prefix.length()); - if (!ZOOKEEPER_CLIENT_TLS_PROPERTIES.contains(zkKey)) { - continue; - } - String value = entry.getValue(); - // If the value has variables substitutions, need to do a get. - if (value.contains(VARIABLE_START)) { - value = conf.get(key); - } - if (System.getProperty(ZOOKEEPER_JAVA_PROPERTY_PREFIX + zkKey) == null) { - System.setProperty(ZOOKEEPER_JAVA_PROPERTY_PREFIX + zkKey, value); - } + Properties zkProperties = extractZKPropsFromHBaseConfig(conf); + for (Entry entry : zkProperties.entrySet()) { + String key = entry.getKey().toString().trim(); + String value = entry.getValue().toString().trim(); + if (System.getProperty(ZOOKEEPER_JAVA_PROPERTY_PREFIX + key) == null) { + System.setProperty(ZOOKEEPER_JAVA_PROPERTY_PREFIX + key, value); } } } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java index 7418afe5d222..63df9043bae3 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java @@ -17,12 +17,12 @@ */ package org.apache.hadoop.hbase.zookeeper; -import static org.apache.hadoop.hbase.zookeeper.ZKConfig.ZOOKEEPER_CLIENT_TLS_PROPERTIES; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Properties; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -33,6 +33,8 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; + @Category({ MiscTests.class, SmallTests.class }) public class TestZKConfig { @@ -40,6 +42,12 @@ public class TestZKConfig { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestZKConfig.class); + /** Supported ZooKeeper client TLS properties */ + private static final Set ZOOKEEPER_CLIENT_TLS_PROPERTIES = ImmutableSet.of( + "client.secure", "clientCnxnSocket", "ssl.keyStore.location", "ssl.keyStore.password", + "ssl.keyStore.passwordPath", "ssl.keyStore.type", "ssl.trustStore.location", + "ssl.trustStore.password", "ssl.trustStore.passwordPath", "ssl.trustStore.type"); + @Test public void testZKConfigLoading() throws Exception { Configuration conf = HBaseConfiguration.create(); @@ -133,6 +141,21 @@ public void testZooKeeperTlsPropertiesServer() { } } + @Test + public void testZooKeeperPropertiesDoesntOverwriteSystem() { + // Arrange + System.setProperty("zookeeper.a.b.c", "foo"); + Configuration conf = HBaseConfiguration.create(); + conf.set(HConstants.ZK_CFG_PROPERTY_PREFIX + "a.b.c", "bar"); + + // Act + ZKConfig.getZKQuorumServersString(conf); + + // Assert + assertEquals("foo", System.getProperty("zookeeper.a.b.c")); + System.clearProperty("zookeeper.a.b.c"); + } + private void testKey(String ensemble, int port, String znode) throws IOException { testKey(ensemble, port, znode, false); // not support multiple client ports } From 96a447f0e64ae45c33e1ed23e7ba39cd3b0796a3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 18 Feb 2024 18:31:04 +0800 Subject: [PATCH 102/109] HBASE-28378 Bump cryptography in /dev-support/git-jira-release-audit (#5687) Bumps [cryptography](https://github.com/pyca/cryptography) from 42.0.0 to 42.0.2. - [Changelog](https://github.com/pyca/cryptography/blob/main/CHANGELOG.rst) - [Commits](https://github.com/pyca/cryptography/compare/42.0.0...42.0.2) --- updated-dependencies: - dependency-name: cryptography dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: Duo Zhang --- dev-support/git-jira-release-audit/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/git-jira-release-audit/requirements.txt b/dev-support/git-jira-release-audit/requirements.txt index e9fc0361d8d8..5e402a1875f4 100644 --- a/dev-support/git-jira-release-audit/requirements.txt +++ b/dev-support/git-jira-release-audit/requirements.txt @@ -19,7 +19,7 @@ blessed==1.17.0 certifi==2023.7.22 cffi==1.13.2 chardet==3.0.4 -cryptography==42.0.0 +cryptography==42.0.2 defusedxml==0.6.0 enlighten==1.4.0 gitdb2==2.0.6 From b5175bad9deaf9caff8686e8957f8dd475e61f58 Mon Sep 17 00:00:00 2001 From: Anchal Kejriwal <55595137+anchal246@users.noreply.github.com> Date: Mon, 19 Feb 2024 01:24:18 +0530 Subject: [PATCH 103/109] HBASE-28142 Region Server Logs getting spammed with warning when storefile has no reader Signed-off-by: Nihal Jain Signed-off-by: Rajeshbabu Chintaguntla --- .../java/org/apache/hadoop/hbase/regionserver/HStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 5e2bf00f85be..dccfd0c0af7b 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 @@ -1616,7 +1616,7 @@ protected void refreshStoreSizeAndTotalBytes() throws IOException { for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) { StoreFileReader r = hsf.getReader(); if (r == null) { - LOG.warn("StoreFile {} has a null Reader", hsf); + LOG.debug("StoreFile {} has a null Reader", hsf); continue; } this.storeSize.addAndGet(r.length()); @@ -1785,7 +1785,7 @@ public int getCompactedFilesCount() { private LongStream getStoreFileAgeStream() { return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> { if (sf.getReader() == null) { - LOG.warn("StoreFile {} has a null Reader", sf); + LOG.debug("StoreFile {} has a null Reader", sf); return false; } else { return true; From 5398b13bab9769113a7b52038c31c7a6a3f77402 Mon Sep 17 00:00:00 2001 From: Kerasone <42959153+Kerasone@users.noreply.github.com> Date: Mon, 19 Feb 2024 12:21:07 +0800 Subject: [PATCH 104/109] HBASE-28238 rpcservice should perform some important admin operation to priority ADMIN_QOS (#5558) HBASE-28238 rpcservice should perform some important admin operation to priority ADMIN_QOS (#5558) Co-authored-by: selina.yan --- .../java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java | 1 + .../org/apache/hadoop/hbase/regionserver/RSRpcServices.java | 2 ++ .../apache/hadoop/hbase/regionserver/TestRSQosFunction.java | 3 +++ 3 files changed, 6 insertions(+) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java index 291b38acb322..b2a0e7803624 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java @@ -310,6 +310,7 @@ public final GetBootstrapNodesResponse getBootstrapNodes(RpcController controlle } @Override + @QosPriority(priority = HConstants.ADMIN_QOS) public UpdateConfigurationResponse updateConfiguration(RpcController controller, UpdateConfigurationRequest request) throws ServiceException { try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 0538b9706e89..4926aa30c8a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -1554,6 +1554,7 @@ public CompactRegionResponse compactRegion(final RpcController controller, } @Override + @QosPriority(priority = HConstants.ADMIN_QOS) public CompactionSwitchResponse compactionSwitch(RpcController controller, CompactionSwitchRequest request) throws ServiceException { rpcPreCheck("compactionSwitch"); @@ -2232,6 +2233,7 @@ public ReplicateWALEntryResponse replicateWALEntry(final RpcController controlle * @param request the request */ @Override + @QosPriority(priority = HConstants.ADMIN_QOS) public RollWALWriterResponse rollWALWriter(final RpcController controller, final RollWALWriterRequest request) throws ServiceException { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSQosFunction.java index c6bd8967e124..15ee32397e9a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSQosFunction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSQosFunction.java @@ -72,5 +72,8 @@ public void testAnnotations() { checkMethod(conf, "CloseRegion", HConstants.ADMIN_QOS, qosFunction); checkMethod(conf, "CompactRegion", HConstants.ADMIN_QOS, qosFunction); checkMethod(conf, "FlushRegion", HConstants.ADMIN_QOS, qosFunction); + checkMethod(conf, "UpdateConfiguration", HConstants.ADMIN_QOS, qosFunction); + checkMethod(conf, "CompactionSwitch", HConstants.ADMIN_QOS, qosFunction); + checkMethod(conf, "RollWALWriter", HConstants.ADMIN_QOS, qosFunction); } } From e23a7e7b8a38fefb5794705e04820bf71dd37b03 Mon Sep 17 00:00:00 2001 From: guluo Date: Thu, 30 Nov 2023 22:02:46 +0800 Subject: [PATCH 105/109] NPE when flushing a non-existing column family --- .../hadoop/hbase/regionserver/HRegion.java | 4 ++- .../hbase/regionserver/TestHRegion.java | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index ae4045b1216b..dd50b6128c1e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -2607,7 +2607,9 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque private Collection getSpecificStores(List families) { Collection specificStoresToFlush = new ArrayList<>(); for (byte[] family : families) { - specificStoresToFlush.add(stores.get(family)); + if (stores.keySet().contains(family)) { + specificStoresToFlush.add(stores.get(family)); + } } return specificStoresToFlush; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 6a1c285bf8db..cfc0b089b8ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5806,6 +5806,36 @@ public void testFlushResult() throws IOException { } } + @Test + public void testFlushWithSpecifiedFamily() throws IOException { + byte[] family = Bytes.toBytes("family"); + this.region = initHRegion(tableName, method, family); + + Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); + this.region.put(put); + + HRegion.FlushResult fr = + this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); + assertFalse(fr.isFlushSucceeded()); + assertFalse(fr.isCompactionNeeded()); + } + + @Test + public void testFlushWithSpecifiedNoSuchFamily() throws IOException { + byte[] family = Bytes.toBytes("family"); + byte[] noSuchFamily = Bytes.toBytes("noSuchFamily"); + this.region = initHRegion(tableName, method, family); + + Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); + this.region.put(put); + + HRegion.FlushResult fr = + this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); + LOG.error(fr.getResult().toString()); + assertFalse(fr.isFlushSucceeded()); + assertFalse(fr.isCompactionNeeded()); + } + protected Configuration initSplit() { // Always compact if there is more than one store file. CONF.setInt("hbase.hstore.compactionThreshold", 2); From 8daf232b1c3f19dbc3975e5d571e72a6e7c72fc9 Mon Sep 17 00:00:00 2001 From: guluo Date: Fri, 1 Dec 2023 00:16:02 +0800 Subject: [PATCH 106/109] Update incorrect assertions --- .../apache/hadoop/hbase/regionserver/TestHRegion.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index cfc0b089b8ab..86307904c358 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5816,8 +5816,8 @@ public void testFlushWithSpecifiedFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); - assertFalse(fr.isFlushSucceeded()); - assertFalse(fr.isCompactionNeeded()); + assertTrue(fr.isFlushSucceeded()); + assertTrue(fr.isCompactionNeeded()); } @Test @@ -5831,9 +5831,8 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); - LOG.error(fr.getResult().toString()); - assertFalse(fr.isFlushSucceeded()); - assertFalse(fr.isCompactionNeeded()); + assertTrue(fr.isFlushSucceeded()); + assertTrue(fr.isCompactionNeeded()); } protected Configuration initSplit() { From 6bc73cac3eb67ff1e8bf844d13e0ff06bbca40c6 Mon Sep 17 00:00:00 2001 From: guluo Date: Fri, 1 Dec 2023 00:36:29 +0800 Subject: [PATCH 107/109] Update assertions --- .../java/org/apache/hadoop/hbase/regionserver/TestHRegion.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 86307904c358..7f65332c51bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5817,7 +5817,6 @@ public void testFlushWithSpecifiedFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family), false, FlushLifeCycleTracker.DUMMY); assertTrue(fr.isFlushSucceeded()); - assertTrue(fr.isCompactionNeeded()); } @Test @@ -5832,7 +5831,6 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); assertTrue(fr.isFlushSucceeded()); - assertTrue(fr.isCompactionNeeded()); } protected Configuration initSplit() { From 8ceb8f0a7b177c0ea4b4890740c1b1c623600fa7 Mon Sep 17 00:00:00 2001 From: guluo Date: Mon, 11 Dec 2023 21:42:29 +0800 Subject: [PATCH 108/109] flush failed-fast if there are non-existing families at client side --- .../hbase/client/RawAsyncHBaseAdmin.java | 79 ++++++++++++++----- .../hadoop/hbase/regionserver/HRegion.java | 18 ++++- .../hbase/client/TestFlushFromClient.java | 36 +++++++++ .../hbase/regionserver/TestHRegion.java | 8 +- 4 files changed, 115 insertions(+), 26 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 103a64e520a1..4005eaf9450b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -81,6 +81,7 @@ import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.quotas.QuotaTableUtil; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; @@ -966,6 +967,8 @@ public CompletableFuture flush(TableName tableName) { @Override public CompletableFuture flush(TableName tableName, byte[] columnFamily) { + Preconditions.checkNotNull(columnFamily, + "columnFamily is null, If you don't specify a columnFamily, use flush(TableName) instead."); return flush(tableName, Collections.singletonList(columnFamily)); } @@ -975,6 +978,8 @@ public CompletableFuture flush(TableName tableName, List columnFam // If the server version is lower than the client version, it's possible that the // flushTable method is not present in the server side, if so, we need to fall back // to the old implementation. + Preconditions.checkNotNull(columnFamilyList, + "columnFamily is null, If you don't specify a columnFamily, use flush(TableName) instead."); List columnFamilies = columnFamilyList.stream() .filter(cf -> cf != null && cf.length > 0).distinct().collect(Collectors.toList()); FlushTableRequest request = RequestConverter.buildFlushTableRequest(tableName, columnFamilies, @@ -983,21 +988,40 @@ public CompletableFuture flush(TableName tableName, List columnFam tableName, request, (s, c, req, done) -> s.flushTable(c, req, done), (resp) -> resp.getProcId(), new FlushTableProcedureBiConsumer(tableName)); CompletableFuture future = new CompletableFuture<>(); - addListener(procFuture, (ret, error) -> { + addListener(getDescriptor(tableName), (tDesc, error) -> { if (error != null) { - if (error instanceof TableNotFoundException || error instanceof TableNotEnabledException) { - future.completeExceptionally(error); - } else if (error instanceof DoNotRetryIOException) { - // usually this is caused by the method is not present on the server or - // the hbase hadoop version does not match the running hadoop version. - // if that happens, we need fall back to the old flush implementation. - LOG.info("Unrecoverable error in master side. Fallback to FlushTableProcedure V1", error); - legacyFlush(future, tableName, columnFamilies); + future.completeExceptionally(error); + } else { + List nonFaimilies = columnFamilies.stream().filter(cf -> !tDesc.hasColumnFamily(cf)) + .map(cf -> Bytes.toString(cf)).collect(Collectors.toList()); + if (nonFaimilies.size() > 0) { + String noSuchFamiliesMsg = + String.format("There are non-existing families %s, we cannot flush the table %s", + nonFaimilies, tableName.getNameAsString()); + future.completeExceptionally(new NoSuchColumnFamilyException(noSuchFamiliesMsg)); } else { - future.completeExceptionally(error); + addListener(procFuture, (ret, error2) -> { + if (error2 != null) { + if ( + error2 instanceof TableNotFoundException + || error2 instanceof TableNotEnabledException + ) { + future.completeExceptionally(error2); + } else if (error2 instanceof DoNotRetryIOException) { + // usually this is caused by the method is not present on the server or + // the hbase hadoop version does not match the running hadoop version. + // if that happens, we need fall back to the old flush implementation. + LOG.info("Unrecoverable error in master side. Fallback to FlushTableProcedure V1", + error2); + legacyFlush(future, tableName, columnFamilies); + } else { + future.completeExceptionally(error2); + } + } else { + future.complete(ret); + } + }); } - } else { - future.complete(ret); } }); return future; @@ -1071,14 +1095,29 @@ CompletableFuture flushRegionInternal(byte[] regionName, by .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName))); return; } - addListener(flush(serverName, location.getRegion(), columnFamily, writeFlushWALMarker), - (ret, err2) -> { - if (err2 != null) { - future.completeExceptionally(err2); - } else { - future.complete(ret); - } - }); + TableName tableName = location.getRegion().getTable(); + addListener(getDescriptor(tableName), (tDesc, error2) -> { + if (error2 != null) { + future.completeExceptionally(error2); + return; + } + if (columnFamily != null && !tDesc.hasColumnFamily(columnFamily)) { + String noSuchFamiliedMsg = String.format( + "There are non-existing family %s, we cannot flush the region %s, in table %s", + Bytes.toString(columnFamily), location.getRegion().getRegionNameAsString(), + tableName.getNameAsString()); + future.completeExceptionally(new NoSuchColumnFamilyException(noSuchFamiliedMsg)); + return; + } + addListener(flush(serverName, location.getRegion(), columnFamily, writeFlushWALMarker), + (ret, error3) -> { + if (error3 != null) { + future.completeExceptionally(error3); + } else { + future.complete(ret); + } + }); + }); }); return future; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index dd50b6128c1e..02dc5e71b185 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -2522,6 +2522,20 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque LOG.debug(msg); return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); } + // cannot flush non-existing column families, and fail-fast + if (families != null) { + List noSuchFamilies = + families.stream().filter(cf -> !getTableDescriptor().hasColumnFamily(cf)) + .map(cf -> Bytes.toString(cf)).collect(Collectors.toList()); + if (noSuchFamilies.size() > 0) { + String noSuchFamiliesMsg = String.format( + "There are non-existing families %s, we cannot flush the region %s, in table %s.", + noSuchFamilies, getRegionInfo().getRegionNameAsString(), + getTableDescriptor().getTableName().getNameAsString()); + LOG.warn(noSuchFamiliesMsg); + return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, noSuchFamiliesMsg, false); + } + } MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this); status.setStatus("Acquiring readlock on region"); // block waiting for the lock for flushing cache @@ -2607,9 +2621,7 @@ public FlushResultImpl flushcache(List families, boolean writeFlushReque private Collection getSpecificStores(List families) { Collection specificStoresToFlush = new ArrayList<>(); for (byte[] family : families) { - if (stores.keySet().contains(family)) { - specificStoresToFlush.add(stores.get(family)); - } + specificStoresToFlush.add(stores.get(family)); } return specificStoresToFlush; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java index 4756a5b22304..6659b6164f01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java @@ -19,10 +19,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -30,9 +34,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.io.IOUtils; import org.junit.After; @@ -43,6 +49,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +73,9 @@ public class TestFlushFromClient { @Rule public TestName name = new TestName(); + @Rule + public ExpectedException exception = ExpectedException.none(); + public TableName tableName; @BeforeClass @@ -184,6 +194,32 @@ public void testAsyncFlushRegionFamily() throws Exception { } } + @Test + public void testAsyncFlushTableWithNonExistingFamilies() throws IOException { + AsyncAdmin admin = asyncConn.getAdmin(); + List families = new ArrayList<>(); + families.add(FAMILY_1); + families.add(FAMILY_2); + families.add(Bytes.toBytes("non_family01")); + families.add(Bytes.toBytes("non_family02")); + CompletableFuture future = CompletableFuture.allOf(admin.flush(tableName, families)); + exception.expect(NoSuchColumnFamilyException.class); + FutureUtils.get(future); + } + + @Test + public void testAsyncFlushRegionWithNonExistingFamily() throws IOException { + AsyncAdmin admin = asyncConn.getAdmin(); + List regions = getRegionInfo(); + assertNotNull(regions); + assertTrue(regions.size() > 0); + HRegion region = regions.get(0); + CompletableFuture future = CompletableFuture.allOf( + admin.flushRegion(region.getRegionInfo().getRegionName(), Bytes.toBytes("non_family"))); + exception.expect(NoSuchColumnFamilyException.class); + FutureUtils.get(future); + } + @Test public void testFlushRegionServer() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 7f65332c51bd..3e4199439f7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -5828,9 +5828,11 @@ public void testFlushWithSpecifiedNoSuchFamily() throws IOException { Put put = new Put(tableName.toBytes()).addColumn(family, family, tableName.toBytes()); this.region.put(put); - HRegion.FlushResult fr = - this.region.flushcache(Arrays.asList(noSuchFamily), false, FlushLifeCycleTracker.DUMMY); - assertTrue(fr.isFlushSucceeded()); + HRegion.FlushResult fr = this.region.flushcache(Arrays.asList(family, noSuchFamily), false, + FlushLifeCycleTracker.DUMMY); + assertEquals(HRegion.FlushResult.Result.CANNOT_FLUSH, fr.getResult()); + assertFalse("Cannot flush the Region because of non-existing column families.", + fr.isFlushSucceeded()); } protected Configuration initSplit() { From ac16dfe7a5f7b65e5f14d74b922022aa5ba08e60 Mon Sep 17 00:00:00 2001 From: guluo Date: Mon, 19 Feb 2024 19:09:13 +0800 Subject: [PATCH 109/109] update --- .../apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 4005eaf9450b..9943aa936ea4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -982,11 +982,6 @@ public CompletableFuture flush(TableName tableName, List columnFam "columnFamily is null, If you don't specify a columnFamily, use flush(TableName) instead."); List columnFamilies = columnFamilyList.stream() .filter(cf -> cf != null && cf.length > 0).distinct().collect(Collectors.toList()); - FlushTableRequest request = RequestConverter.buildFlushTableRequest(tableName, columnFamilies, - ng.getNonceGroup(), ng.newNonce()); - CompletableFuture procFuture = this. procedureCall( - tableName, request, (s, c, req, done) -> s.flushTable(c, req, done), - (resp) -> resp.getProcId(), new FlushTableProcedureBiConsumer(tableName)); CompletableFuture future = new CompletableFuture<>(); addListener(getDescriptor(tableName), (tDesc, error) -> { if (error != null) { @@ -1000,6 +995,11 @@ public CompletableFuture flush(TableName tableName, List columnFam nonFaimilies, tableName.getNameAsString()); future.completeExceptionally(new NoSuchColumnFamilyException(noSuchFamiliesMsg)); } else { + FlushTableRequest request = RequestConverter.buildFlushTableRequest(tableName, columnFamilies, + ng.getNonceGroup(), ng.newNonce()); + CompletableFuture procFuture = this. procedureCall( + tableName, request, (s, c, req, done) -> s.flushTable(c, req, done), + (resp) -> resp.getProcId(), new FlushTableProcedureBiConsumer(tableName)); addListener(procFuture, (ret, error2) -> { if (error2 != null) { if (