[GitHub] [hbase] apurtell commented on a change in pull request #2574: HBASE-25212 Optionally abort requests in progress after deciding a region should close

2020-11-03 Thread GitBox


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

2020-10-29 Thread GitBox


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

2020-10-29 Thread GitBox


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

2020-10-29 Thread GitBox


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

2020-10-29 Thread GitBox


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

2020-10-29 Thread GitBox


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

2020-10-28 Thread GitBox


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

2020-10-27 Thread GitBox


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

2020-10-27 Thread GitBox


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

2020-10-27 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-26 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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

2020-10-22 Thread GitBox


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