[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r516166902 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4565,17 +4663,23 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount()); locked = true; + // From this point until memstore update this operation should not be interrupted. + disableInterrupts(); Review comment: Yes, here I can put a checkInterrupt. Let me do that. If we remove the handler from the map at this point, no other mini batch will be eligible for interrupt. batchMutate does this: while (!batchOp.isDone()) { ... doMiniBatchMutate(batchOp); ... } If we remove the handler from the map only the first iteration through the loop can be interrupted. ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4565,17 +4663,23 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount()); locked = true; + // From this point until memstore update this operation should not be interrupted. + disableInterrupts(); Review comment: Yes, here I can put a checkInterrupt. Let me do that. If we remove the handler from the map at this point, no other mini batch will be eligible for interrupt. batchMutate does this: while (!batchOp.isDone()) { ... doMiniBatchMutate(batchOp); ... } ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8303,9 +8428,14 @@ public void processRowsWithLocks(RowProcessor processor, long timeout, prevRowLock = rowLock; } } + // STEP 3. Region lock lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size()); locked = true; + +// From this point until memstore update this operation should not be interrupted. +disableInterrupts(); Review comment: Will add a checkInterrupt here too. ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -7623,6 +7740,9 @@ protected boolean nextRow(ScannerContext scannerContext, Cell curRowCell) throws Cell next; while ((next = this.storeHeap.peek()) != null && CellUtil.matchingRows(next, curRowCell)) { +// Check for thread interrupt status in case we have been signaled from +// #interruptRegionOperation. +checkInterrupt(); Review comment: Good point. Agreed it would be a good follow up. ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8303,9 +8428,14 @@ public void processRowsWithLocks(RowProcessor processor, long timeout, prevRowLock = rowLock; } } + // STEP 3. Region lock lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size()); locked = true; + +// From this point until memstore update this operation should not be interrupted. +disableInterrupts(); Review comment: This code is not executed in a loop so removing the thread from the map would be fine. However the reason why I also do disable/enable interrupt is as follows: We already have to do this for doMiniBatchMutation. The disable/enable interrupt pair is a new code discipline. This section of the code also has the same requirements. Apply the new code discipline here too for consistency. ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8303,9 +8428,14 @@ public void processRowsWithLocks(RowProcessor processor, long timeout, prevRowLock = rowLock; } } + // STEP 3. Region lock lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size()); locked = true; + +// From this point until memstore update this operation should not be interrupted. +disableInterrupts(); Review comment: This code is not executed in a loop so removing the thread from the map would be fine. However the reason why I also do disable/enable interrupt is as follows: We already have to do this for doMiniBatchMutation. (The disable/enable pair in doMiniBatchMutation protects WAL update and memstore insert from interrupt and makes them "atomic" in this sense, and disable/enable makes sense there because doMiniBatchMutation is inside a loop.) The disable/enable interrupt pair is a
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r514400950 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -9000,6 +9173,51 @@ public long getReadPoint() { return getReadPoint(IsolationLevel.READ_COMMITTED); } + /** + * Interrupt any region options that have acquired the region lock via + * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)}, + * or {@link #startBulkRegionOperation(boolean)}. + */ + private void interruptRegionOperations() { +for (Map.Entry entry: regionLockHolders.entrySet()) { + // An entry in this map will have a boolean value indicating if it is currently + // eligible for interrupt; if so, we should interrupt it. + if (entry.getValue().booleanValue()) { Review comment: The generated code will be the same. I opted for this to be pedantic in the code. It should be fine, right? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r514400489 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8777,23 +8936,36 @@ public void closeRegionOperation(Operation operation) throws IOException { if (operation == Operation.SNAPSHOT) { stores.values().forEach(HStore::postSnapshotOperation); } +regionLockHolders.remove(Thread.currentThread()); lock.readLock().unlock(); if (coprocessorHost != null) { coprocessorHost.postCloseRegionOperation(operation); } } + /** + * If a handler thread is eligible for interrupt, make it ineligible. Should be paired + * with {{@link #enableInterrupts()}. + */ + protected void disableInterrupts() { +regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> false); + } + + /** + * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()}, + * make it eligible again. No-op if interrupts are already enabled. + */ + protected void enableInterrupts() { +regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> true); + } + /** * This method needs to be called before any public call that reads or * modifies stores in bulk. It has to be called just before a try. * #closeBulkRegionOperation needs to be called in the try's finally block * Acquires a writelock and checks if the region is closing or closed. - * @throws NotServingRegionException when the region is closing or closed Review comment: I can put them back but I believe this was done in response to feedback from another reviewer This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r514337192 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8730,12 +8868,22 @@ public void startRegionOperation() throws IOException { @Override public void startRegionOperation(Operation op) throws IOException { +boolean isInterruptableOp = false; switch (op) { - case GET: // read operations + case GET: // interruptible read operations case SCAN: +isInterruptableOp = true; Review comment: There are some checkInterrupts placed inside the scanner inner loops. Get, Exist, and Scan operations all share this path. I did not put checkInterrupt into the checkAndXXX operations because these are point ops that will complete quickly, but could if you feel this represents missing coverage. @Reidd On the write path there are checkInterrupts placed at points in doMiniBatchMutation to catch the cases where we are likely to run a long time, and the row mutation processor also checks for interrupts. Again not every operation has a check, like Increment or Append, where the unit of work is small, but we could add them there too if you feel this represents missing coverage. My opinion is the long running cases are where the check is definitely worth it -- scanners on the read side, batch mutations on the write side -- and others are marginal (and therefore not included). This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r514337192 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8730,12 +8868,22 @@ public void startRegionOperation() throws IOException { @Override public void startRegionOperation(Operation op) throws IOException { +boolean isInterruptableOp = false; switch (op) { - case GET: // read operations + case GET: // interruptible read operations case SCAN: +isInterruptableOp = true; Review comment: There are some checkInterrupts placed inside the scanner inner loops. Get, Exist, and Scan operations all share this path. Row mutation processor also checks for interrupts. I did not put checkInterrupt into the checkAndXXX operations because these are point ops that will complete quickly, but could if you feel this represents missing coverage. @Reidd This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r514337192 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8730,12 +8868,22 @@ public void startRegionOperation() throws IOException { @Override public void startRegionOperation(Operation op) throws IOException { +boolean isInterruptableOp = false; switch (op) { - case GET: // read operations + case GET: // interruptible read operations case SCAN: +isInterruptableOp = true; Review comment: There are some checkInterrupts placed inside the scanner inner loops. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r513631004 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -688,14 +689,17 @@ void sawNoSuchFamily() { // Last flush time for each Store. Useful when we are flushing for each column private final ConcurrentMap lastStoreFlushTimeMap = new ConcurrentHashMap<>(); - final RegionServerServices rsServices; + protected RegionServerServices rsServices; private RegionServerAccounting rsAccounting; private long flushCheckInterval; // flushPerChanges is to prevent too many changes in memstore private long flushPerChanges; private long blockingMemStoreSize; // Used to guard closes final ReentrantReadWriteLock lock; + // Used to track interruptible holders of the region lock + // Currently that is only RPC handler threads + final ConcurrentHashMap regionLockHolders; Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512965703 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4569,13 +4665,29 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { // We should record the timestamp only after we have acquired the rowLock, // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer // timestamp + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + long now = EnvironmentEdgeManager.currentTime(); batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks); // STEP 3. Build WAL edit + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + List> walEdits = batchOp.buildWALEdits(miniBatchOp); // STEP 4. Append the WALEdits to WAL and sync. + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. This is the last place we can do it "safely" before + // WAL appends. + checkInterrupt(); + Review comment: Since we are on the topic of WAL syncs We could protect against interrupting a sync using a couple of different approaches. The simplest in the context of this proposed change would be to remove the current handler thread from the regionLockHolders set whenever we decide interrupting it is no longer the best choice, like a point of no return on the mutation code path. startRegionOperation(Operation.PUT); // add thread to the set so it is interruptible try { ... startWALOperation(); // remove thread from the set so it will not be interrupted try { ... } finally { endWALOperation(); // add thread to the set so it is eligible to be interrupted again } ... } finally { endRegionOperation(); // remove thread from the set as it is no longer actively handling RPC } However we already have issues with sync timeouts or regionserver aborts during sync such that we are not introducing anything new here. In other words, if it is problematic that a WAL sync can be aborted before it completes, this already happens for other reasons, like timeouts (presumably because HDFS is slow), or HDFS level pipeline recovery failure, or server shutdown. Thinking about this though makes me think of the English expression "opening a can of worms". I don't know of a Chinese equivalent. I'm going to implement the proposal above out of an abundance of caution and we can decide upon further review if it makes sense. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512957365 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4569,13 +4665,29 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { // We should record the timestamp only after we have acquired the rowLock, // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer // timestamp + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + long now = EnvironmentEdgeManager.currentTime(); batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks); // STEP 3. Build WAL edit + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + List> walEdits = batchOp.buildWALEdits(miniBatchOp); // STEP 4. Append the WALEdits to WAL and sync. + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. This is the last place we can do it "safely" before + // WAL appends. + checkInterrupt(); + Review comment: @Reidd I would agree checking beyond WAL sync would be useful if and only if we decide to protect WAL syncs from interrupts. Otherwise the sync will be interrupted and we will break out of the RPC processing with an IOException with cause InterruptedException (from the WAL sync future). This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512937688 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4569,13 +4665,29 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { // We should record the timestamp only after we have acquired the rowLock, // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer // timestamp + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + long now = EnvironmentEdgeManager.currentTime(); batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks); // STEP 3. Build WAL edit + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); + List> walEdits = batchOp.buildWALEdits(miniBatchOp); // STEP 4. Append the WALEdits to WAL and sync. + + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. This is the last place we can do it "safely" before + // WAL appends. + checkInterrupt(); + Review comment: The steps here are each loops, so there's a chance that a fair amount of time elapses between each step, which is why I thought checking after each step offered value beyond just checking once. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512176656 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,356 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TableNameTestRule name = new TableNameTestRule(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { +Configuration conf = TEST_UTIL.getConfiguration(); +conf.setClass(HConstants.REGION_IMPL, InterruptInterceptingHRegion.class, Region.class); +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); Review comment: Done This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512168388 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,356 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TableNameTestRule name = new TableNameTestRule(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { +Configuration conf = TEST_UTIL.getConfiguration(); +conf.setClass(HConstants.REGION_IMPL, InterruptInterceptingHRegion.class, Region.class); +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); +conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + } + + @Before + public void setUp() throws Exception { +TEST_UTIL.startMiniCluster(); + } + + @After + public void tearDown() throws Exception { +TEST_UTIL.shutdownMiniCluster(); + } + + @Test(timeout=12) Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512165464 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -1679,22 +1688,82 @@ public void setTimeoutForWriteLock(long timeoutForWriteLock) { } } -if (timeoutForWriteLock == null -|| timeoutForWriteLock == Long.MAX_VALUE) { - // block waiting for the lock for closing - lock.writeLock().lock(); // FindBugs: Complains UL_UNRELEASED_LOCK_EXCEPTION_PATH but seems fine +// Set the closing flag +// From this point new arrivals at the region lock will get NSRE. + +this.closing.set(true); +LOG.info("Closing region {}", this); + +// Acquire the close lock + +// The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both +// the new regionserver abort condition and interrupts for running requests. +// If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior, +// we will not attempt to interrupt threads servicing requests nor crash out +// the regionserver if something remains stubborn. + +boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT); +boolean useTimedWait = false; +if (timeoutForWriteLock == null || timeoutForWriteLock == Long.MAX_VALUE) { + if (canAbort) { +timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME); +useTimedWait = true; + } } else { + // convert legacy use of timeoutForWriteLock in seconds to new use in millis + timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock); + useTimedWait = true; +} +if (LOG.isDebugEnabled()) { + LOG.debug((useTimedWait ? "Time limited wait" : "Waiting") + " for close lock on " + this); Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512090958 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,356 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TableNameTestRule name = new TableNameTestRule(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { +Configuration conf = TEST_UTIL.getConfiguration(); +conf.setClass(HConstants.REGION_IMPL, InterruptInterceptingHRegion.class, Region.class); +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); Review comment: Let me restate this as what I think you were getting at: "Can we set a shorter abort interval so this test test will not run for a long time if interrupts do not work as expected?" Sure, ok. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512090958 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,356 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TableNameTestRule name = new TableNameTestRule(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { +Configuration conf = TEST_UTIL.getConfiguration(); +conf.setClass(HConstants.REGION_IMPL, InterruptInterceptingHRegion.class, Region.class); +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); Review comment: Let me restate this as what I think you were getting at: "Can we set a shorter abort interval so this test will not run for a long time if interrupts do not work as expected?" Sure, ok. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512090958 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,356 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +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.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TableNameTestRule name = new TableNameTestRule(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { +Configuration conf = TEST_UTIL.getConfiguration(); +conf.setClass(HConstants.REGION_IMPL, InterruptInterceptingHRegion.class, Region.class); +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); Review comment: Let me restate this as what I think you were getting at: "Can we set this shorter so it doesn't cause the test to run for a long time and/or timeout if interrupts do not work as expected?" Sure, ok. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512089577 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -688,14 +688,17 @@ void sawNoSuchFamily() { // Last flush time for each Store. Useful when we are flushing for each column private final ConcurrentMap lastStoreFlushTimeMap = new ConcurrentHashMap<>(); - final RegionServerServices rsServices; + protected RegionServerServices rsServices; Review comment: In the HRegion tests I mock RSS with mockito, to test if the HRegion code calls RSS#abort as it should. We don't need to make a real cluster test that aborts a server (and adds a lot of time to the unit test suite) just to confirm this. So, it needs to be nonfinal for mocking, but it does not need to be exposed as a settable thing, because nowhere else in the code is there a use for that. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512087407 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -9000,6 +9129,49 @@ public long getReadPoint() { return getReadPoint(IsolationLevel.READ_COMMITTED); } + /** + * Interrupt any region options that have acquired the region lock via + * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)}, + * or {@link #startBulkRegionOperation(boolean)}. + */ + private void interruptRegionOperations() { +for (Thread t: regionLockHolders) { + t.interrupt(); +} + } + + /** + * Check thread interrupt status and throw an exception if interrupted. + * @throws NotServingRegionException if region is closing + * @throws InterruptedIOException if interrupted but region is not closing + */ + // Package scope for tests + void checkInterrupt() throws NotServingRegionException, InterruptedIOException { +if (Thread.interrupted()) { + if (this.closing.get()) { +throw new NotServingRegionException( + getRegionInfo().getRegionNameAsString() + " is closing"); + } + throw new InterruptedIOException(); +} + } + + /** + * Throw the correct exception upon interrupt + * @param t cause + * @throws NotServingRegionException if region is closing + * @throws InterruptedIOException in all cases except if region is closing Review comment: Old stuff, will update This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512087013 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8793,7 +8921,7 @@ public void closeRegionOperation(Operation operation) throws IOException { * @throws InterruptedIOException if interrupted while waiting for a lock */ private void startBulkRegionOperation(boolean writeLockNeeded) - throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { + throws NotServingRegionException, IOException { Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512083604 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7366,157 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test + public void testCloseNoInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Disable close thread interrupt and server abort behavior +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false); +region = initHRegion(tableName, method, conf, families); + +final CountDownLatch latch = new CountDownLatch(1); +final AtomicBoolean holderInterrupted = new AtomicBoolean(); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + try { +Thread.sleep(10*1000); + } catch (InterruptedException e) { +LOG.info("Interrupted"); +holderInterrupted.set(true); Review comment: If you throw an assertion inside a Thread runnable it won't unwind to the JUnit test method that forked the thread, it will be caught by Thread.UncaughtExceptionHandler, which is not what I think we want. So I set the boolean and then test it in the unit test method where an assertion will propagate the way I want. Maybe JUnit installs its own UEH, didn't test, but the way I have done it makes more sense to me. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512084799 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -1679,22 +1688,82 @@ public void setTimeoutForWriteLock(long timeoutForWriteLock) { } } -if (timeoutForWriteLock == null -|| timeoutForWriteLock == Long.MAX_VALUE) { - // block waiting for the lock for closing - lock.writeLock().lock(); // FindBugs: Complains UL_UNRELEASED_LOCK_EXCEPTION_PATH but seems fine +// Set the closing flag +// From this point new arrivals at the region lock will get NSRE. + +this.closing.set(true); +LOG.info("Closing region {}", this); + +// Acquire the close lock + +// The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both +// the new regionserver abort condition and interrupts for running requests. +// If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior, +// we will not attempt to interrupt threads servicing requests nor crash out +// the regionserver if something remains stubborn. + +boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT); +boolean useTimedWait = false; +if (timeoutForWriteLock == null || timeoutForWriteLock == Long.MAX_VALUE) { + if (canAbort) { +timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME); +useTimedWait = true; + } } else { + // convert legacy use of timeoutForWriteLock in seconds to new use in millis + timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock); Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512083604 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7366,157 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test + public void testCloseNoInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Disable close thread interrupt and server abort behavior +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false); +region = initHRegion(tableName, method, conf, families); + +final CountDownLatch latch = new CountDownLatch(1); +final AtomicBoolean holderInterrupted = new AtomicBoolean(); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + try { +Thread.sleep(10*1000); + } catch (InterruptedException e) { +LOG.info("Interrupted"); +holderInterrupted.set(true); Review comment: If you throw an assertion inside a Thread runnable it won't unwind to the Junit test method, it will be caught by Thread.UncaughtExceptionHandler, which is not what I think we want. So I set the boolean and then test it in the unit test method where an assertion will propagate the way I want. Maybe JUnit installs its own UEH, didn't test, but the way I have done it makes more sense to me. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512084523 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7366,157 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test + public void testCloseNoInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Disable close thread interrupt and server abort behavior +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false); +region = initHRegion(tableName, method, conf, families); + +final CountDownLatch latch = new CountDownLatch(1); +final AtomicBoolean holderInterrupted = new AtomicBoolean(); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + try { +Thread.sleep(10*1000); + } catch (InterruptedException e) { +LOG.info("Interrupted"); +holderInterrupted.set(true); + } +} catch (Exception e) { + throw new RuntimeException(e); +} finally { + try { +region.closeRegionOperation(); + } catch (IOException e) { + } + LOG.info("Stopped region operation holder"); +} + } +}); + +holder.start(); +latch.await(); +region.close(); +holder.join(); +region = null; + +assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get()); + } + + @Test + public void testCloseInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Enable close thread interrupt and server abort behavior +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); +region = initHRegion(tableName, method, conf, families); + +final CountDownLatch latch = new CountDownLatch(1); +final AtomicBoolean holderInterrupted = new AtomicBoolean(); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + try { +Thread.sleep(10*1000); + } catch (InterruptedException e) { +LOG.info("Interrupted"); +holderInterrupted.set(true); + } +} catch (Exception e) { + throw new RuntimeException(e); +} finally { + try { +region.closeRegionOperation(); + } catch (IOException e) { + } + LOG.info("Stopped region operation holder"); +} + } +}); + +holder.start(); +latch.await(); +region.close(); +holder.join(); +region = null; + +assertTrue("Region lock holder was not interrupted", holderInterrupted.get()); + } + + @Test + public void testCloseAbort() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Enable close thread interrupt and server abort behavior +// Set the close lock acquisition wait time to 5 seconds +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true); +conf.setInt(HRegion.CLOSE_WAIT_TIME, 5*1000); +region = initHRegion(tableName, method, conf, families); +RegionServerServices rsServices = mock(RegionServerServices.class); +region.rsServices = rsServices; + +final CountDownLatch latch = new CountDownLatch(1); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + // Hold the lock for 10 seconds no matter how many times we are interrupted + int timeRemaining = 10 * 1000; + while (timeRemaining > 0) { +long start = EnvironmentEdgeManager.currentTime(); Review comment: We always use EnvironmentEdgeManager, especially in tests, per our coding conventions. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r512083604 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7366,157 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test + public void testCloseNoInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = new Configuration(CONF); +// Disable close thread interrupt and server abort behavior +conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false); +region = initHRegion(tableName, method, conf, families); + +final CountDownLatch latch = new CountDownLatch(1); +final AtomicBoolean holderInterrupted = new AtomicBoolean(); +Thread holder = new Thread(new Runnable() { + @Override + public void run() { +try { + LOG.info("Starting region operation holder"); + region.startRegionOperation(Operation.SCAN); + latch.countDown(); + try { +Thread.sleep(10*1000); + } catch (InterruptedException e) { +LOG.info("Interrupted"); +holderInterrupted.set(true); Review comment: If you throw an assertion inside a Thread runnable it won't unwind to the Junit test method, it will be caught by Thread.UncaughtExceptionHandler, which is not what I think we want. So I set the boolean and then test it in the unit test method where an assertion will propagate the way I want. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510320642 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInterrupt.java ## @@ -0,0 +1,357 @@ +/* + * 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.assertTrue; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; +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.Append; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Scan; +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.coprocessor.ObserverContext; +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.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({RegionServerTests.class, LargeTests.class}) +public class TestRegionInterrupt { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestRegionInterrupt.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Logger LOG = LoggerFactory.getLogger(TestRegionInterrupt.class); + + static final int SLEEP_TIME = 10 * 1000; + static final byte[] FAMILY = Bytes.toBytes("info"); + + @Rule + public TestName name = new TestName(); Review comment: Did not know about this, ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510320411 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7367,154 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test(timeout=2) + public void testCloseNoInterrupt() throws Exception { +byte[] cf1 = Bytes.toBytes("CF1"); +byte[][] families = { cf1 }; + +Configuration conf = HBaseConfiguration.create(); Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510320339 ## File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ## @@ -7364,4 +7367,154 @@ protected HStoreForTesting(final HRegion region, return super.doCompaction(cr, filesToCompact, user, compactionStartTime, newFiles); } } + + @Test(timeout=2) Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510319462 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -6588,8 +6677,10 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock success = true; return result; } catch (InterruptedException ie) { - LOG.warn("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, -getRegionInfo().getRegionNameAsString()); + if (LOG.isDebugEnabled()) { +LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, Review comment: This was an existing case where the client would get back an IIOE. I didn't change it so the current semantics are preserved. It can be changed to use throwOnInterrupt This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510319462 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -6588,8 +6677,10 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock success = true; return result; } catch (InterruptedException ie) { - LOG.warn("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, -getRegionInfo().getRegionNameAsString()); + if (LOG.isDebugEnabled()) { +LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, Review comment: This was an existing case where the client would get back an IIOE. I didn't change it so the current semantics are preserved. It can be changed to use throwIfInterrupted This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510317299 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4550,6 +4619,10 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { /** Keep track of the locks we hold so we can release them in finally clause */ List acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size()); try { + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); Review comment: Yes, but on the other hand we are basically checking interrupt status manually, as one has to do in Java, so can check and exit early at only what we think are safe points, which is nice. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510317538 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -8748,6 +8863,11 @@ public void startRegionOperation(Operation op) throws IOException { throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); } lock(lock.readLock()); +// Update regionLockHolders ONLY for any startRegionOperation call that is invoked from an RPC handler +Thread thisThread = Thread.currentThread(); +if (isInterruptableOp) { + regionLockHolders.put(thisThread.hashCode(), thisThread); Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510317299 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -4550,6 +4619,10 @@ private void doMiniBatchMutate(BatchOperation batchOp) throws IOException { /** Keep track of the locks we hold so we can release them in finally clause */ List acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size()); try { + // Check for thread interrupt status in case we have been signaled from + // #interruptRegionOperation. + checkInterrupt(); Review comment: Yes, but on the other hand we are basically checking interrupt status manually, as one has to do in Java, so can check and exit early only at only what we think are safe points. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close
apurtell commented on a change in pull request #2574: URL: https://github.com/apache/hbase/pull/2574#discussion_r510316643 ## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ## @@ -1174,7 +1178,7 @@ public HStore call() throws IOException { LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this); } } catch (InterruptedException e) { -throw (InterruptedIOException)new InterruptedIOException().initCause(e); +throwOnInterrupt(e); Review comment: Ok This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org