Return-Path: Delivered-To: apmail-incubator-cassandra-commits-archive@minotaur.apache.org Received: (qmail 74548 invoked from network); 8 Jun 2009 19:18:39 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 8 Jun 2009 19:18:39 -0000 Received: (qmail 68917 invoked by uid 500); 8 Jun 2009 19:18:50 -0000 Delivered-To: apmail-incubator-cassandra-commits-archive@incubator.apache.org Received: (qmail 68889 invoked by uid 500); 8 Jun 2009 19:18:50 -0000 Mailing-List: contact cassandra-commits-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: cassandra-dev@incubator.apache.org Delivered-To: mailing list cassandra-commits@incubator.apache.org Received: (qmail 68875 invoked by uid 99); 8 Jun 2009 19:18:50 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 08 Jun 2009 19:18:50 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 08 Jun 2009 19:18:40 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 7BE0A23888A0; Mon, 8 Jun 2009 19:18:19 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r782731 - in /incubator/cassandra/branches/cassandra-0.3: interface/ interface/gen-java/org/apache/cassandra/service/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/ src/java/org/apa... Date: Mon, 08 Jun 2009 19:18:19 -0000 To: cassandra-commits@incubator.apache.org From: jbellis@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20090608191819.7BE0A23888A0@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: jbellis Date: Mon Jun 8 19:18:18 2009 New Revision: 782731 URL: http://svn.apache.org/viewvc?rev=782731&view=rev Log: r/m 'touch' code that populates a cache that is never used (and never updated on compaction either, so it's buggy too). patch by jbellis; reviewed by goffinet for CASSANDRA-222 Removed: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/TouchMessage.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/TouchVerbHandler.java Modified: incubator/cassandra/branches/cassandra-0.3/interface/cassandra.thrift incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/Table.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/IFileReader.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SSTable.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SequenceFile.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/CassandraServer.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageProxy.java incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageService.java Modified: incubator/cassandra/branches/cassandra-0.3/interface/cassandra.thrift URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/interface/cassandra.thrift?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/interface/cassandra.thrift (original) +++ incubator/cassandra/branches/cassandra-0.3/interface/cassandra.thrift Mon Jun 8 19:18:18 2009 @@ -137,8 +137,6 @@ void batch_insert_superColumn(1:batch_mutation_super_t batchMutationSuper, 2:bool block=0) throws (1: InvalidRequestException ire, 2: UnavailableException ue), - oneway void touch(1:string key, 2:bool fData), - # range query: returns matching keys list get_key_range(1:string tablename, 2:string startWith="", 3:string stopAt="", 4:i32 maxResults=1000) throws (1: InvalidRequestException ire), Modified: incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java (original) +++ incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java Mon Jun 8 19:18:18 2009 @@ -46,8 +46,6 @@ public void batch_insert_superColumn(batch_mutation_super_t batchMutationSuper, boolean block) throws InvalidRequestException, UnavailableException, TException; - public void touch(String key, boolean fData) throws TException; - public List get_key_range(String tablename, String startWith, String stopAt, int maxResults) throws InvalidRequestException, TException; public String getStringProperty(String propertyName) throws TException; @@ -568,22 +566,6 @@ return; } - public void touch(String key, boolean fData) throws TException - { - send_touch(key, fData); - } - - public void send_touch(String key, boolean fData) throws TException - { - oprot_.writeMessageBegin(new TMessage("touch", TMessageType.CALL, seqid_)); - touch_args args = new touch_args(); - args.key = key; - args.fData = fData; - args.write(oprot_); - oprot_.writeMessageEnd(); - oprot_.getTransport().flush(); - } - public List get_key_range(String tablename, String startWith, String stopAt, int maxResults) throws InvalidRequestException, TException { send_get_key_range(tablename, startWith, stopAt, maxResults); @@ -773,7 +755,6 @@ processMap_.put("get_slice_super_by_names", new get_slice_super_by_names()); processMap_.put("get_superColumn", new get_superColumn()); processMap_.put("batch_insert_superColumn", new batch_insert_superColumn()); - processMap_.put("touch", new touch()); processMap_.put("get_key_range", new get_key_range()); processMap_.put("getStringProperty", new getStringProperty()); processMap_.put("getStringListProperty", new getStringListProperty()); @@ -1161,17 +1142,6 @@ } - private class touch implements ProcessFunction { - public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException - { - touch_args args = new touch_args(); - args.read(iprot); - iprot.readMessageEnd(); - iface_.touch(args.key, args.fData); - return; - } - } - private class get_key_range implements ProcessFunction { public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException { @@ -10084,275 +10054,6 @@ } - public static class touch_args implements TBase, java.io.Serializable, Cloneable { - private static final TStruct STRUCT_DESC = new TStruct("touch_args"); - private static final TField KEY_FIELD_DESC = new TField("key", TType.STRING, (short)1); - private static final TField F_DATA_FIELD_DESC = new TField("fData", TType.BOOL, (short)2); - - public String key; - public static final int KEY = 1; - public boolean fData; - public static final int FDATA = 2; - - private final Isset __isset = new Isset(); - private static final class Isset implements java.io.Serializable { - public boolean fData = false; - } - - public static final Map metaDataMap = Collections.unmodifiableMap(new HashMap() {{ - put(KEY, new FieldMetaData("key", TFieldRequirementType.DEFAULT, - new FieldValueMetaData(TType.STRING))); - put(FDATA, new FieldMetaData("fData", TFieldRequirementType.DEFAULT, - new FieldValueMetaData(TType.BOOL))); - }}); - - static { - FieldMetaData.addStructMetaDataMap(touch_args.class, metaDataMap); - } - - public touch_args() { - } - - public touch_args( - String key, - boolean fData) - { - this(); - this.key = key; - this.fData = fData; - this.__isset.fData = true; - } - - /** - * Performs a deep copy on other. - */ - public touch_args(touch_args other) { - if (other.isSetKey()) { - this.key = other.key; - } - __isset.fData = other.__isset.fData; - this.fData = other.fData; - } - - @Override - public touch_args clone() { - return new touch_args(this); - } - - public String getKey() { - return this.key; - } - - public void setKey(String key) { - this.key = key; - } - - public void unsetKey() { - this.key = null; - } - - // Returns true if field key is set (has been asigned a value) and false otherwise - public boolean isSetKey() { - return this.key != null; - } - - public void setKeyIsSet(boolean value) { - if (!value) { - this.key = null; - } - } - - public boolean isFData() { - return this.fData; - } - - public void setFData(boolean fData) { - this.fData = fData; - this.__isset.fData = true; - } - - public void unsetFData() { - this.__isset.fData = false; - } - - // Returns true if field fData is set (has been asigned a value) and false otherwise - public boolean isSetFData() { - return this.__isset.fData; - } - - public void setFDataIsSet(boolean value) { - this.__isset.fData = value; - } - - public void setFieldValue(int fieldID, Object value) { - switch (fieldID) { - case KEY: - if (value == null) { - unsetKey(); - } else { - setKey((String)value); - } - break; - - case FDATA: - if (value == null) { - unsetFData(); - } else { - setFData((Boolean)value); - } - break; - - default: - throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!"); - } - } - - public Object getFieldValue(int fieldID) { - switch (fieldID) { - case KEY: - return getKey(); - - case FDATA: - return new Boolean(isFData()); - - default: - throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!"); - } - } - - // Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise - public boolean isSet(int fieldID) { - switch (fieldID) { - case KEY: - return isSetKey(); - case FDATA: - return isSetFData(); - default: - throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!"); - } - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof touch_args) - return this.equals((touch_args)that); - return false; - } - - public boolean equals(touch_args that) { - if (that == null) - return false; - - boolean this_present_key = true && this.isSetKey(); - boolean that_present_key = true && that.isSetKey(); - if (this_present_key || that_present_key) { - if (!(this_present_key && that_present_key)) - return false; - if (!this.key.equals(that.key)) - return false; - } - - boolean this_present_fData = true; - boolean that_present_fData = true; - if (this_present_fData || that_present_fData) { - if (!(this_present_fData && that_present_fData)) - return false; - if (this.fData != that.fData) - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - public void read(TProtocol iprot) throws TException { - TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == TType.STOP) { - break; - } - switch (field.id) - { - case KEY: - if (field.type == TType.STRING) { - this.key = iprot.readString(); - } else { - TProtocolUtil.skip(iprot, field.type); - } - break; - case FDATA: - if (field.type == TType.BOOL) { - this.fData = iprot.readBool(); - this.__isset.fData = true; - } else { - TProtocolUtil.skip(iprot, field.type); - } - break; - default: - TProtocolUtil.skip(iprot, field.type); - break; - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - - // check for required fields of primitive type, which can't be checked in the validate method - validate(); - } - - public void write(TProtocol oprot) throws TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.key != null) { - oprot.writeFieldBegin(KEY_FIELD_DESC); - oprot.writeString(this.key); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(F_DATA_FIELD_DESC); - oprot.writeBool(this.fData); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("touch_args("); - boolean first = true; - - sb.append("key:"); - if (this.key == null) { - sb.append("null"); - } else { - sb.append(this.key); - } - first = false; - if (!first) sb.append(", "); - sb.append("fData:"); - sb.append(this.fData); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws TException { - // check for required fields - // check that fields of type enum have valid values - } - - } - public static class get_key_range_args implements TBase, java.io.Serializable, Cloneable { private static final TStruct STRUCT_DESC = new TStruct("get_key_range_args"); private static final TField TABLENAME_FIELD_DESC = new TField("tablename", TType.STRING, (short)1); Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Mon Jun 8 19:18:18 2009 @@ -90,8 +90,6 @@ private static String partitionerClass_; /* if the size of columns or super-columns are more than this, indexing will kick in */ private static int columnIndexSizeInKB_; - /* Size of touch key cache */ - private static int touchKeyCacheSize_ = 1024; /* Number of hours to keep a memtable in memory */ private static int memtableLifetime_ = 6; /* Size of the memtable in memory before it is dumped */ @@ -209,12 +207,6 @@ if (port != null) thriftPort_ = Integer.parseInt(port); - - /* Touch Key Cache Size */ - String touchKeyCacheSize = xmlUtils.getNodeValue("/Storage/TouchKeyCacheSize"); - if ( touchKeyCacheSize != null ) - touchKeyCacheSize_ = Integer.parseInt(touchKeyCacheSize); - /* Number of days to keep the memtable around w/o flushing */ String lifetime = xmlUtils.getNodeValue("/Storage/MemtableLifetimeInDays"); if ( lifetime != null ) @@ -565,11 +557,6 @@ return applicationColumnFamilies_.contains(columnFamily); } - public static int getTouchKeyCacheSize() - { - return touchKeyCacheSize_; - } - public static String getJobJarLocation() { return jobJarFileLocation_; Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Mon Jun 8 19:18:18 2009 @@ -258,34 +258,6 @@ } } - void touch(String key, boolean fData) throws IOException - { - /* Scan the SSTables on disk first */ - lock_.readLock().lock(); - try - { - List files = new ArrayList(ssTables_); - for (String file : files) - { - /* - * Get the BloomFilter associated with this file. Check if the key - * is present in the BloomFilter. If not continue to the next file. - */ - boolean bVal = SSTable.isKeyInFile(key, file); - if (!bVal) - { - continue; - } - SSTable ssTable = new SSTable(file, StorageService.getPartitioner()); - ssTable.touch(key, fData); - } - } - finally - { - lock_.readLock().unlock(); - } - } - /* * This method forces a compaction of the SSTables on disk. We wait * for the process to complete by waiting on a future pointer. Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/Table.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/Table.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/Table.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/db/Table.java Mon Jun 8 19:18:18 2009 @@ -501,27 +501,6 @@ } } - - /* - * This method is used to ensure that all keys - * prior to the specified key, as dtermined by - * the SSTable index bucket it falls in, are in - * buffer cache. - */ - public void touch(String key, boolean fData) throws IOException - { - Set columnFamilies = tableMetadata_.getColumnFamilies(); - for ( String columnFamily : columnFamilies ) - { - if ( DatabaseDescriptor.isApplicationColumnFamily(columnFamily) ) - { - ColumnFamilyStore cfStore = columnFamilyStores_.get( columnFamily ); - if ( cfStore != null ) - cfStore.touch(key, fData); - } - } - } - /* * Clear the existing snapshots in the system */ Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/IFileReader.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/IFileReader.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/IFileReader.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/IFileReader.java Mon Jun 8 19:18:18 2009 @@ -54,16 +54,6 @@ public long readLong() throws IOException; /** - * This functions is used to help out subsequent reads - * on the specified key. It reads the keys prior to this - * one on disk so that the buffer cache is hot. - * - * @param key key for which we are performing the touch. - * @param fData true implies we fetch the data into buffer cache. - */ - public long touch(String key , boolean fData) throws IOException; - - /** * This method helps is retrieving the offset of the specified * key in the file using the block index. * @@ -72,15 +62,6 @@ public long getPositionFromBlockIndex(String key) throws IOException; /** - * This method returns the position of the specified key and the - * size of its data segment from the block index. - * - * @param key key whose block metadata we are interested in. - * @return an instance of the block metadata for this key. - */ - public SSTable.BlockMetadata getBlockMetadata(String key) throws IOException; - - /** * This method dumps the next key/value into the DataOuputStream * passed in. * Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SSTable.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SSTable.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SSTable.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SSTable.java Mon Jun 8 19:18:18 2009 @@ -20,28 +20,16 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Hashtable; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; +import java.util.*; -import org.apache.log4j.Logger; - -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.RowMutation; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.BasicUtilities; import org.apache.cassandra.utils.BloomFilter; import org.apache.cassandra.utils.FileUtils; import org.apache.cassandra.utils.LogUtil; +import org.apache.log4j.Logger; /** * This class is built on top of the SequenceFile. It stores @@ -100,8 +88,6 @@ * If not we can avoid scanning it. */ private static Map bfs_ = new Hashtable(); - /* Maintains a touched set of keys */ - private static LinkedHashMap touchCache_ = new TouchedKeyCache(DatabaseDescriptor.getTouchKeyCacheSize()); /** * This class holds the position of a key in a block @@ -521,52 +507,6 @@ return getFile(dataFile_); } - /* - * Seeks to the specified key on disk. - */ - public void touch(final String clientKey, boolean fData) throws IOException - { - if (touchCache_.containsKey(dataFile_ + ":" + clientKey)) - { - return; - } - - IFileReader dataReader = SequenceFile.reader(dataFile_); - try - { - /* Morph the key */ - String decoratedKey = partitioner_.decorateKey(clientKey); - Coordinate fileCoordinate = getCoordinates(decoratedKey, dataReader, partitioner_); - /* Get offset of key from block Index */ - dataReader.seek(fileCoordinate.end_); - BlockMetadata blockMetadata = dataReader.getBlockMetadata(decoratedKey); - if (blockMetadata.position_ != -1L) - { - touchCache_.put(dataFile_ + ":" + clientKey, blockMetadata.position_); - } - - if (fData) - { - /* Read the data associated with this key and pull it into the Buffer Cache */ - if (blockMetadata.position_ != -1L) - { - dataReader.seek(blockMetadata.position_); - DataOutputBuffer bufOut = new DataOutputBuffer(); - dataReader.next(bufOut); - bufOut.reset(); - logger_.debug("Finished the touch of the key to pull it into buffer cache."); - } - } - } - finally - { - if (dataReader != null) - { - dataReader.close(); - } - } - } - private long beforeAppend(String decoratedKey) throws IOException { if (decoratedKey == null) Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SequenceFile.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SequenceFile.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SequenceFile.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/io/SequenceFile.java Mon Jun 8 19:18:18 2009 @@ -600,103 +600,6 @@ } /** - * Return the block index metadata for a given key. - */ - public SSTable.BlockMetadata getBlockMetadata(String key) throws IOException - { - SSTable.BlockMetadata blockMetadata = SSTable.BlockMetadata.NULL; - /* read the block key. */ - String blockIndexKey = file_.readUTF(); - if (!blockIndexKey.equals(SSTable.blockIndexKey_)) - throw new IOException("Unexpected position to be reading the block index from."); - /* read the size of the block index */ - int size = file_.readInt(); - - /* Read the entire block index. */ - byte[] bytes = new byte[size]; - file_.readFully(bytes); - - DataInputBuffer bufIn = new DataInputBuffer(); - bufIn.reset(bytes, bytes.length); - - /* Number of keys in the block. */ - int keys = bufIn.readInt(); - for (int i = 0; i < keys; ++i) - { - if (bufIn.readUTF().equals(key)) - { - long position = bufIn.readLong(); - long dataSize = bufIn.readLong(); - blockMetadata = new SSTable.BlockMetadata(position, dataSize); - break; - } - else - { - /* - * This is not the key we are looking for. So read its position - * and the size of the data associated with it. This was strored - * as the BlockMetadata. - */ - bufIn.readLong(); - bufIn.readLong(); - } - } - - return blockMetadata; - } - - /** - * This function seeks to the position where the key data is present in the file - * in order to get the buffer cache populated with the key-data. This is done as - * a hint before the user actually queries the data. - * - * @param key the key whose data is being touched - * @param fData - */ - public long touch(String key, boolean fData) throws IOException - { - long bytesRead = -1L; - if (isEOF()) - return bytesRead; - - long startPosition = file_.getFilePointer(); - String keyInDisk = file_.readUTF(); - if (keyInDisk != null) - { - /* - * If key on disk is greater than requested key - * we can bail out since we exploit the property - * of the SSTable format. - */ - if (keyInDisk.compareTo(key) > 0) - return bytesRead; - - /* - * If we found the key then we populate the buffer that - * is passed in. If not then we skip over this key and - * position ourselves to read the next one. - */ - int dataSize = file_.readInt(); - if (keyInDisk.equals(key)) - { - /* return 0L to signal the key has been touched. */ - bytesRead = 0L; - return bytesRead; - } - else - { - /* skip over data portion */ - file_.seek(dataSize + file_.getFilePointer()); - } - - long endPosition = file_.getFilePointer(); - bytesRead = endPosition - startPosition; - } - - return bytesRead; - } - - /** * This method seek the disk head to the block index, finds * the offset of the key within the block and seeks to that * offset. Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/CassandraServer.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/CassandraServer.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/CassandraServer.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/CassandraServer.java Mon Jun 8 19:18:18 2009 @@ -527,17 +527,5 @@ return StorageProxy.getKeyRange(new RangeCommand(tablename, startWith, stopAt, maxResults)); } - /* - * This method is used to ensure that all keys - * prior to the specified key, as dtermined by - * the SSTable index bucket it falls in, are in - * buffer cache. - */ - public void touch (String key, boolean fData) - { - logger.debug("touch"); - StorageProxy.touchProtocol(DatabaseDescriptor.getTables().get(0), key, fData, StorageService.ConsistencyLevel.WEAK); - } - // main method moved to CassandraDaemon } Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageProxy.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageProxy.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageProxy.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageProxy.java Mon Jun 8 19:18:18 2009 @@ -262,83 +262,6 @@ return response.row(); } - static void touch_local(String tablename, String key, boolean fData ) throws IOException - { - Table table = Table.open( tablename ); - table.touch(key, fData); - } - - static void weakTouchProtocol(String tablename, String key, boolean fData) throws IOException - { - EndPoint endPoint = null; - try - { - endPoint = StorageService.instance().findSuitableEndPoint(key); - } - catch( Throwable ex) - { - ex.printStackTrace(); - } - if(endPoint != null) - { - if(endPoint.equals(StorageService.getLocalStorageEndPoint())) - { - touch_local(tablename, key, fData); - return; - } - TouchMessage touchMessage = null; - touchMessage = new TouchMessage(tablename, key, fData); - Message message = TouchMessage.makeTouchMessage(touchMessage); - MessagingService.getMessagingInstance().sendOneWay(message, endPoint); - } - } - - static void strongTouchProtocol(String tablename, String key, boolean fData) throws IOException - { - Map endpointMap = StorageService.instance().getNStorageEndPointMap(key); - Set endpoints = endpointMap.keySet(); - TouchMessage touchMessage = null; - touchMessage = new TouchMessage(tablename, key, fData); - Message message = TouchMessage.makeTouchMessage(touchMessage); - for(EndPoint endpoint : endpoints) - { - MessagingService.getMessagingInstance().sendOneWay(message, endpoint); - } - } - - /* - * Only touch data on the most suitable end point. - */ - public static void touchProtocol(String tablename, String key, boolean fData, StorageService.ConsistencyLevel consistencyLevel) - { - long startTime = System.currentTimeMillis(); - try - { - switch ( consistencyLevel ) - { - case WEAK: - weakTouchProtocol(tablename, key, fData); - break; - - case STRONG: - strongTouchProtocol(tablename, key, fData); - break; - - default: - weakTouchProtocol(tablename, key, fData); - break; - } - } - catch (IOException ex) - { - throw new RuntimeException("error touching key " + key, ex); - } - finally - { - readStats.add(System.currentTimeMillis() - startTime); - } - } - /** * Performs the actual reading of a row out of the StorageService, fetching * a specific set of column names from a given column family. Modified: incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageService.java URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageService.java?rev=782731&r1=782730&r2=782731&view=diff ============================================================================== --- incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageService.java (original) +++ incubator/cassandra/branches/cassandra-0.3/src/java/org/apache/cassandra/service/StorageService.java Mon Jun 8 19:18:18 2009 @@ -21,16 +21,7 @@ import java.io.File; import java.io.IOException; import java.lang.management.ManagementFactory; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Timer; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -40,65 +31,21 @@ import javax.management.MBeanServer; import javax.management.ObjectName; -import org.apache.log4j.Logger; - import org.apache.cassandra.analytics.AnalyticsContext; -import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor; -import org.apache.cassandra.concurrent.MultiThreadedStage; -import org.apache.cassandra.concurrent.SingleThreadedStage; -import org.apache.cassandra.concurrent.StageManager; -import org.apache.cassandra.concurrent.ThreadFactoryImpl; +import org.apache.cassandra.concurrent.*; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.BinaryVerbHandler; -import org.apache.cassandra.db.CalloutDeployVerbHandler; -import org.apache.cassandra.db.DBManager; -import org.apache.cassandra.db.DataFileVerbHandler; -import org.apache.cassandra.db.HintedHandOffManager; -import org.apache.cassandra.db.LoadVerbHandler; -import org.apache.cassandra.db.Memtable; -import org.apache.cassandra.db.ReadRepairVerbHandler; -import org.apache.cassandra.db.ReadVerbHandler; -import org.apache.cassandra.db.Row; -import org.apache.cassandra.db.RowMutationVerbHandler; -import org.apache.cassandra.db.SystemTable; -import org.apache.cassandra.db.Table; -import org.apache.cassandra.db.TouchVerbHandler; -import org.apache.cassandra.db.ReadCommand; -import org.apache.cassandra.dht.BootStrapper; -import org.apache.cassandra.dht.BootstrapInitiateMessage; -import org.apache.cassandra.dht.BootstrapMetadataVerbHandler; -import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.gms.IEndPointStateChangeSubscriber; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.EndPointState; -import org.apache.cassandra.gms.FailureDetector; -import org.apache.cassandra.net.EndPoint; -import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.Message; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.SelectorManager; +import org.apache.cassandra.db.*; +import org.apache.cassandra.dht.*; +import org.apache.cassandra.gms.*; +import org.apache.cassandra.locator.*; +import org.apache.cassandra.net.*; import org.apache.cassandra.net.http.HttpConnection; import org.apache.cassandra.net.io.StreamContextManager; -import org.apache.cassandra.locator.IEndPointSnitch; -import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.locator.IReplicaPlacementStrategy; -import org.apache.cassandra.locator.EndPointSnitch; -import org.apache.cassandra.locator.RackUnawareStrategy; -import org.apache.cassandra.locator.RackAwareStrategy; -import org.apache.cassandra.utils.LogUtil; -import org.apache.cassandra.utils.FileUtils; import org.apache.cassandra.tools.MembershipCleanerVerbHandler; - -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.cassandra.utils.FileUtils; +import org.apache.cassandra.utils.LogUtil; +import org.apache.log4j.Logger; import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.data.Stat; /* * This abstraction contains the token/identifier of this node @@ -134,7 +81,6 @@ public final static String mbrshipCleanerVerbHandler_ = "MBRSHIP-CLEANER-VERB-HANDLER"; public final static String bsMetadataVerbHandler_ = "BS-METADATA-VERB-HANDLER"; public final static String calloutDeployVerbHandler_ = "CALLOUT-DEPLOY-VERB-HANDLER"; - public final static String touchVerbHandler_ = "TOUCH-VERB-HANDLER"; public static String rangeVerbHandler_ = "RANGE-VERB-HANDLER"; public static enum ConsistencyLevel @@ -295,7 +241,6 @@ MessagingService.getMessagingInstance().registerVerbHandlers(StorageService.mbrshipCleanerVerbHandler_, new MembershipCleanerVerbHandler() ); MessagingService.getMessagingInstance().registerVerbHandlers(StorageService.bsMetadataVerbHandler_, new BootstrapMetadataVerbHandler() ); MessagingService.getMessagingInstance().registerVerbHandlers(StorageService.calloutDeployVerbHandler_, new CalloutDeployVerbHandler() ); - MessagingService.getMessagingInstance().registerVerbHandlers(StorageService.touchVerbHandler_, new TouchVerbHandler()); MessagingService.getMessagingInstance().registerVerbHandlers(StorageService.rangeVerbHandler_, new RangeVerbHandler()); /* register the stage for the mutations */