Author: jbellis Date: Tue Nov 30 14:26:07 2010 New Revision: 1040536 URL: http://svn.apache.org/viewvc?rev=1040536&view=rev Log: reduce log messages on startup when key/row cache saving is off patch by mdennis; reviewed by jbellis for CASSANDRA-1789
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1040536&r1=1040535&r2=1040536&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue Nov 30 14:26:07 2010 @@ -215,7 +215,6 @@ public class ColumnFamilyStore implement // scan for sstables corresponding to this cf and load them ssTables = new SSTableTracker(table.name, columnFamilyName); Set<DecoratedKey> savedKeys = readSavedCache(DatabaseDescriptor.getSerializedKeyCachePath(table.name, columnFamilyName)); - logger.info("read " + savedKeys.size() + " from saved key cache"); List<SSTableReader> sstables = new ArrayList<SSTableReader>(); for (Map.Entry<Descriptor,Set<Component>> sstableFiles : files(table.name, columnFamilyName, false).entrySet()) { @@ -270,8 +269,7 @@ public class ColumnFamilyStore implement if (path.exists()) { - if (logger.isDebugEnabled()) - logger.debug(String.format("reading saved cache from %s", path)); + logger.info(String.format("reading saved cache %s", path)); ObjectInputStream in = new ObjectInputStream(new BufferedInputStream(new FileInputStream(path))); while (in.available() > 0) { @@ -282,7 +280,7 @@ public class ColumnFamilyStore implement } in.close(); if (logger.isDebugEnabled()) - logger.debug(String.format("completed reading (%d ms; %d keys) from saved cache at %s", + logger.debug(String.format("completed reading (%d ms; %d keys) saved cache from %s", System.currentTimeMillis() - start, keys.size(), path)); } } @@ -504,18 +502,20 @@ public class ColumnFamilyStore implement // must be called after all sstables are loaded since row cache merges all row versions public void initRowCache() { - String msgSuffix = String.format(" row cache for %s of %s", columnFamily, table.name); int rowCacheSavePeriodInSeconds = DatabaseDescriptor.getTableMetaData(table.name).get(columnFamily).getRowCacheSavePeriodInSeconds(); int keyCacheSavePeriodInSeconds = DatabaseDescriptor.getTableMetaData(table.name).get(columnFamily).getKeyCacheSavePeriodInSeconds(); long start = System.currentTimeMillis(); - logger.info(String.format("loading%s", msgSuffix)); // sort the results on read because there are few reads and many writes and reads only happen at startup Set<DecoratedKey> savedKeys = readSavedCache(DatabaseDescriptor.getSerializedRowCachePath(table.name, columnFamily)); for (DecoratedKey key : savedKeys) cacheRow(key); - logger.info(String.format("completed loading (%d ms; %d keys) %s", - System.currentTimeMillis()-start, ssTables.getRowCache().getSize(), msgSuffix)); + if (ssTables.getRowCache().getSize() > 0) + logger.info(String.format("completed loading (%d ms; %d keys) row cache for %s.%s", + System.currentTimeMillis()-start, + ssTables.getRowCache().getSize(), + table.name, + columnFamily)); if (rowCacheSavePeriodInSeconds > 0) { cacheSavingExecutor.scheduleWithFixedDelay(rowCacheSaverTask, Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=1040536&r1=1040535&r2=1040536&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/Descriptor.java Tue Nov 30 14:26:07 2010 @@ -82,7 +82,7 @@ public class Descriptor { return filenameFor(component.name()); } - + /** * @param suffix A component suffix, such as 'Data.db'/'Index.db'/etc * @return A filename for this descriptor with the given suffix. Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1040536&r1=1040535&r2=1040536&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java Tue Nov 30 14:26:07 2010 @@ -167,7 +167,7 @@ public class SSTableReader extends SSTab assert partitioner != null; long start = System.currentTimeMillis(); - logger.info("Sampling index for " + descriptor); + logger.info("Opening " + descriptor); EstimatedHistogram rowSizes; EstimatedHistogram columnCounts; Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java?rev=1040536&r1=1040535&r2=1040536&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java Tue Nov 30 14:26:07 2010 @@ -71,8 +71,8 @@ public class SSTableTracker implements I public void saveCache(JMXInstrumentedCache<K, V> cache, File savedCachePath, Function<K, ByteBuffer> converter) throws IOException { long start = System.currentTimeMillis(); - String msgSuffix = " " + savedCachePath.getName() + " for " + cfname + " of " + ksname; - logger.debug("saving" + msgSuffix); + String msgSuffix = savedCachePath.getName() + " for " + cfname + " of " + ksname; + logger.info("saving " + msgSuffix); int count = 0; File tmpFile = File.createTempFile(savedCachePath.getName(), null, savedCachePath.getParentFile()); FileOutputStream fout = new FileOutputStream(tmpFile); @@ -91,7 +91,7 @@ public class SSTableTracker implements I if (!tmpFile.renameTo(savedCachePath)) throw new IOException("Unable to rename cache to " + savedCachePath); if (logger.isDebugEnabled()) - logger.debug("saved " + count + " keys in " + (System.currentTimeMillis() - start) + " ms from" + msgSuffix); + logger.debug("saved " + count + " keys in " + (System.currentTimeMillis() - start) + " ms from " + msgSuffix); } }