Author: gdusbabek
Date: Wed Aug 18 19:15:24 2010
New Revision: 986863
URL: http://svn.apache.org/viewvc?rev=986863&view=rev
Log:
remove underscores from CFS members. patch by gdusbabek, reviewed by jbellis.
CASSANDRA-1385
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java
cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
Modified: cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java?rev=986863&r1=986862&r2=986863&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java Wed
Aug 18 19:15:24 2010
@@ -122,7 +122,7 @@ public class BinaryMemtable implements I
{
logger.info("Writing " + this);
String path = cfs.getFlushPath();
- SSTableWriter writer = new SSTableWriter(path, sortedKeys.size(),
cfs.metadata, cfs.partitioner_);
+ SSTableWriter writer = new SSTableWriter(path, sortedKeys.size(),
cfs.metadata, cfs.partitioner);
for (DecoratedKey key : sortedKeys)
{
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=986863&r1=986862&r2=986863&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed
Aug 18 19:15:24 2010
@@ -71,7 +71,7 @@ import javax.management.ObjectName;
public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
- private static Logger logger_ =
LoggerFactory.getLogger(ColumnFamilyStore.class);
+ private static Logger logger =
LoggerFactory.getLogger(ColumnFamilyStore.class);
/*
* submitFlush first puts [Binary]Memtable.getSortedContents on the
flushSorter executor,
@@ -88,21 +88,21 @@ public class ColumnFamilyStore implement
* which is necessary for replay in case of a restart since CommitLog
assumes that when onMF is
* called, all data up to the given context has been persisted to SSTables.
*/
- private static ExecutorService flushSorter_
+ private static ExecutorService flushSorter
= new JMXEnabledThreadPoolExecutor(1,
Runtime.getRuntime().availableProcessors(),
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new
LinkedBlockingQueue<Runnable>(Runtime.getRuntime().availableProcessors()),
new
NamedThreadFactory("FLUSH-SORTER-POOL"));
- private static ExecutorService flushWriter_
+ private static ExecutorService flushWriter
= new JMXEnabledThreadPoolExecutor(1,
DatabaseDescriptor.getFlushWriters(),
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new
LinkedBlockingQueue<Runnable>(DatabaseDescriptor.getFlushWriters()),
new
NamedThreadFactory("FLUSH-WRITER-POOL"));
- private static ExecutorService postFlushExecutor_ = new
JMXEnabledThreadPoolExecutor("MEMTABLE-POST-FLUSHER");
+ private static ExecutorService postFlushExecutor = new
JMXEnabledThreadPoolExecutor("MEMTABLE-POST-FLUSHER");
private static final FilenameFilter DB_NAME_FILTER = new FilenameFilter()
{
@@ -114,45 +114,45 @@ public class ColumnFamilyStore implement
private Set<Memtable> memtablesPendingFlush = new
ConcurrentSkipListSet<Memtable>();
- public final String table_;
- public final String columnFamily_;
- public final IPartitioner partitioner_;
+ public final String table;
+ public final String columnFamily;
+ public final IPartitioner partitioner;
private final String mbeanName;
private volatile int memtableSwitchCount = 0;
/* This is used to generate the next index for a SSTable */
- private AtomicInteger fileIndexGenerator_ = new AtomicInteger(0);
+ private AtomicInteger fileIndexGenerator = new AtomicInteger(0);
/* active memtable associated with this ColumnFamilyStore. */
- private Memtable memtable_;
+ private Memtable memtable;
- private final Map<byte[], ColumnFamilyStore> indexedColumns_;
+ private final Map<byte[], ColumnFamilyStore> indexedColumns;
// TODO binarymemtable ops are not threadsafe (do they need to be?)
- private AtomicReference<BinaryMemtable> binaryMemtable_;
+ private AtomicReference<BinaryMemtable> binaryMemtable;
/* SSTables on disk for this column family */
- private SSTableTracker ssTables_;
+ private SSTableTracker ssTables;
- private LatencyTracker readStats_ = new LatencyTracker();
- private LatencyTracker writeStats_ = new LatencyTracker();
+ private LatencyTracker readStats = new LatencyTracker();
+ private LatencyTracker writeStats = new LatencyTracker();
final CFMetaData metadata;
private ColumnFamilyStore(String table, String columnFamilyName,
IPartitioner partitioner, int generation, CFMetaData metadata)
{
assert metadata != null : "null metadata for " + table + ":" +
columnFamilyName;
- table_ = table;
- columnFamily_ = columnFamilyName;
+ this.table = table;
+ columnFamily = columnFamilyName;
this.metadata = metadata;
- this.partitioner_ = partitioner;
- fileIndexGenerator_.set(generation);
- memtable_ = new Memtable(this, partitioner_);
- binaryMemtable_ = new AtomicReference<BinaryMemtable>(new
BinaryMemtable(this));
+ this.partitioner = partitioner;
+ fileIndexGenerator.set(generation);
+ memtable = new Memtable(this, this.partitioner);
+ binaryMemtable = new AtomicReference<BinaryMemtable>(new
BinaryMemtable(this));
- if (logger_.isDebugEnabled())
- logger_.debug("Starting CFS {}", columnFamily_);
+ if (logger.isDebugEnabled())
+ logger.debug("Starting CFS {}", columnFamily);
// scan for data files corresponding to this CF
List<File> sstableFiles = new ArrayList<File>();
@@ -177,19 +177,19 @@ public class ColumnFamilyStore implement
SSTableReader sstable;
try
{
- sstable =
SSTableReader.open(Descriptor.fromFilename(filename), metadata, partitioner_);
+ sstable =
SSTableReader.open(Descriptor.fromFilename(filename), metadata,
this.partitioner);
}
catch (IOException ex)
{
- logger_.error("Corrupt file " + filename + "; skipped", ex);
+ logger.error("Corrupt file " + filename + "; skipped", ex);
continue;
}
sstables.add(sstable);
}
- ssTables_ = new SSTableTracker(table, columnFamilyName);
- ssTables_.add(sstables);
+ ssTables = new SSTableTracker(table, columnFamilyName);
+ ssTables.add(sstables);
- indexedColumns_ = new TreeMap<byte[],
ColumnFamilyStore>(BytesType.instance);
+ indexedColumns = new TreeMap<byte[],
ColumnFamilyStore>(BytesType.instance);
for (Map.Entry<byte[], ColumnDefinition> entry :
metadata.column_metadata.entrySet())
{
byte[] column = entry.getKey();
@@ -197,7 +197,7 @@ public class ColumnFamilyStore implement
if (info.index_type == null)
continue;
- String indexedCfName = columnFamily_ + "." + (info.index_name ==
null ? FBUtilities.bytesToHex(column) : info.index_name);
+ String indexedCfName = columnFamily + "." + (info.index_name ==
null ? FBUtilities.bytesToHex(column) : info.index_name);
IPartitioner rowPartitioner = StorageService.getPartitioner();
AbstractType columnComparator = (rowPartitioner instanceof
OrderPreservingPartitioner || rowPartitioner instanceof ByteOrderedPartitioner)
? BytesType.instance
@@ -220,11 +220,11 @@ public class ColumnFamilyStore implement
indexedCfName,
new LocalPartitioner(metadata.column_metadata.get(column).validator),
indexedCfMetadata);
- indexedColumns_.put(column, indexedCfs);
+ indexedColumns.put(column, indexedCfs);
}
- String type = this.partitioner_ instanceof LocalPartitioner ?
"IndexColumnFamilies" : "ColumnFamilies";
- mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" +
table_ + ",columnfamily=" + columnFamily_;
+ String type = this.partitioner instanceof LocalPartitioner ?
"IndexColumnFamilies" : "ColumnFamilies";
+ mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" +
this.table + ",columnfamily=" + columnFamily;
try
{
MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@@ -247,20 +247,20 @@ public class ColumnFamilyStore implement
ObjectName nameObj = new ObjectName(mbeanName);
if (mbs.isRegistered(nameObj))
mbs.unregisterMBean(nameObj);
- for (ColumnFamilyStore index : indexedColumns_.values())
+ for (ColumnFamilyStore index : indexedColumns.values())
index.unregisterMBean();
}
catch (Exception e)
{
// this shouldn't block anything.
- logger_.warn(e.getMessage(), e);
+ logger.warn(e.getMessage(), e);
}
}
public long getMinRowSize()
{
long min = 0;
- for (SSTableReader sstable : ssTables_)
+ for (SSTableReader sstable : ssTables)
{
if (min == 0 || sstable.getEstimatedRowSize().min() < min)
min = sstable.getEstimatedRowSize().min();
@@ -271,7 +271,7 @@ public class ColumnFamilyStore implement
public long getMaxRowSize()
{
long max = 0;
- for (SSTableReader sstable : ssTables_)
+ for (SSTableReader sstable : ssTables)
{
if (sstable.getEstimatedRowSize().max() > max)
max = sstable.getEstimatedRowSize().max();
@@ -283,7 +283,7 @@ public class ColumnFamilyStore implement
{
long sum = 0;
long count = 0;
- for (SSTableReader sstable : ssTables_)
+ for (SSTableReader sstable : ssTables)
{
sum += sstable.getEstimatedRowSize().median();
count++;
@@ -295,7 +295,7 @@ public class ColumnFamilyStore implement
{
long sum = 0;
int count = 0;
- for (SSTableReader sstable : ssTables_)
+ for (SSTableReader sstable : ssTables)
{
sum += sstable.getEstimatedColumnCount().median();
count++;
@@ -356,7 +356,7 @@ public class ColumnFamilyStore implement
String basePath = matcher.group(1);
if (!new File(basePath + "-Data.db").exists())
{
- logger_.info(String.format("Removing orphan %s",
file.getAbsolutePath()));
+ logger.info(String.format("Removing orphan %s",
file.getAbsolutePath()));
try
{
FileUtils.deleteWithConfirm(file);
@@ -439,7 +439,7 @@ public class ColumnFamilyStore implement
*/
public String getColumnFamilyName()
{
- return columnFamily_;
+ return columnFamily;
}
private static String getColumnFamilyFromFileName(String filename)
@@ -460,7 +460,7 @@ public class ColumnFamilyStore implement
public String getFlushPath()
{
long guessedSize = 2 * DatabaseDescriptor.getMemtableThroughput() *
1024*1024; // 2* adds room for keys, column indexes
- String location =
DatabaseDescriptor.getDataFileLocationForTable(table_, guessedSize);
+ String location =
DatabaseDescriptor.getDataFileLocationForTable(table, guessedSize);
if (location == null)
throw new RuntimeException("Insufficient disk space to flush");
return getTempSSTablePath(location);
@@ -469,9 +469,9 @@ public class ColumnFamilyStore implement
public String getTempSSTablePath(String directory)
{
Descriptor desc = new Descriptor(new File(directory),
- table_,
- columnFamily_,
-
fileIndexGenerator_.incrementAndGet(),
+ table,
+ columnFamily,
+
fileIndexGenerator.incrementAndGet(),
true);
return desc.filenameFor("Data.db");
}
@@ -490,23 +490,23 @@ public class ColumnFamilyStore implement
if (oldMemtable.isFrozen())
return null;
- assert memtable_ == oldMemtable;
- memtable_.freeze();
+ assert memtable == oldMemtable;
+ memtable.freeze();
final CommitLogSegment.CommitLogContext ctx = writeCommitLog ?
CommitLog.instance().getContext() : null;
- logger_.info("switching in a fresh Memtable for " + columnFamily_
+ " at " + ctx);
+ logger.info("switching in a fresh Memtable for " + columnFamily +
" at " + ctx);
// submit the memtable for any indexed sub-cfses, and our own
- final CountDownLatch latch = new CountDownLatch(1 +
indexedColumns_.size());
- for (ColumnFamilyStore cfs :
Iterables.concat(indexedColumns_.values(), Collections.singleton(this)))
+ final CountDownLatch latch = new CountDownLatch(1 +
indexedColumns.size());
+ for (ColumnFamilyStore cfs :
Iterables.concat(indexedColumns.values(), Collections.singleton(this)))
{
- submitFlush(cfs.memtable_, latch);
- cfs.memtable_ = new Memtable(cfs, cfs.partitioner_);
+ submitFlush(cfs.memtable, latch);
+ cfs.memtable = new Memtable(cfs, cfs.partitioner);
}
// when all the memtables have been written, including for
indexes, mark the flush in the commitlog header.
// a second executor makes sure the onMemtableFlushes get called
in the right order,
// while keeping the wait-for-flush (future.get) out of anything
latency-sensitive.
- return postFlushExecutor_.submit(new WrappedRunnable()
+ return postFlushExecutor.submit(new WrappedRunnable()
{
public void runMayThrow() throws InterruptedException,
IOException
{
@@ -515,7 +515,7 @@ public class ColumnFamilyStore implement
{
// if we're not writing to the commit log, we are
replaying the log, so marking
// the log header with "you can discard anything
written before the context" is not valid
- logger_.debug("Discarding {}", metadata.cfId);
+ logger.debug("Discarding {}", metadata.cfId);
CommitLog.instance().discardCompletedSegments(metadata.cfId, ctx);
}
}
@@ -534,22 +534,22 @@ public class ColumnFamilyStore implement
void switchBinaryMemtable(DecoratedKey key, byte[] buffer)
{
- binaryMemtable_.set(new BinaryMemtable(this));
- binaryMemtable_.get().put(key, buffer);
+ binaryMemtable.set(new BinaryMemtable(this));
+ binaryMemtable.get().put(key, buffer);
}
public void forceFlushIfExpired()
{
- if (memtable_.isExpired())
+ if (memtable.isExpired())
forceFlush();
}
public Future<?> forceFlush()
{
- if (memtable_.isClean())
+ if (memtable.isClean())
return null;
- return maybeSwitchMemtable(memtable_, true);
+ return maybeSwitchMemtable(memtable, true);
}
public void forceBlockingFlush() throws ExecutionException,
InterruptedException
@@ -561,10 +561,10 @@ public class ColumnFamilyStore implement
public void forceFlushBinary()
{
- if (binaryMemtable_.get().isClean())
+ if (binaryMemtable.get().isClean())
return;
- submitFlush(binaryMemtable_.get(), new CountDownLatch(1));
+ submitFlush(binaryMemtable.get(), new CountDownLatch(1));
}
/**
@@ -578,11 +578,11 @@ public class ColumnFamilyStore implement
{
long start = System.nanoTime();
- boolean flushRequested = memtable_.isThresholdViolated();
- memtable_.put(key, columnFamily);
- writeStats_.addNano(System.nanoTime() - start);
+ boolean flushRequested = memtable.isThresholdViolated();
+ memtable.put(key, columnFamily);
+ writeStats.addNano(System.nanoTime() - start);
- return flushRequested ? memtable_ : null;
+ return flushRequested ? memtable : null;
}
/*
@@ -593,8 +593,8 @@ public class ColumnFamilyStore implement
void applyBinary(DecoratedKey key, byte[] buffer)
{
long start = System.nanoTime();
- binaryMemtable_.get().put(key, buffer);
- writeStats_.addNano(System.nanoTime() - start);
+ binaryMemtable.get().put(key, buffer);
+ writeStats.addNano(System.nanoTime() - start);
}
public static ColumnFamily removeDeletedCF(ColumnFamily cf, int gcBefore)
@@ -704,7 +704,7 @@ public class ColumnFamilyStore implement
*/
public void addSSTable(SSTableReader sstable)
{
- ssTables_.add(Arrays.asList(sstable));
+ ssTables.add(Arrays.asList(sstable));
CompactionManager.instance.submitMinorIfNeeded(this);
}
@@ -748,22 +748,22 @@ public class ColumnFamilyStore implement
public Table getTable()
{
- return Table.open(table_);
+ return Table.open(table);
}
void markCompacted(Collection<SSTableReader> sstables)
{
- ssTables_.markCompacted(sstables);
+ ssTables.markCompacted(sstables);
}
boolean isCompleteSSTables(Collection<SSTableReader> sstables)
{
- return ssTables_.getSSTables().equals(new
HashSet<SSTableReader>(sstables));
+ return ssTables.getSSTables().equals(new
HashSet<SSTableReader>(sstables));
}
void replaceCompactedSSTables(Collection<SSTableReader> sstables,
Iterable<SSTableReader> replacements)
{
- ssTables_.replace(sstables, replacements);
+ ssTables.replace(sstables, replacements);
}
/**
@@ -777,8 +777,8 @@ public class ColumnFamilyStore implement
*/
void submitFlush(IFlushable flushable, CountDownLatch latch)
{
- logger_.info("Enqueuing flush of {}", flushable);
- flushable.flushAndSignal(latch, flushSorter_, flushWriter_);
+ logger.info("Enqueuing flush of {}", flushable);
+ flushable.flushAndSignal(latch, flushSorter, flushWriter);
}
public int getMemtableColumnsCount()
@@ -811,7 +811,7 @@ public class ColumnFamilyStore implement
Table.flusherLock.readLock().lock();
try
{
- return memtable_;
+ return memtable;
}
finally
{
@@ -821,32 +821,32 @@ public class ColumnFamilyStore implement
public Collection<SSTableReader> getSSTables()
{
- return ssTables_.getSSTables();
+ return ssTables.getSSTables();
}
public long getReadCount()
{
- return readStats_.getOpCount();
+ return readStats.getOpCount();
}
public double getRecentReadLatencyMicros()
{
- return readStats_.getRecentLatencyMicros();
+ return readStats.getRecentLatencyMicros();
}
public long[] getLifetimeReadLatencyHistogramMicros()
{
- return readStats_.getTotalLatencyHistogramMicros();
+ return readStats.getTotalLatencyHistogramMicros();
}
public long[] getRecentReadLatencyHistogramMicros()
{
- return readStats_.getRecentLatencyHistogramMicros();
+ return readStats.getRecentLatencyHistogramMicros();
}
public long getTotalReadLatencyMicros()
{
- return readStats_.getTotalLatencyMicros();
+ return readStats.getTotalLatencyMicros();
}
// TODO this actually isn't a good meature of pending tasks
@@ -857,27 +857,27 @@ public class ColumnFamilyStore implement
public long getWriteCount()
{
- return writeStats_.getOpCount();
+ return writeStats.getOpCount();
}
public long getTotalWriteLatencyMicros()
{
- return writeStats_.getTotalLatencyMicros();
+ return writeStats.getTotalLatencyMicros();
}
public double getRecentWriteLatencyMicros()
{
- return writeStats_.getRecentLatencyMicros();
+ return writeStats.getRecentLatencyMicros();
}
public long[] getLifetimeWriteLatencyHistogramMicros()
{
- return writeStats_.getTotalLatencyHistogramMicros();
+ return writeStats.getTotalLatencyHistogramMicros();
}
public long[] getRecentWriteLatencyHistogramMicros()
{
- return writeStats_.getRecentLatencyHistogramMicros();
+ return writeStats.getRecentLatencyHistogramMicros();
}
public ColumnFamily getColumnFamily(DecoratedKey key, QueryPath path,
byte[] start, byte[] finish, List<byte[]> bitmasks, boolean reversed, int limit)
@@ -903,24 +903,24 @@ public class ColumnFamilyStore implement
private ColumnFamily cacheRow(DecoratedKey key)
{
ColumnFamily cached;
- if ((cached = ssTables_.getRowCache().get(key)) == null)
+ if ((cached = ssTables.getRowCache().get(key)) == null)
{
- cached = getTopLevelColumns(QueryFilter.getIdentityFilter(key, new
QueryPath(columnFamily_)), Integer.MIN_VALUE);
+ cached = getTopLevelColumns(QueryFilter.getIdentityFilter(key, new
QueryPath(columnFamily)), Integer.MIN_VALUE);
if (cached == null)
return null;
- ssTables_.getRowCache().put(key, cached);
+ ssTables.getRowCache().put(key, cached);
}
return cached;
}
private ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore)
{
- assert columnFamily_.equals(filter.getColumnFamilyName());
+ assert columnFamily.equals(filter.getColumnFamilyName());
long start = System.nanoTime();
try
{
- if (ssTables_.getRowCache().getCapacity() == 0)
+ if (ssTables.getRowCache().getCapacity() == 0)
{
ColumnFamily cf = getTopLevelColumns(filter, gcBefore);
// TODO this is necessary because when we collate supercolumns
together, we don't check
@@ -936,7 +936,7 @@ public class ColumnFamilyStore implement
}
finally
{
- readStats_.addNano(System.nanoTime() - start);
+ readStats.addNano(System.nanoTime() - start);
}
}
@@ -1025,7 +1025,7 @@ public class ColumnFamilyStore implement
}
/* add the SSTables on disk */
- for (SSTableReader sstable : ssTables_)
+ for (SSTableReader sstable : ssTables)
{
iter = filter.getSSTableColumnIterator(sstable);
if (iter.getColumnFamily() != null)
@@ -1058,7 +1058,7 @@ public class ColumnFamilyStore implement
}
catch (Throwable th)
{
- logger_.error("error closing " + ci, th);
+ logger.error("error closing " + ci, th);
}
}
}
@@ -1084,13 +1084,13 @@ public class ColumnFamilyStore implement
DecoratedKey startWith = new DecoratedKey(range.left, (byte[])null);
DecoratedKey stopAt = new DecoratedKey(range.right, (byte[])null);
- QueryFilter filter = new QueryFilter(null, new
QueryPath(columnFamily_, superColumn, null), columnFilter);
+ QueryFilter filter = new QueryFilter(null, new QueryPath(columnFamily,
superColumn, null), columnFilter);
Collection<Memtable> memtables = new ArrayList<Memtable>();
memtables.add(getMemtableThreadSafe());
memtables.addAll(memtablesPendingFlush);
Collection<SSTableReader> sstables = new ArrayList<SSTableReader>();
- Iterables.addAll(sstables, ssTables_);
+ Iterables.addAll(sstables, ssTables);
RowIterator iterator = RowIteratorFactory.getIterator(memtables,
sstables, startWith, stopAt, filter, getComparator(), this);
@@ -1110,8 +1110,8 @@ public class ColumnFamilyStore implement
if(range instanceof Bounds || !first || !key.equals(startWith))
{
rows.add(current);
- if (logger_.isDebugEnabled())
- logger_.debug("scanned " + key);
+ if (logger.isDebugEnabled())
+ logger.debug("scanned " + key);
}
first = false;
@@ -1140,7 +1140,7 @@ public class ColumnFamilyStore implement
IndexExpression first = highestSelectivityPredicate(clause);
ColumnFamilyStore indexCFS =
getIndexedColumnFamilyStore(first.column_name);
assert indexCFS != null;
- DecoratedKey indexKey = indexCFS.partitioner_.decorateKey(first.value);
+ DecoratedKey indexKey = indexCFS.partitioner.decorateKey(first.value);
List<Row> rows = new ArrayList<Row>();
byte[] startKey = clause.start_key;
@@ -1171,12 +1171,12 @@ public class ColumnFamilyStore implement
continue;
dataKey = column.name();
n++;
- DecoratedKey dk = partitioner_.decorateKey(dataKey);
- if (!range.right.equals(partitioner_.getMinimumToken()) &&
range.right.compareTo(dk.token) < 0)
+ DecoratedKey dk = partitioner.decorateKey(dataKey);
+ if (!range.right.equals(partitioner.getMinimumToken()) &&
range.right.compareTo(dk.token) < 0)
break outer;
if (!range.contains(dk.token))
continue;
- ColumnFamily data = getColumnFamily(new QueryFilter(dk, new
QueryPath(columnFamily_), dataFilter));
+ ColumnFamily data = getColumnFamily(new QueryFilter(dk, new
QueryPath(columnFamily), dataFilter));
if (satisfies(data, clause, first))
rows.add(new Row(dk, data));
if (rows.size() == clause.count)
@@ -1271,14 +1271,14 @@ public class ColumnFamilyStore implement
throw new AssertionError(e);
}
- for (SSTableReader ssTable : ssTables_)
+ for (SSTableReader ssTable : ssTables)
{
try
{
// mkdir
File sourceFile = new File(ssTable.getFilename());
File dataDirectory =
sourceFile.getParentFile().getParentFile();
- String snapshotDirectoryPath =
Table.getSnapshotPath(dataDirectory.getAbsolutePath(), table_, snapshotName);
+ String snapshotDirectoryPath =
Table.getSnapshotPath(dataDirectory.getAbsolutePath(), table, snapshotName);
FileUtils.createDirectory(snapshotDirectoryPath);
// hard links
@@ -1292,8 +1292,8 @@ public class ColumnFamilyStore implement
sourceFile = new File(ssTable.filterFilename());
targetLink = new File(snapshotDirectoryPath,
sourceFile.getName());
FileUtils.createHardLink(sourceFile, targetLink);
- if (logger_.isDebugEnabled())
- logger_.debug("Snapshot for " + table_ + " table data file
" + sourceFile.getAbsolutePath() +
+ if (logger.isDebugEnabled())
+ logger.debug("Snapshot for " + table + " table data file "
+ sourceFile.getAbsolutePath() +
" created as " + targetLink.getAbsolutePath());
}
catch (IOException e)
@@ -1308,12 +1308,12 @@ public class ColumnFamilyStore implement
{
if (metadata.preloadRowCache)
{
- logger_.debug(String.format("Loading cache for
keyspace/columnfamily %s/%s", table_, columnFamily_));
+ logger.debug(String.format("Loading cache for
keyspace/columnfamily %s/%s", table, columnFamily));
int ROWS = 4096;
- Token min = partitioner_.getMinimumToken();
+ Token min = partitioner.getMinimumToken();
Token start = min;
long i = 0;
- while (i < ssTables_.getRowCache().getCapacity())
+ while (i < ssTables.getRowCache().getCapacity())
{
List<Row> result;
try
@@ -1326,47 +1326,47 @@ public class ColumnFamilyStore implement
}
for (Row row : result)
- ssTables_.getRowCache().put(row.key, row.cf);
+ ssTables.getRowCache().put(row.key, row.cf);
i += result.size();
if (result.size() < ROWS)
break;
- start = partitioner_.getToken(result.get(ROWS - 1).key.key);
+ start = partitioner.getToken(result.get(ROWS - 1).key.key);
}
- logger_.info(String.format("Loaded %s rows into the %s cache", i,
columnFamily_));
+ logger.info(String.format("Loaded %s rows into the %s cache", i,
columnFamily));
}
}
public boolean hasUnreclaimedSpace()
{
- return ssTables_.getLiveSize() < ssTables_.getTotalSize();
+ return ssTables.getLiveSize() < ssTables.getTotalSize();
}
public long getTotalDiskSpaceUsed()
{
- return ssTables_.getTotalSize();
+ return ssTables.getTotalSize();
}
public long getLiveDiskSpaceUsed()
{
- return ssTables_.getLiveSize();
+ return ssTables.getLiveSize();
}
public int getLiveSSTableCount()
{
- return ssTables_.size();
+ return ssTables.size();
}
/** raw cached row -- does not fetch the row if it is not present. not
counted in cache statistics. */
public ColumnFamily getRawCachedRow(DecoratedKey key)
{
- return ssTables_.getRowCache().getCapacity() == 0 ? null :
ssTables_.getRowCache().getInternal(key);
+ return ssTables.getRowCache().getCapacity() == 0 ? null :
ssTables.getRowCache().getInternal(key);
}
void invalidateCachedRow(DecoratedKey key)
{
- ssTables_.getRowCache().remove(key);
+ ssTables.getRowCache().remove(key);
}
public void forceMajorCompaction()
@@ -1376,12 +1376,12 @@ public class ColumnFamilyStore implement
public void invalidateRowCache()
{
- ssTables_.getRowCache().clear();
+ ssTables.getRowCache().clear();
}
public int getKeyCacheSize()
{
- return ssTables_.getKeyCache().getCapacity();
+ return ssTables.getKeyCache().getCapacity();
}
public static Iterable<ColumnFamilyStore> all()
@@ -1412,8 +1412,8 @@ public class ColumnFamilyStore implement
*/
void clearUnsafe()
{
- memtable_.clearUnsafe();
- ssTables_.clearUnsafe();
+ memtable.clearUnsafe();
+ ssTables.clearUnsafe();
}
@@ -1450,7 +1450,7 @@ public class ColumnFamilyStore implement
// putting markCompacted on the commitlogUpdater thread
ensures it will run
// after any compactions that were in progress when truncate
was called, are finished
List<SSTableReader> truncatedSSTables = new
ArrayList<SSTableReader>();
- for (SSTableReader sstable : ssTables_.getSSTables())
+ for (SSTableReader sstable : ssTables.getSSTables())
{
if (!sstable.newSince(truncatedAt))
truncatedSSTables.add(sstable);
@@ -1462,12 +1462,12 @@ public class ColumnFamilyStore implement
}
};
- return postFlushExecutor_.submit(runnable);
+ return postFlushExecutor.submit(runnable);
}
public static Future<?> submitPostFlush(Runnable runnable)
{
- return postFlushExecutor_.submit(runnable);
+ return postFlushExecutor.submit(runnable);
}
public long getBloomFilterFalsePositives()
@@ -1520,30 +1520,30 @@ public class ColumnFamilyStore implement
public Set<byte[]> getIndexedColumns()
{
- return indexedColumns_.keySet();
+ return indexedColumns.keySet();
}
public ColumnFamilyStore getIndexedColumnFamilyStore(byte[] column)
{
- return indexedColumns_.get(column);
+ return indexedColumns.get(column);
}
public ColumnFamily newIndexedColumnFamily(byte[] column)
{
- return ColumnFamily.create(indexedColumns_.get(column).metadata);
+ return ColumnFamily.create(indexedColumns.get(column).metadata);
}
public DecoratedKey<LocalToken> getIndexKeyFor(byte[] name, byte[] value)
{
- return indexedColumns_.get(name).partitioner_.decorateKey(value);
+ return indexedColumns.get(name).partitioner.decorateKey(value);
}
@Override
public String toString()
{
return "ColumnFamilyStore(" +
- "table='" + table_ + '\'' +
- ", columnFamily='" + columnFamily_ + '\'' +
+ "table='" + table + '\'' +
+ ", columnFamily='" + columnFamily + '\'' +
')';
}
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java?rev=986863&r1=986862&r2=986863&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java Wed
Aug 18 19:15:24 2010
@@ -140,7 +140,7 @@ public class CompactionManager implement
logger.debug("Compaction is currently disabled.");
return 0;
}
- logger.debug("Checking to see if compaction of " +
cfs.columnFamily_ + " would be useful");
+ logger.debug("Checking to see if compaction of " +
cfs.columnFamily + " would be useful");
Set<List<SSTableReader>> buckets =
getBuckets(convertSSTablesToPairs(cfs.getSSTables()), 50L * 1024L * 1024L);
updateEstimateFor(cfs, buckets);
@@ -294,7 +294,7 @@ public class CompactionManager implement
// it is not empty, it may compact down to nothing if all rows are
deleted.
Table table = cfs.getTable();
if (DatabaseDescriptor.isSnapshotBeforeCompaction())
- table.snapshot("compact-" + cfs.columnFamily_);
+ table.snapshot("compact-" + cfs.columnFamily);
logger.info("Compacting [" + StringUtils.join(sstables, ",") + "]");
String compactionFileLocation =
table.getDataFileLocation(cfs.getExpectedCompactedFileSize(sstables));
// If the compaction file path is null that means we have no space
left for this compaction.
@@ -343,7 +343,7 @@ public class CompactionManager implement
}
String newFilename = new
File(cfs.getTempSSTablePath(compactionFileLocation)).getAbsolutePath();
- writer = new SSTableWriter(newFilename, expectedBloomFilterSize,
cfs.metadata, cfs.partitioner_);
+ writer = new SSTableWriter(newFilename, expectedBloomFilterSize,
cfs.metadata, cfs.partitioner);
while (nni.hasNext())
{
AbstractCompactedRow row = nni.next();
@@ -433,7 +433,7 @@ public class CompactionManager implement
{
FileUtils.createDirectory(compactionFileLocation);
String newFilename = new
File(cfs.getTempSSTablePath(compactionFileLocation)).getAbsolutePath();
- writer = new SSTableWriter(newFilename,
expectedBloomFilterSize, cfs.metadata, cfs.partitioner_);
+ writer = new SSTableWriter(newFilename,
expectedBloomFilterSize, cfs.metadata, cfs.partitioner);
}
writer.append(row);
totalkeysWritten++;
@@ -619,7 +619,7 @@ public class CompactionManager implement
{
public void run ()
{
- logger.debug("Estimating compactions for " +
cfs.columnFamily_);
+ logger.debug("Estimating compactions for " +
cfs.columnFamily);
final Set<List<SSTableReader>> buckets =
getBuckets(convertSSTablesToPairs(cfs.getSSTables()), 50L * 1024L * 1024L);
updateEstimateFor(cfs, buckets);
}
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java?rev=986863&r1=986862&r2=986863&view=diff
==============================================================================
---
cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
(original)
+++
cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
Wed Aug 18 19:15:24 2010
@@ -21,7 +21,6 @@ package org.apache.cassandra.service;
import java.net.InetAddress;
import java.util.*;
import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.ThreadPoolExecutor;
@@ -71,7 +70,7 @@ public class AntiEntropyServiceTest exte
// generate a fake endpoint for which we can spoof receiving/sending
trees
REMOTE = InetAddress.getByName("127.0.0.2");
store =
Table.open(tablename).getColumnFamilyStores().iterator().next();
- cfname = store.columnFamily_;
+ cfname = store.columnFamily;
}
@Before