keith-turner commented on code in PR #3677:
URL: https://github.com/apache/accumulo/pull/3677#discussion_r1289153602


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java:
##########
@@ -901,6 +902,39 @@ public void close(boolean saveState) throws IOException {
   void initiateClose(boolean saveState) {
     log.trace("initiateClose(saveState={}) {}", saveState, getExtent());
 
+    // Check to see if last or current minc is failing. If so, then throw
+    // an exception before closing the compactable and leaving the tablet
+    // in a half-closed state. Don't throw IllegalStateException because
+    // calling code will just continue to retry.
+    if (saveState) {
+      if (!isLastMinCSuccessful()) {
+        if (isMinorCompactionRunning()) {
+          // Then current minor compaction is retrying and failure is being
+          // reported.
+          String msg = "Aborting close on " + extent
+              + " because last minor compaction was not successful. Check the 
server log.";
+          log.warn(msg);
+          throw new RuntimeException(msg);
+        } else {
+          // We don't know when the last minc occurred. Kick one off now. It's 
possible
+          // that a minc will not be initiated (if there is no data in the 
table for example)
+          if (initiateMinorCompaction(MinorCompactionReason.CLOSE)) {
+            getTabletMemory().waitForMinC();
+          }
+        }
+      }
+      // We don't want to initiate the close process on the tablet if the last 
minor compaction
+      // failed. Let the user resolve whatever the issue may be so that we get 
a successful
+      // minor compaction on the tablet before closing it.
+      if (!isLastMinCSuccessful()) {
+        String msg = "Aborting close on " + extent
+            + " because last minor compaction was not successful. Check the 
server log.";
+        log.warn(msg);
+        throw new RuntimeException(msg);
+      }
+
+    }
+

Review Comment:
   If I understand correctly, the observed issue was fixed by catching 
Exception instead of IOException in MinorCompaction task.  If that is correct 
then I think it may be best to remove this code for the following reasons.
   
    * Think its better to make the minor compaction code aggressively retry (as 
was done in other parts of this PR) instead of attempting to handle it not 
retrying here.  If we find more places where the minor compaction code could 
have retried and it did not, then I think we should modify that minor 
compaction code to make it retry there.
    * This code has some race conditions, for example a minor compaction could 
fail in another thread right after the check in this function.  So the checks 
here do not completely solve the problem.
    * I think there is a bug in the code.  The call to waitForMinC() must be 
called with the tablet lock held or it will throw an exception, I do not think 
the tablet lock is held when its called.
    * This function is already initiating a minor compaction.  If we are going 
to initiate another minor compaction in the function it may be best to remove 
the later one.  However that is tricky because that later minc is coordinated 
with updating the flush id by the function.
   
   ```suggestion
   ```



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -1473,6 +1473,13 @@ public CompactionServiceId 
getConfiguredService(CompactionKind kind) {
     try {
       var dispatcher = 
tablet.getTableConfiguration().getCompactionDispatcher();
 
+      if (dispatcher == null) {

Review Comment:
   Was this if stmt added to avoid logging an NPE later in the code?



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -1473,6 +1473,13 @@ public CompactionServiceId 
getConfiguredService(CompactionKind kind) {
     try {
       var dispatcher = 
tablet.getTableConfiguration().getCompactionDispatcher();
 
+      if (dispatcher == null) {
+        log.error(
+            "Failed to dispatch compaction {} kind:{} hints:{}, falling back 
to {} service. Check server log.",

Review Comment:
   ```suggestion
               "Failed to dispatch compaction {} kind:{} hints:{}, falling back 
to {} service. Unable to instantiate dispatcher plugin.  Check server log.",
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java:
##########
@@ -178,6 +178,14 @@ private static CompactionDispatcher 
createCompactionDispatcher(AccumuloConfigura
     CompactionDispatcher newDispatcher = 
Property.createTableInstanceFromPropertyName(conf,
         Property.TABLE_COMPACTION_DISPATCHER, CompactionDispatcher.class, 
null);
 
+    if (newDispatcher == null) {
+      // return early to prevent NPE
+      log.error(

Review Comment:
   Not sure if this matters.  The log level of error here does not match the 
log level of the message in `ConfigurationTypeHelper.getClassInstance()` which 
is warn.  The log message from `ConfigurationTypeHelper.getClassInstance()` 
will probably contain the information the user needs to determine why the 
compaction dispatcher plugin did not load.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java:
##########
@@ -1009,6 +1009,9 @@ public void splitTablet(TInfo tinfo, TCredentials 
credentials, TKeyExtent tkeyEx
       } catch (IOException e) {
         log.warn("Failed to split " + keyExtent, e);
         throw new RuntimeException(e);
+      } catch (RuntimeException re) {
+        log.warn("Failed to split " + keyExtent, re);

Review Comment:
   Was this code only added to log additional information? 



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java:
##########
@@ -190,6 +190,7 @@ enum CompactionState {
   private CompactableImpl compactable;
 
   private volatile CompactionState minorCompactionState = null;
+  private volatile boolean lastMinCSuccessful = true;

Review Comment:
   Could remove this if the code that checks it in close is removed.



##########
test/src/main/java/org/apache/accumulo/test/functional/HalfClosedTabletIT.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.clientImpl.AccumuloServerException;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.accumulo.test.util.Wait;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Sets;
+
+// This IT tests the cases seen in 
https://github.com/apache/accumulo/issues/3674
+// where a failing minor compaction causes a Tablet.initiateClose to leave the
+// Tablet in a half-closed state. The half-closed Tablet cannot be unloaded and
+// the TabletServer cannot be shutdown normally. Because the minor compaction 
has
+// been failing the Tablet needs to be recovered when it's ultimately 
re-assigned.
+public class HalfClosedTabletIT extends SharedMiniClusterBase {
+
+  public static class HalfClosedTabletITConfiguration implements 
MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration 
coreSite) {
+      cfg.setNumTservers(1);
+    }
+
+  }
+
+  @BeforeAll
+  public static void startup() throws Exception {
+    SharedMiniClusterBase.startMiniClusterWithConfig(new 
HalfClosedTabletITConfiguration());
+  }
+
+  @AfterAll
+  public static void shutdown() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testSplitWithInvalidContext() throws Exception {
+
+    // In this scenario the table has been mis-configured with an invalid 
context name.
+    // The minor compaction task is failing because classes like the volume 
chooser or
+    // user iterators cannot be loaded. The user calls Tablet.split which 
calls initiateClose.
+    // This test ensures that the Tablet can still be unloaded normally by 
taking if offline
+    // after the split call with an invalid context. The context property is 
removed after the
+    // split call below to get the minor compaction task to succeed on a 
subsequent run. Because
+    // the minor compaction task backs off when retrying, this could take some 
time.
+
+    String tableName = getUniqueNames(1)[0];
+
+    try (final var client = 
Accumulo.newClient().from(getClientProps()).build()) {
+      final var tops = client.tableOperations();
+      tops.create(tableName);
+      TableId tableId = TableId.of(tops.tableIdMap().get(tableName));
+      try (final var bw = client.createBatchWriter(tableName)) {
+        final var m1 = new Mutation("a");
+        final var m2 = new Mutation("b");
+        m1.put(new Text("cf"), new Text(), new Value());
+        m2.put(new Text("cf"), new Text(), new Value());
+        bw.addMutation(m1);
+        bw.addMutation(m2);
+      }
+
+      
setInvalidClassLoaderContextPropertyWithoutValidation(getCluster().getServerContext(),
+          tableId);
+
+      Thread.sleep(500);
+
+      // This should fail to split, but not leave the tablets in a state where 
they can't
+      // be unloaded
+      assertThrows(AccumuloServerException.class,
+          () -> tops.addSplits(tableName, Sets.newTreeSet(List.of(new 
Text("b")))));
+
+      
removeInvalidClassLoaderContextPropertyWithoutValidation(getCluster().getServerContext(),
+          tableId);

Review Comment:
   I tried removing the code in initiateClose and running the test locally.  
Made the following changes to this test to get it running.
   
   ```suggestion
         Thread configFixer = new Thread(() -> {
           UtilWaitThread.sleep(3000);
           
removeInvalidClassLoaderContextPropertyWithoutValidation(getCluster().getServerContext(),
                   tableId);
         });
   
         // grab this time before starting the thread starts and before it 
sleeps.
         long t1 = System.nanoTime();
         configFixer.start();
   
         // The split will probably start running w/ bad config that will cause 
it to get stuck.  However once the config is fixed by the background thread it 
should continue.
         tops.addSplits(tableName, Sets.newTreeSet(List.of(new Text("b"))));
         long t2 = System.nanoTime();
   
         // expect that split took at least 3 seconds because that is the time 
it takes to fix the config
         assertTrue(TimeUnit.NANOSECONDS.toMillis(t2-t1) > 3000);
   ```



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to