Author: jbellis
Date: Wed Mar 3 15:35:02 2010
New Revision: 918514
URL: http://svn.apache.org/viewvc?rev=918514&view=rev
Log:
remove underscores from DD.java. patch by Brandon Williams; reviewed by
jbellis for CASSANDRA-843
Modified:
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
Modified:
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=918514&r1=918513&r2=918514&view=diff
==============================================================================
---
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
(original)
+++
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Wed Mar 3 15:35:02 2010
@@ -49,7 +49,7 @@
public class DatabaseDescriptor
{
- private static Logger logger_ = Logger.getLogger(DatabaseDescriptor.class);
+ private static Logger logger = Logger.getLogger(DatabaseDescriptor.class);
public static final String STREAMING_SUBDIR = "stream";
// don't capitalize these; we need them to match what's in the config file
for CLS.valueOf to parse
@@ -65,71 +65,71 @@
standard,
}
- public static final String random_ = "RANDOM";
- public static final String ophf_ = "OPHF";
- private static int storagePort_ = 7000;
- private static int controlPort_ = 7001;
- private static int thriftPort_ = 9160;
- private static boolean thriftFramed_ = false;
- private static InetAddress listenAddress_; // leave null so we can fall
through to getLocalHost
- private static InetAddress thriftAddress_;
- private static String clusterName_ = "Test";
- private static long rpcTimeoutInMillis_ = 2000;
- private static Set<InetAddress> seeds_ = new HashSet<InetAddress>();
+ public static final String random = "RANDOM";
+ public static final String ophf = "OPHF";
+ private static int storagePort = 7000;
+ private static int controlPort = 7001;
+ private static int thriftPort = 9160;
+ private static boolean thriftFramed = false;
+ private static InetAddress listenAddress; // leave null so we can fall
through to getLocalHost
+ private static InetAddress thriftAddress;
+ private static String clusterName = "Test";
+ private static long rpcTimeoutInMillis = 2000;
+ private static Set<InetAddress> seeds = new HashSet<InetAddress>();
/* Keeps the list of data file directories */
- private static String[] dataFileDirectories_;
+ private static String[] dataFileDirectories;
/* Current index into the above list of directories */
- private static int currentIndex_ = 0;
- private static String logFileDirectory_;
- private static int consistencyThreads_ = 4; // not configurable
- private static int concurrentReaders_ = 8;
- private static int concurrentWriters_ = 32;
-
- private static double flushDataBufferSizeInMB_ = 32;
- private static double flushIndexBufferSizeInMB_ = 8;
- private static int slicedReadBufferSizeInKB_ = 64;
+ private static int currentIndex = 0;
+ private static String logFileDirectory;
+ private static int consistencyThreads = 4; // not configurable
+ private static int concurrentReaders = 8;
+ private static int concurrentWriters = 32;
+
+ private static double flushDataBufferSizeInMB = 32;
+ private static double flushIndexBufferSizeInMB = 8;
+ private static int slicedReadBufferSizeInKB = 64;
- static Map<String, KSMetaData> tables_ = new HashMap<String, KSMetaData>();
- private static int bmtThreshold_ = 256;
+ static Map<String, KSMetaData> tables = new HashMap<String, KSMetaData>();
+ private static int bmtThreshold = 256;
/* Hashing strategy Random or OPHF */
- private static IPartitioner partitioner_;
+ private static IPartitioner partitioner;
/* if the size of columns or super-columns are more than this, indexing
will kick in */
- private static int columnIndexSizeInKB_;
+ private static int columnIndexSizeInKB;
/* Number of minutes to keep a memtable in memory */
- private static int memtableLifetimeMs_ = 60 * 60 * 1000;
+ private static int memtableLifetimeMs = 60 * 60 * 1000;
/* Size of the memtable in memory before it is dumped */
- private static int memtableThroughput_ = 64;
+ private static int memtableThroughput = 64;
/* Number of objects in millions in the memtable before it is dumped */
- private static double memtableOperations_ = 0.1;
+ private static double memtableOperations = 0.1;
/*
* This parameter enables or disables consistency checks.
* If set to false the read repairs are disable for very
* high throughput on reads but at the cost of consistency.
*/
- private static boolean doConsistencyCheck_ = true;
+ private static boolean doConsistencyCheck = true;
/* Job Jar Location */
- private static String jobJarFileLocation_;
+ private static String jobJarFileLocation;
/* Address where to run the job tracker */
- private static String jobTrackerHost_;
+ private static String jobTrackerHost;
/* time to wait before garbage collecting tombstones (deletion markers) */
- private static int gcGraceInSeconds_ = 10 * 24 * 3600; // 10 days
+ private static int gcGraceInSeconds = 10 * 24 * 3600; // 10 days
// the path qualified config file (storage-conf.xml) name
- private static String configFileName_;
+ private static String configFileName;
/* initial token in the ring */
- private static String initialToken_ = null;
+ private static String initialToken = null;
- private static CommitLogSync commitLogSync_;
- private static double commitLogSyncBatchMS_;
- private static int commitLogSyncPeriodMS_;
+ private static CommitLogSync commitLogSync;
+ private static double commitLogSyncBatchMS;
+ private static int commitLogSyncPeriodMS;
- private static DiskAccessMode diskAccessMode_;
- private static DiskAccessMode indexAccessMode_;
+ private static DiskAccessMode diskAccessMode;
+ private static DiskAccessMode indexAccessMode;
- private static boolean snapshotBeforeCompaction_;
- private static boolean autoBootstrap_ = false;
+ private static boolean snapshotBeforeCompaction;
+ private static boolean autoBootstrap = false;
private static IAuthenticator authenticator = new AllowAllAuthenticator();
@@ -155,32 +155,32 @@
{
try
{
- configFileName_ = getStorageConfigPath();
- if (logger_.isDebugEnabled())
- logger_.debug("Loading settings from " + configFileName_);
- XMLUtils xmlUtils = new XMLUtils(configFileName_);
+ configFileName = getStorageConfigPath();
+ if (logger.isDebugEnabled())
+ logger.debug("Loading settings from " + configFileName);
+ XMLUtils xmlUtils = new XMLUtils(configFileName);
/* Cluster Name */
- clusterName_ = xmlUtils.getNodeValue("/Storage/ClusterName");
+ clusterName = xmlUtils.getNodeValue("/Storage/ClusterName");
String syncRaw = xmlUtils.getNodeValue("/Storage/CommitLogSync");
try
{
- commitLogSync_ = CommitLogSync.valueOf(syncRaw);
+ commitLogSync = CommitLogSync.valueOf(syncRaw);
}
catch (IllegalArgumentException e)
{
throw new ConfigurationException("CommitLogSync must be either
'periodic' or 'batch'");
}
- if (commitLogSync_ == null)
+ if (commitLogSync == null)
{
throw new ConfigurationException("Missing required directive
CommitLogSync");
}
- else if (commitLogSync_ == CommitLogSync.batch)
+ else if (commitLogSync == CommitLogSync.batch)
{
try
{
- commitLogSyncBatchMS_ =
Double.valueOf(xmlUtils.getNodeValue("/Storage/CommitLogSyncBatchWindowInMS"));
+ commitLogSyncBatchMS =
Double.valueOf(xmlUtils.getNodeValue("/Storage/CommitLogSyncBatchWindowInMS"));
}
catch (Exception e)
{
@@ -190,14 +190,14 @@
{
throw new ConfigurationException("Batch sync specified,
but CommitLogSyncPeriodInMS found. Only specify CommitLogSyncBatchWindowInMS
when using batch sync.");
}
- logger_.debug("Syncing log with a batch window of " +
commitLogSyncBatchMS_);
+ logger.debug("Syncing log with a batch window of " +
commitLogSyncBatchMS);
}
else
{
- assert commitLogSync_ == CommitLogSync.periodic;
+ assert commitLogSync == CommitLogSync.periodic;
try
{
- commitLogSyncPeriodMS_ =
Integer.valueOf(xmlUtils.getNodeValue("/Storage/CommitLogSyncPeriodInMS"));
+ commitLogSyncPeriodMS =
Integer.valueOf(xmlUtils.getNodeValue("/Storage/CommitLogSyncPeriodInMS"));
}
catch (Exception e)
{
@@ -207,32 +207,32 @@
{
throw new ConfigurationException("Periodic sync specified,
but CommitLogSyncBatchWindowInMS found. Only specify CommitLogSyncPeriodInMS
when using periodic sync.");
}
- logger_.debug("Syncing log with a period of " +
commitLogSyncPeriodMS_);
+ logger.debug("Syncing log with a period of " +
commitLogSyncPeriodMS);
}
String modeRaw = xmlUtils.getNodeValue("/Storage/DiskAccessMode");
try
{
- diskAccessMode_ = DiskAccessMode.valueOf(modeRaw);
+ diskAccessMode = DiskAccessMode.valueOf(modeRaw);
}
catch (IllegalArgumentException e)
{
throw new ConfigurationException("DiskAccessMode must be
either 'auto', 'mmap', 'mmap_index_only', or 'standard'");
}
- if (diskAccessMode_ == DiskAccessMode.auto)
+ if (diskAccessMode == DiskAccessMode.auto)
{
- diskAccessMode_ = System.getProperty("os.arch").contains("64")
? DiskAccessMode.mmap : DiskAccessMode.standard;
- indexAccessMode_ = diskAccessMode_;
- logger_.info("Auto DiskAccessMode determined to be " +
diskAccessMode_);
+ diskAccessMode = System.getProperty("os.arch").contains("64")
? DiskAccessMode.mmap : DiskAccessMode.standard;
+ indexAccessMode = diskAccessMode;
+ logger.info("Auto DiskAccessMode determined to be " +
diskAccessMode);
}
- else if (diskAccessMode_ == DiskAccessMode.mmap_index_only)
+ else if (diskAccessMode == DiskAccessMode.mmap_index_only)
{
- diskAccessMode_ = DiskAccessMode.standard;
- indexAccessMode_ = DiskAccessMode.mmap;
+ diskAccessMode = DiskAccessMode.standard;
+ indexAccessMode = DiskAccessMode.mmap;
}
else
{
- indexAccessMode_ = diskAccessMode_;
+ indexAccessMode = diskAccessMode;
}
/* Authentication and authorization backend, implementing
IAuthenticator */
@@ -259,7 +259,7 @@
try
{
Class cls = Class.forName(partitionerClassName);
- partitioner_ = (IPartitioner)
cls.getConstructor().newInstance();
+ partitioner = (IPartitioner)
cls.getConstructor().newInstance();
}
catch (ClassNotFoundException e)
{
@@ -267,92 +267,92 @@
}
/* JobTracker address */
- jobTrackerHost_ = xmlUtils.getNodeValue("/Storage/JobTrackerHost");
+ jobTrackerHost = xmlUtils.getNodeValue("/Storage/JobTrackerHost");
/* Job Jar file location */
- jobJarFileLocation_ =
xmlUtils.getNodeValue("/Storage/JobJarFileLocation");
+ jobJarFileLocation =
xmlUtils.getNodeValue("/Storage/JobJarFileLocation");
String gcGrace = xmlUtils.getNodeValue("/Storage/GCGraceSeconds");
if ( gcGrace != null )
- gcGraceInSeconds_ = Integer.parseInt(gcGrace);
+ gcGraceInSeconds = Integer.parseInt(gcGrace);
- initialToken_ = xmlUtils.getNodeValue("/Storage/InitialToken");
+ initialToken = xmlUtils.getNodeValue("/Storage/InitialToken");
/* RPC Timeout */
- String rpcTimeoutInMillis =
xmlUtils.getNodeValue("/Storage/RpcTimeoutInMillis");
- if ( rpcTimeoutInMillis != null )
- rpcTimeoutInMillis_ = Integer.parseInt(rpcTimeoutInMillis);
+ String rpcTimeout =
xmlUtils.getNodeValue("/Storage/RpcTimeoutInMillis");
+ if ( rpcTimeout != null )
+ rpcTimeoutInMillis = Integer.parseInt(rpcTimeout);
/* Thread per pool */
String rawReaders =
xmlUtils.getNodeValue("/Storage/ConcurrentReads");
if (rawReaders != null)
{
- concurrentReaders_ = Integer.parseInt(rawReaders);
+ concurrentReaders = Integer.parseInt(rawReaders);
}
String rawWriters =
xmlUtils.getNodeValue("/Storage/ConcurrentWrites");
if (rawWriters != null)
{
- concurrentWriters_ = Integer.parseInt(rawWriters);
+ concurrentWriters = Integer.parseInt(rawWriters);
}
String rawFlushData =
xmlUtils.getNodeValue("/Storage/FlushDataBufferSizeInMB");
if (rawFlushData != null)
{
- flushDataBufferSizeInMB_ = Double.parseDouble(rawFlushData);
+ flushDataBufferSizeInMB = Double.parseDouble(rawFlushData);
}
String rawFlushIndex =
xmlUtils.getNodeValue("/Storage/FlushIndexBufferSizeInMB");
if (rawFlushIndex != null)
{
- flushIndexBufferSizeInMB_ = Double.parseDouble(rawFlushIndex);
+ flushIndexBufferSizeInMB = Double.parseDouble(rawFlushIndex);
}
String rawSlicedBuffer =
xmlUtils.getNodeValue("/Storage/SlicedBufferSizeInKB");
if (rawSlicedBuffer != null)
{
- slicedReadBufferSizeInKB_ = Integer.parseInt(rawSlicedBuffer);
+ slicedReadBufferSizeInKB = Integer.parseInt(rawSlicedBuffer);
}
- String bmtThreshold =
xmlUtils.getNodeValue("/Storage/BinaryMemtableThroughputInMB");
- if (bmtThreshold != null)
+ String bmtThresh =
xmlUtils.getNodeValue("/Storage/BinaryMemtableThroughputInMB");
+ if (bmtThresh != null)
{
- bmtThreshold_ = Integer.parseInt(bmtThreshold);
+ bmtThreshold = Integer.parseInt(bmtThresh);
}
/* TCP port on which the storage system listens */
String port = xmlUtils.getNodeValue("/Storage/StoragePort");
if ( port != null )
- storagePort_ = Integer.parseInt(port);
+ storagePort = Integer.parseInt(port);
/* Local IP or hostname to bind services to */
- String listenAddress =
xmlUtils.getNodeValue("/Storage/ListenAddress");
- if (listenAddress != null)
+ String listenAddr =
xmlUtils.getNodeValue("/Storage/ListenAddress");
+ if (listenAddr != null)
{
- if (listenAddress.equals("0.0.0.0"))
+ if (listenAddr.equals("0.0.0.0"))
throw new ConfigurationException("ListenAddress must be a
single interface. See
http://wiki.apache.org/cassandra/FAQ#cant_listen_on_ip_any");
try
{
- listenAddress_ = InetAddress.getByName(listenAddress);
+ listenAddress = InetAddress.getByName(listenAddr);
}
catch (UnknownHostException e)
{
- throw new ConfigurationException("Unknown ListenAddress '"
+ listenAddress + "'");
+ throw new ConfigurationException("Unknown ListenAddress '"
+ listenAddr + "'");
}
}
/* Local IP or hostname to bind thrift server to */
- String thriftAddress =
xmlUtils.getNodeValue("/Storage/ThriftAddress");
- if ( thriftAddress != null )
- thriftAddress_ = InetAddress.getByName(thriftAddress);
+ String thriftAddr =
xmlUtils.getNodeValue("/Storage/ThriftAddress");
+ if ( thriftAddr != null )
+ thriftAddress = InetAddress.getByName(thriftAddr);
/* UDP port for control messages */
port = xmlUtils.getNodeValue("/Storage/ControlPort");
if ( port != null )
- controlPort_ = Integer.parseInt(port);
+ controlPort = Integer.parseInt(port);
/* get the thrift port from conf file */
port = xmlUtils.getNodeValue("/Storage/ThriftPort");
if (port != null)
- thriftPort_ = Integer.parseInt(port);
+ thriftPort = Integer.parseInt(port);
/* Framed (Thrift) transport (default to "no") */
String framedRaw =
xmlUtils.getNodeValue("/Storage/ThriftFramedTransport");
@@ -360,7 +360,7 @@
{
if (framedRaw.equalsIgnoreCase("true") ||
framedRaw.equalsIgnoreCase("false"))
{
- thriftFramed_ = Boolean.valueOf(framedRaw);
+ thriftFramed = Boolean.valueOf(framedRaw);
}
else
{
@@ -374,9 +374,9 @@
{
if (sbc.equalsIgnoreCase("true") ||
sbc.equalsIgnoreCase("false"))
{
- if (logger_.isDebugEnabled())
- logger_.debug("setting snapshotBeforeCompaction to " +
sbc);
- snapshotBeforeCompaction_ = Boolean.valueOf(sbc);
+ if (logger.isDebugEnabled())
+ logger.debug("setting snapshotBeforeCompaction to " +
sbc);
+ snapshotBeforeCompaction = Boolean.valueOf(sbc);
}
else
{
@@ -385,14 +385,14 @@
}
/* snapshot-before-compaction. defaults to false */
- String autoBootstrap =
xmlUtils.getNodeValue("/Storage/AutoBootstrap");
- if (autoBootstrap != null)
+ String autoBootstr =
xmlUtils.getNodeValue("/Storage/AutoBootstrap");
+ if (autoBootstr != null)
{
- if (autoBootstrap.equalsIgnoreCase("true") ||
autoBootstrap.equalsIgnoreCase("false"))
+ if (autoBootstr.equalsIgnoreCase("true") ||
autoBootstr.equalsIgnoreCase("false"))
{
- if (logger_.isDebugEnabled())
- logger_.debug("setting autoBootstrap to " +
autoBootstrap);
- autoBootstrap_ = Boolean.valueOf(autoBootstrap);
+ if (logger.isDebugEnabled())
+ logger.debug("setting autoBootstrap to " +
autoBootstr);
+ autoBootstrap = Boolean.valueOf(autoBootstr);
}
else
{
@@ -403,17 +403,17 @@
/* Number of days to keep the memtable around w/o flushing */
String lifetime =
xmlUtils.getNodeValue("/Storage/MemtableFlushAfterMinutes");
if (lifetime != null)
- memtableLifetimeMs_ = Integer.parseInt(lifetime) * 60 * 1000;
+ memtableLifetimeMs = Integer.parseInt(lifetime) * 60 * 1000;
/* Size of the memtable in memory in MB before it is dumped */
String memtableSize =
xmlUtils.getNodeValue("/Storage/MemtableThroughputInMB");
if ( memtableSize != null )
- memtableThroughput_ = Integer.parseInt(memtableSize);
+ memtableThroughput = Integer.parseInt(memtableSize);
/* Number of objects in millions in the memtable before it is
dumped */
String memtableObjectCount =
xmlUtils.getNodeValue("/Storage/MemtableOperationsInMillions");
if ( memtableObjectCount != null )
- memtableOperations_ = Double.parseDouble(memtableObjectCount);
- if (memtableOperations_ <= 0)
+ memtableOperations = Double.parseDouble(memtableObjectCount);
+ if (memtableOperations <= 0)
{
throw new ConfigurationException("Memtable object count must
be a positive double");
}
@@ -421,24 +421,24 @@
/* This parameter enables or disables consistency checks.
* If set to false the read repairs are disable for very
* high throughput on reads but at the cost of consistency.*/
- String doConsistencyCheck =
xmlUtils.getNodeValue("/Storage/DoConsistencyChecksBoolean");
- if ( doConsistencyCheck != null )
- doConsistencyCheck_ = Boolean.parseBoolean(doConsistencyCheck);
+ String doConsistency =
xmlUtils.getNodeValue("/Storage/DoConsistencyChecksBoolean");
+ if ( doConsistency != null )
+ doConsistencyCheck = Boolean.parseBoolean(doConsistency);
/* read the size at which we should do column indexes */
- String columnIndexSizeInKB =
xmlUtils.getNodeValue("/Storage/ColumnIndexSizeInKB");
- if(columnIndexSizeInKB == null)
+ String columnIndexSize =
xmlUtils.getNodeValue("/Storage/ColumnIndexSizeInKB");
+ if(columnIndexSize == null)
{
- columnIndexSizeInKB_ = 64;
+ columnIndexSizeInKB = 64;
}
else
{
- columnIndexSizeInKB_ = Integer.parseInt(columnIndexSizeInKB);
+ columnIndexSizeInKB = Integer.parseInt(columnIndexSize);
}
/* data file and commit log directories. they get created later,
when they're needed. */
- dataFileDirectories_ =
xmlUtils.getNodeValues("/Storage/DataFileDirectories/DataFileDirectory");
- logFileDirectory_ =
xmlUtils.getNodeValue("/Storage/CommitLogDirectory");
+ dataFileDirectories =
xmlUtils.getNodeValues("/Storage/DataFileDirectories/DataFileDirectory");
+ logFileDirectory =
xmlUtils.getNodeValue("/Storage/CommitLogDirectory");
/* threshold after which commit log should be rotated. */
String value =
xmlUtils.getNodeValue("/Storage/CommitLogRotationThresholdInMB");
@@ -446,12 +446,12 @@
CommitLog.setSegmentSize(Integer.parseInt(value) * 1024 *
1024);
readTablesFromXml();
- if (tables_.isEmpty())
+ if (tables.isEmpty())
throw new ConfigurationException("No keyspaces configured");
// Hardcoded system tables
KSMetaData systemMeta = new KSMetaData(Table.SYSTEM_TABLE, null,
-1, null);
- tables_.put(Table.SYSTEM_TABLE, systemMeta);
+ tables.put(Table.SYSTEM_TABLE, systemMeta);
systemMeta.cfMetaData.put(SystemTable.STATUS_CF, new
CFMetaData(Table.SYSTEM_TABLE,
SystemTable.STATUS_CF,
"Standard",
@@ -471,19 +471,19 @@
0.01));
/* Load the seeds for node contact points */
- String[] seeds = xmlUtils.getNodeValues("/Storage/Seeds/Seed");
- if (seeds.length <= 0)
+ String[] seedsxml = xmlUtils.getNodeValues("/Storage/Seeds/Seed");
+ if (seedsxml.length <= 0)
{
throw new ConfigurationException("A minimum of one seed is
required.");
}
- for( int i = 0; i < seeds.length; ++i )
+ for( int i = 0; i < seedsxml.length; ++i )
{
- seeds_.add(InetAddress.getByName(seeds[i]));
+ seeds.add(InetAddress.getByName(seedsxml[i]));
}
}
catch (ConfigurationException e)
{
- logger_.error("Fatal error: " + e.getMessage());
+ logger.error("Fatal error: " + e.getMessage());
System.err.println("Bad configuration; unable to start server");
System.exit(1);
}
@@ -498,7 +498,7 @@
XMLUtils xmlUtils = null;
try
{
- xmlUtils = new XMLUtils(configFileName_);
+ xmlUtils = new XMLUtils(configFileName);
}
catch (ParserConfigurationException e)
{
@@ -522,12 +522,12 @@
/* Read the table related stuff from config */
try
{
- NodeList tables =
xmlUtils.getRequestedNodeList("/Storage/Keyspaces/Keyspace");
- int size = tables.getLength();
+ NodeList tablesxml =
xmlUtils.getRequestedNodeList("/Storage/Keyspaces/Keyspace");
+ int size = tablesxml.getLength();
for ( int i = 0; i < size; ++i )
{
String value = null;
- Node table = tables.item(i);
+ Node table = tablesxml.item(i);
/* parsing out the table ksName */
String ksName = XMLUtils.getAttributeValue(table, "Name");
@@ -648,7 +648,7 @@
{
keyCacheSize = Double.valueOf(value);
// TODO: KeysCachedFraction deprecated: remove in 1.0
- logger_.warn("KeysCachedFraction is deprecated: use
KeysCached instead.");
+ logger.warn("KeysCachedFraction is deprecated: use
KeysCached instead.");
}
if ((value = XMLUtils.getAttributeValue(columnFamily,
"KeysCached")) != null)
{
@@ -669,7 +669,7 @@
meta.cfMetaData.put(cfName, new CFMetaData(tableName,
cfName, columnType, comparator, subcolumnComparator, comment, rowCacheSize,
keyCacheSize));
}
- tables_.put(meta.name, meta);
+ tables.put(meta.name, meta);
}
}
catch (XPathExpressionException e)
@@ -693,7 +693,7 @@
public static boolean isThriftFramed()
{
- return thriftFramed_;
+ return thriftFramed;
}
private static AbstractType getComparator(Node columnFamily, String attr)
throws ConfigurationException
@@ -764,28 +764,28 @@
public static void createAllDirectories() throws IOException
{
try {
- if (dataFileDirectories_.length == 0)
+ if (dataFileDirectories.length == 0)
{
throw new ConfigurationException("At least one
DataFileDirectory must be specified");
}
- for ( String dataFileDirectory : dataFileDirectories_ )
+ for ( String dataFileDirectory : dataFileDirectories )
FileUtils.createDirectory(dataFileDirectory);
- if (logFileDirectory_ == null)
+ if (logFileDirectory == null)
{
throw new ConfigurationException("CommitLogDirectory must be
specified");
}
- FileUtils.createDirectory(logFileDirectory_);
+ FileUtils.createDirectory(logFileDirectory);
}
catch (ConfigurationException ex) {
- logger_.error("Fatal error: " + ex.getMessage());
+ logger.error("Fatal error: " + ex.getMessage());
System.err.println("Bad configuration; unable to start server");
System.exit(1);
}
/* make sure we have a directory for each table */
- for (String dataFile : dataFileDirectories_)
+ for (String dataFile : dataFileDirectories)
{
FileUtils.createDirectory(dataFile + File.separator +
Table.SYSTEM_TABLE);
- for (String table : tables_.keySet())
+ for (String table : tables.keySet())
{
String oneDir = dataFile + File.separator + table;
FileUtils.createDirectory(oneDir);
@@ -805,16 +805,16 @@
public static void storeMetadata() throws IOException
{
int cfId = 0;
- Set<String> tables = tables_.keySet();
+ Set<String> tableset = tables.keySet();
- for (String table : tables)
+ for (String table : tableset)
{
Table.TableMetadata tmetadata =
Table.TableMetadata.instance(table);
if (tmetadata.isEmpty())
{
tmetadata = Table.TableMetadata.instance(table);
/* Column families associated with this table */
- Map<String, CFMetaData> columnFamilies =
tables_.get(table).cfMetaData;
+ Map<String, CFMetaData> columnFamilies =
tables.get(table).cfMetaData;
for (String columnFamily : columnFamilies.keySet())
{
@@ -826,77 +826,77 @@
public static int getGcGraceInSeconds()
{
- return gcGraceInSeconds_;
+ return gcGraceInSeconds;
}
public static IPartitioner getPartitioner()
{
- return partitioner_;
+ return partitioner;
}
public static IEndPointSnitch getEndPointSnitch(String table)
{
- return tables_.get(table).epSnitch;
+ return tables.get(table).epSnitch;
}
public static Class<? extends AbstractReplicationStrategy>
getReplicaPlacementStrategyClass(String table)
{
- return tables_.get(table).repStratClass;
+ return tables.get(table).repStratClass;
}
public static String getJobTrackerAddress()
{
- return jobTrackerHost_;
+ return jobTrackerHost;
}
public static int getColumnIndexSize()
{
- return columnIndexSizeInKB_ * 1024;
+ return columnIndexSizeInKB * 1024;
}
public static int getMemtableLifetimeMS()
{
- return memtableLifetimeMs_;
+ return memtableLifetimeMs;
}
public static String getInitialToken()
{
- return initialToken_;
+ return initialToken;
}
public static int getMemtableThroughput()
{
- return memtableThroughput_;
+ return memtableThroughput;
}
public static double getMemtableOperations()
{
- return memtableOperations_;
+ return memtableOperations;
}
public static boolean getConsistencyCheck()
{
- return doConsistencyCheck_;
+ return doConsistencyCheck;
}
public static String getClusterName()
{
- return clusterName_;
+ return clusterName;
}
public static String getConfigFileName() {
- return configFileName_;
+ return configFileName;
}
public static String getJobJarLocation()
{
- return jobJarFileLocation_;
+ return jobJarFileLocation;
}
public static Map<String, CFMetaData> getTableMetaData(String tableName)
{
assert tableName != null;
- KSMetaData ksm = tables_.get(tableName);
+ KSMetaData ksm = tables.get(tableName);
assert ksm != null;
return Collections.unmodifiableMap(ksm.cfMetaData);
}
@@ -909,7 +909,7 @@
public static CFMetaData getCFMetaData(String tableName, String cfName)
{
assert tableName != null;
- KSMetaData ksm = tables_.get(tableName);
+ KSMetaData ksm = tables.get(tableName);
if (ksm == null)
return null;
return ksm.cfMetaData.get(cfName);
@@ -927,64 +927,64 @@
public static Set<String> getTables()
{
- return tables_.keySet();
+ return tables.keySet();
}
public static List<String> getNonSystemTables()
{
- List<String> tables = new ArrayList<String>(tables_.keySet());
- tables.remove(Table.SYSTEM_TABLE);
- return Collections.unmodifiableList(tables);
+ List<String> tableslist = new ArrayList<String>(tables.keySet());
+ tableslist.remove(Table.SYSTEM_TABLE);
+ return Collections.unmodifiableList(tableslist);
}
public static int getStoragePort()
{
- return storagePort_;
+ return storagePort;
}
public static int getControlPort()
{
- return controlPort_;
+ return controlPort;
}
public static int getThriftPort()
{
- return thriftPort_;
+ return thriftPort;
}
public static int getReplicationFactor(String table)
{
- return tables_.get(table).replicationFactor;
+ return tables.get(table).replicationFactor;
}
public static int getQuorum(String table)
{
- return (tables_.get(table).replicationFactor / 2) + 1;
+ return (tables.get(table).replicationFactor / 2) + 1;
}
public static long getRpcTimeout()
{
- return rpcTimeoutInMillis_;
+ return rpcTimeoutInMillis;
}
public static int getConsistencyThreads()
{
- return consistencyThreads_;
+ return consistencyThreads;
}
public static int getConcurrentReaders()
{
- return concurrentReaders_;
+ return concurrentReaders;
}
public static int getConcurrentWriters()
{
- return concurrentWriters_;
+ return concurrentWriters;
}
public static String[] getAllDataFileLocations()
{
- return dataFileDirectories_;
+ return dataFileDirectories;
}
/**
@@ -996,11 +996,11 @@
*/
public static String[] getAllDataFileLocationsForTable(String table)
{
- String[] tableLocations = new String[dataFileDirectories_.length];
+ String[] tableLocations = new String[dataFileDirectories.length];
- for (int i = 0; i < dataFileDirectories_.length; i++)
+ for (int i = 0; i < dataFileDirectories.length; i++)
{
- tableLocations[i] = dataFileDirectories_[i] + File.separator +
table;
+ tableLocations[i] = dataFileDirectories[i] + File.separator +
table;
}
return tableLocations;
@@ -1008,19 +1008,19 @@
public synchronized static String getNextAvailableDataLocation()
{
- String dataFileDirectory = dataFileDirectories_[currentIndex_];
- currentIndex_ = (currentIndex_ + 1) % dataFileDirectories_.length;
+ String dataFileDirectory = dataFileDirectories[currentIndex];
+ currentIndex = (currentIndex + 1) % dataFileDirectories.length;
return dataFileDirectory;
}
public static String getLogFileLocation()
{
- return logFileDirectory_;
+ return logFileDirectory;
}
public static Set<InetAddress> getSeeds()
{
- return seeds_;
+ return seeds;
}
public static String getColumnFamilyType(String tableName, String cfName)
@@ -1059,11 +1059,11 @@
if( expectedCompactedFileSize < maxFreeDisk )
{
dataFileDirectory = dataDirectoryForTable[maxDiskIndex];
- currentIndex_ = (maxDiskIndex + 1 )%dataDirectoryForTable.length ;
+ currentIndex = (maxDiskIndex + 1 )%dataDirectoryForTable.length ;
}
else
{
- currentIndex_ = maxDiskIndex;
+ currentIndex = maxDiskIndex;
}
return dataFileDirectory;
}
@@ -1113,70 +1113,70 @@
public static InetAddress getListenAddress()
{
- return listenAddress_;
+ return listenAddress;
}
public static InetAddress getThriftAddress()
{
- return thriftAddress_;
+ return thriftAddress;
}
public static double getCommitLogSyncBatchWindow()
{
- return commitLogSyncBatchMS_;
+ return commitLogSyncBatchMS;
}
public static int getCommitLogSyncPeriod() {
- return commitLogSyncPeriodMS_;
+ return commitLogSyncPeriodMS;
}
public static CommitLogSync getCommitLogSync()
{
- return commitLogSync_;
+ return commitLogSync;
}
public static DiskAccessMode getDiskAccessMode()
{
- return diskAccessMode_;
+ return diskAccessMode;
}
public static DiskAccessMode getIndexAccessMode()
{
- return indexAccessMode_;
+ return indexAccessMode;
}
public static double getFlushDataBufferSizeInMB()
{
- return flushDataBufferSizeInMB_;
+ return flushDataBufferSizeInMB;
}
public static double getFlushIndexBufferSizeInMB()
{
- return flushIndexBufferSizeInMB_;
+ return flushIndexBufferSizeInMB;
}
public static int getIndexedReadBufferSizeInKB()
{
- return columnIndexSizeInKB_;
+ return columnIndexSizeInKB;
}
public static int getSlicedReadBufferSizeInKB()
{
- return slicedReadBufferSizeInKB_;
+ return slicedReadBufferSizeInKB;
}
public static int getBMTThreshold()
{
- return bmtThreshold_;
+ return bmtThreshold;
}
public static boolean isSnapshotBeforeCompaction()
{
- return snapshotBeforeCompaction_;
+ return snapshotBeforeCompaction;
}
public static boolean isAutoBootstrap()
{
- return autoBootstrap_;
+ return autoBootstrap;
}
}
Modified:
incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java?rev=918514&r1=918513&r2=918514&view=diff
==============================================================================
---
incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
(original)
+++
incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
Wed Mar 3 15:35:02 2010
@@ -52,7 +52,7 @@
@Test
public void testKSMetaDataSerialization() throws IOException
{
- for (KSMetaData ksm : DatabaseDescriptor.tables_.values())
+ for (KSMetaData ksm : DatabaseDescriptor.tables.values())
{
byte[] ser = KSMetaData.serialize(ksm);
KSMetaData ksmDupe = KSMetaData.deserialize(new
ByteArrayInputStream(ser));