HBASE-12145 Fix javadoc and findbugs so new folks aren't freaked when they see them
Fix javadoc warnings. Fixup findbugs warnings mostly by adding annotations saying 'working as expected'. In RpcRetryingCallerWithReadReplicas made following change which findbugs spotted: - if (completed == null) tasks.wait(); + while (completed == null) tasks.wait(); In RecoverableZooKeeper, made all zk accesses synchronized -- we were doing it half-ways previously. In RatioBasedCompactionPolicy we were making an instance of Random on each invocation of getNextMajorCompactionTime
This commit is contained in:
parent
4ee6a73490
commit
43301167db
|
@ -823,7 +823,6 @@ class ConnectionManager {
|
|||
|
||||
/**
|
||||
* An identifier that will remain the same for a given connection.
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public String toString(){
|
||||
|
@ -2485,7 +2484,6 @@ class ConnectionManager {
|
|||
/**
|
||||
* Connects to the master to get the table descriptor.
|
||||
* @param tableName table name
|
||||
* @return
|
||||
* @throws IOException if the connection to master fails or if the table
|
||||
* is not found.
|
||||
*/
|
||||
|
|
|
@ -191,7 +191,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
/**
|
||||
* Constructor.
|
||||
* See {@link #HBaseAdmin(HConnection connection)}
|
||||
* See {@link #HBaseAdmin(Connection connection)}
|
||||
*
|
||||
* @param c Configuration object. Copied internally.
|
||||
*/
|
||||
|
@ -210,10 +210,10 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
|
||||
/**
|
||||
* Constructor for externally managed HConnections.
|
||||
* Constructor for externally managed Connections.
|
||||
* The connection to master will be created when required by admin functions.
|
||||
*
|
||||
* @param connection The HConnection instance to use
|
||||
* @param connection The Connection instance to use
|
||||
* @throws MasterNotRunningException, ZooKeeperConnectionException are not
|
||||
* thrown anymore but kept into the interface for backward api compatibility
|
||||
* @deprecated Do not use this internal ctor.
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
|||
* Similar to {@link RegionServerCallable} but for the AdminService interface. This service callable
|
||||
* assumes a Table and row and thus does region locating similar to RegionServerCallable.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD",
|
||||
justification="stub used by ipc")
|
||||
public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<T> {
|
||||
|
||||
protected final ClusterConnection connection;
|
||||
|
|
|
@ -378,6 +378,8 @@ public class RpcRetryingCallerWithReadReplicas {
|
|||
}
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE",
|
||||
justification="Is this an issue?")
|
||||
@Override
|
||||
public Result get(long timeout, TimeUnit unit)
|
||||
throws InterruptedException, ExecutionException, TimeoutException {
|
||||
|
@ -390,7 +392,7 @@ public class RpcRetryingCallerWithReadReplicas {
|
|||
}
|
||||
unit.timedWait(tasks, timeout);
|
||||
}
|
||||
|
||||
// Findbugs says this null check is redundant. Will result be set across the wait above?
|
||||
if (result != null) {
|
||||
return result;
|
||||
}
|
||||
|
@ -398,7 +400,7 @@ public class RpcRetryingCallerWithReadReplicas {
|
|||
throw exeEx;
|
||||
}
|
||||
|
||||
throw new TimeoutException();
|
||||
throw new TimeoutException("timeout=" + timeout + ", " + unit);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -416,7 +418,7 @@ public class RpcRetryingCallerWithReadReplicas {
|
|||
|
||||
public QueueingFuture take() throws InterruptedException {
|
||||
synchronized (tasks) {
|
||||
if (completed == null) tasks.wait();
|
||||
while (completed == null) tasks.wait();
|
||||
}
|
||||
return completed;
|
||||
}
|
||||
|
|
|
@ -18,17 +18,8 @@
|
|||
package org.apache.hadoop.hbase.protobuf;
|
||||
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.ListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Parser;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
|
@ -45,6 +36,7 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -56,7 +48,6 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -96,7 +87,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||
|
@ -127,13 +117,14 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaType;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaScope;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaType;
|
||||
import org.apache.hadoop.hbase.quotas.ThrottleType;
|
||||
import org.apache.hadoop.hbase.security.access.Permission;
|
||||
import org.apache.hadoop.hbase.security.access.TablePermission;
|
||||
|
@ -151,27 +142,23 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.ParameterizedType;
|
||||
import java.lang.reflect.Type;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.ListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Parser;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
/**
|
||||
* Protobufs utility.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED",
|
||||
justification="None. Address sometime.")
|
||||
public final class ProtobufUtil {
|
||||
|
||||
private ProtobufUtil() {
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -391,8 +390,14 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
|
|||
if (peer == null) {
|
||||
return false;
|
||||
}
|
||||
((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).putIfAbsent(peerId, peer);
|
||||
LOG.info("Added new peer cluster " + peer.getPeerConfig().getClusterKey());
|
||||
ReplicationPeerZKImpl previous =
|
||||
((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).putIfAbsent(peerId, peer);
|
||||
if (previous == null) {
|
||||
LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey());
|
||||
} else {
|
||||
LOG.info("Peer already present, " + previous.getPeerConfig().getClusterKey() +
|
||||
", new cluster=" + peer.getPeerConfig().getClusterKey());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -105,6 +105,8 @@ public class RecoverableZooKeeper {
|
|||
null);
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
|
||||
justification="None. Its always been this way.")
|
||||
public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
|
||||
Watcher watcher, int maxRetries, int retryIntervalMillis, String identifier)
|
||||
throws IOException {
|
||||
|
@ -690,23 +692,23 @@ public class RecoverableZooKeeper {
|
|||
return newData;
|
||||
}
|
||||
|
||||
public long getSessionId() {
|
||||
return zk == null ? null : zk.getSessionId();
|
||||
public synchronized long getSessionId() {
|
||||
return zk == null ? -1 : zk.getSessionId();
|
||||
}
|
||||
|
||||
public void close() throws InterruptedException {
|
||||
public synchronized void close() throws InterruptedException {
|
||||
if (zk != null) zk.close();
|
||||
}
|
||||
|
||||
public States getState() {
|
||||
public synchronized States getState() {
|
||||
return zk == null ? null : zk.getState();
|
||||
}
|
||||
|
||||
public ZooKeeper getZooKeeper() {
|
||||
public synchronized ZooKeeper getZooKeeper() {
|
||||
return zk;
|
||||
}
|
||||
|
||||
public byte[] getSessionPasswd() {
|
||||
public synchronized byte[] getSessionPasswd() {
|
||||
return zk == null ? null : zk.getSessionPasswd();
|
||||
}
|
||||
|
||||
|
|
|
@ -44,7 +44,8 @@ public class PrettyPrinter {
|
|||
return human.toString();
|
||||
}
|
||||
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG",
|
||||
justification="Will not overflow")
|
||||
private static String humanReadableTTL(final long interval){
|
||||
StringBuilder sb = new StringBuilder();
|
||||
int days, hours, minutes, seconds;
|
||||
|
|
|
@ -45,7 +45,6 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
protected ByteBuffer block;
|
||||
protected boolean includeMvccVersion;
|
||||
protected PrefixTreeArraySearcher ptSearcher;
|
||||
protected boolean movedToPrevious = false;
|
||||
|
||||
public PrefixTreeSeeker(boolean includeMvccVersion) {
|
||||
this.includeMvccVersion = includeMvccVersion;
|
||||
|
|
|
@ -28,13 +28,12 @@ import java.nio.ByteBuffer;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
|
@ -62,7 +61,7 @@ import com.google.common.base.Preconditions;
|
|||
* information from the block index are required to read a block.
|
||||
* <li>In version 2 a block is structured as follows:
|
||||
* <ul>
|
||||
* <li>header (see {@link Writer#finishBlock()})
|
||||
* <li>header (see Writer#finishBlock())
|
||||
* <ul>
|
||||
* <li>Magic record identifying the block type (8 bytes)
|
||||
* <li>Compressed block size, excluding header, including checksum (4 bytes)
|
||||
|
|
|
@ -538,7 +538,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
@Override
|
||||
public void processHistogram(MetricName name, Histogram histogram, PrintStream stream) {
|
||||
super.processHistogram(name, histogram, stream);
|
||||
stream.printf(Locale.getDefault(), " count = %d\n", histogram.count());
|
||||
stream.printf(Locale.getDefault(), " count = %d%n", histogram.count());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -218,6 +218,8 @@ public class HFileOutputFormat2
|
|||
* @return A WriterLength, containing a new StoreFile.Writer.
|
||||
* @throws IOException
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="BX_UNBOXING_IMMEDIATELY_REBOXED",
|
||||
justification="Not important")
|
||||
private WriterLength getNewWriter(byte[] family, Configuration conf)
|
||||
throws IOException {
|
||||
WriterLength wl = new WriterLength();
|
||||
|
|
|
@ -65,7 +65,7 @@ import org.apache.hadoop.util.StringUtils;
|
|||
* class ExampleTIF extends TableInputFormatBase implements JobConfigurable {
|
||||
*
|
||||
* public void configure(JobConf job) {
|
||||
* Connection connection =
|
||||
* Connection connection =
|
||||
* ConnectionFactory.createConnection(HBaseConfiguration.create(job));
|
||||
* TableName tableName = TableName.valueOf("exampleTable");
|
||||
* // mandatory
|
||||
|
@ -91,7 +91,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
|
||||
final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
|
||||
|
||||
/** Holds the details for the internal scanner.
|
||||
/** Holds the details for the internal scanner.
|
||||
*
|
||||
* @see Scan */
|
||||
private Scan scan = null;
|
||||
|
@ -106,10 +106,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
/** The reverse DNS lookup cache mapping: IPAddress => HostName */
|
||||
private HashMap<InetAddress, String> reverseDNSCacheMap =
|
||||
new HashMap<InetAddress, String>();
|
||||
|
||||
/** The NameServer address */
|
||||
private String nameServer = null;
|
||||
|
||||
|
||||
/**
|
||||
* Builds a {@link TableRecordReader}. If no {@link TableRecordReader} was provided, uses
|
||||
* the default.
|
||||
|
@ -161,9 +158,6 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
if (table == null) {
|
||||
throw new IOException("No table was provided.");
|
||||
}
|
||||
// Get the name server address and the default value is null.
|
||||
this.nameServer =
|
||||
context.getConfiguration().get("hbase.nameserver.address", null);
|
||||
|
||||
RegionSizeCalculator sizeCalculator = new RegionSizeCalculator((HTable) table);
|
||||
|
||||
|
@ -229,7 +223,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
}
|
||||
return splits;
|
||||
}
|
||||
|
||||
|
||||
public String reverseDNS(InetAddress ipAddress) throws NamingException, UnknownHostException {
|
||||
String hostName = this.reverseDNSCacheMap.get(ipAddress);
|
||||
if (hostName == null) {
|
||||
|
@ -277,8 +271,8 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
|
||||
/**
|
||||
* Allows subclasses to get the {@link HTable}.
|
||||
*
|
||||
* @deprecated Use {@link #getTable()} and {@link #getRegionLocator()} instead.
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
@Deprecated
|
||||
protected HTable getHTable() {
|
||||
|
@ -287,8 +281,8 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
|
||||
/**
|
||||
* Allows subclasses to set the {@link HTable}.
|
||||
*
|
||||
* @param table The {@link HTable} to get the data from.
|
||||
*
|
||||
* @param table The table to get the data from.
|
||||
* @deprecated Use {@link #initializeTable(Connection, TableName)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
|
|
|
@ -55,7 +55,7 @@ extends RecordReader<ImmutableBytesWritable, Result> {
|
|||
/**
|
||||
* Sets the HBase table.
|
||||
*
|
||||
* @param htable The {@link HTable} to scan.
|
||||
* @param htable The table to scan.
|
||||
* @deprecated Use setTable() instead.
|
||||
*/
|
||||
@Deprecated
|
||||
|
|
|
@ -540,6 +540,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
|||
}
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NM_FIELD_NAMING_CONVENTION",
|
||||
justification="Mistake. Too disruptive to change now")
|
||||
public static Action NullAction = new Action(Type.NULL);
|
||||
|
||||
public void doAction(Action action) {
|
||||
|
@ -785,6 +787,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
|||
}
|
||||
};
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SBSC_USE_STRINGBUFFER_CONCATENATION",
|
||||
justification="Not important but should be fixed")
|
||||
@Override
|
||||
public String toString() {
|
||||
String desc = "Cluster{" +
|
||||
|
|
|
@ -405,6 +405,8 @@ public class CompactSplitThread implements CompactionRequestor {
|
|||
return this.regionSplitLimit;
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
|
||||
justification="Contrived use of compareTo")
|
||||
private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
|
||||
private final Store store;
|
||||
private final HRegion region;
|
||||
|
|
|
@ -29,9 +29,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
@Deprecated
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
|
||||
justification="Temporary glue. To be removed")
|
||||
public class RowTooBigException extends org.apache.hadoop.hbase.client.RowTooBigException {
|
||||
|
||||
public RowTooBigException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
|
|
|
@ -27,13 +27,12 @@ import java.util.Random;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
|
@ -321,6 +320,15 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
|
|||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used calculation jitter
|
||||
*/
|
||||
private final Random random = new Random();
|
||||
|
||||
/**
|
||||
* @param filesToCompact
|
||||
* @return When to run next major compaction
|
||||
*/
|
||||
public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
|
||||
// default = 24hrs
|
||||
long ret = comConf.getMajorCompactionPeriod();
|
||||
|
@ -332,10 +340,15 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
|
|||
// deterministic jitter avoids a major compaction storm on restart
|
||||
Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
|
||||
if (seed != null) {
|
||||
double rnd = (new Random(seed)).nextDouble();
|
||||
// Synchronized to ensure one user of random instance at a time.
|
||||
double rnd = -1;
|
||||
synchronized (this) {
|
||||
this.random.setSeed(seed);
|
||||
rnd = this.random.nextDouble();
|
||||
}
|
||||
ret += jitter - Math.round(2L * jitter * rnd);
|
||||
} else {
|
||||
ret = 0; // no storefiles == no major compaction
|
||||
ret = 0; // If seed is null, then no storefiles == no major compaction
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -735,6 +735,8 @@ public class ReplicationSource extends Thread
|
|||
* @return true if we're done with the current file, false if we should
|
||||
* continue trying to read from it
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
|
||||
justification="Yeah, this is how it works")
|
||||
protected boolean processEndOfFile() {
|
||||
if (this.queue.size() != 0) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
|
|
@ -220,6 +220,8 @@ public class SchemaResource extends ResourceBase {
|
|||
return update(model, false, uriInfo);
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
|
||||
justification="Expected")
|
||||
@DELETE
|
||||
public Response delete(final @Context UriInfo uriInfo) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
|
|
@ -144,7 +144,7 @@ public class RegionSplitter {
|
|||
* {@link HexStringSplit} to partition their table and set it as default, but
|
||||
* provided this for your custom algorithm. To use, create a new derived class
|
||||
* from this interface and call {@link RegionSplitter#createPresplitTable} or
|
||||
* {@link RegionSplitter#rollingSplit(String, SplitAlgorithm, Configuration)} with the
|
||||
* RegionSplitter#rollingSplit(TableName, SplitAlgorithm, Configuration) with the
|
||||
* argument splitClassName giving the name of your class.
|
||||
*/
|
||||
public interface SplitAlgorithm {
|
||||
|
|
Loading…
Reference in New Issue