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));


Reply via email to