cassandra git commit: Prevent AssertionError from SizeEstimatesRecorder
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 b66092f90 - bd4842410 Prevent AssertionError from SizeEstimatesRecorder patch by Carl Yeksigian; reviewed by Stefania Alborghetti for CASSANDRA-9034 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bd484241 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bd484241 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bd484241 Branch: refs/heads/cassandra-2.1 Commit: bd4842410e73574dff8f3a51bd95e414f76ed506 Parents: b66092f Author: Carl Yeksigian c...@apache.org Authored: Thu Mar 26 14:58:46 2015 -0400 Committer: Aleksey Yeschenko alek...@apache.org Committed: Fri Mar 27 10:08:54 2015 +0300 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java | 6 ++ 2 files changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd484241/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index dba397c..3f5571e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.4 + * Prevent AssertionError from SizeEstimatesRecorder (CASSANDRA-9034) * Avoid overwriting index summaries for sstables with an older format that does not support downsampling; rebuild summaries on startup when this is detected (CASSANDRA-8993) http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd484241/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java -- diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java index 1472c11..13d9c60 100644 --- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java +++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java @@ -55,6 +55,12 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable public void run() { +if (StorageService.instance.isStarting()) +{ +logger.debug(Node has not yet joined; not recording size estimates); +return; +} + logger.debug(Recording size estimates); // find primary token ranges for the local node.
[jira] [Commented] (CASSANDRA-9048) Delimited File Bulk Loader
[ https://issues.apache.org/jira/browse/CASSANDRA-9048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383404#comment-14383404 ] Aleksey Yeschenko commented on CASSANDRA-9048: -- We've just recently committed CASSANDRA-8225 that should increase {{COPY}} performance roughly 10x over CASSANDRA-7405, which was another 10x improvement over {{COPY}} in 2.0. That's a total improvement of ~100x between 2.0 and 2.1.4. We'll have 2.1.4 released soon. I believe it should be sufficiently fast for now, until the new Spark-based loader arrives. But if it isn't, there is at least a few more lhf x left there if we add prepared statements there w/ single-partition batching and use more than one node. bq. Hmm. My expectation was slightly different – I was looking for Spark to handle Cassandra-to-Cassandra transformations (CREATE TABLE AS, INSERT INTO ... SELECT FROM) and for COPY to continue simple load-from-file operations. Unless we get file parsing for free via Spark somehow (I don't think we do) then I don't see that as a particularly natural fit. I'm looking for Spark there for both as an input backend. We get it all for free - https://github.com/databricks/spark-csv and https://github.com/datastax/spark-cassandra-connector Delimited File Bulk Loader -- Key: CASSANDRA-9048 URL: https://issues.apache.org/jira/browse/CASSANDRA-9048 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brian Hess Fix For: 3.0 Attachments: CASSANDRA-9048.patch There is a strong need for bulk loading data from delimited files into Cassandra. Starting with delimited files means that the data is not currently in the SSTable format, and therefore cannot immediately leverage Cassandra's bulk loading tool, sstableloader, directly. A tool supporting delimited files much closer matches the format of the data more often than the SSTable format itself, and a tool that loads from delimited files is very useful. In order for this bulk loader to be more generally useful to customers, it should handle a number of options at a minimum: - support specifying the input file or to read the data from stdin (so other command-line programs can pipe into the loader) - supply the CQL schema for the input data - support all data types other than collections (collections is a stretch goal/need) - an option to specify the delimiter - an option to specify comma as the decimal delimiter (for international use casese) - an option to specify how NULL values are specified in the file (e.g., the empty string or the string NULL) - an option to specify how BOOLEAN values are specified in the file (e.g., TRUE/FALSE or 0/1) - an option to specify the Date and Time format - an option to skip some number of rows at the beginning of the file - an option to only read in some number of rows from the file - an option to indicate how many parse errors to tolerate - an option to specify a file that will contain all the lines that did not parse correctly (up to the maximum number of parse errors) - an option to specify the CQL port to connect to (with 9042 as the default). Additional options would be useful, but this set of options/features is a start. A word on COPY. COPY comes via CQLSH which requires the client to be the same version as the server (e.g., 2.0 CQLSH does not work with 2.1 Cassandra, etc). This tool should be able to connect to any version of Cassandra (within reason). For example, it should be able to handle 2.0.x and 2.1.x. Moreover, CQLSH's COPY command does not support a number of the options above. Lastly, the performance of COPY in 2.0.x is not high enough to be considered a bulk ingest tool. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9048) Delimited File Bulk Loader
[ https://issues.apache.org/jira/browse/CASSANDRA-9048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383436#comment-14383436 ] Aleksey Yeschenko commented on CASSANDRA-9048: -- bq. Spark certainly also makes sense for the CREATE-TABLE-AS-SELECT (which is what CASSANDRA-8234 is about, not about loading). That's how the ticket is currently worded, but it's meant to do more. Perhaps I should open a new one, to detail everything explicitly. bq. Aleksey Yeschenko - The use case here is that a client machine has a pile of delimited files that need to be loaded in bulk to Cassandra - a common use case we see. In the Spark-based tool, you would have to have Spark on the client (perhaps that's okay) but moreover it would be reading files from the local filesystem (or stdin), not from a distributed file system, so there would be no parallelism from Spark. Spark will be bundled with C* itself in a near version. Running the tool in Spark local mode would be the equivalent of today's COPY. Delimited File Bulk Loader -- Key: CASSANDRA-9048 URL: https://issues.apache.org/jira/browse/CASSANDRA-9048 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brian Hess Fix For: 3.0 Attachments: CASSANDRA-9048.patch There is a strong need for bulk loading data from delimited files into Cassandra. Starting with delimited files means that the data is not currently in the SSTable format, and therefore cannot immediately leverage Cassandra's bulk loading tool, sstableloader, directly. A tool supporting delimited files much closer matches the format of the data more often than the SSTable format itself, and a tool that loads from delimited files is very useful. In order for this bulk loader to be more generally useful to customers, it should handle a number of options at a minimum: - support specifying the input file or to read the data from stdin (so other command-line programs can pipe into the loader) - supply the CQL schema for the input data - support all data types other than collections (collections is a stretch goal/need) - an option to specify the delimiter - an option to specify comma as the decimal delimiter (for international use casese) - an option to specify how NULL values are specified in the file (e.g., the empty string or the string NULL) - an option to specify how BOOLEAN values are specified in the file (e.g., TRUE/FALSE or 0/1) - an option to specify the Date and Time format - an option to skip some number of rows at the beginning of the file - an option to only read in some number of rows from the file - an option to indicate how many parse errors to tolerate - an option to specify a file that will contain all the lines that did not parse correctly (up to the maximum number of parse errors) - an option to specify the CQL port to connect to (with 9042 as the default). Additional options would be useful, but this set of options/features is a start. A word on COPY. COPY comes via CQLSH which requires the client to be the same version as the server (e.g., 2.0 CQLSH does not work with 2.1 Cassandra, etc). This tool should be able to connect to any version of Cassandra (within reason). For example, it should be able to handle 2.0.x and 2.1.x. Moreover, CQLSH's COPY command does not support a number of the options above. Lastly, the performance of COPY in 2.0.x is not high enough to be considered a bulk ingest tool. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9034) AssertionError in SizeEstimatesRecorder
[ https://issues.apache.org/jira/browse/CASSANDRA-9034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383449#comment-14383449 ] Aleksey Yeschenko commented on CASSANDRA-9034: -- bq. Aleksey Yeschenko if you are happy could you take care of committing please? Doubt I'll ever be happy, so let's not wait for that. Committed to both 2.1 and 3.0, as {{bd48424}}, thanks. AssertionError in SizeEstimatesRecorder --- Key: CASSANDRA-9034 URL: https://issues.apache.org/jira/browse/CASSANDRA-9034 Project: Cassandra Issue Type: Bug Environment: Trunk (52ddfe412a) Reporter: Stefania Assignee: Carl Yeksigian Priority: Minor Fix For: 3.0 Attachments: 9034-trunk.txt One of the dtests of CASSANDRA-8236 (https://github.com/stef1927/cassandra-dtest/tree/8236) raises the following exception unless I set {{-Dcassandra.size_recorder_interval=0}}: {code} ERROR [OptionalTasks:1] 2015-03-25 12:58:47,015 CassandraDaemon.java:179 - Exception in thread Thread[OptionalTasks:1,5,main] java.lang.AssertionError: null at org.apache.cassandra.service.StorageService.getLocalTokens(StorageService.java:2235) ~[main/:na] at org.apache.cassandra.db.SizeEstimatesRecorder.run(SizeEstimatesRecorder.java:61) ~[main/:na] at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:82) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_76] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304) [na:1.7.0_76] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178) [na:1.7.0_76] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.7.0_76] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_76] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_76] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_76] INFO [RMI TCP Connection(2)-127.0.0.1] 2015-03-25 12:59:23,189 StorageService.java:863 - Joining ring by operator request {code} The test is {{start_node_without_join_test}} in _pushed_notifications_test.py_ but starting a node that won't join the ring might be sufficient to reproduce the exception (I haven't tried though). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/2] cassandra git commit: Prevent AssertionError from SizeEstimatesRecorder
Repository: cassandra Updated Branches: refs/heads/trunk 5d9574fc0 - 04f351d57 Prevent AssertionError from SizeEstimatesRecorder patch by Carl Yeksigian; reviewed by Stefania Alborghetti for CASSANDRA-9034 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bd484241 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bd484241 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bd484241 Branch: refs/heads/trunk Commit: bd4842410e73574dff8f3a51bd95e414f76ed506 Parents: b66092f Author: Carl Yeksigian c...@apache.org Authored: Thu Mar 26 14:58:46 2015 -0400 Committer: Aleksey Yeschenko alek...@apache.org Committed: Fri Mar 27 10:08:54 2015 +0300 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java | 6 ++ 2 files changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd484241/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index dba397c..3f5571e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.4 + * Prevent AssertionError from SizeEstimatesRecorder (CASSANDRA-9034) * Avoid overwriting index summaries for sstables with an older format that does not support downsampling; rebuild summaries on startup when this is detected (CASSANDRA-8993) http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd484241/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java -- diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java index 1472c11..13d9c60 100644 --- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java +++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java @@ -55,6 +55,12 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable public void run() { +if (StorageService.instance.isStarting()) +{ +logger.debug(Node has not yet joined; not recording size estimates); +return; +} + logger.debug(Recording size estimates); // find primary token ranges for the local node.
[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/04f351d5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/04f351d5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/04f351d5 Branch: refs/heads/trunk Commit: 04f351d57422b91dd1be3822fa28a3220a42056a Parents: 5d9574f bd48424 Author: Aleksey Yeschenko alek...@apache.org Authored: Fri Mar 27 10:11:31 2015 +0300 Committer: Aleksey Yeschenko alek...@apache.org Committed: Fri Mar 27 10:11:31 2015 +0300 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java | 6 ++ 2 files changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/04f351d5/CHANGES.txt -- diff --cc CHANGES.txt index 1780249,3f5571e..4a25079 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,83 -1,5 +1,84 @@@ +3.0 + * Compressed Commit Log (CASSANDRA-6809) + * Optimise IntervalTree (CASSANDRA-8988) + * Add a key-value payload for third party usage (CASSANDRA-8553) + * Bump metrics-reporter-config dependency for metrics 3.0 (CASSANDRA-8149) + * Partition intra-cluster message streams by size, not type (CASSANDRA-8789) + * Add WriteFailureException to native protocol, notify coordinator of + write failures (CASSANDRA-8592) + * Convert SequentialWriter to nio (CASSANDRA-8709) + * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761, 8850) + * Record client ip address in tracing sessions (CASSANDRA-8162) + * Indicate partition key columns in response metadata for prepared + statements (CASSANDRA-7660) + * Merge UUIDType and TimeUUIDType parse logic (CASSANDRA-8759) + * Avoid memory allocation when searching index summary (CASSANDRA-8793) + * Optimise (Time)?UUIDType Comparisons (CASSANDRA-8730) + * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836) + * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714) + * Avoid accessing partitioner through StorageProxy (CASSANDRA-8244, 8268) + * Upgrade Metrics library and remove depricated metrics (CASSANDRA-5657) + * Serializing Row cache alternative, fully off heap (CASSANDRA-7438) + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6707) + * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560) + * Support direct buffer decompression for reads (CASSANDRA-8464) + * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039) + * Group sstables for anticompaction correctly (CASSANDRA-8578) + * Add ReadFailureException to native protocol, respond + immediately when replicas encounter errors while handling + a read request (CASSANDRA-7886) + * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308) + * Allow mixing token and partition key restrictions (CASSANDRA-7016) + * Support index key/value entries on map collections (CASSANDRA-8473) + * Modernize schema tables (CASSANDRA-8261) + * Support for user-defined aggregation functions (CASSANDRA-8053) + * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419) + * Refactor SelectStatement, return IN results in natural order instead + of IN value list order and ignore duplicate values in partition key IN restrictions (CASSANDRA-7981) + * Support UDTs, tuples, and collections in user-defined + functions (CASSANDRA-7563) + * Fix aggregate fn results on empty selection, result column name, + and cqlsh parsing (CASSANDRA-8229) + * Mark sstables as repaired after full repair (CASSANDRA-7586) + * Extend Descriptor to include a format value and refactor reader/writer + APIs (CASSANDRA-7443) + * Integrate JMH for microbenchmarks (CASSANDRA-8151) + * Keep sstable levels when bootstrapping (CASSANDRA-7460) + * Add Sigar library and perform basic OS settings check on startup (CASSANDRA-7838) + * Support for aggregation functions (CASSANDRA-4914) + * Remove cassandra-cli (CASSANDRA-7920) + * Accept dollar quoted strings in CQL (CASSANDRA-7769) + * Make assassinate a first class command (CASSANDRA-7935) + * Support IN clause on any partition key column (CASSANDRA-7855) + * Support IN clause on any clustering column (CASSANDRA-4762) + * Improve compaction logging (CASSANDRA-7818) + * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917) + * Do anticompaction in groups (CASSANDRA-6851) + * Support user-defined functions (CASSANDRA-7395, 7526, 7562, 7740, 7781, 7929, + 7924, 7812, 8063, 7813, 7708) + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416) + * Move sstable RandomAccessReader to nio2, which allows using the + FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050) + * Remove CQL2
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383541#comment-14383541 ] Marcus Eriksson commented on CASSANDRA-9045: [~r0mant] did you see the compacting large row-message for the row you deleted in cqlsh.txt between 18:07 and 19:39? Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-8717: - Reviewer: Aleksey Yeschenko Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383628#comment-14383628 ] Erik Forsberg commented on CASSANDRA-9036: -- After applying patch: {noformat} INFO [CompactionExecutor:12] 2015-03-27 10:16:38,930 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') DEBUG [CompactionExecutor:12] 2015-03-27 10:16:39,423 Directories.java (line 265) removing candidate /cassandra, usable=732825808896, requested=933404582552 ERROR [CompactionExecutor:12] 2015-03-27 10:16:39,424 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:12,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} The number it reports as usable corresponds quite well with output from df: {noformat} # df /cassandra Filesystem 1K-blocks Used Available Use% Mounted on /dev/sda7 1893666392 1178016188 715650204 63% /cassandra {noformat} The number it reports as requested doesn't at all correspond with the actual file size: {noformat} # ls -l /cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 234667877465 Mar 21 04:42 /cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db {noformat} The file is compressed, we're using DeflateCompressor: {noformat} # sstablemetadata /cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db|grep Compr Compression ratio: 0.21589549046598225 {noformat} No quota. Filesystem is XFS. Is the estimation of space needed for compaction taking compression into account? disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now.
[jira] [Commented] (CASSANDRA-8808) CQLSSTableWriter: close does not work + more than one table throws ex
[ https://issues.apache.org/jira/browse/CASSANDRA-8808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383660#comment-14383660 ] Sebastian YEPES FERNANDEZ commented on CASSANDRA-8808: -- For anyone interested, I have just created a related issue that was introduced in 2.1.3 https://issues.apache.org/jira/browse/CASSANDRA-9052 CQLSSTableWriter: close does not work + more than one table throws ex - Key: CASSANDRA-8808 URL: https://issues.apache.org/jira/browse/CASSANDRA-8808 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sebastian YEPES FERNANDEZ Assignee: Benjamin Lerer Labels: cql Fix For: 2.1.4, 2.0.14 Attachments: CASSANDRA-8808-2.0-V2.txt, CASSANDRA-8808-2.0.txt, CASSANDRA-8808-2.1-V2.txt, CASSANDRA-8808-2.1.txt, CASSANDRA-8808-trunk-V2.txt, CASSANDRA-8808-trunk.txt I have encountered the following two issues: - When closing the CQLSSTableWriter it just hangs the process and does nothing. (https://issues.apache.org/jira/browse/CASSANDRA-8281) - When writing more than one table throws ex. (https://issues.apache.org/jira/browse/CASSANDRA-8251) These issue can be reproduced with the following code: {code:title=test.java|borderStyle=solid} import org.apache.cassandra.config.Config; import org.apache.cassandra.io.sstable.CQLSSTableWriter; public static void main(String[] args) { Config.setClientMode(true); CQLSSTableWriter w1 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t1) .forTable(CREATE TABLE kspc.t1 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t1 (id) VALUES ( ? );) .build(); CQLSSTableWriter w2 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t2) .forTable(CREATE TABLE kspc.t2 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t2 (id) VALUES ( ? );) .build(); try { w1.addRow(1); w2.addRow(1); w1.close(); w2.close(); } catch (Exception e) { System.out.println(e); } } {code} {code:title=The error|borderStyle=solid} Exception in thread main java.lang.ExceptionInInitializerError at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) at org.apache.cassandra.db.Keyspace.init(Keyspace.java:277) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:96) at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101) at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:226) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:145) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:120) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite$PojoCachedMethodSite.invoke(PojoMetaMethodSite.java:189) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite.call(PojoMetaMethodSite.java:53) at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:45) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:108) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:120) at com.allthingsmonitoring.utils.BulkDataLoader.main(BulkDataLoader.groovy:415) Caused by: java.lang.NullPointerException at org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters(DatabaseDescriptor.java:1053) at org.apache.cassandra.db.ColumnFamilyStore.clinit(ColumnFamilyStore.java:85) ... 18 more {code} I have just tested the in the cassandra-2.1 branch and the issue still persists. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8180) Optimize disk seek using min/max colunm name meta data when the LIMIT clause is used
[ https://issues.apache.org/jira/browse/CASSANDRA-8180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383642#comment-14383642 ] Stefania commented on CASSANDRA-8180: - Using {{MergeIterator}} is a great idea, I still have some details to iron out but it's already looking much better. I have one question : the iterator is over atoms but the sstable min and max column names are lists of ByteBuffer, which I can compare with atoms using the ClusteringComparator but it would be nice to convert the lower bound to an atom, so we can have only one generic type (the {{In}} type) in the MergeIterator specialization, which must feed atoms upstream. Is there a way to do this or do I just have to settle for having two different (comparable) types in MergeIterator? Optimize disk seek using min/max colunm name meta data when the LIMIT clause is used Key: CASSANDRA-8180 URL: https://issues.apache.org/jira/browse/CASSANDRA-8180 Project: Cassandra Issue Type: Improvement Components: Core Environment: Cassandra 2.0.10 Reporter: DOAN DuyHai Assignee: Stefania Priority: Minor Fix For: 3.0 I was working on an example of sensor data table (timeseries) and face a use case where C* does not optimize read on disk. {code} cqlsh:test CREATE TABLE test(id int, col int, val text, PRIMARY KEY(id,col)) WITH CLUSTERING ORDER BY (col DESC); cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 10, '10'); ... nodetool flush test test ... cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 20, '20'); ... nodetool flush test test ... cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 30, '30'); ... nodetool flush test test {code} After that, I activate request tracing: {code} cqlsh:test SELECT * FROM test WHERE id=1 LIMIT 1; activity | timestamp| source| source_elapsed ---+--+---+ execute_cql3_query | 23:48:46,498 | 127.0.0.1 | 0 Parsing SELECT * FROM test WHERE id=1 LIMIT 1; | 23:48:46,498 | 127.0.0.1 | 74 Preparing statement | 23:48:46,499 | 127.0.0.1 |253 Executing single-partition query on test | 23:48:46,499 | 127.0.0.1 |930 Acquiring sstable references | 23:48:46,499 | 127.0.0.1 |943 Merging memtable tombstones | 23:48:46,499 | 127.0.0.1 | 1032 Key cache hit for sstable 3 | 23:48:46,500 | 127.0.0.1 | 1160 Seeking to partition beginning in data file | 23:48:46,500 | 127.0.0.1 | 1173 Key cache hit for sstable 2 | 23:48:46,500 | 127.0.0.1 | 1889 Seeking to partition beginning in data file | 23:48:46,500 | 127.0.0.1 | 1901 Key cache hit for sstable 1 | 23:48:46,501 | 127.0.0.1 | 2373 Seeking to partition beginning in data file | 23:48:46,501 | 127.0.0.1 | 2384 Skipped 0/3 non-slice-intersecting sstables, included 0 due to tombstones | 23:48:46,501 | 127.0.0.1 | 2768 Merging data from memtables and 3 sstables | 23:48:46,501 | 127.0.0.1 | 2784 Read 2 live and 0 tombstoned cells | 23:48:46,501 | 127.0.0.1 | 2976 Request complete | 23:48:46,501 | 127.0.0.1 | 3551 {code} We can clearly see that C* hits 3 SSTables on disk instead of just one, although it has the min/max column meta data to decide which SSTable contains the most recent data. Funny enough, if we add a clause on the clustering column to the select, this time C* optimizes the read path: {code} cqlsh:test SELECT * FROM test WHERE id=1 AND col 25 LIMIT 1; activity | timestamp| source| source_elapsed ---+--+---+ execute_cql3_query | 23:52:31,888 | 127.0.0.1 | 0
[jira] [Commented] (CASSANDRA-8180) Optimize disk seek using min/max colunm name meta data when the LIMIT clause is used
[ https://issues.apache.org/jira/browse/CASSANDRA-8180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383652#comment-14383652 ] Sylvain Lebresne commented on CASSANDRA-8180: - bq. it would be nice to convert the lower bound to an atom I suspect you don't need to have an Atom, only a Clusterable, in which case you can convert the lower bound to a Clustering with something like {{new SimpleClustering(sstable.minClusteringValues.toArray(new ByteBuffer\[metadata.clusteringColumns().size()\]))}}. Optimize disk seek using min/max colunm name meta data when the LIMIT clause is used Key: CASSANDRA-8180 URL: https://issues.apache.org/jira/browse/CASSANDRA-8180 Project: Cassandra Issue Type: Improvement Components: Core Environment: Cassandra 2.0.10 Reporter: DOAN DuyHai Assignee: Stefania Priority: Minor Fix For: 3.0 I was working on an example of sensor data table (timeseries) and face a use case where C* does not optimize read on disk. {code} cqlsh:test CREATE TABLE test(id int, col int, val text, PRIMARY KEY(id,col)) WITH CLUSTERING ORDER BY (col DESC); cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 10, '10'); ... nodetool flush test test ... cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 20, '20'); ... nodetool flush test test ... cqlsh:test INSERT INTO test(id, col , val ) VALUES ( 1, 30, '30'); ... nodetool flush test test {code} After that, I activate request tracing: {code} cqlsh:test SELECT * FROM test WHERE id=1 LIMIT 1; activity | timestamp| source| source_elapsed ---+--+---+ execute_cql3_query | 23:48:46,498 | 127.0.0.1 | 0 Parsing SELECT * FROM test WHERE id=1 LIMIT 1; | 23:48:46,498 | 127.0.0.1 | 74 Preparing statement | 23:48:46,499 | 127.0.0.1 |253 Executing single-partition query on test | 23:48:46,499 | 127.0.0.1 |930 Acquiring sstable references | 23:48:46,499 | 127.0.0.1 |943 Merging memtable tombstones | 23:48:46,499 | 127.0.0.1 | 1032 Key cache hit for sstable 3 | 23:48:46,500 | 127.0.0.1 | 1160 Seeking to partition beginning in data file | 23:48:46,500 | 127.0.0.1 | 1173 Key cache hit for sstable 2 | 23:48:46,500 | 127.0.0.1 | 1889 Seeking to partition beginning in data file | 23:48:46,500 | 127.0.0.1 | 1901 Key cache hit for sstable 1 | 23:48:46,501 | 127.0.0.1 | 2373 Seeking to partition beginning in data file | 23:48:46,501 | 127.0.0.1 | 2384 Skipped 0/3 non-slice-intersecting sstables, included 0 due to tombstones | 23:48:46,501 | 127.0.0.1 | 2768 Merging data from memtables and 3 sstables | 23:48:46,501 | 127.0.0.1 | 2784 Read 2 live and 0 tombstoned cells | 23:48:46,501 | 127.0.0.1 | 2976 Request complete | 23:48:46,501 | 127.0.0.1 | 3551 {code} We can clearly see that C* hits 3 SSTables on disk instead of just one, although it has the min/max column meta data to decide which SSTable contains the most recent data. Funny enough, if we add a clause on the clustering column to the select, this time C* optimizes the read path: {code} cqlsh:test SELECT * FROM test WHERE id=1 AND col 25 LIMIT 1; activity | timestamp| source| source_elapsed ---+--+---+ execute_cql3_query | 23:52:31,888 | 127.0.0.1 | 0 Parsing SELECT * FROM test WHERE id=1 AND col 25 LIMIT 1; | 23:52:31,888 | 127.0.0.1 | 60 Preparing statement | 23:52:31,888 | 127.0.0.1 |277
[jira] [Created] (CASSANDRA-9052) CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281
Sebastian YEPES FERNANDEZ created CASSANDRA-9052: Summary: CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281 Key: CASSANDRA-9052 URL: https://issues.apache.org/jira/browse/CASSANDRA-9052 Project: Cassandra Issue Type: Bug Environment: cassandra-all:2.1.2 cassandra-all:2.1.3 Reporter: Sebastian YEPES FERNANDEZ Hello, I have just noticed that the last C* version 2.1.3 reintroduced an old bug CASSANDRA-8281. When closing the CQLSSTableWriter after adding rows it generated the following Exception: {code:title=Exception|borderStyle=solid} Exception in thread main java.lang.ExceptionInInitializerError at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) at org.apache.cassandra.db.Keyspace.init(Keyspace.java:277) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:96) at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101) at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:225) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:144) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:119) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite$PojoCachedMethodSite.invoke(PojoMetaMethodSite.java:189) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite.call(PojoMetaMethodSite.java:53) at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:45) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:110) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:122) at BulkDataLoader.main(BulkDataLoader.groovy:26) Caused by: java.lang.NullPointerException at org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters(DatabaseDescriptor.java:1053) at org.apache.cassandra.db.ColumnFamilyStore.clinit(ColumnFamilyStore.java:85) ... 18 more {code} Note that is this works correctly in 2.1.2 and not with 2.1.3, we can reproduce this issue with the following code: {code:title=test.java|borderStyle=solid} import org.apache.cassandra.config.Config; import org.apache.cassandra.io.sstable.CQLSSTableWriter; public static void main(String[] args) { Config.setClientMode(true); // These folders needs must exist: mkdir -p /tmp/kspc/t1 CQLSSTableWriter w1 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t1) .forTable(CREATE TABLE kspc.t1 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t1 (id) VALUES ( ? );) .build(); try { w1.addRow(1); w1.close(); } catch (Exception e) { System.out.println(e); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383613#comment-14383613 ] Aleksey Yeschenko commented on CASSANDRA-8717: -- On second thought, this looks reasonable enough for at least 3.0 inclusion - especially if this eventually allows you guys to get rid of that C* fork. Still, I want to hear from [~slebresne] and [~beobal], the latter planning to do some C* API refactoring for a while, before proceeding. Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8099) Refactor and modernize the storage engine
[ https://issues.apache.org/jira/browse/CASSANDRA-8099?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383633#comment-14383633 ] Benjamin Lerer commented on CASSANDRA-8099: --- As the patch is relatively large I have choose to split my review of the CQL layer into chunks and give my comments for each chunk as soon as I have finished reviewing it. I think it will make the things more manageable for Sylvain and me. For the first chunk I focused on the {{restrictions}}: * I am not a big fan of big class hierachies but I wonder if it will not be better to have two sub-classes for {{PrimaryKeyRestrictionSet}} one for the partition key and one for the clustering columns rather than having a boolean variable. * In {{PrimaryKeyRestrictionSet}} the method {{addColumnFilterTo}} can be simplified based on the fact that we know if the restrictions are on the partition key components or on the clustering key columns. * The {{AbstractPrimaryKeyRestrictions.toByteBuffers}} method can be moved down as it is only used in {{PrimaryKeyRestrictionSet}} * In {{MultiColumnRestriction}} the method {{isPartitionKey()}} is not used (in case you have forgotten: {{MultiColumnRestriction}} only apply to clustering key columns). * I understand why you renamed {{?Restriction.Slice}} to {{?Restriction.SliceRestriction}} but now the class names look a bit inconsistent. May be we should rename the other classes too. * In {{ColumnFilter}} the {{add(Expression expression)}} method is not used. * In {{Operator}} the {{reverse}} method is not needed anymore and can be removed. * In {{StatementRestrictions}} I do not understand the use of {{useFiltering}}. My understanding was that we should return an error message specifying that {{ALLOW FILTERING}} is required and that this problem should have been handled by {{checkNeedsFiltering}} in {{SelectStatement}}. Could you explain? * In {{StatementRestrictions}} the {{nonPKRestrictedColumns}} method look wrong to me as it can return some primary key columns. Refactor and modernize the storage engine - Key: CASSANDRA-8099 URL: https://issues.apache.org/jira/browse/CASSANDRA-8099 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Fix For: 3.0 Attachments: 8099-nit The current storage engine (which for this ticket I'll loosely define as the code implementing the read/write path) is suffering from old age. One of the main problem is that the only structure it deals with is the cell, which completely ignores the more high level CQL structure that groups cell into (CQL) rows. This leads to many inefficiencies, like the fact that during a reads we have to group cells multiple times (to count on replica, then to count on the coordinator, then to produce the CQL resultset) because we forget about the grouping right away each time (so lots of useless cell names comparisons in particular). But outside inefficiencies, having to manually recreate the CQL structure every time we need it for something is hindering new features and makes the code more complex that it should be. Said storage engine also has tons of technical debt. To pick an example, the fact that during range queries we update {{SliceQueryFilter.count}} is pretty hacky and error prone. Or the overly complex ways {{AbstractQueryPager}} has to go into to simply remove the last query result. So I want to bite the bullet and modernize this storage engine. I propose to do 2 main things: # Make the storage engine more aware of the CQL structure. In practice, instead of having partitions be a simple iterable map of cells, it should be an iterable list of row (each being itself composed of per-column cells, though obviously not exactly the same kind of cell we have today). # Make the engine more iterative. What I mean here is that in the read path, we end up reading all cells in memory (we put them in a ColumnFamily object), but there is really no reason to. If instead we were working with iterators all the way through, we could get to a point where we're basically transferring data from disk to the network, and we should be able to reduce GC substantially. Please note that such refactor should provide some performance improvements right off the bat but it's not it's primary goal either. It's primary goal is to simplify the storage engine and adds abstraction that are better suited to further optimizations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-9052) CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281
[ https://issues.apache.org/jira/browse/CASSANDRA-9052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer reassigned CASSANDRA-9052: - Assignee: Benjamin Lerer CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281 - Key: CASSANDRA-9052 URL: https://issues.apache.org/jira/browse/CASSANDRA-9052 Project: Cassandra Issue Type: Bug Environment: cassandra-all:2.1.2 cassandra-all:2.1.3 Reporter: Sebastian YEPES FERNANDEZ Assignee: Benjamin Lerer Labels: API, CQL, SSTableWriter Hello, I have just noticed that the last C* version 2.1.3 reintroduced an old bug CASSANDRA-8281. When closing the CQLSSTableWriter after adding rows it generated the following Exception: {code:title=Exception|borderStyle=solid} Exception in thread main java.lang.ExceptionInInitializerError at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) at org.apache.cassandra.db.Keyspace.init(Keyspace.java:277) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:96) at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101) at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:225) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:144) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:119) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite$PojoCachedMethodSite.invoke(PojoMetaMethodSite.java:189) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite.call(PojoMetaMethodSite.java:53) at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:45) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:110) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:122) at BulkDataLoader.main(BulkDataLoader.groovy:26) Caused by: java.lang.NullPointerException at org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters(DatabaseDescriptor.java:1053) at org.apache.cassandra.db.ColumnFamilyStore.clinit(ColumnFamilyStore.java:85) ... 18 more {code} Note that is this works correctly in 2.1.2 and not with 2.1.3, we can reproduce this issue with the following code: {code:title=test.java|borderStyle=solid} import org.apache.cassandra.config.Config; import org.apache.cassandra.io.sstable.CQLSSTableWriter; public static void main(String[] args) { Config.setClientMode(true); // These folders needs must exist: mkdir -p /tmp/kspc/t1 CQLSSTableWriter w1 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t1) .forTable(CREATE TABLE kspc.t1 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t1 (id) VALUES ( ? );) .build(); try { w1.addRow(1); w1.close(); } catch (Exception e) { System.out.println(e); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383640#comment-14383640 ] Robbie Strickland commented on CASSANDRA-8717: -- FWIW, I spoke with several other teams at Spark Summit last week that would really like this patch for the same reason. Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383648#comment-14383648 ] Robert Stupp commented on CASSANDRA-9036: - OK, thanks for trying the patch. Then there's a bug in calculation of the sstable size for compaction - it requests 870GB, usable is 680GB ({{usable=732825808896, requested=933404582552}}). So it is correct to ignore that directory candidate - [~krummas]? disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now. Could it be some bug in the calculation of the expected size of the new sstable, perhaps? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383665#comment-14383665 ] Robert Stupp commented on CASSANDRA-9036: - Right - the request size is the _uncompressed_ size. Will provide a patch that multiples that with the (previous) compression ratio soon. (so yes, it is related to CASSANDRA-7386) disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now. Could it be some bug in the calculation of the expected size of the new sstable, perhaps? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383698#comment-14383698 ] Andrés de la Peña commented on CASSANDRA-8717: -- On our side, we would very much like to abandon the fork and distribute our index as a plugin once you guys agree that the proposed changes regarding top-K queries are a go. Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (CASSANDRA-8150) Revaluate Default JVM tuning parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-8150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-8150: - Comment: was deleted (was: Dear sender, I am giving a training and will be back on Tuesday 31-03-2015 During breaks I hope to respond on your email. Your email will not be forwarded. For urgent matters regarding GRTC / RTPE contact Peter v/d Koolwijk (peter.van.de.koolw...@ing.nl / 06-54660211 Or alternatively my manager Coos v/d Berg (coos.van.den.b...@ing.nl / 06-22018780) Best regards, Hans van der Linde - ATTENTION: The information in this electronic mail message is private and confidential, and only intended for the addressee. Should you receive this message by mistake, you are hereby notified that any disclosure, reproduction, distribution or use of this message is strictly prohibited. Please inform the sender by reply transmission and delete the message without copying or opening it. Messages and attachments are scanned for all viruses known. If this message contains password-protected attachments, the files have NOT been scanned for viruses by the ING mail domain. Always scan attachments before opening them. - ) Revaluate Default JVM tuning parameters --- Key: CASSANDRA-8150 URL: https://issues.apache.org/jira/browse/CASSANDRA-8150 Project: Cassandra Issue Type: Improvement Components: Config Reporter: Matt Stump Assignee: Ryan McGuire Attachments: upload.png It's been found that the old twitter recommendations of 100m per core up to 800m is harmful and should no longer be used. Instead the formula used should be 1/3 or 1/4 max heap with a max of 2G. 1/3 or 1/4 is debatable and I'm open to suggestions. If I were to hazard a guess 1/3 is probably better for releases greater than 2.1. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9051) Error in cqlsh command line while querying
Naresh Palaiya created CASSANDRA-9051: - Summary: Error in cqlsh command line while querying Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Critical Fix For: 2.1.2 Aggregation queries on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6680) Clock skew detection via gossip
[ https://issues.apache.org/jira/browse/CASSANDRA-6680?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383548#comment-14383548 ] Sergio Bossa commented on CASSANDRA-6680: - Hybrid Logical Clocks could be relevant for this issue: http://www.cse.buffalo.edu/tech-reports/2014-04.pdf Clock skew detection via gossip --- Key: CASSANDRA-6680 URL: https://issues.apache.org/jira/browse/CASSANDRA-6680 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Brandon Williams Assignee: Stefania Priority: Minor Fix For: 3.0 Gossip's HeartbeatState keeps the generation (local timestamp the node was started) and version (monotonically increasing per gossip interval) which could be used to roughly calculate the node's current time, enabling detection of gossip messages too far in the future for the clocks to be synced. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Naresh Palaiya updated CASSANDRA-9051: -- Description: Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds was: Aggregation queries on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Critical Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8984) Introduce Transactional API for behaviours that can corrupt system state
[ https://issues.apache.org/jira/browse/CASSANDRA-8984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383602#comment-14383602 ] Benedict commented on CASSANDRA-8984: - Rebased to trunk [here|https://github.com/belliottsmith/cassandra/tree/8984] Introduce Transactional API for behaviours that can corrupt system state Key: CASSANDRA-8984 URL: https://issues.apache.org/jira/browse/CASSANDRA-8984 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Fix For: 2.1.4 Attachments: 8984_windows_timeout.txt As a penultimate (and probably final for 2.1, if we agree to introduce it there) round of changes to the internals managing sstable writing, I've introduced a new API called Transactional that I hope will make it much easier to write correct behaviour. As things stand we conflate a lot of behaviours into methods like close - the recent changes unpicked some of these, but didn't go far enough. My proposal here introduces an interface designed to support four actions (on top of their normal function): * prepareToCommit * commit * abort * cleanup In normal operation, once we have finished constructing a state change we call prepareToCommit; once all such state changes are prepared, we call commit. If at any point everything fails, abort is called. In _either_ case, cleanup is called at the very last. These transactional objects are all AutoCloseable, with the behaviour being to rollback any changes unless commit has completed successfully. The changes are actually less invasive than it might sound, since we did recently introduce abort in some places, as well as have commit like methods. This simply formalises the behaviour, and makes it consistent between all objects that interact in this way. Much of the code change is boilerplate, such as moving an object into a try-declaration, although the change is still non-trivial. What it _does_ do is eliminate a _lot_ of special casing that we have had since 2.1 was released. The data tracker API changes and compaction leftover cleanups should finish the job with making this much easier to reason about, but this change I think is worthwhile considering for 2.1, since we've just overhauled this entire area (and not released these changes), and this change is essentially just the finishing touches, so the risk is minimal and the potential gains reasonably significant. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8993) EffectiveIndexInterval calculation is incorrect
[ https://issues.apache.org/jira/browse/CASSANDRA-8993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383561#comment-14383561 ] Benedict commented on CASSANDRA-8993: - I hate it when the explanation is simply that I'm an idiot (or if I'm charitable, that I forgot the reason). Given this explanation, I think either piece of code is as good as the other, so since it's your baby perhaps you can decide which you prefer? EffectiveIndexInterval calculation is incorrect --- Key: CASSANDRA-8993 URL: https://issues.apache.org/jira/browse/CASSANDRA-8993 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Blocker Fix For: 2.1.4 Attachments: 8993-2.1-v2.txt, 8993-2.1.txt, 8993.txt I'm not familiar enough with the calculation itself to understand why this is happening, but see discussion on CASSANDRA-8851 for the background. I've introduced a test case to look for this during downsampling, but it seems to pass just fine, so it may be an artefact of upgrading. The problem was, unfortunately, not manifesting directly because it would simply result in a failed lookup. This was only exposed when early opening used firstKeyBeyond, which does not use the effective interval, and provided the result to getPosition(). I propose a simple fix that ensures a bug here cannot break correctness. Perhaps [~thobbs] can follow up with an investigation as to how it actually went wrong? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9052) CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281
[ https://issues.apache.org/jira/browse/CASSANDRA-9052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383787#comment-14383787 ] Benjamin Lerer commented on CASSANDRA-9052: --- This ticket has nothing to do with CASSANDRA-8281. In CASSANDRA-8281 the problem was that some non daemon thread was preventing the JVM to shutdown there was no Exception. According to the stacktrace the Exception is trigger by the call to {{addRow}} not by the call to {{close}}. This ticket is in fact a duplicate of the other ticket that you have open CASSANDRA-8808. CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281 - Key: CASSANDRA-9052 URL: https://issues.apache.org/jira/browse/CASSANDRA-9052 Project: Cassandra Issue Type: Bug Environment: cassandra-all:2.1.2 cassandra-all:2.1.3 Reporter: Sebastian YEPES FERNANDEZ Assignee: Benjamin Lerer Labels: API, CQL, SSTableWriter Hello, I have just noticed that the last C* version 2.1.3 reintroduced an old bug CASSANDRA-8281. When closing the CQLSSTableWriter after adding rows it generated the following Exception: {code:title=Exception|borderStyle=solid} Exception in thread main java.lang.ExceptionInInitializerError at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) at org.apache.cassandra.db.Keyspace.init(Keyspace.java:277) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:96) at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101) at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:225) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:144) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:119) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite$PojoCachedMethodSite.invoke(PojoMetaMethodSite.java:189) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite.call(PojoMetaMethodSite.java:53) at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:45) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:110) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:122) at BulkDataLoader.main(BulkDataLoader.groovy:26) Caused by: java.lang.NullPointerException at org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters(DatabaseDescriptor.java:1053) at org.apache.cassandra.db.ColumnFamilyStore.clinit(ColumnFamilyStore.java:85) ... 18 more {code} Note that is this works correctly in 2.1.2 and not with 2.1.3, we can reproduce this issue with the following code: {code:title=test.java|borderStyle=solid} import org.apache.cassandra.config.Config; import org.apache.cassandra.io.sstable.CQLSSTableWriter; public static void main(String[] args) { Config.setClientMode(true); // These folders needs must exist: mkdir -p /tmp/kspc/t1 CQLSSTableWriter w1 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t1) .forTable(CREATE TABLE kspc.t1 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t1 (id) VALUES ( ? );) .build(); try { w1.addRow(1); w1.close(); } catch (Exception e) { System.out.println(e); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383704#comment-14383704 ] Sylvain Lebresne commented on CASSANDRA-8717: - I don't have a problem with this in theory, at least in 3.0 (I tend to agree with Aleksey on that part), though I could argue that what you fundamentally ask is not specific to indexing. What you want is a way to transform the result of internal queries. It's rather close to aggregation except that instead of transforming multiple rows into a single, you want to transform some rows into other rows (sorting them being just one particular use case of that). The fact that the results you want to transform is the result of your custom index is kind of incidental. So I do feel that implementing this as the more general concept of results transformation would be cleaner (and more generic). However, doing so is probably a little bit more involved so I'm happy to hijack the 2ndary index API for that in the short term and leave generalization to later, provided we agree that we may generalize that better and thus slightly break those new APIs. Now on the patch, I do think {{requiresFullScan}} somewhat break the {{concurrencyFactor}} computation in {{getRangeSlice}} as {{remainingRows}} can become negative. This is not a huge deal in the sense that the code ensure the {{concurrentFactor}} is never smaller than 1, but it still is kind of wrong in principle. In fact, that method is really about modifying the query limit internally (up until the combine method has been applied), and that's imo the proper way to expose it. Another nit is that we should rename the {{sort}} method in something more generic (as said above, sorting is somewhat of a special case and no reason to imply a limitation to that). It could be renamed {{combine}} or, imo a bit better, something like {{postReconciliationProcessing}}. Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-9052) CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281
[ https://issues.apache.org/jira/browse/CASSANDRA-9052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer resolved CASSANDRA-9052. --- Resolution: Duplicate CQLSSTableWriter close does not work - Regression bug: CASSANDRA-8281 - Key: CASSANDRA-9052 URL: https://issues.apache.org/jira/browse/CASSANDRA-9052 Project: Cassandra Issue Type: Bug Environment: cassandra-all:2.1.2 cassandra-all:2.1.3 Reporter: Sebastian YEPES FERNANDEZ Assignee: Benjamin Lerer Labels: API, CQL, SSTableWriter Hello, I have just noticed that the last C* version 2.1.3 reintroduced an old bug CASSANDRA-8281. When closing the CQLSSTableWriter after adding rows it generated the following Exception: {code:title=Exception|borderStyle=solid} Exception in thread main java.lang.ExceptionInInitializerError at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) at org.apache.cassandra.db.Keyspace.init(Keyspace.java:277) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:96) at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101) at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:225) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:144) at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:119) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite$PojoCachedMethodSite.invoke(PojoMetaMethodSite.java:189) at org.codehaus.groovy.runtime.callsite.PojoMetaMethodSite.call(PojoMetaMethodSite.java:53) at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:45) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:110) at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:122) at BulkDataLoader.main(BulkDataLoader.groovy:26) Caused by: java.lang.NullPointerException at org.apache.cassandra.config.DatabaseDescriptor.getFlushWriters(DatabaseDescriptor.java:1053) at org.apache.cassandra.db.ColumnFamilyStore.clinit(ColumnFamilyStore.java:85) ... 18 more {code} Note that is this works correctly in 2.1.2 and not with 2.1.3, we can reproduce this issue with the following code: {code:title=test.java|borderStyle=solid} import org.apache.cassandra.config.Config; import org.apache.cassandra.io.sstable.CQLSSTableWriter; public static void main(String[] args) { Config.setClientMode(true); // These folders needs must exist: mkdir -p /tmp/kspc/t1 CQLSSTableWriter w1 = CQLSSTableWriter.builder() .inDirectory(/tmp/kspc/t1) .forTable(CREATE TABLE kspc.t1 ( id int, PRIMARY KEY (id));) .using(INSERT INTO kspc.t1 (id) VALUES ( ? );) .build(); try { w1.addRow(1); w1.close(); } catch (Exception e) { System.out.println(e); } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8717) Top-k queries with custom secondary indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383812#comment-14383812 ] Andrés de la Peña commented on CASSANDRA-8717: -- I agree with your idea about doing this for the short term and leave generalization for later. We can deal with future API changes without problems. What we would need at 2i level is some way to specify that we need to scan all the nodes and the aforementioned method so as to combine the partial results. Indeed, sort is not the most fortunate name for this method... Top-k queries with custom secondary indexes --- Key: CASSANDRA-8717 URL: https://issues.apache.org/jira/browse/CASSANDRA-8717 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Andrés de la Peña Assignee: Andrés de la Peña Priority: Minor Labels: 2i, secondary_index, sort, sorting, top-k Fix For: 3.0 Attachments: 0001-Add-support-for-top-k-queries-in-2i.patch As presented in [Cassandra Summit Europe 2014|https://www.youtube.com/watch?v=Hg5s-hXy_-M], secondary indexes can be modified to support general top-k queries with minimum changes in Cassandra codebase. This way, custom 2i implementations could provide relevance search, sorting by columns, etc. Top-k queries retrieve the k best results for a certain query. That implies querying the k best rows in each token range and then sort them in order to obtain the k globally best rows. For doing that, we propose two additional methods in class SecondaryIndexSearcher: {code:java} public boolean requiresFullScan(ListIndexExpression clause) { return false; } public ListRow sort(ListIndexExpression clause, ListRow rows) { return rows; } {code} The first one indicates if a query performed in the index requires querying all the nodes in the ring. It is necessary in top-k queries because we do not know which node are the best results. The second method specifies how to sort all the partial node results according to the query. Then we add two similar methods to the class AbstractRangeCommand: {code:java} this.searcher = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily).indexManager.searcher(rowFilter); public boolean requiresFullScan() { return searcher == null ? false : searcher.requiresFullScan(rowFilter); } public ListRow combine(ListRow rows) { return searcher == null ? trim(rows) : trim(searcher.sort(rowFilter, rows)); } {code} Finnally, we modify StorageProxy#getRangeSlice to use the previous method, as shown in the attached patch. We think that the proposed approach provides very useful functionality with minimum impact in current codebase. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383844#comment-14383844 ] Sylvain Lebresne commented on CASSANDRA-8374: - So, I think it's time to decide where we go here. As I said in my previous comment, my preference here is to go with no default and an explicit choice. And doing so still leaves the ability to change our mind later and add a default, so I suggest we implement that and if someone feels strongly about a particular default, he can open a follow up ticket. [~snazy] can you rebase your branch, make sure it implements what's above and remind us where that branch actually is? Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7807) Push notification when tracing completes for an operation
[ https://issues.apache.org/jira/browse/CASSANDRA-7807?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-7807: Attachment: 7807-v3.txt I worked in all of your comments. {{SimpleClient}}: Regarding {{TransportException}} - unfortunately it’s an interface - not an (unchecked) exception class. {{Event}}: I decided to check the version at the top-level methods (so removed the additional, paranoid checks in the individual implementations) {{TraceState}}: Added functionality to {{Connection}}/{{ServerConnection}} that checks for registration. {{TraceCompleteTest}}: removed that unnecessary stuff (so it doesn’t waste time) {{MessagePayloadTest}}: Apologies for that. Push notification when tracing completes for an operation - Key: CASSANDRA-7807 URL: https://issues.apache.org/jira/browse/CASSANDRA-7807 Project: Cassandra Issue Type: Sub-task Components: Core Reporter: Tyler Hobbs Assignee: Robert Stupp Priority: Minor Labels: client-impacting, protocolv4 Fix For: 3.0 Attachments: 7807-v2.txt, 7807-v3.txt, 7807.txt Tracing is an asynchronous operation, and drivers currently poll to determine when the trace is complete (in a loop with sleeps). Instead, the server could push a notification to the driver when the trace completes. I'm guessing that most of the work for this will be around pushing notifications to a single connection instead of all connections that have registered listeners for a particular event type. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-9036: Attachment: 9036-3.0.txt 9036-2.1.txt 9036-2.0.txt Patch(es) are about to fix the issue in {{ColumnFamilyStore.getExpectedCompactedFileSize}}, which returned the on-disk size for non-CLEANUP-compactions but the uncompressed size for CLEANUP-compactions. [~krummas] can you review? disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp Attachments: 9036-2.0.txt, 9036-2.1.txt, 9036-3.0.txt I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now. Could it be some bug in the calculation of the expected size of the new sstable, perhaps? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7304) Ability to distinguish between NULL and UNSET values in Prepared Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-7304: Reviewer: Benjamin Lerer (was: Sylvain Lebresne) Ability to distinguish between NULL and UNSET values in Prepared Statements --- Key: CASSANDRA-7304 URL: https://issues.apache.org/jira/browse/CASSANDRA-7304 Project: Cassandra Issue Type: Sub-task Reporter: Drew Kutcharian Assignee: Oded Peer Labels: cql, protocolv4 Fix For: 3.0 Attachments: 7304-03.patch, 7304-04.patch, 7304-2.patch, 7304.patch Currently Cassandra inserts tombstones when a value of a column is bound to NULL in a prepared statement. At higher insert rates managing all these tombstones becomes an unnecessary overhead. This limits the usefulness of the prepared statements since developers have to either create multiple prepared statements (each with a different combination of column names, which at times is just unfeasible because of the sheer number of possible combinations) or fall back to using regular (non-prepared) statements. This JIRA is here to explore the possibility of either: A. Have a flag on prepared statements that once set, tells Cassandra to ignore null columns or B. Have an UNSET value which makes Cassandra skip the null columns and not tombstone them Basically, in the context of a prepared statement, a null value means delete, but we don’t have anything that means ignore (besides creating a new prepared statement without the ignored column). Please refer to the original conversation on DataStax Java Driver mailing list for more background: https://groups.google.com/a/lists.datastax.com/d/topic/java-driver-user/cHE3OOSIXBU/discussion *EDIT 18/12/14 - [~odpeer] Implementation Notes:* The motivation hasn't changed. Protocol version 4 specifies that bind variables do not require having a value when executing a statement. Bind variables without a value are called 'unset'. The 'unset' bind variable is serialized as the int value '-2' without following bytes. \\ \\ * An unset bind variable in an EXECUTE or BATCH request ** On a {{value}} does not modify the value and does not create a tombstone ** On the {{ttl}} clause is treated as 'unlimited' ** On the {{timestamp}} clause is treated as 'now' ** On a map key or a list index throws {{InvalidRequestException}} ** On a {{counter}} increment or decrement operation does not change the counter value, e.g. {{UPDATE my_tab SET c = c - ? WHERE k = 1}} does change the value of counter {{c}} ** On a tuple field or UDT field throws {{InvalidRequestException}} * An unset bind variable in a QUERY request ** On a partition column, clustering column or index column in the {{WHERE}} clause throws {{InvalidRequestException}} ** On the {{limit}} clause is treated as 'unlimited' -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7304) Ability to distinguish between NULL and UNSET values in Prepared Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383826#comment-14383826 ] Sylvain Lebresne commented on CASSANDRA-7304: - I apologize for not getting back on this one, it kind of slipped out of my review list, but we need to get that ready for 3.0. [~odpeer] would you have the time to rebase this to trunk? Ability to distinguish between NULL and UNSET values in Prepared Statements --- Key: CASSANDRA-7304 URL: https://issues.apache.org/jira/browse/CASSANDRA-7304 Project: Cassandra Issue Type: Sub-task Reporter: Drew Kutcharian Assignee: Oded Peer Labels: cql, protocolv4 Fix For: 3.0 Attachments: 7304-03.patch, 7304-04.patch, 7304-2.patch, 7304.patch Currently Cassandra inserts tombstones when a value of a column is bound to NULL in a prepared statement. At higher insert rates managing all these tombstones becomes an unnecessary overhead. This limits the usefulness of the prepared statements since developers have to either create multiple prepared statements (each with a different combination of column names, which at times is just unfeasible because of the sheer number of possible combinations) or fall back to using regular (non-prepared) statements. This JIRA is here to explore the possibility of either: A. Have a flag on prepared statements that once set, tells Cassandra to ignore null columns or B. Have an UNSET value which makes Cassandra skip the null columns and not tombstone them Basically, in the context of a prepared statement, a null value means delete, but we don’t have anything that means ignore (besides creating a new prepared statement without the ignored column). Please refer to the original conversation on DataStax Java Driver mailing list for more background: https://groups.google.com/a/lists.datastax.com/d/topic/java-driver-user/cHE3OOSIXBU/discussion *EDIT 18/12/14 - [~odpeer] Implementation Notes:* The motivation hasn't changed. Protocol version 4 specifies that bind variables do not require having a value when executing a statement. Bind variables without a value are called 'unset'. The 'unset' bind variable is serialized as the int value '-2' without following bytes. \\ \\ * An unset bind variable in an EXECUTE or BATCH request ** On a {{value}} does not modify the value and does not create a tombstone ** On the {{ttl}} clause is treated as 'unlimited' ** On the {{timestamp}} clause is treated as 'now' ** On a map key or a list index throws {{InvalidRequestException}} ** On a {{counter}} increment or decrement operation does not change the counter value, e.g. {{UPDATE my_tab SET c = c - ? WHERE k = 1}} does change the value of counter {{c}} ** On a tuple field or UDT field throws {{InvalidRequestException}} * An unset bind variable in a QUERY request ** On a partition column, clustering column or index column in the {{WHERE}} clause throws {{InvalidRequestException}} ** On the {{limit}} clause is treated as 'unlimited' -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383840#comment-14383840 ] Erik Forsberg commented on CASSANDRA-9036: -- Applied the 2.0 version and put in production on one of my nodes. Cleanup of my rather-large-file now started without exception. So I'm now officially happy! :-) disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp Attachments: 9036-2.0.txt, 9036-2.1.txt, 9036-3.0.txt I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now. Could it be some bug in the calculation of the expected size of the new sstable, perhaps? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8241) Use javac instead of javassist
[ https://issues.apache.org/jira/browse/CASSANDRA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383859#comment-14383859 ] Sylvain Lebresne commented on CASSANDRA-8241: - For what it's worth, I do think the lack of auto-boxing will be a pain in the ass. Not to say that I have the perfect solution. Use javac instead of javassist -- Key: CASSANDRA-8241 URL: https://issues.apache.org/jira/browse/CASSANDRA-8241 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Labels: udf Fix For: 3.0 Attachments: 8241-ecj.txt, udf-java-javac.txt Using JDK's built-in Java-Compiler API has some advantages over javassist. Although compilation feels a bit slower, Java compiler API has some advantages: * boxing + unboxing works * generics work * compiler error messages are better (or at least known) and have line/column numbers The implementation does not use any temp files. Everything's in memory. Patch attached to this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7304) Ability to distinguish between NULL and UNSET values in Prepared Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383897#comment-14383897 ] Oded Peer commented on CASSANDRA-7304: -- Yes. I will rebase next week. Thanks Ability to distinguish between NULL and UNSET values in Prepared Statements --- Key: CASSANDRA-7304 URL: https://issues.apache.org/jira/browse/CASSANDRA-7304 Project: Cassandra Issue Type: Sub-task Reporter: Drew Kutcharian Assignee: Oded Peer Labels: cql, protocolv4 Fix For: 3.0 Attachments: 7304-03.patch, 7304-04.patch, 7304-2.patch, 7304.patch Currently Cassandra inserts tombstones when a value of a column is bound to NULL in a prepared statement. At higher insert rates managing all these tombstones becomes an unnecessary overhead. This limits the usefulness of the prepared statements since developers have to either create multiple prepared statements (each with a different combination of column names, which at times is just unfeasible because of the sheer number of possible combinations) or fall back to using regular (non-prepared) statements. This JIRA is here to explore the possibility of either: A. Have a flag on prepared statements that once set, tells Cassandra to ignore null columns or B. Have an UNSET value which makes Cassandra skip the null columns and not tombstone them Basically, in the context of a prepared statement, a null value means delete, but we don’t have anything that means ignore (besides creating a new prepared statement without the ignored column). Please refer to the original conversation on DataStax Java Driver mailing list for more background: https://groups.google.com/a/lists.datastax.com/d/topic/java-driver-user/cHE3OOSIXBU/discussion *EDIT 18/12/14 - [~odpeer] Implementation Notes:* The motivation hasn't changed. Protocol version 4 specifies that bind variables do not require having a value when executing a statement. Bind variables without a value are called 'unset'. The 'unset' bind variable is serialized as the int value '-2' without following bytes. \\ \\ * An unset bind variable in an EXECUTE or BATCH request ** On a {{value}} does not modify the value and does not create a tombstone ** On the {{ttl}} clause is treated as 'unlimited' ** On the {{timestamp}} clause is treated as 'now' ** On a map key or a list index throws {{InvalidRequestException}} ** On a {{counter}} increment or decrement operation does not change the counter value, e.g. {{UPDATE my_tab SET c = c - ? WHERE k = 1}} does change the value of counter {{c}} ** On a tuple field or UDT field throws {{InvalidRequestException}} * An unset bind variable in a QUERY request ** On a partition column, clustering column or index column in the {{WHERE}} clause throws {{InvalidRequestException}} ** On the {{limit}} clause is treated as 'unlimited' -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383873#comment-14383873 ] Philip Thompson edited comment on CASSANDRA-9051 at 3/27/15 2:21 PM: - The error you are seeing is just a timeout. For non-trivial datasets, {{select count(\*) from table}} will time out. It's just a result of the C* architecture. You will need to use hadoop or spark analytic jobs to get row counts at scale. was (Author: philipthompson): The error you are seeing is just a timeout. For non-trivial datasets, {{select count(*) from table}} will time out. It's just a result of the C* architecture. You will need to use hadoop or spark analytic jobs to get row counts at scale. Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383923#comment-14383923 ] Robert Stupp commented on CASSANDRA-8374: - Sure Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383926#comment-14383926 ] Roman Tkachenko edited comment on CASSANDRA-9045 at 3/27/15 2:36 PM: - I did: {code} INFO [ValidationExecutor:8] 2015-03-26 18:53:41,404 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279555898 bytes) incrementally {code} was (Author: r0mant): I did: INFO [ValidationExecutor:8] 2015-03-26 18:53:41,404 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279555898 bytes) incrementally Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson resolved CASSANDRA-9051. Resolution: Not a Problem Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9051: --- Priority: Minor (was: Critical) The error you are seeing is just a timeout. For non-trivial datasets, {{select count(*) from table}} will time out. It's just a result of the C* architecture. You will need to use hadoop or spark analytic jobs to get row counts at scale. Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383895#comment-14383895 ] Naresh Palaiya commented on CASSANDRA-9051: --- [~philipthompson] I don'tr get this error. While executing the same query from Datastax driver for c#. I get the proper number, why is it so? Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383895#comment-14383895 ] Naresh Palaiya edited comment on CASSANDRA-9051 at 3/27/15 2:20 PM: [~philipthompson] I don't get this error while executing the same query from Datastax driver for c#. I get the proper number, why is it so? was (Author: palaiya): [~philipthompson] I don'tr get this error. While executing the same query from Datastax driver for c#. I get the proper number, why is it so? Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383926#comment-14383926 ] Roman Tkachenko commented on CASSANDRA-9045: I did: INFO [ValidationExecutor:8] 2015-03-26 18:53:41,404 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279555898 bytes) incrementally Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8241) Use javac instead of javassist
[ https://issues.apache.org/jira/browse/CASSANDRA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383933#comment-14383933 ] Robert Stupp commented on CASSANDRA-8241: - Using Oracle's {{JavaCompiler}} API implementation requires installation of a JDK (not a good option). We can bundle the Eclipse thing (which btw also implements the {{JavaCompiler}} API beside its native API. I see three options for this one (preferably option 3): # stick with javassist (effectively resolving this ticket as 'later') # use ecj with its native API (stick with ecj) # use ecj with {{JavaCompiler}} API (allows to use ecj and JDK) Use javac instead of javassist -- Key: CASSANDRA-8241 URL: https://issues.apache.org/jira/browse/CASSANDRA-8241 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Labels: udf Fix For: 3.0 Attachments: 8241-ecj.txt, udf-java-javac.txt Using JDK's built-in Java-Compiler API has some advantages over javassist. Although compilation feels a bit slower, Java compiler API has some advantages: * boxing + unboxing works * generics work * compiler error messages are better (or at least known) and have line/column numbers The implementation does not use any temp files. Everything's in memory. Patch attached to this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9051) Error in cqlsh command line while querying
[ https://issues.apache.org/jira/browse/CASSANDRA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383948#comment-14383948 ] Philip Thompson commented on CASSANDRA-9051: Discussion on expected is better taken up with the user mailing list or on IRC. Unless you encounter server errors in your system.log, timeouts on {{select count(\*) from table}} will not be considered a bug, regardless of whether they occur consistently or not. Error in cqlsh command line while querying -- Key: CASSANDRA-9051 URL: https://issues.apache.org/jira/browse/CASSANDRA-9051 Project: Cassandra Issue Type: Bug Components: Core Reporter: Naresh Palaiya Priority: Minor Fix For: 2.1.2 Aggregation queries (select count(*) from TABLE_NAME ) on Cassandra cluster results in the following error. Even after increasing the read_request_timeout_in_ms and range_request_timeout_in_ms parameters. For more information on the bug. You can refer the this stack overflow link. http://stackoverflow.com/questions/29205005/error-in-cqlsh-command-line-while-querying errors={}, last_host=localhost Statement trace did not complete within 10 seconds -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383946#comment-14383946 ] Robert Stupp commented on CASSANDRA-8374: - So (just for my understanding)... It should look like this (requiring the user to choose either {{RETURNS NULL ON NULL INPUT}} or {{CALLED ON NULL INPUT}} without a default): {code} CREATE FUNCTION foo ... [ RETURNS NULL ON NULL INPUT | CALLED ON NULL INPUT ] ... {code} Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8241) Use javac instead of javassist
[ https://issues.apache.org/jira/browse/CASSANDRA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383950#comment-14383950 ] Aleksey Yeschenko commented on CASSANDRA-8241: -- Before going any further w/ Eclipse, make sure we can bundle it at all, license-wise. Eclipse Distribution License is ASF-compatible (see https://www.apache.org/legal/resolved.html#category-a), but there is nothing there about Eclipse Public License. I'm not sure which one the Eclipse thing is covered by, but if it's not EDL, it might have to be vetted, though ultimately I expect it to be compatible. Use javac instead of javassist -- Key: CASSANDRA-8241 URL: https://issues.apache.org/jira/browse/CASSANDRA-8241 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Labels: udf Fix For: 3.0 Attachments: 8241-ecj.txt, udf-java-javac.txt Using JDK's built-in Java-Compiler API has some advantages over javassist. Although compilation feels a bit slower, Java compiler API has some advantages: * boxing + unboxing works * generics work * compiler error messages are better (or at least known) and have line/column numbers The implementation does not use any temp files. Everything's in memory. Patch attached to this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9048) Delimited File Bulk Loader
[ https://issues.apache.org/jira/browse/CASSANDRA-9048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383081#comment-14383081 ] Jonathan Ellis edited comment on CASSANDRA-9048 at 3/27/15 2:50 PM: How performant is this compared to CASSANDRA-7405? (Edit: thanks to Aleksey for the reminder that I meant CASSANDRA-8225.) was (Author: jbellis): How performant is this compared to CASSANDRA-7405? Delimited File Bulk Loader -- Key: CASSANDRA-9048 URL: https://issues.apache.org/jira/browse/CASSANDRA-9048 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brian Hess Fix For: 3.0 Attachments: CASSANDRA-9048.patch There is a strong need for bulk loading data from delimited files into Cassandra. Starting with delimited files means that the data is not currently in the SSTable format, and therefore cannot immediately leverage Cassandra's bulk loading tool, sstableloader, directly. A tool supporting delimited files much closer matches the format of the data more often than the SSTable format itself, and a tool that loads from delimited files is very useful. In order for this bulk loader to be more generally useful to customers, it should handle a number of options at a minimum: - support specifying the input file or to read the data from stdin (so other command-line programs can pipe into the loader) - supply the CQL schema for the input data - support all data types other than collections (collections is a stretch goal/need) - an option to specify the delimiter - an option to specify comma as the decimal delimiter (for international use casese) - an option to specify how NULL values are specified in the file (e.g., the empty string or the string NULL) - an option to specify how BOOLEAN values are specified in the file (e.g., TRUE/FALSE or 0/1) - an option to specify the Date and Time format - an option to skip some number of rows at the beginning of the file - an option to only read in some number of rows from the file - an option to indicate how many parse errors to tolerate - an option to specify a file that will contain all the lines that did not parse correctly (up to the maximum number of parse errors) - an option to specify the CQL port to connect to (with 9042 as the default). Additional options would be useful, but this set of options/features is a start. A word on COPY. COPY comes via CQLSH which requires the client to be the same version as the server (e.g., 2.0 CQLSH does not work with 2.1 Cassandra, etc). This tool should be able to connect to any version of Cassandra (within reason). For example, it should be able to handle 2.0.x and 2.1.x. Moreover, CQLSH's COPY command does not support a number of the options above. Lastly, the performance of COPY in 2.0.x is not high enough to be considered a bulk ingest tool. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383960#comment-14383960 ] Sylvain Lebresne commented on CASSANDRA-8374: - Yes. Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Clearer logic for first index summary entry in builder
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 bd4842410 - f7856c225 Clearer logic for first index summary entry in builder Patch by Tyler Hobbs; reviewed by Benedict Elliot Smith as a follow up for CASSANDRA-8993 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f7856c22 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f7856c22 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f7856c22 Branch: refs/heads/cassandra-2.1 Commit: f7856c225857e8bb975289f46e2b9b001888ea0c Parents: bd48424 Author: Tyler Hobbs tylerho...@apache.org Authored: Fri Mar 27 10:00:37 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Fri Mar 27 10:02:44 2015 -0500 -- .../org/apache/cassandra/io/sstable/IndexSummaryBuilder.java| 5 - 1 file changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7856c22/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java index 54e8dd2..ff06c10 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java @@ -109,7 +109,10 @@ public class IndexSummaryBuilder implements AutoCloseable maxExpectedEntries = Math.max(1, (maxExpectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL); offsets = new SafeMemoryWriter(4 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); entries = new SafeMemoryWriter(40 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); -setNextSamplePosition(-minIndexInterval); + +// the summary will always contain the first index entry (downsampling will never remove it) +nextSamplePosition = 0; +indexIntervalMatches++; } // the index file has been flushed to the provided position; stash it and use that to recalculate our max readable boundary
[1/2] cassandra git commit: Clearer logic for first index summary entry in builder
Repository: cassandra Updated Branches: refs/heads/trunk 04f351d57 - db900a374 Clearer logic for first index summary entry in builder Patch by Tyler Hobbs; reviewed by Benedict Elliot Smith as a follow up for CASSANDRA-8993 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f7856c22 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f7856c22 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f7856c22 Branch: refs/heads/trunk Commit: f7856c225857e8bb975289f46e2b9b001888ea0c Parents: bd48424 Author: Tyler Hobbs tylerho...@apache.org Authored: Fri Mar 27 10:00:37 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Fri Mar 27 10:02:44 2015 -0500 -- .../org/apache/cassandra/io/sstable/IndexSummaryBuilder.java| 5 - 1 file changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7856c22/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java index 54e8dd2..ff06c10 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java @@ -109,7 +109,10 @@ public class IndexSummaryBuilder implements AutoCloseable maxExpectedEntries = Math.max(1, (maxExpectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL); offsets = new SafeMemoryWriter(4 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); entries = new SafeMemoryWriter(40 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); -setNextSamplePosition(-minIndexInterval); + +// the summary will always contain the first index entry (downsampling will never remove it) +nextSamplePosition = 0; +indexIntervalMatches++; } // the index file has been flushed to the provided position; stash it and use that to recalculate our max readable boundary
[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Conflicts: src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/db900a37 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/db900a37 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/db900a37 Branch: refs/heads/trunk Commit: db900a37465d4e3a0c915d16725c890f3ee375e9 Parents: 04f351d f7856c2 Author: Tyler Hobbs tylerho...@apache.org Authored: Fri Mar 27 10:04:06 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Fri Mar 27 10:04:06 2015 -0500 -- .../org/apache/cassandra/io/sstable/IndexSummaryBuilder.java| 5 - 1 file changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/db900a37/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java -- diff --cc src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java index 2e96d03,ff06c10..696bbf8 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java @@@ -107,9 -107,12 +107,12 @@@ public class IndexSummaryBuilder implem // for initializing data structures, adjust our estimates based on the sampling level maxExpectedEntries = Math.max(1, (maxExpectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL); -offsets = new SafeMemoryWriter(4 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); -entries = new SafeMemoryWriter(40 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder()); +offsets = new SafeMemoryWriter(4 * maxExpectedEntries).order(ByteOrder.nativeOrder()); +entries = new SafeMemoryWriter(40 * maxExpectedEntries).order(ByteOrder.nativeOrder()); - setNextSamplePosition(-minIndexInterval); + + // the summary will always contain the first index entry (downsampling will never remove it) + nextSamplePosition = 0; + indexIntervalMatches++; } // the index file has been flushed to the provided position; stash it and use that to recalculate our max readable boundary
[jira] [Commented] (CASSANDRA-9050) Add debug level logging to Directories.getWriteableLocation()
[ https://issues.apache.org/jira/browse/CASSANDRA-9050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383971#comment-14383971 ] Yuki Morishita commented on CASSANDRA-9050: --- +1 Add debug level logging to Directories.getWriteableLocation() - Key: CASSANDRA-9050 URL: https://issues.apache.org/jira/browse/CASSANDRA-9050 Project: Cassandra Issue Type: Improvement Reporter: Robert Stupp Assignee: Robert Stupp Fix For: 2.0.14 Attachments: 9050-2.0.txt, 9050-2.1.txt Add some debug level logging to log * blacklisted directories that are excluded * directories not matching requested size -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8241) Use javac instead of javassist
[ https://issues.apache.org/jira/browse/CASSANDRA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383972#comment-14383972 ] Robert Stupp commented on CASSANDRA-8241: - Damn - yes, EPL is nod EDL. Probably EPL1.0 is not compatible with ASF2.0 (according to [this comparison|https://en.wikipedia.org/wiki/Comparison_of_free_and_open-source_software_licenses]). Tried to find a definitive answer on the web but had no luck. Use javac instead of javassist -- Key: CASSANDRA-8241 URL: https://issues.apache.org/jira/browse/CASSANDRA-8241 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Labels: udf Fix For: 3.0 Attachments: 8241-ecj.txt, udf-java-javac.txt Using JDK's built-in Java-Compiler API has some advantages over javassist. Although compilation feels a bit slower, Java compiler API has some advantages: * boxing + unboxing works * generics work * compiler error messages are better (or at least known) and have line/column numbers The implementation does not use any temp files. Everything's in memory. Patch attached to this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8993) EffectiveIndexInterval calculation is incorrect
[ https://issues.apache.org/jira/browse/CASSANDRA-8993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383976#comment-14383976 ] Tyler Hobbs commented on CASSANDRA-8993: I'm partial to the more explicit {{nextSamplePosition = 0}}, so I've committed that as f7856c22. Thanks, I think that should wrap this ticket up. EffectiveIndexInterval calculation is incorrect --- Key: CASSANDRA-8993 URL: https://issues.apache.org/jira/browse/CASSANDRA-8993 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Blocker Fix For: 2.1.4 Attachments: 8993-2.1-v2.txt, 8993-2.1.txt, 8993.txt I'm not familiar enough with the calculation itself to understand why this is happening, but see discussion on CASSANDRA-8851 for the background. I've introduced a test case to look for this during downsampling, but it seems to pass just fine, so it may be an artefact of upgrading. The problem was, unfortunately, not manifesting directly because it would simply result in a failed lookup. This was only exposed when early opening used firstKeyBeyond, which does not use the effective interval, and provided the result to getPosition(). I propose a simple fix that ensures a bug here cannot break correctness. Perhaps [~thobbs] can follow up with an investigation as to how it actually went wrong? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6096) Look into a Pig Macro to url encode URLs passed to CqlStorage
[ https://issues.apache.org/jira/browse/CASSANDRA-6096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384158#comment-14384158 ] Philip Thompson commented on CASSANDRA-6096: [~brandon.williams], is this patch still desired? I'll move to {{Patch Available}} Look into a Pig Macro to url encode URLs passed to CqlStorage - Key: CASSANDRA-6096 URL: https://issues.apache.org/jira/browse/CASSANDRA-6096 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Jeremy Hanna Priority: Minor Labels: lhf Attachments: trunk-6096.txt In the evolution of CqlStorage, the URL went from non-encoded to encoded. It would be great to somehow keep the URL readable, perhaps using the Pig macro interface to do expansion: http://pig.apache.org/docs/r0.9.2/cont.html#macros See also CASSANDRA-6073 and CASSANDRA-5867 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8314) C* 2.1.1: AssertionError: stream can only read forward
[ https://issues.apache.org/jira/browse/CASSANDRA-8314?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384175#comment-14384175 ] Philip Thompson commented on CASSANDRA-8314: Do you still have this problem when running upgradesstables with nodetool from 2.1.3? C* 2.1.1: AssertionError: stream can only read forward - Key: CASSANDRA-8314 URL: https://issues.apache.org/jira/browse/CASSANDRA-8314 Project: Cassandra Issue Type: Bug Components: Core Reporter: Donald Smith Fix For: 2.1.4 I see this multiple nodes on a 2.1.1 cluster running on CentOS 6.4: {noformat} ERROR [STREAM-IN-/10.6.1.104] 2014-11-13 14:13:16,565 StreamSession.java (line 470) [Stream #45bdfe30-6b81-11e4-a7ca-b150b4554347] Streaming error occurred java.io.IOException: Too many retries for Header (cfId: aaefa7d7-9d72-3d18-b5f0-02b30cee5bd7, #29, version: jb, estimated keys: 12672, transfer size: 130005779, compressed?: true, repairedAt: 0) at org.apache.cassandra.streaming.StreamSession.doRetry(StreamSession.java:594) [apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:53) [apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38) [apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) [apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:245) [apache-cassandra-2.1.1.jar:2.1.1] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_60] Caused by: java.lang.AssertionError: stream can only read forward. at org.apache.cassandra.streaming.compress.CompressedInputStream.position(CompressedInputStream.java:107) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:85) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:48) [apache-cassandra-2.1.1.jar:2.1.1] ... 4 common frames omitted {noformat} We couldn't upgrade SStables due to exceptions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8472) Streams hang in repair
[ https://issues.apache.org/jira/browse/CASSANDRA-8472?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-8472: --- Fix Version/s: 2.0.14 Streams hang in repair -- Key: CASSANDRA-8472 URL: https://issues.apache.org/jira/browse/CASSANDRA-8472 Project: Cassandra Issue Type: Bug Reporter: Jimmy Mårdell Fix For: 2.0.14 Attachments: errlogs In general streaming is working much better in 2.0.x than before, but we still get occasional hanging stream sessions. One of the nodes, the follower, throws IOException: Broken pipe, causing all streams to fail with the initiator node. But the initiator node still thinks its sending and receiving files from the follower, causing the streaming to hang forever. Relevant lines from the logs of the follower attached. There's nothing relevant in the logs on the initiator node. There are no indications of retry attempts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8534) The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set.
[ https://issues.apache.org/jira/browse/CASSANDRA-8534?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384184#comment-14384184 ] Philip Thompson commented on CASSANDRA-8534: [~mikea], am I correct in thinking that based on your response, this is not a problem? The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set. --- Key: CASSANDRA-8534 URL: https://issues.apache.org/jira/browse/CASSANDRA-8534 Project: Cassandra Issue Type: Bug Components: Config, Core Environment: Ubuntu 14.04 64-bit C* 2.1.2 Reporter: Andrew Trimble Priority: Minor Fix For: 2.1.4 Attachments: error.txt In the class org.apache.cassandra.config.YamlConfigurationLoader, the DEFAULT_CONFIGURATION is set to cassandra.yaml. This is improperly formatted as it does not contain the prefix file://. If this value is used, a ConfigurationException is thrown (see line 73 of the same class). A solution is to set the cassandra.config system property, but this does not solve the underlying problem. A vanilla Cassandra installation will throw this error. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7814) enable describe on indices
[ https://issues.apache.org/jira/browse/CASSANDRA-7814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384189#comment-14384189 ] Benjamin Lerer commented on CASSANDRA-7814: --- After removing the existing driver that was on my System path I ran into the following error: {quote}Python Cassandra driver not installed, or not on PYTHONPATH. You might try pip install cassandra-driver.{quote} I printed out the the path that cqlsh was using to load the cassandra driver and found: {{myPathToTheBinDirectory..\lib\cassandra-driver-internal-only-2.1.4.post.3d578f9b69.zip\cassandra-driver-2.1.4.post.3d578f9b69}} I looked into the zip file and found out that the root directory was: {{cassandra-driver-2.1.4.post0}} which explain why the driver could not be loaded. enable describe on indices -- Key: CASSANDRA-7814 URL: https://issues.apache.org/jira/browse/CASSANDRA-7814 Project: Cassandra Issue Type: Improvement Components: Core Reporter: radha Assignee: Stefania Priority: Minor Fix For: 2.1.4 Describe index should be supported, right now, the only way is to export the schema and find what it really is before updating/dropping the index. verified in [cqlsh 3.1.8 | Cassandra 1.2.18.1 | CQL spec 3.0.0 | Thrift protocol 19.36.2] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8951) Add smallint (and possibly byte) type
[ https://issues.apache.org/jira/browse/CASSANDRA-8951?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-8951: Fix Version/s: 3.0 Add smallint (and possibly byte) type - Key: CASSANDRA-8951 URL: https://issues.apache.org/jira/browse/CASSANDRA-8951 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Fix For: 3.0 We have {{int}} and {{bigint}}, but we don't have a {{smallint}} (2 bytes). I see no reason not to add it. And while we're at it, it doesn't cost much to add a {{byte}} type either. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8951) Add smallint (and possibly byte) type
[ https://issues.apache.org/jira/browse/CASSANDRA-8951?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-8951: Assignee: Benjamin Lerer Add smallint (and possibly byte) type - Key: CASSANDRA-8951 URL: https://issues.apache.org/jira/browse/CASSANDRA-8951 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Benjamin Lerer Fix For: 3.0 We have {{int}} and {{bigint}}, but we don't have a {{smallint}} (2 bytes). I see no reason not to add it. And while we're at it, it doesn't cost much to add a {{byte}} type either. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-5330) make it possible to run unittests in any order
[ https://issues.apache.org/jira/browse/CASSANDRA-5330?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-5330: --- Issue Type: Test (was: Bug) make it possible to run unittests in any order -- Key: CASSANDRA-5330 URL: https://issues.apache.org/jira/browse/CASSANDRA-5330 Project: Cassandra Issue Type: Test Components: Tests Reporter: Marcus Eriksson Priority: Minor SchemaLoader does its thing @BeforeClass and @AfterClass, should ideally be done @Before and @After - otherwise tests will se leftovers from earlier unit tests guessing this caused most of the issues in CASSANDRA-5315 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-5683) Nodetool command for modifying rpc_timeout
[ https://issues.apache.org/jira/browse/CASSANDRA-5683?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-5683: --- Summary: Nodetool command for modifying rpc_timeout (was: Write timeout in multi-dc environment ) Nodetool command for modifying rpc_timeout -- Key: CASSANDRA-5683 URL: https://issues.apache.org/jira/browse/CASSANDRA-5683 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: apache cassandra 1.1.9 Reporter: Boole Guo Labels: write Fix For: 3.0 When writing in mutil-dc environment, there are many timeout exception. As I know, this version have do good to reduce network bandwidth. Can we assign rpc timeout when writing like consistenylevel? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8951) Add smallint (and possibly byte) type
[ https://issues.apache.org/jira/browse/CASSANDRA-8951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384162#comment-14384162 ] Aleksey Yeschenko commented on CASSANDRA-8951: -- Would it make sense to at least make them non-emptiable? I remember the issue of null in primary key columns, but I'm not sure if that's that much of a deal, or if allowing null there would be a better overall solution if it is. Add smallint (and possibly byte) type - Key: CASSANDRA-8951 URL: https://issues.apache.org/jira/browse/CASSANDRA-8951 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Benjamin Lerer Fix For: 3.0 We have {{int}} and {{bigint}}, but we don't have a {{smallint}} (2 bytes). I see no reason not to add it. And while we're at it, it doesn't cost much to add a {{byte}} type either. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6702) Upgrading node uses the wrong port in gossiping
[ https://issues.apache.org/jira/browse/CASSANDRA-6702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384160#comment-14384160 ] Philip Thompson commented on CASSANDRA-6702: Has this been reproduced on an upgrade from 1.2 - 2.0, or only from 1.1 - 1.2? Upgrading node uses the wrong port in gossiping --- Key: CASSANDRA-6702 URL: https://issues.apache.org/jira/browse/CASSANDRA-6702 Project: Cassandra Issue Type: Bug Components: Core Environment: 1.1.7, AWS, Ec2MultiRegionSnitch Reporter: Minh Do Priority: Minor Fix For: 2.0.14 When upgrading a node in 1.1.7 (or 1.1.11) cluster to 1.2.15 and inspecting the gossip information on port/Ip, I could see that the upgrading node (1.2 version) communicates to one other node in the same region using Public IP and non-encrypted port. For the rest, the upgrading node uses the correct ports and IPs to communicate in this manner: Same region: private IP and non-encrypted port and Different region: public IP and encrypted port Because there is one node like this (or 2 out of 12 nodes cluster in which nodes are split equally on 2 AWS regions), we have to modify Security Group to allow the new traffics. Without modifying the SG, the 95th and 99th latencies for both reads and writes in the cluster are very bad (due to RPC timeout). Inspecting closer, that upgraded node (1.2 node) is contributing to all of the high latencies whenever it acts as a coordinator node. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8472) Streams hang in repair
[ https://issues.apache.org/jira/browse/CASSANDRA-8472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384198#comment-14384198 ] Yuki Morishita commented on CASSANDRA-8472: --- Yes, and I'm working on CASSANDRA-8621 as well and that can be fix for this as well. Streams hang in repair -- Key: CASSANDRA-8472 URL: https://issues.apache.org/jira/browse/CASSANDRA-8472 Project: Cassandra Issue Type: Bug Reporter: Jimmy Mårdell Fix For: 2.0.14 Attachments: errlogs In general streaming is working much better in 2.0.x than before, but we still get occasional hanging stream sessions. One of the nodes, the follower, throws IOException: Broken pipe, causing all streams to fail with the initiator node. But the initiator node still thinks its sending and receiving files from the follower, causing the streaming to hang forever. Relevant lines from the logs of the follower attached. There's nothing relevant in the logs on the initiator node. There are no indications of retry attempts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-8472) Streams hang in repair
[ https://issues.apache.org/jira/browse/CASSANDRA-8472?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita reassigned CASSANDRA-8472: - Assignee: Yuki Morishita Streams hang in repair -- Key: CASSANDRA-8472 URL: https://issues.apache.org/jira/browse/CASSANDRA-8472 Project: Cassandra Issue Type: Bug Reporter: Jimmy Mårdell Assignee: Yuki Morishita Fix For: 2.0.14 Attachments: errlogs In general streaming is working much better in 2.0.x than before, but we still get occasional hanging stream sessions. One of the nodes, the follower, throws IOException: Broken pipe, causing all streams to fail with the initiator node. But the initiator node still thinks its sending and receiving files from the follower, causing the streaming to hang forever. Relevant lines from the logs of the follower attached. There's nothing relevant in the logs on the initiator node. There are no indications of retry attempts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8951) Add smallint (and possibly byte) type
[ https://issues.apache.org/jira/browse/CASSANDRA-8951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384216#comment-14384216 ] Sylvain Lebresne commented on CASSANDRA-8951: - bq. Would it make sense to at least make them non-emptiable? That's trivial to do and it's not very risky either since if we find a case where that's problematic, we can always change our minds and allow it. The only downside is the lack of consistency with the rest of the types (so I'd have preferred a more general solution to this problem), but again, happy to start with non-emptiable and reconsider if this confuse too many people. Add smallint (and possibly byte) type - Key: CASSANDRA-8951 URL: https://issues.apache.org/jira/browse/CASSANDRA-8951 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Benjamin Lerer Fix For: 3.0 We have {{int}} and {{bigint}}, but we don't have a {{smallint}} (2 bytes). I see no reason not to add it. And while we're at it, it doesn't cost much to add a {{byte}} type either. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-4339) Add TTL support to ColumnFamilyRecordWriter and Pig
[ https://issues.apache.org/jira/browse/CASSANDRA-4339?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-4339: --- Component/s: Hadoop Fix Version/s: 3.0 Add TTL support to ColumnFamilyRecordWriter and Pig --- Key: CASSANDRA-4339 URL: https://issues.apache.org/jira/browse/CASSANDRA-4339 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Jeremy Hanna Fix For: 3.0 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9054) Break DatabaseDescriptor up into multiple classes.
[ https://issues.apache.org/jira/browse/CASSANDRA-9054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9054: --- Fix Version/s: 3.0 Break DatabaseDescriptor up into multiple classes. -- Key: CASSANDRA-9054 URL: https://issues.apache.org/jira/browse/CASSANDRA-9054 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jeremiah Jordan Fix For: 3.0 Right now to get at Config stuff you go through DatabaseDescriptor. But when you instantiate DatabaseDescriptor it actually opens system tables and such, which triggers commit log replays, and other things if the right flags aren't set ahead of time. This makes getting at config stuff from tools annoying, as you have to be very careful about instantiation orders. It would be nice if we could break DatabaseDescriptor up into multiple classes, so that getting at config stuff from tools wasn't such a pain. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8951) Add smallint (and possibly byte) type
[ https://issues.apache.org/jira/browse/CASSANDRA-8951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384157#comment-14384157 ] Sylvain Lebresne commented on CASSANDRA-8951: - I suggest we try to get this in 3.0 because it's reasonably trivial and it'll allow to add a code in the v4 protocol for this/those types. bq. Just a note to start the ball rolling with byte-order comparability for smallint... I suggest leaving that to whatever patch will change it for other types. I don't think it makes sense to sense to have a inconsistent (with other current encoding) encoding of {{smallint}} at least in the native protocol and bothering with conversion between internal and external encoding in this ticket doesn't make sense. And once we'll handle the other types, handling {{smallint}} at the same time will be a completely negligible overhead, so it's easier to leave that aside for now. Add smallint (and possibly byte) type - Key: CASSANDRA-8951 URL: https://issues.apache.org/jira/browse/CASSANDRA-8951 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Benjamin Lerer Fix For: 3.0 We have {{int}} and {{bigint}}, but we don't have a {{smallint}} (2 bytes). I see no reason not to add it. And while we're at it, it doesn't cost much to add a {{byte}} type either. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7113) Unhandled ClassCastException Around Migrations
[ https://issues.apache.org/jira/browse/CASSANDRA-7113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384170#comment-14384170 ] Philip Thompson commented on CASSANDRA-7113: [~thobbs], is this still an issue? Assigning to you to find out. Unhandled ClassCastException Around Migrations -- Key: CASSANDRA-7113 URL: https://issues.apache.org/jira/browse/CASSANDRA-7113 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Fix For: 2.1.4 I'm having trouble reproducing this, but a run of the pycassa integration tests against the latest 2.1 left this in the logs: {noformat} INFO [MigrationStage:1] 2014-04-29 18:42:22,088 DefsTables.java:388 - Loading org.apache.cassandra.config.CFMetaData@52f225c1[cfId=e8c08650-cff7-11e3-8109-6b09a6cc3d5a,ksName=PycassaTestKeyspace,cfName=SingleComposite,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.IntegerType),comment=,readRepairChance=0.1,dclocalReadRepairChance=0.0,gcGraceSeconds=864000,defaultValidator=org.apache.cassandra.db.marshal.BytesType,keyValidator=org.apache.cassandra.db.marshal.BytesType,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata={java.nio.HeapByteBuffer[pos=0 lim=3 cap=3]=ColumnDefinition{name=key, type=org.apache.cassandra.db.marshal.BytesType, kind=PARTITION_KEY, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=5 cap=5]=ColumnDefinition{name=value, type=org.apache.cassandra.db.marshal.BytesType, kind=COMPACT_VALUE, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=7 cap=7]=ColumnDefinition{name=column1, type=org.apache.cassandra.db.marshal.IntegerType, kind=CLUSTERING_COLUMN, componentIndex=0, indexName=null, indexType=null}},compactionStrategyClass=class org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy,compactionStrategyOptions={},compressionParameters={sstable_compression=org.apache.cassandra.io.compress.LZ4Compressor},bloomFilterFpChance=null,memtableFlushPeriod=0,caching={keys:ALL, rows_per_partition:NONE},defaultTimeToLive=0,minIndexInterval=128,maxIndexInterval=2048,speculativeRetry=NONE,droppedColumns={},triggers={}] INFO [MigrationStage:1] 2014-04-29 18:42:22,090 ColumnFamilyStore.java:285 - Initializing PycassaTestKeyspace.SingleComposite INFO [CompactionExecutor:9] 2014-04-29 18:42:22,096 CompactionTask.java:252 - Compacted 4 sstables to [/var/lib/cassandra/data/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-169,]. 18,036 bytes to 17,507 (~97% of original) in 23ms = 0.725912MB/s. 7 total partitions merged to 4. Partition merge counts were {1:3, 4:1, } ERROR [Thrift:24] 2014-04-29 18:42:22,109 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:34] 2014-04-29 18:42:22,130 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:25] 2014-04-29 18:42:22,173 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:11] 2014-04-29 18:42:22,258 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:16] 2014-04-29 18:42:22,422 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:26] 2014-04-29 18:42:22,747 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null INFO [Thrift:7] 2014-04-29 18:42:22,790 MigrationManager.java:220 - Create new ColumnFamily: org.apache.cassandra.config.CFMetaData@461a7048[cfId=e9380040-cff7-11e3-8109-6b09a6cc3d5a,ksName=PycassaTestKeyspace,cfName=UUIDComposite,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.IntegerType),org.apache.cassandra.db.marshal.TimeUUIDType),comment=,readRepairChance=0.1,dclocalReadRepairChance=0.0,gcGraceSeconds=864000,defaultValidator=org.apache.cassandra.db.marshal.UTF8Type,keyValidator=org.apache.cassandra.db.marshal.TimeUUIDType,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata={java.nio.HeapByteBuffer[pos=0 lim=3 cap=3]=ColumnDefinition{name=key, type=org.apache.cassandra.db.marshal.TimeUUIDType, kind=PARTITION_KEY, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=7 cap=7]=ColumnDefinition{name=column2, type=org.apache.cassandra.db.marshal.TimeUUIDType, kind=CLUSTERING_COLUMN,
[jira] [Updated] (CASSANDRA-7113) Unhandled ClassCastException Around Migrations
[ https://issues.apache.org/jira/browse/CASSANDRA-7113?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-7113: --- Assignee: Tyler Hobbs Unhandled ClassCastException Around Migrations -- Key: CASSANDRA-7113 URL: https://issues.apache.org/jira/browse/CASSANDRA-7113 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Tyler Hobbs Fix For: 2.1.4 I'm having trouble reproducing this, but a run of the pycassa integration tests against the latest 2.1 left this in the logs: {noformat} INFO [MigrationStage:1] 2014-04-29 18:42:22,088 DefsTables.java:388 - Loading org.apache.cassandra.config.CFMetaData@52f225c1[cfId=e8c08650-cff7-11e3-8109-6b09a6cc3d5a,ksName=PycassaTestKeyspace,cfName=SingleComposite,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.IntegerType),comment=,readRepairChance=0.1,dclocalReadRepairChance=0.0,gcGraceSeconds=864000,defaultValidator=org.apache.cassandra.db.marshal.BytesType,keyValidator=org.apache.cassandra.db.marshal.BytesType,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata={java.nio.HeapByteBuffer[pos=0 lim=3 cap=3]=ColumnDefinition{name=key, type=org.apache.cassandra.db.marshal.BytesType, kind=PARTITION_KEY, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=5 cap=5]=ColumnDefinition{name=value, type=org.apache.cassandra.db.marshal.BytesType, kind=COMPACT_VALUE, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=7 cap=7]=ColumnDefinition{name=column1, type=org.apache.cassandra.db.marshal.IntegerType, kind=CLUSTERING_COLUMN, componentIndex=0, indexName=null, indexType=null}},compactionStrategyClass=class org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy,compactionStrategyOptions={},compressionParameters={sstable_compression=org.apache.cassandra.io.compress.LZ4Compressor},bloomFilterFpChance=null,memtableFlushPeriod=0,caching={keys:ALL, rows_per_partition:NONE},defaultTimeToLive=0,minIndexInterval=128,maxIndexInterval=2048,speculativeRetry=NONE,droppedColumns={},triggers={}] INFO [MigrationStage:1] 2014-04-29 18:42:22,090 ColumnFamilyStore.java:285 - Initializing PycassaTestKeyspace.SingleComposite INFO [CompactionExecutor:9] 2014-04-29 18:42:22,096 CompactionTask.java:252 - Compacted 4 sstables to [/var/lib/cassandra/data/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-169,]. 18,036 bytes to 17,507 (~97% of original) in 23ms = 0.725912MB/s. 7 total partitions merged to 4. Partition merge counts were {1:3, 4:1, } ERROR [Thrift:24] 2014-04-29 18:42:22,109 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:34] 2014-04-29 18:42:22,130 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:25] 2014-04-29 18:42:22,173 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:11] 2014-04-29 18:42:22,258 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:16] 2014-04-29 18:42:22,422 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null ERROR [Thrift:26] 2014-04-29 18:42:22,747 CustomTThreadPoolServer.java:219 - Error occurred during processing of message. java.lang.ClassCastException: null INFO [Thrift:7] 2014-04-29 18:42:22,790 MigrationManager.java:220 - Create new ColumnFamily: org.apache.cassandra.config.CFMetaData@461a7048[cfId=e9380040-cff7-11e3-8109-6b09a6cc3d5a,ksName=PycassaTestKeyspace,cfName=UUIDComposite,cfType=Standard,comparator=org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.IntegerType),org.apache.cassandra.db.marshal.TimeUUIDType),comment=,readRepairChance=0.1,dclocalReadRepairChance=0.0,gcGraceSeconds=864000,defaultValidator=org.apache.cassandra.db.marshal.UTF8Type,keyValidator=org.apache.cassandra.db.marshal.TimeUUIDType,minCompactionThreshold=4,maxCompactionThreshold=32,columnMetadata={java.nio.HeapByteBuffer[pos=0 lim=3 cap=3]=ColumnDefinition{name=key, type=org.apache.cassandra.db.marshal.TimeUUIDType, kind=PARTITION_KEY, componentIndex=null, indexName=null, indexType=null}, java.nio.HeapByteBuffer[pos=0 lim=7 cap=7]=ColumnDefinition{name=column2, type=org.apache.cassandra.db.marshal.TimeUUIDType, kind=CLUSTERING_COLUMN, componentIndex=1, indexName=null, indexType=null},
[jira] [Commented] (CASSANDRA-8481) ghost node in gossip
[ https://issues.apache.org/jira/browse/CASSANDRA-8481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384181#comment-14384181 ] Philip Thompson commented on CASSANDRA-8481: What Cassandra version are you running? ghost node in gossip Key: CASSANDRA-8481 URL: https://issues.apache.org/jira/browse/CASSANDRA-8481 Project: Cassandra Issue Type: Bug Reporter: Alexey Larkov Priority: Minor After inaccurate removing nodes from cluster nodetool gossipinfo and jmx org.apache.cassandra.net.FailureDetector.AllEndpointsStates shows the node status is LEFT. Name Value TypeDisplay NameUpdate Interval Description /192.168.58.75 generation:3 heartbeat:0 REMOVAL_COORDINATOR:REMOVER,f9a28f8c-3244-42d1-986e-592aafe1406c STATUS:LEFT,-3361705224534889554,141446785 jmx org.apache.cassandra.net.FailureDetector.DownEndpointCount is 1 node 58.75 is absent in nodetool status and system.peers table. Before node got LEFT status it was in REMOVING state. I've done unsafeassassinateendpoint and it's status became LEFT, but DownEndpointCount is still 1. And org.apache.cassandra.net.FailureDetector.SimpleStates is still DOWN. How to remove this node from gossip? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8564) Harmless exception logged as ERROR
[ https://issues.apache.org/jira/browse/CASSANDRA-8564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384223#comment-14384223 ] Tyler Hobbs commented on CASSANDRA-8564: Linking this to CASSANDRA-8996, as it's one of the few remaining causes for failing dtests against trunk Harmless exception logged as ERROR -- Key: CASSANDRA-8564 URL: https://issues.apache.org/jira/browse/CASSANDRA-8564 Project: Cassandra Issue Type: Bug Reporter: Philip Thompson Assignee: Benedict Priority: Minor Fix For: 2.1.4 After CASSANDRA-8474, when running the dtest counter_test.py:TestCounters.upgrade_test, we now see the following in the log: {code} ERROR [CompactionExecutor:2] 2015-01-05 13:59:51,003 CassandraDaemon.java:170 - Exception in thread Thread[CompactionExecutor:2,1,main] java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@5e8ea989 rejected from org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@7fc92f94[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 5] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048) ~[na:1.7.0_67] at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:325) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:530) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.execute(ScheduledThreadPoolExecutor.java:619) ~[na:1.7.0_67] at org.apache.cassandra.io.sstable.SSTableReader.scheduleTidy(SSTableReader.java:638) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableReader.tidy(SSTableReader.java:619) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1650) ~[main/:na] at org.apache.cassandra.db.DataTracker.replaceReaders(DataTracker.java:409) ~[main/:na] at org.apache.cassandra.db.DataTracker.replaceWithNewInstances(DataTracker.java:303) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableRewriter.moveStarts(SSTableRewriter.java:254) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableRewriter.abort(SSTableRewriter.java:180) ~[main/:na] at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:205) ~[main/:na] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[main/:na] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75) ~[main/:na] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[main/:na] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:226) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_67] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_67] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_67] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_67] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67] Suppressed: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@681c91de rejected from org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@7fc92f94[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 5] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048) ~[na:1.7.0_67] at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:325) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:530) ~[na:1.7.0_67] at java.util.concurrent.ScheduledThreadPoolExecutor.execute(ScheduledThreadPoolExecutor.java:619) ~[na:1.7.0_67] at org.apache.cassandra.io.sstable.SSTableReader.scheduleTidy(SSTableReader.java:638) ~[main/:na] at
[jira] [Commented] (CASSANDRA-8381) CFStats should record keys of largest N requests for time interval
[ https://issues.apache.org/jira/browse/CASSANDRA-8381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384233#comment-14384233 ] Philip Thompson commented on CASSANDRA-8381: Bump. [~mstump], should we close as duplicate of CASSANDRA-7974? CFStats should record keys of largest N requests for time interval -- Key: CASSANDRA-8381 URL: https://issues.apache.org/jira/browse/CASSANDRA-8381 Project: Cassandra Issue Type: Improvement Reporter: Matt Stump Priority: Critical Isolating the problem partition for a CF is right now incredibly difficult. If we could keep the primary key of the largest N read or write requests for the pervious interval or since counter has been cleared it would be extremely useful. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8751) C* should always listen to both ssl/non-ssl ports
[ https://issues.apache.org/jira/browse/CASSANDRA-8751?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-8751: --- Fix Version/s: 3.0 C* should always listen to both ssl/non-ssl ports - Key: CASSANDRA-8751 URL: https://issues.apache.org/jira/browse/CASSANDRA-8751 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Minh Do Assignee: Minh Do Priority: Critical Fix For: 3.0 Since there is always one thread dedicated on server socket listener and it does not use much resource, we should always have these two listeners up no matter what users set for internode_encryption. The reason behind this is that we need to switch back and forth between different internode_encryption modes and we need C* servers to keep running in transient state or during mode switching. Currently this is not possible. For example, we have a internode_encryption=dc cluster in a multi-region AWS environment and want to set internode_encryption=all by rolling restart C* nodes. However, the node with internode_encryption=all does not open to listen to non-ssl port. As a result, we have a splitted brain cluster here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9054) Break DatabaseDescriptor up into multiple classes.
Jeremiah Jordan created CASSANDRA-9054: -- Summary: Break DatabaseDescriptor up into multiple classes. Key: CASSANDRA-9054 URL: https://issues.apache.org/jira/browse/CASSANDRA-9054 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jeremiah Jordan Right now to get at Config stuff you go through DatabaseDescriptor. But when you instantiate DatabaseDescriptor it actually opens system tables and such, which triggers commit log replays, and other things if the right flags aren't set ahead of time. This makes getting at config stuff from tools annoying, as you have to be very careful about instantiation orders. It would be nice if we could break DatabaseDescriptor up into multiple classes, so that getting at config stuff from tools wasn't such a pain. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-7645) cqlsh: show partial trace if incomplete after max_trace_wait
[ https://issues.apache.org/jira/browse/CASSANDRA-7645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian reassigned CASSANDRA-7645: - Assignee: Carl Yeksigian (was: Tyler Hobbs) cqlsh: show partial trace if incomplete after max_trace_wait Key: CASSANDRA-7645 URL: https://issues.apache.org/jira/browse/CASSANDRA-7645 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Tyler Hobbs Assignee: Carl Yeksigian Priority: Trivial Fix For: 2.1.4 If a trace hasn't completed within {{max_trace_wait}}, cqlsh will say the trace is unavailable and not show anything. It (and the underlying python driver) determines when the trace is complete by checking if the {{duration}} column in {{system_traces.sessions}} is non-null. If {{duration}} is null but we still have some trace events when the timeout is hit, cqlsh should print whatever trace events we have along with a warning about it being incomplete. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384030#comment-14384030 ] Marcus Eriksson commented on CASSANDRA-9045: I grasping for straws here, but have you made sure that the clocks are synced on all the nodes? Are the nodes agreeing on schemas? Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-5683) Nodetool command for modifying rpc_timeout
[ https://issues.apache.org/jira/browse/CASSANDRA-5683?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-5683: --- Fix Version/s: 3.0 Issue Type: New Feature (was: Bug) Nodetool command for modifying rpc_timeout -- Key: CASSANDRA-5683 URL: https://issues.apache.org/jira/browse/CASSANDRA-5683 Project: Cassandra Issue Type: New Feature Components: Core Affects Versions: 1.1.9 Environment: apache cassandra 1.1.9 Reporter: Boole Guo Labels: write Fix For: 3.0 When writing in mutil-dc environment, there are many timeout exception. As I know, this version have do good to reduce network bandwidth. Can we assign rpc timeout when writing like consistenylevel? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8472) Streams hang in repair
[ https://issues.apache.org/jira/browse/CASSANDRA-8472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384177#comment-14384177 ] Philip Thompson commented on CASSANDRA-8472: [~yukim], do we consider this a bug since stream timeout can be set, preventing hanging on failed repairs? Streams hang in repair -- Key: CASSANDRA-8472 URL: https://issues.apache.org/jira/browse/CASSANDRA-8472 Project: Cassandra Issue Type: Bug Reporter: Jimmy Mårdell Fix For: 2.0.14 Attachments: errlogs In general streaming is working much better in 2.0.x than before, but we still get occasional hanging stream sessions. One of the nodes, the follower, throws IOException: Broken pipe, causing all streams to fail with the initiator node. But the initiator node still thinks its sending and receiving files from the follower, causing the streaming to hang forever. Relevant lines from the logs of the follower attached. There's nothing relevant in the logs on the initiator node. There are no indications of retry attempts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8475) Altering Table's tombstone_threshold stalls compaction until restart
[ https://issues.apache.org/jira/browse/CASSANDRA-8475?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-8475: --- Description: Compaction won't move forward on the table until a restart takes place and the temp table is ignored. My hunch is that running CompactionTasks are killed and there are still pre-opened temp files ref'd but they get deleted with the CompactionTask dies? {code} Exception: 2014-12-12_22:03:19.84572 ERROR 22:03:19 Exception in thread Thread[CompactionExecutor:671,1,main] 2014-12-12_22:03:19.84576 java.lang.RuntimeException: java.io.FileNotFoundException: /data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db (No such file or directory) 2014-12-12_22:03:19.84576 at org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:52) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84577 at org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1895) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84578 at org.apache.cassandra.io.sstable.SSTableScanner.init(SSTableScanner.java:67) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84579 at org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1681) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84579 at org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1693) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84580 at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getScanners(LeveledCompactionStrategy.java:181) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84581 at org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:320) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84581 at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:340) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84582 at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:151) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84583 at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84583 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84583 at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84584 at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84584 at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:232) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84585 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_65] 2014-12-12_22:03:19.84586 at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_65] 2014-12-12_22:03:19.84586 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_65] 2014-12-12_22:03:19.84587 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_65] 2014-12-12_22:03:19.84587 at java.lang.Thread.run(Thread.java:745) [na:1.7.0_65] 2014-12-12_22:03:19.84587 Caused by: java.io.FileNotFoundException: /data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db (No such file or directory) 2014-12-12_22:03:19.84588 at java.io.RandomAccessFile.open(Native Method) ~[na:1.7.0_65] 2014-12-12_22:03:19.84588 at java.io.RandomAccessFile.init(RandomAccessFile.java:241) ~[na:1.7.0_65] 2014-12-12_22:03:19.84589 at org.apache.cassandra.io.util.RandomAccessReader.init(RandomAccessReader.java:58) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84590 at org.apache.cassandra.io.compress.CompressedRandomAccessReader.init(CompressedRandomAccessReader.java:77) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84590 at org.apache.cassandra.io.compress.CompressedThrottledReader.init(CompressedThrottledReader.java:34) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84591 at org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:48) ~[apache-cassandra-2.1.2.jar:2.1.2] 2014-12-12_22:03:19.84591 ... 18 common frames omitted{code} was: Compaction won't move forward on the table until a restart takes place and the temp table is ignored. My hunch is that running CompactionTasks are
[jira] [Commented] (CASSANDRA-9037) Terminal UDFs evaluated at prepare time throw protocol version error
[ https://issues.apache.org/jira/browse/CASSANDRA-9037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384227#comment-14384227 ] Tyler Hobbs commented on CASSANDRA-9037: Linking to CASSANDRA-8996, as this is responsible for the failure of the {{ user_functions_test.TestUserFunctions.udf_overload_test}} dtest. Terminal UDFs evaluated at prepare time throw protocol version error Key: CASSANDRA-9037 URL: https://issues.apache.org/jira/browse/CASSANDRA-9037 Project: Cassandra Issue Type: Bug Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Fix For: 3.0 When a pure function with only terminal arguments (or with no arguments) is used in a where clause, it's executed at prepare time and {{Server.CURRENT_VERSION}} passed as the protocol version for serialization purposes. For native functions, this isn't a problem, but UDFs use classes in the bundled java-driver-core jar for (de)serialization of args and return values. When {{Server.CURRENT_VERSION}} is greater than the highest version supported by the bundled java driver the execution fails with the following exception: {noformat} ERROR [SharedPool-Worker-1] 2015-03-24 18:10:59,391 QueryMessage.java:132 - Unexpected error during query org.apache.cassandra.exceptions.FunctionExecutionException: execution of 'ks.overloaded[text]' failed: java.lang.IllegalArgumentException: No protocol version matching integer version 4 at org.apache.cassandra.exceptions.FunctionExecutionException.create(FunctionExecutionException.java:35) ~[main/:na] at org.apache.cassandra.cql3.udf.gen.Cksoverloaded_1.execute(Cksoverloaded_1.java) ~[na:na] at org.apache.cassandra.cql3.functions.FunctionCall.executeInternal(FunctionCall.java:78) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall.access$200(FunctionCall.java:34) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.execute(FunctionCall.java:176) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.prepare(FunctionCall.java:161) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.toTerm(SingleColumnRelation.java:108) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.newEQRestriction(SingleColumnRelation.java:143) ~[main/:na] at org.apache.cassandra.cql3.Relation.toRestriction(Relation.java:127) ~[main/:na] at org.apache.cassandra.cql3.restrictions.StatementRestrictions.init(StatementRestrictions.java:126) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepareRestrictions(SelectStatement.java:787) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:740) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:488) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:252) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:246) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:475) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:371) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) [netty-all-4.0.23.Final.jar:4.0.23.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_71] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_71] Caused by: java.lang.IllegalArgumentException: No protocol version matching integer version 4 at com.datastax.driver.core.ProtocolVersion.fromInt(ProtocolVersion.java:89) ~[cassandra-driver-core-2.1.2.jar:na] at org.apache.cassandra.cql3.functions.UDFunction.compose(UDFunction.java:177) ~[main/:na] ... 25 common frames
[jira] [Comment Edited] (CASSANDRA-9037) Terminal UDFs evaluated at prepare time throw protocol version error
[ https://issues.apache.org/jira/browse/CASSANDRA-9037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384227#comment-14384227 ] Tyler Hobbs edited comment on CASSANDRA-9037 at 3/27/15 5:46 PM: - Linking to CASSANDRA-8996, as this is responsible for the failure of the {{user_functions_test.TestUserFunctions.udf_overload_test}} dtest. was (Author: thobbs): Linking to CASSANDRA-8996, as this is responsible for the failure of the {{ user_functions_test.TestUserFunctions.udf_overload_test}} dtest. Terminal UDFs evaluated at prepare time throw protocol version error Key: CASSANDRA-9037 URL: https://issues.apache.org/jira/browse/CASSANDRA-9037 Project: Cassandra Issue Type: Bug Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Fix For: 3.0 When a pure function with only terminal arguments (or with no arguments) is used in a where clause, it's executed at prepare time and {{Server.CURRENT_VERSION}} passed as the protocol version for serialization purposes. For native functions, this isn't a problem, but UDFs use classes in the bundled java-driver-core jar for (de)serialization of args and return values. When {{Server.CURRENT_VERSION}} is greater than the highest version supported by the bundled java driver the execution fails with the following exception: {noformat} ERROR [SharedPool-Worker-1] 2015-03-24 18:10:59,391 QueryMessage.java:132 - Unexpected error during query org.apache.cassandra.exceptions.FunctionExecutionException: execution of 'ks.overloaded[text]' failed: java.lang.IllegalArgumentException: No protocol version matching integer version 4 at org.apache.cassandra.exceptions.FunctionExecutionException.create(FunctionExecutionException.java:35) ~[main/:na] at org.apache.cassandra.cql3.udf.gen.Cksoverloaded_1.execute(Cksoverloaded_1.java) ~[na:na] at org.apache.cassandra.cql3.functions.FunctionCall.executeInternal(FunctionCall.java:78) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall.access$200(FunctionCall.java:34) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.execute(FunctionCall.java:176) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.prepare(FunctionCall.java:161) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.toTerm(SingleColumnRelation.java:108) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.newEQRestriction(SingleColumnRelation.java:143) ~[main/:na] at org.apache.cassandra.cql3.Relation.toRestriction(Relation.java:127) ~[main/:na] at org.apache.cassandra.cql3.restrictions.StatementRestrictions.init(StatementRestrictions.java:126) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepareRestrictions(SelectStatement.java:787) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:740) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:488) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:252) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:246) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:475) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:371) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) [netty-all-4.0.23.Final.jar:4.0.23.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_71] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_71] Caused by: java.lang.IllegalArgumentException: No protocol version matching integer version 4 at
[jira] [Updated] (CASSANDRA-5322) Make dtest logging more granular
[ https://issues.apache.org/jira/browse/CASSANDRA-5322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-5322: --- Fix Version/s: 3.0 Make dtest logging more granular - Key: CASSANDRA-5322 URL: https://issues.apache.org/jira/browse/CASSANDRA-5322 Project: Cassandra Issue Type: Test Reporter: Ryan McGuire Assignee: Ryan McGuire Fix For: 3.0 From Brandon: We need a way (might need to go in ccm, I haven't looked) to just set one class to DEBUG or TRACE, like we'd do in conf/log4-server.properties but with an env var preferably, so I can control it via buildbot, since it's better at reproducing some issues than I am sometimes, but I don't want to run the full hammer debug all the time. Also, a way to set Tester.allow_log_errors to false via an env var, since sometimes there's an error there that takes a while to fix but is cosmetic, and in the meantime I want to catch new failures so we don't fall behind. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384107#comment-14384107 ] Roman Tkachenko commented on CASSANDRA-9045: Yep. I triple checked that clocks are synchronized. Also I checked multiple times that the schema version is the same for all nodes in the cluster. Were you able to reproduce the issue? FWIW I did some more research and it *seems* like it affects only certain columns in the row. Like, in my yesterday's test (the one I attached cqlsh output from) I also removed one more column and it did not reappear after repair. In tracing logs for it I did not see the digest mismatch thing, unlike the other one that did reappear. Not sure if it's related at all. Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8970) Allow custom time_format on cqlsh COPY TO
[ https://issues.apache.org/jira/browse/CASSANDRA-8970?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-8970: -- Reviewer: Carl Yeksigian (was: Tyler Hobbs) Allow custom time_format on cqlsh COPY TO - Key: CASSANDRA-8970 URL: https://issues.apache.org/jira/browse/CASSANDRA-8970 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Aaron Ploetz Priority: Trivial Labels: cqlsh Fix For: 2.1.4 Attachments: CASSANDRA-8970.patch Original Estimate: 4h Remaining Estimate: 4h When executing a COPY TO from cqlsh, the user is currently has no control over the format of exported timestamp columns. If the user has indicated a {{time_format}} in their cqlshrc file, that format will be used. Otherwise, the system default format will be used. The problem comes into play when the timestamp format used on a COPY TO, is not valid when the data is sent back into Cassandra with a COPY FROM. For instance, if a user has {{time_format = %Y-%m-%d %H:%M:%S%Z}} specified in their cqlshrc, COPY TO will format timestamp columns like this: {{userid|posttime|postcontent}} {{0|2015-03-14 14:59:00CDT|rtyeryerweh}} {{0|2015-03-14 14:58:00CDT|sdfsdfsdgfjdsgojr}} {{0|2015-03-12 14:27:00CDT|sdgfjdsgojr}} Executing a COPY FROM on that same file will produce an unable to coerce to formatted date(long) error. Right now, the only way to change the way timestamps are formatted is to exit cqlsh, modify the {{time_format}} property in cqlshrc, and restart cqlsh. The ability to specify a COPY option of TIME_FORMAT with a Python strftime format, would allow the user to quickly alter the timestamp format for export, without reconfiguring cqlsh. {{aploetz@cqlsh:stackoverflow COPY posts1 TO '/home/aploetz/posts1.csv' WITH DELIMITER='|' AND HEADER=true AND TIME_FORMAT='%Y-%m-%d %H:%M:%S%z;}} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7212) Allow to switch user within CQLSH session
[ https://issues.apache.org/jira/browse/CASSANDRA-7212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-7212: -- Reviewer: Carl Yeksigian (was: Tyler Hobbs) Allow to switch user within CQLSH session - Key: CASSANDRA-7212 URL: https://issues.apache.org/jira/browse/CASSANDRA-7212 Project: Cassandra Issue Type: Improvement Components: API Environment: [cqlsh 4.1.1 | Cassandra 2.0.7.31 | CQL spec 3.1.1 | Thrift protocol 19.39.0] Reporter: Jose Martinez Poblete Labels: cqlsh Attachments: 7212_1.patch Once a user is logged into CQLSH, it is not possible to switch to another user without exiting and relaunch This is a feature offered in postgres and probably other databases: http://secure.encivasolutions.com/knowledgebase.php?action=displayarticleid=1126 Perhaps this could be implemented on CQLSH as part of the USE directive: USE Keyspace [USER] [password] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9037) Terminal UDFs evaluated at prepare time throw protocol version error
[ https://issues.apache.org/jira/browse/CASSANDRA-9037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384113#comment-14384113 ] Tyler Hobbs commented on CASSANDRA-9037: It just occurred to me that we can't execute terminal functions at prepare-time if they return a collection. The reason is that the protocol version isn't known yet, and the serialization of the collection depends on the protocol version. So, we should either avoid executing if the return type is a collection type (or tuple or UDT that contains a collection), or execute anyway and handle re-serialization at execute time if the protocol version is not v3+. I'm guessing the first option is significantly simpler to implement. Terminal UDFs evaluated at prepare time throw protocol version error Key: CASSANDRA-9037 URL: https://issues.apache.org/jira/browse/CASSANDRA-9037 Project: Cassandra Issue Type: Bug Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Fix For: 3.0 When a pure function with only terminal arguments (or with no arguments) is used in a where clause, it's executed at prepare time and {{Server.CURRENT_VERSION}} passed as the protocol version for serialization purposes. For native functions, this isn't a problem, but UDFs use classes in the bundled java-driver-core jar for (de)serialization of args and return values. When {{Server.CURRENT_VERSION}} is greater than the highest version supported by the bundled java driver the execution fails with the following exception: {noformat} ERROR [SharedPool-Worker-1] 2015-03-24 18:10:59,391 QueryMessage.java:132 - Unexpected error during query org.apache.cassandra.exceptions.FunctionExecutionException: execution of 'ks.overloaded[text]' failed: java.lang.IllegalArgumentException: No protocol version matching integer version 4 at org.apache.cassandra.exceptions.FunctionExecutionException.create(FunctionExecutionException.java:35) ~[main/:na] at org.apache.cassandra.cql3.udf.gen.Cksoverloaded_1.execute(Cksoverloaded_1.java) ~[na:na] at org.apache.cassandra.cql3.functions.FunctionCall.executeInternal(FunctionCall.java:78) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall.access$200(FunctionCall.java:34) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.execute(FunctionCall.java:176) ~[main/:na] at org.apache.cassandra.cql3.functions.FunctionCall$Raw.prepare(FunctionCall.java:161) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.toTerm(SingleColumnRelation.java:108) ~[main/:na] at org.apache.cassandra.cql3.SingleColumnRelation.newEQRestriction(SingleColumnRelation.java:143) ~[main/:na] at org.apache.cassandra.cql3.Relation.toRestriction(Relation.java:127) ~[main/:na] at org.apache.cassandra.cql3.restrictions.StatementRestrictions.init(StatementRestrictions.java:126) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepareRestrictions(SelectStatement.java:787) ~[main/:na] at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:740) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:488) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:252) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:246) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:475) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:371) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) [netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) [netty-all-4.0.23.Final.jar:4.0.23.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_71] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at
[jira] [Assigned] (CASSANDRA-8051) Add SERIAL and LOCAL_SERIAL consistency levels to cqlsh
[ https://issues.apache.org/jira/browse/CASSANDRA-8051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian reassigned CASSANDRA-8051: - Assignee: Carl Yeksigian (was: Tyler Hobbs) Add SERIAL and LOCAL_SERIAL consistency levels to cqlsh --- Key: CASSANDRA-8051 URL: https://issues.apache.org/jira/browse/CASSANDRA-8051 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Nicolas Favre-Felix Assignee: Carl Yeksigian Priority: Minor Labels: cqlsh cqlsh does not support setting the serial consistency level. The default CL.SERIAL does not let users safely execute LWT alongside an app that runs at LOCAL_SERIAL, and can prevent any LWT from running when a DC is down (e.g. with 2 DCs, RF=3 in each.) Implementing this well is a bit tricky. A user setting the serial CL will probably not want all of their statements to have a serial CL attached, but only the conditional updates. At the same time it would be useful to support serial reads. WITH CONSISTENCY LEVEL used to provide this flexibility. I believe that it is currently impossible to run a SELECT at SERIAL or LOCAL_SERIAL; the only workaround seems to be to run a conditional update with a predicate that always resolves to False, and to rely on the CAS response to read the data. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9053) Convert dtests that use cassandra-cli to Thrift API
Tyler Hobbs created CASSANDRA-9053: -- Summary: Convert dtests that use cassandra-cli to Thrift API Key: CASSANDRA-9053 URL: https://issues.apache.org/jira/browse/CASSANDRA-9053 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Fix For: 3.0 The following dtests need to be changed to use the Thrift API directly instead of going through cassandra-cli: * {{global_row_key_cache_test.TestGlobalRowKeyCache.functional_test}} * {{cql_tests.TestCQL.cql3_insert_thrift_test}} * {{cql_tests.TestCQL.rename_test}} * {{super_column_cache_test.TestSCCache.sc_with_row_cache_test}} * {{upgrade_supercolumns_test.TestSCUpgrade.upgrade_with_index_creation_test}} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384118#comment-14384118 ] Roman Tkachenko commented on CASSANDRA-9045: And it also does not explain why those original zombie columns were finally purged when I increased in memory compaction limit setting. Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8241) Use javac instead of javassist
[ https://issues.apache.org/jira/browse/CASSANDRA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383979#comment-14383979 ] Jonathan Ellis commented on CASSANDRA-8241: --- Probably best to ask the legal team. https://issues.apache.org/jira/browse/LEGAL Use javac instead of javassist -- Key: CASSANDRA-8241 URL: https://issues.apache.org/jira/browse/CASSANDRA-8241 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Labels: udf Fix For: 3.0 Attachments: 8241-ecj.txt, udf-java-javac.txt Using JDK's built-in Java-Compiler API has some advantages over javassist. Although compilation feels a bit slower, Java compiler API has some advantages: * boxing + unboxing works * generics work * compiler error messages are better (or at least known) and have line/column numbers The implementation does not use any temp files. Everything's in memory. Patch attached to this issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-8985) java.lang.AssertionError: Added column does not sort as the last column
[ https://issues.apache.org/jira/browse/CASSANDRA-8985?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian reassigned CASSANDRA-8985: - Assignee: Carl Yeksigian (was: Tyler Hobbs) java.lang.AssertionError: Added column does not sort as the last column --- Key: CASSANDRA-8985 URL: https://issues.apache.org/jira/browse/CASSANDRA-8985 Project: Cassandra Issue Type: Bug Environment: Cassandra 2.0.13 OracleJDK1.7 Debian 7.8 Reporter: Maxim Assignee: Carl Yeksigian Fix For: 2.0.14 After upgrade Cassandra from 2.0.12 to 2.0.13 I begin to receive an error: {code}ERROR [ReadStage:1823] 2015-03-18 09:03:27,091 CassandraDaemon.java (line 199) Exception in thread Thread[ReadStage:1823,5,main] java.lang.AssertionError: Added column does not sort as the last column at org.apache.cassandra.db.ArrayBackedSortedColumns.addColumn(ArrayBackedSortedColumns.java:116) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:121) at org.apache.cassandra.db.ColumnFamily.addIfRelevant(ColumnFamily.java:115) at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:211) at org.apache.cassandra.db.filter.ExtendedFilter$WithClauses.prune(ExtendedFilter.java:290) at org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1792) at org.apache.cassandra.db.index.keys.KeysSearcher.search(KeysSearcher.java:54) at org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:551) at org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:1755) at org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:135) at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745){code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384117#comment-14384117 ] Roman Tkachenko commented on CASSANDRA-9045: And it also does not explain why those original zombie columns were finally purged when I increased in memory compaction limit setting. Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)