JIRA is not read-only, you should be able to create a ticket at https://issues.apache.org/jira/browse/CASSANDRA, though that probably require that you create an account.
-- Sylvain On Thu, Oct 13, 2011 at 3:20 PM, Thomas Richter <t...@tricnet.de> wrote: > Hi Aaron, > > the fix does the trick. I wonder why nobody else ran into this before... > I checked org/apache/cassandra/db/ColumnIndexer.java in 0.7.9, 0.8.7 and > 1.0.0-rc2 and all seem to be affected. > > Looks like public Jira is readonly - so I'm not sure about how to continue. > > Best, > > Thomas > > On 10/13/2011 10:52 AM, Thomas Richter wrote: >> Hi Aaron, >> >> I guess i found it :-). >> >> I added logging for the used IndexInfo to >> SSTableNamesIterator.readIndexedColumns and got negative index postions >> for the missing columns. This is the reason why the columns are not >> loaded from sstable. >> >> So I had a look at ColumnIndexer.serializeInternal and there it is: >> >> int endPosition = 0, startPosition = -1; >> >> Should be: >> >> long endPosition = 0, startPosition = -1; >> >> I'm currently running a compaction with a fixed version to verify. >> >> Best, >> >> Thomas >> >> On 10/12/2011 11:54 PM, aaron morton wrote: >>> Sounds a lot like the column is deleted. >>> >>> IIRC this is where the columns from various SSTables are reduced >>> https://github.com/apache/cassandra/blob/cassandra-0.8/src/java/org/apache/cassandra/db/filter/QueryFilter.java#L117 >>> >>> The call to ColumnFamily.addColumn() is where the column instance may be >>> merged with other instances. >>> >>> A >>> >>> ----------------- >>> Aaron Morton >>> Freelance Cassandra Developer >>> @aaronmorton >>> http://www.thelastpickle.com >>> >>> ----------------- >>> Aaron Morton >>> Freelance Cassandra Developer >>> @aaronmorton >>> http://www.thelastpickle.com >>> >>> On 13/10/2011, at 5:33 AM, Thomas Richter wrote: >>> >>>> Hi Aaron, >>>> >>>> I cannot read the column with a slice query. >>>> The slice query only returns data till a certain column and after that i >>>> only get empty results. >>>> >>>> I added log output to QueryFilter.isRelevant to see if the filter is >>>> dropping the column(s) but it doesn't even show up there. >>>> >>>> Next thing i will check check is the diff between columns contained in >>>> json export and columns fetched with the slice query, maybe this gives >>>> more clue... >>>> >>>> Any other ideas where to place more debugging output to see what's >>>> happening? >>>> >>>> Best, >>>> >>>> Thomas >>>> >>>> On 10/11/2011 12:46 PM, aaron morton wrote: >>>>> kewl, >>>>> >>>>>> * Row is not deleted (other columns can be read, row survives compaction >>>>>> with GCGraceSeconds=0) >>>>> >>>>> IIRC row tombstones can hang around for a while (until gc grace has >>>>> passed), and they only have an effect on columns that have a lower >>>>> timstamp. So it's possible to read columns from a row with a tombstone. >>>>> >>>>> Can you read the column using a slice range rather than specifying it's >>>>> name ? >>>>> >>>>> Aaron >>>>> >>>>> ----------------- >>>>> Aaron Morton >>>>> Freelance Cassandra Developer >>>>> @aaronmorton >>>>> http://www.thelastpickle.com >>>>> >>>>> On 11/10/2011, at 11:15 PM, Thomas Richter wrote: >>>>> >>>>>> Hi Aaron, >>>>>> >>>>>> i invalidated the caches but nothing changed. I didn't get the mentioned >>>>>> log line either, but as I read the code SliceByNamesReadCommand uses >>>>>> NamesQueryFilter and not SliceQueryFilter. >>>>>> >>>>>> Next, there is only one SSTable. >>>>>> >>>>>> I can rule out that the row is deleted because I deleted all other rows >>>>>> in that CF to reduce data size and speed up testing. I set >>>>>> GCGraceSeconds to zero and ran a compaction. All other rows are gone, >>>>>> but i can still access at least one column from the left row. >>>>>> So as far as I understand it, there should not be a tombstone on row >>>>>> level. >>>>>> >>>>>> To make it a list: >>>>>> >>>>>> * One SSTable, one row >>>>>> * >>>>>> * Row is not deleted (other columns can be read, row survives compaction >>>>>> with GCGraceSeconds=0) >>>>>> * Most columns can be read by get['row']['col'] from cassandra-cli >>>>>> * Some columns can not be read by get['row']['col'] from cassandra-cli >>>>>> but can be found in output of sstable2json >>>>>> * unreadable data survives compaction with GCGraceSeconds=0 (checked >>>>>> with sstable2json) >>>>>> * Invalidation caches does not help >>>>>> * Nothing in the logs >>>>>> >>>>>> Does that point into any direction where i should look next? >>>>>> >>>>>> Best, >>>>>> >>>>>> Thomas >>>>>> >>>>>> On 10/11/2011 10:30 AM, aaron morton wrote: >>>>>>> Nothing jumps out. The obvious answer is that the column has been >>>>>>> deleted. Did you check all the SSTables ? >>>>>>> >>>>>>> It looks like query returned from row cache, otherwise you would see >>>>>>> this as well… >>>>>>> >>>>>>> DEBUG [ReadStage:34] 2011-10-11 21:11:11,484 SliceQueryFilter.java >>>>>>> (line 123) collecting 0 of 2147483647: >>>>>>> 1318294191654059:false:354@1318294191654861 >>>>>>> >>>>>>> Which would mean a version of the column was found. >>>>>>> >>>>>>> If you invalidate the cache with nodetool and run the query and the log >>>>>>> message appears it will mean the column was read from (all of the) >>>>>>> sstables. If you do not get a column returned I would say there is a >>>>>>> tombstone in place. It's either a row level or a column level one. >>>>>>> >>>>>>> Hope that helps. >>>>>>> >>>>>>> ----------------- >>>>>>> Aaron Morton >>>>>>> Freelance Cassandra Developer >>>>>>> @aaronmorton >>>>>>> http://www.thelastpickle.com >>>>>>> >>>>>>> On 11/10/2011, at 10:35 AM, Thomas Richter wrote: >>>>>>> >>>>>>>> Hi Aaron, >>>>>>>> >>>>>>>> normally we use hector to access cassandra, but for debugging I >>>>>>>> switched >>>>>>>> to cassandra-cli. >>>>>>>> >>>>>>>> Column can not be read by a simple >>>>>>>> get CFName['rowkey']['colname']; >>>>>>>> >>>>>>>> Response is "Value was not found" >>>>>>>> if i query another column, everything is just fine. >>>>>>>> >>>>>>>> Serverlog for unsuccessful read (keyspace and CF names replaced): >>>>>>>> >>>>>>>> DEBUG [pool-1-thread-1] 2011-10-10 23:15:29,739 CassandraServer.java >>>>>>>> (line 280) get >>>>>>>> >>>>>>>> DEBUG [pool-1-thread-1] 2011-10-10 23:15:29,744 StorageProxy.java (line >>>>>>>> 320) Command/ConsistencyLevel is >>>>>>>> SliceByNamesReadCommand(table='Keyspace', >>>>>>>> key=61636162626139322d396638312d343562382d396637352d393162303337383030393762, >>>>>>>> columnParent='QueryPath(columnFamilyName='ColumnFamily', >>>>>>>> superColumnName='null', columnName='null')', >>>>>>>> columns=[574c303030375030,])/ONE >>>>>>>> >>>>>>>> DEBUG [pool-1-thread-1] 2011-10-10 23:15:29,750 ReadCallback.java (line >>>>>>>> 86) Blockfor/repair is 1/true; setting up requests to >>>>>>>> localhost/127.0.0.1 >>>>>>>> >>>>>>>> DEBUG [pool-1-thread-1] 2011-10-10 23:15:29,750 StorageProxy.java (line >>>>>>>> 343) reading data locally >>>>>>>> >>>>>>>> DEBUG [ReadStage:33] 2011-10-10 23:15:29,751 StorageProxy.java (line >>>>>>>> 448) LocalReadRunnable reading >>>>>>>> SliceByNamesReadCommand(table='Keyspace', >>>>>>>> key=61636162626139322d396638312d343562382d396637352d393162303337383030393762, >>>>>>>> columnParent='QueryPath(columnFamilyName='ColumnFamily', >>>>>>>> superColumnName='null', columnName='null')', >>>>>>>> columns=[574c303030375030,]) >>>>>>>> >>>>>>>> DEBUG [pool-1-thread-1] 2011-10-10 23:15:29,818 StorageProxy.java (line >>>>>>>> 393) Read: 67 ms. >>>>>>>> >>>>>>>> Log looks fine to me, but no result is returned. >>>>>>>> >>>>>>>> Best, >>>>>>>> >>>>>>>> Thomas >>>>>>>> >>>>>>>> On 10/10/2011 10:00 PM, aaron morton wrote: >>>>>>>>> How are they unreadable ? You need to go into some details about what >>>>>>>>> is going wrong. >>>>>>>>> >>>>>>>>> What sort of read ? >>>>>>>>> What client ? >>>>>>>>> What is in the logging on client and server side ? >>>>>>>>> >>>>>>>>> >>>>>>>>> Try turning the logging up to DEBUG on the server to watch what >>>>>>>>> happens. >>>>>>>>> >>>>>>>>> Cheers >>>>>>>>> >>>>>>>>> ----------------- >>>>>>>>> Aaron Morton >>>>>>>>> Freelance Cassandra Developer >>>>>>>>> @aaronmorton >>>>>>>>> http://www.thelastpickle.com >>>>>>>>> >>>>>>>>> On 10/10/2011, at 9:23 PM, Thomas Richter wrote: >>>>>>>>> >>>>>>>>>> Hi, >>>>>>>>>> >>>>>>>>>> no errors in the server logs. The columns are unreadable on all >>>>>>>>>> nodes at >>>>>>>>>> any consistency level (ONE, QUORUM, ALL). We started with 0.7.3 and >>>>>>>>>> upgraded to 0.7.6-2 two months ago. >>>>>>>>>> >>>>>>>>>> Best, >>>>>>>>>> >>>>>>>>>> Thomas >>>>>>>>>> >>>>>>>>>> On 10/10/2011 10:03 AM, aaron morton wrote: >>>>>>>>>>> What error are you seeing in the server logs ? Are the columns >>>>>>>>>>> unreadable at all Consistency Levels ? i.e. are the columns >>>>>>>>>>> unreadable on all nodes. >>>>>>>>>>> >>>>>>>>>>> What is the upgrade history of the cluster ? What version did it >>>>>>>>>>> start at ? >>>>>>>>>>> >>>>>>>>>>> Cheers >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> ----------------- >>>>>>>>>>> Aaron Morton >>>>>>>>>>> Freelance Cassandra Developer >>>>>>>>>>> @aaronmorton >>>>>>>>>>> http://www.thelastpickle.com >>>>>>>>>>> >>>>>>>>>>> On 10/10/2011, at 7:42 AM, Thomas Richter wrote: >>>>>>>>>>> >>>>>>>>>>>> Hi, >>>>>>>>>>>> >>>>>>>>>>>> here is some further information. Compaction did not help, but >>>>>>>>>>>> data is >>>>>>>>>>>> still there when I dump the row with sstable2json. >>>>>>>>>>>> >>>>>>>>>>>> Best, >>>>>>>>>>>> >>>>>>>>>>>> Thomas >>>>>>>>>>>> >>>>>>>>>>>> On 10/08/2011 11:30 PM, Thomas Richter wrote: >>>>>>>>>>>>> Hi, >>>>>>>>>>>>> >>>>>>>>>>>>> we are running a 3 node cassandra (0.7.6-2) cluster and some of >>>>>>>>>>>>> our >>>>>>>>>>>>> column families contain quite large rows (400k+ columns, 4-6GB >>>>>>>>>>>>> row size). >>>>>>>>>>>>> Replicaton factor is 3 for all keyspaces. The cluster is running >>>>>>>>>>>>> fine >>>>>>>>>>>>> for several months now and we never experienced any serious >>>>>>>>>>>>> trouble. >>>>>>>>>>>>> >>>>>>>>>>>>> Some days ago we noticed, that some previously written columns >>>>>>>>>>>>> could not >>>>>>>>>>>>> be read. This does not always happen, and only some dozen columns >>>>>>>>>>>>> out of >>>>>>>>>>>>> 400k are affected. >>>>>>>>>>>>> >>>>>>>>>>>>> After ruling out application logic as a cause I dumped the row in >>>>>>>>>>>>> question with sstable2json and the columns are there (and are not >>>>>>>>>>>>> marked >>>>>>>>>>>>> for deletion). >>>>>>>>>>>>> >>>>>>>>>>>>> Next thing was setting up a fresh single node cluster and copying >>>>>>>>>>>>> the >>>>>>>>>>>>> column family data to that node. Columns could not be read either. >>>>>>>>>>>>> Right now I'm running a nodetool compact for the cf to see if >>>>>>>>>>>>> data could >>>>>>>>>>>>> be read afterwards. >>>>>>>>>>>>> >>>>>>>>>>>>> Is there any explanation for such behavior? Are there any >>>>>>>>>>>>> suggestions >>>>>>>>>>>>> for further investigation? >>>>>>>>>>>>> >>>>>>>>>>>>> TIA, >>>>>>>>>>>>> >>>>>>>>>>>>> Thomas >>>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >>> >> > >