[ https://issues.apache.org/jira/browse/TEPHRA-215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856391#comment-15856391 ]
ASF GitHub Bot commented on TEPHRA-215: --------------------------------------- Github user poornachandra commented on a diff in the pull request: https://github.com/apache/incubator-tephra/pull/32#discussion_r99880416 --- Diff: tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java --- @@ -18,55 +18,48 @@ package org.apache.tephra.hbase.txprune; +import com.google.common.util.concurrent.AbstractIdleService; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.TableName; -import org.apache.tephra.hbase.coprocessor.TransactionProcessor; import java.io.IOException; +import java.util.Iterator; +import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; /** * Thread that will write the the prune upper bound */ -public class PruneUpperBoundWriter { - private static final Log LOG = LogFactory.getLog(TransactionProcessor.class); +public class PruneUpperBoundWriter extends AbstractIdleService { + private static final Log LOG = LogFactory.getLog(PruneUpperBoundWriter.class); - private final TableName pruneStateTable; private final DataJanitorState dataJanitorState; - private final byte[] regionName; - private final String regionNameAsString; private final long pruneFlushInterval; - private final AtomicLong pruneUpperBound; - private final AtomicBoolean shouldFlush; + private final Queue<PruneInfo> pruneEntries; private Thread flushThread; private long lastChecked; - public PruneUpperBoundWriter(DataJanitorState dataJanitorState, TableName pruneStateTable, String regionNameAsString, - byte[] regionName, long pruneFlushInterval) { - this.pruneStateTable = pruneStateTable; + public PruneUpperBoundWriter(DataJanitorState dataJanitorState, long pruneFlushInterval) { this.dataJanitorState = dataJanitorState; - this.regionName = regionName; - this.regionNameAsString = regionNameAsString; this.pruneFlushInterval = pruneFlushInterval; - this.pruneUpperBound = new AtomicLong(); - this.shouldFlush = new AtomicBoolean(false); + this.pruneEntries = new ConcurrentLinkedQueue<>(); startFlushThread(); } - public boolean isAlive() { - return flushThread.isAlive(); + public void persistPruneEntry(byte[] regionName, long pruneUpperBound) { + pruneEntries.add(new PruneInfo(regionName, pruneUpperBound)); --- End diff -- I think it would be good to limit the number of entries in the `pruneEntries` queue. If we are over the limit then we can drop earlier elements. > Share PruneUpperBoundWriter across all TransactionProcessors on the same > region server > -------------------------------------------------------------------------------------- > > Key: TEPHRA-215 > URL: https://issues.apache.org/jira/browse/TEPHRA-215 > Project: Tephra > Issue Type: Improvement > Affects Versions: 0.11.0-incubating > Reporter: Gokul Gunasekaran > Assignee: Gokul Gunasekaran > Fix For: 0.11.0-incubating > > > Currently we start one prune upperbound writer thread per > TransactionProcessor coprocessor. Instead we should be able to share only one > thread that flushes writes to the prune state table periodically. -- This message was sent by Atlassian JIRA (v6.3.15#6346)