dlmarion commented on code in PR #5543:
URL: https://github.com/apache/accumulo/pull/5543#discussion_r2084523016


##########
server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/SharedBatchWriter.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.coordinator;
+
+import java.util.ArrayList;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.util.Timer;
+import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.server.ServerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SharedBatchWriter {
+  private static final Logger log = 
LoggerFactory.getLogger(SharedBatchWriter.class);
+
+  private static class Work {
+    private final Mutation mutation;
+    private final CompletableFuture<Void> future;
+
+    private Work(Mutation mutation) {
+      this.mutation = mutation;
+      this.future = new CompletableFuture<>();
+    }
+  }
+
+  private final BlockingQueue<Work> mutations;
+  private final String table;
+  private final ServerContext context;
+
+  public SharedBatchWriter(String table, ServerContext context, int queueSize) 
{
+    this.table = table;
+    this.context = context;
+    this.mutations = new ArrayBlockingQueue<>(queueSize);
+    var thread =
+        Threads.createCriticalThread("shared batch writer for " + table, 
this::processMutations);
+    thread.start();
+  }
+
+  public void write(Mutation m) {
+    try {
+      var work = new Work(m);
+      mutations.put(work);
+      work.future.join();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(e);
+    }
+  }
+
+  private void processMutations() {
+    Timer timer = Timer.startNew();
+    while (true) {
+      ArrayList<Work> batch = new ArrayList<>();
+      try {
+        batch.add(mutations.take());
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(e);
+      }
+
+      var config = new BatchWriterConfig().setMaxWriteThreads(16);
+      try (var writer = context.createBatchWriter(table, config)) {
+        mutations.drainTo(batch);
+        timer.restart();
+        for (var work : batch) {
+          writer.addMutation(work.mutation);
+        }
+        writer.flush();
+        log.trace("Wrote {} mutations in {}ms", batch.size(), 
timer.elapsed(TimeUnit.MILLISECONDS));
+        batch.forEach(work -> work.future.complete(null));
+      } catch (TableNotFoundException | MutationsRejectedException e) {
+        batch.forEach(work -> work.future.completeExceptionally(e));

Review Comment:
   Should we log the failures, even if to debug or trace?



##########
core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java:
##########
@@ -66,4 +71,17 @@ public static Thread createThread(String name, OptionalInt 
priority, Runnable r,
     return thread;
   }
 
+  public static Thread createCriticalThread(String name, Runnable r) {
+    Runnable wrapped = () -> {
+      try {
+        r.run();
+      } catch (RuntimeException e) {
+        String msg = "Critical thread " + name + " died";
+        log.error(msg, e);

Review Comment:
   I think you should use `System.err.println` here. If log4j2 is configured to 
use async logging, which I think is the default in our configuration, then this 
log message may never make it to the log.



##########
core/src/main/java/org/apache/accumulo/core/util/threads/Threads.java:
##########
@@ -66,4 +71,17 @@ public static Thread createThread(String name, OptionalInt 
priority, Runnable r,
     return thread;
   }
 
+  public static Thread createCriticalThread(String name, Runnable r) {
+    Runnable wrapped = () -> {
+      try {
+        r.run();
+      } catch (RuntimeException e) {
+        String msg = "Critical thread " + name + " died";
+        log.error(msg, e);
+        Halt.halt(msg);

Review Comment:
   I think you should just call `Runtime.getRuntime().halt(int);` here. 
`Halt.halt(String)` creates a Thread to log the msg, then calls 
Runtime.halt(int). For the same reason as above, the log msg may never make it 
to the log.  This suggestion and the one above are consistent with the `Error` 
handling in the 
[AccumuloUncaughtExceptionHandler](https://github.com/apache/accumulo/blob/2.1/core/src/main/java/org/apache/accumulo/core/util/threads/AccumuloUncaughtExceptionHandler.java#L67-L76).



##########
server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java:
##########
@@ -99,15 +108,18 @@ public void commitCompaction(ExternalCompactionId ecid, 
KeyExtent extent, long f
     var ecfs =
         new ExternalCompactionFinalState(ecid, extent, FinalState.FINISHED, 
fileSize, fileEntries);
 
-    LOG.debug("Initiating commit for external compaction: {}", ecfs);
+    LOG.trace("Initiating commit for external compaction: {} {}", ecid, ecfs);
 
     // write metadata entry
-    context.getAmple().putExternalCompactionFinalStates(List.of(ecfs));

Review Comment:
   It looks like the only other caller of this method is in `failCompactions` 
below. I'm wondering if we should use the same SharedBatchWriter for the 
failures and then remove `putExternalCompactionFinalStates` from the Ample 
interface. It's possible that someone could shutdown or kill a bunch of 
compactors (maybe via `accumulo-cluster prune`) and then the coordinator has a 
lot of failures to deal with.



-- 
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: notifications-unsubscr...@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to