[jira] [Created] (CASSANDRA-15319) Add support for network topology and tracing to in-JVM dtests.
Jon Meredith created CASSANDRA-15319: Summary: Add support for network topology and tracing to in-JVM dtests. Key: CASSANDRA-15319 URL: https://issues.apache.org/jira/browse/CASSANDRA-15319 Project: Cassandra Issue Type: Improvement Components: Test/dtest Reporter: Jon Meredith While working on CASSANDRA-15318, testing it properly with an in-JVM test requires setting up the network topology and tracing requests to check which nodes performed forwarding. In support of testing, make it possible to create in-JVM clusters with nodes appearing in different datacenter/racks and add support for executing queries with tracing enabled. -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15318) sendMessagesToNonlocalDC() should shuffle targets
Jon Meredith created CASSANDRA-15318: Summary: sendMessagesToNonlocalDC() should shuffle targets Key: CASSANDRA-15318 URL: https://issues.apache.org/jira/browse/CASSANDRA-15318 Project: Cassandra Issue Type: Improvement Components: Messaging/Internode Reporter: Jon Meredith Assignee: Jon Meredith To better spread load and reduce the impact of a node failure before detection (or other issues like issues host replacement), when forwarding messages to other data centers the forwarding non-local dc nodes should be selected at random rather than always selecting the first node in the list of endpoints for a token. -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15260) Add `allocate_tokens_for_dc_rf` yaml option for token allocation
[ https://issues.apache.org/jira/browse/CASSANDRA-15260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mck updated CASSANDRA-15260: Source Control Link: https://github.com/apache/cassandra/commit/068d2d37c6fbdb60546821c4d408a84161fd1cb6 Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed as 068d2d37c6fbdb60546821c4d408a84161fd1cb6 > Add `allocate_tokens_for_dc_rf` yaml option for token allocation > > > Key: CASSANDRA-15260 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15260 > Project: Cassandra > Issue Type: Improvement > Components: Local/Config >Reporter: mck >Assignee: mck >Priority: Normal > Fix For: 4.0-alpha > > > Similar to DSE's option: {{allocate_tokens_for_local_replication_factor}} > Currently the > [ReplicationAwareTokenAllocator|https://www.datastax.com/dev/blog/token-allocation-algorithm] > requires a defined keyspace and a replica factor specified in the current > datacenter. > This is problematic in a number of ways. The real keyspace can not be used > when adding new datacenters as, in practice, all its nodes need to be up and > running before it has the capacity to replicate data into it. New datacenters > (or lift-and-shifting a cluster via datacenter migration) therefore has to be > done using a dummy keyspace that duplicates the replication strategy+factor > of the real keyspace. This gets even more difficult come version 4.0, as the > replica factor can not even be defined in new datacenters before those > datacenters are up and running. > These issues are removed by avoiding the keyspace definition and lookup, and > presuming the replica strategy is by datacenter, ie NTS. This can be done > with the use of an {{allocate_tokens_for_dc_rf}} option. > It may also be of value considering whether {{allocate_tokens_for_dc_rf=3}} > becomes the default? as this is the replication factor for the vast majority > of datacenters in production. I suspect this would be a good improvement over > the existing randomly generated tokens algorithm. > Initial patch is available in > [https://github.com/thelastpickle/cassandra/commit/fc4865b0399570e58f11215565ba17dc4a53da97] > The patch does not remove the existing {{allocate_tokens_for_keyspace}} > option, as that provides the codebase for handling different replication > strategies. > > fyi [~blambov] [~jay.zhuang] [~chovatia.jayd...@gmail.com] [~alokamvenki] > [~alexchueshev] -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] branch trunk updated: Add `allocate_tokens_for_local_rf` yaml option for token allocation that doesn't require keyspace knowledge/existence
This is an automated email from the ASF dual-hosted git repository. mck pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git The following commit(s) were added to refs/heads/trunk by this push: new 068d2d3 Add `allocate_tokens_for_local_rf` yaml option for token allocation that doesn't require keyspace knowledge/existence 068d2d3 is described below commit 068d2d37c6fbdb60546821c4d408a84161fd1cb6 Author: Mick Semb Wever AuthorDate: Mon Aug 5 00:06:30 2019 +0200 Add `allocate_tokens_for_local_rf` yaml option for token allocation that doesn't require keyspace knowledge/existence patch by Mick Semb Wever; reviewed by Branimir Lambov for CASSANDRA-15260 --- CHANGES.txt| 3 +++ conf/cassandra.yaml| 10 +++-- src/java/org/apache/cassandra/config/Config.java | 2 ++ .../cassandra/config/DatabaseDescriptor.java | 5 + .../org/apache/cassandra/dht/BootStrapper.java | 22 +++ .../apache/cassandra/dht/BootstrapDiagnostics.java | 16 ++ .../org/apache/cassandra/dht/BootstrapEvent.java | 6 +- .../dht/tokenallocator/TokenAllocation.java| 25 ++ .../org/apache/cassandra/dht/BootStrapperTest.java | 19 9 files changed, 105 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6425b1f..709e436 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +4.0-alpha2 + * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) + 4.0-alpha1 * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287) * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index e776a5a..f3e5c75 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -26,15 +26,21 @@ num_tokens: 256 # Triggers automatic allocation of num_tokens tokens for this node. The allocation # algorithm attempts to choose tokens in a way that optimizes replicated load over -# the nodes in the datacenter for the replication strategy used by the specified -# keyspace. +# the nodes in the datacenter for the replica factor. # # The load assigned to each node will be close to proportional to its number of # vnodes. # # Only supported with the Murmur3Partitioner. + +# Replica factor is determined via the replication strategy used by the specified +# keyspace. # allocate_tokens_for_keyspace: KEYSPACE +# Replica factor is explicitly set, regardless of keyspace or datacenter. +# This is the replica factor within the datacenter, like NTS. +# allocate_tokens_for_local_replication_factor: 3 + # initial_token allows you to specify tokens manually. While you can use it with # vnodes (num_tokens > 1, above) -- in which case you should provide a # comma-separated list -- it's primarily used when adding nodes to legacy clusters diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index a1fdfdc..b86b7c5 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -86,6 +86,8 @@ public class Config public int num_tokens = 1; /** Triggers automatic allocation of tokens if set, using the replication strategy of the referenced keyspace */ public String allocate_tokens_for_keyspace = null; +/** Triggers automatic allocation of tokens if set, based on the provided replica count for a datacenter */ +public Integer allocate_tokens_for_local_replication_factor = null; public long native_transport_idle_timeout_in_ms = 0L; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 43203e5..e4ea611 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1422,6 +1422,11 @@ public class DatabaseDescriptor return System.getProperty(Config.PROPERTY_PREFIX + "allocate_tokens_for_keyspace", conf.allocate_tokens_for_keyspace); } +public static Integer getAllocateTokensForLocalRf() +{ +return conf.allocate_tokens_for_local_replication_factor; +} + public static Collection tokensFromString(String tokenString) { List tokens = new ArrayList(); diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java index f5c455c..94bf283 100644 --- a/src/java/org/apache/cassandra/dht/BootStrapper.java +++ b/src/java/org/apache/cassandra/dht/BootStrapper.java @@ -24,6 +24,7 @@ import com.google.common.util.concurrent.ListenableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apa
[jira] [Updated] (CASSANDRA-15260) Add `allocate_tokens_for_dc_rf` yaml option for token allocation
[ https://issues.apache.org/jira/browse/CASSANDRA-15260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mck updated CASSANDRA-15260: Reviewers: Branimir Lambov (was: Branimir Lambov, mck) > Add `allocate_tokens_for_dc_rf` yaml option for token allocation > > > Key: CASSANDRA-15260 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15260 > Project: Cassandra > Issue Type: Improvement > Components: Local/Config >Reporter: mck >Assignee: mck >Priority: Normal > Fix For: 4.0-alpha > > > Similar to DSE's option: {{allocate_tokens_for_local_replication_factor}} > Currently the > [ReplicationAwareTokenAllocator|https://www.datastax.com/dev/blog/token-allocation-algorithm] > requires a defined keyspace and a replica factor specified in the current > datacenter. > This is problematic in a number of ways. The real keyspace can not be used > when adding new datacenters as, in practice, all its nodes need to be up and > running before it has the capacity to replicate data into it. New datacenters > (or lift-and-shifting a cluster via datacenter migration) therefore has to be > done using a dummy keyspace that duplicates the replication strategy+factor > of the real keyspace. This gets even more difficult come version 4.0, as the > replica factor can not even be defined in new datacenters before those > datacenters are up and running. > These issues are removed by avoiding the keyspace definition and lookup, and > presuming the replica strategy is by datacenter, ie NTS. This can be done > with the use of an {{allocate_tokens_for_dc_rf}} option. > It may also be of value considering whether {{allocate_tokens_for_dc_rf=3}} > becomes the default? as this is the replication factor for the vast majority > of datacenters in production. I suspect this would be a good improvement over > the existing randomly generated tokens algorithm. > Initial patch is available in > [https://github.com/thelastpickle/cassandra/commit/fc4865b0399570e58f11215565ba17dc4a53da97] > The patch does not remove the existing {{allocate_tokens_for_keyspace}} > option, as that provides the codebase for handling different replication > strategies. > > fyi [~blambov] [~jay.zhuang] [~chovatia.jayd...@gmail.com] [~alokamvenki] > [~alexchueshev] -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15260) Add `allocate_tokens_for_dc_rf` yaml option for token allocation
[ https://issues.apache.org/jira/browse/CASSANDRA-15260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mck updated CASSANDRA-15260: Fix Version/s: (was: 4.x) 4.0-alpha > Add `allocate_tokens_for_dc_rf` yaml option for token allocation > > > Key: CASSANDRA-15260 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15260 > Project: Cassandra > Issue Type: Improvement > Components: Local/Config >Reporter: mck >Assignee: mck >Priority: Normal > Fix For: 4.0-alpha > > > Similar to DSE's option: {{allocate_tokens_for_local_replication_factor}} > Currently the > [ReplicationAwareTokenAllocator|https://www.datastax.com/dev/blog/token-allocation-algorithm] > requires a defined keyspace and a replica factor specified in the current > datacenter. > This is problematic in a number of ways. The real keyspace can not be used > when adding new datacenters as, in practice, all its nodes need to be up and > running before it has the capacity to replicate data into it. New datacenters > (or lift-and-shifting a cluster via datacenter migration) therefore has to be > done using a dummy keyspace that duplicates the replication strategy+factor > of the real keyspace. This gets even more difficult come version 4.0, as the > replica factor can not even be defined in new datacenters before those > datacenters are up and running. > These issues are removed by avoiding the keyspace definition and lookup, and > presuming the replica strategy is by datacenter, ie NTS. This can be done > with the use of an {{allocate_tokens_for_dc_rf}} option. > It may also be of value considering whether {{allocate_tokens_for_dc_rf=3}} > becomes the default? as this is the replication factor for the vast majority > of datacenters in production. I suspect this would be a good improvement over > the existing randomly generated tokens algorithm. > Initial patch is available in > [https://github.com/thelastpickle/cassandra/commit/fc4865b0399570e58f11215565ba17dc4a53da97] > The patch does not remove the existing {{allocate_tokens_for_keyspace}} > option, as that provides the codebase for handling different replication > strategies. > > fyi [~blambov] [~jay.zhuang] [~chovatia.jayd...@gmail.com] [~alokamvenki] > [~alexchueshev] -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15260) Add `allocate_tokens_for_dc_rf` yaml option for token allocation
[ https://issues.apache.org/jira/browse/CASSANDRA-15260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mck updated CASSANDRA-15260: Status: Ready to Commit (was: Review In Progress) > Add `allocate_tokens_for_dc_rf` yaml option for token allocation > > > Key: CASSANDRA-15260 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15260 > Project: Cassandra > Issue Type: Improvement > Components: Local/Config >Reporter: mck >Assignee: mck >Priority: Normal > Fix For: 4.x > > > Similar to DSE's option: {{allocate_tokens_for_local_replication_factor}} > Currently the > [ReplicationAwareTokenAllocator|https://www.datastax.com/dev/blog/token-allocation-algorithm] > requires a defined keyspace and a replica factor specified in the current > datacenter. > This is problematic in a number of ways. The real keyspace can not be used > when adding new datacenters as, in practice, all its nodes need to be up and > running before it has the capacity to replicate data into it. New datacenters > (or lift-and-shifting a cluster via datacenter migration) therefore has to be > done using a dummy keyspace that duplicates the replication strategy+factor > of the real keyspace. This gets even more difficult come version 4.0, as the > replica factor can not even be defined in new datacenters before those > datacenters are up and running. > These issues are removed by avoiding the keyspace definition and lookup, and > presuming the replica strategy is by datacenter, ie NTS. This can be done > with the use of an {{allocate_tokens_for_dc_rf}} option. > It may also be of value considering whether {{allocate_tokens_for_dc_rf=3}} > becomes the default? as this is the replication factor for the vast majority > of datacenters in production. I suspect this would be a good improvement over > the existing randomly generated tokens algorithm. > Initial patch is available in > [https://github.com/thelastpickle/cassandra/commit/fc4865b0399570e58f11215565ba17dc4a53da97] > The patch does not remove the existing {{allocate_tokens_for_keyspace}} > option, as that provides the codebase for handling different replication > strategies. > > fyi [~blambov] [~jay.zhuang] [~chovatia.jayd...@gmail.com] [~alokamvenki] > [~alexchueshev] -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15260) Add `allocate_tokens_for_dc_rf` yaml option for token allocation
[ https://issues.apache.org/jira/browse/CASSANDRA-15260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mck updated CASSANDRA-15260: Reviewers: Branimir Lambov, mck (was: Branimir Lambov) Status: Review In Progress (was: Patch Available) > Add `allocate_tokens_for_dc_rf` yaml option for token allocation > > > Key: CASSANDRA-15260 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15260 > Project: Cassandra > Issue Type: Improvement > Components: Local/Config >Reporter: mck >Assignee: mck >Priority: Normal > Fix For: 4.x > > > Similar to DSE's option: {{allocate_tokens_for_local_replication_factor}} > Currently the > [ReplicationAwareTokenAllocator|https://www.datastax.com/dev/blog/token-allocation-algorithm] > requires a defined keyspace and a replica factor specified in the current > datacenter. > This is problematic in a number of ways. The real keyspace can not be used > when adding new datacenters as, in practice, all its nodes need to be up and > running before it has the capacity to replicate data into it. New datacenters > (or lift-and-shifting a cluster via datacenter migration) therefore has to be > done using a dummy keyspace that duplicates the replication strategy+factor > of the real keyspace. This gets even more difficult come version 4.0, as the > replica factor can not even be defined in new datacenters before those > datacenters are up and running. > These issues are removed by avoiding the keyspace definition and lookup, and > presuming the replica strategy is by datacenter, ie NTS. This can be done > with the use of an {{allocate_tokens_for_dc_rf}} option. > It may also be of value considering whether {{allocate_tokens_for_dc_rf=3}} > becomes the default? as this is the replication factor for the vast majority > of datacenters in production. I suspect this would be a good improvement over > the existing randomly generated tokens algorithm. > Initial patch is available in > [https://github.com/thelastpickle/cassandra/commit/fc4865b0399570e58f11215565ba17dc4a53da97] > The patch does not remove the existing {{allocate_tokens_for_keyspace}} > option, as that provides the codebase for handling different replication > strategies. > > fyi [~blambov] [~jay.zhuang] [~chovatia.jayd...@gmail.com] [~alokamvenki] > [~alexchueshev] -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
svn commit: r35701 - in /release/cassandra: 4.0-alpha1/ debian/dists/40x/ debian/dists/40x/main/ debian/dists/40x/main/binary-amd64/ debian/dists/40x/main/binary-i386/ debian/dists/40x/main/source/ de
Author: mshuler Date: Sun Sep 8 17:15:33 2019 New Revision: 35701 Log: Release Apache Cassandra 4.0-alpha1 Added: release/cassandra/4.0-alpha1/ release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz (with props) release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc.md5 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc.sha1 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.md5 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.sha1 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz (with props) release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz.asc release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz.asc.md5 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz.asc.sha1 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz.md5 release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-src.tar.gz.sha1 release/cassandra/debian/dists/40x/ release/cassandra/debian/dists/40x/InRelease release/cassandra/debian/dists/40x/Release release/cassandra/debian/dists/40x/Release.gpg release/cassandra/debian/dists/40x/main/ release/cassandra/debian/dists/40x/main/binary-amd64/ release/cassandra/debian/dists/40x/main/binary-amd64/Packages release/cassandra/debian/dists/40x/main/binary-amd64/Packages.gz (with props) release/cassandra/debian/dists/40x/main/binary-amd64/Release release/cassandra/debian/dists/40x/main/binary-i386/ release/cassandra/debian/dists/40x/main/binary-i386/Packages release/cassandra/debian/dists/40x/main/binary-i386/Packages.gz (with props) release/cassandra/debian/dists/40x/main/binary-i386/Release release/cassandra/debian/dists/40x/main/source/ release/cassandra/debian/dists/40x/main/source/Release release/cassandra/debian/dists/40x/main/source/Sources.gz (with props) release/cassandra/debian/pool/main/c/cassandra/cassandra-tools_4.0~alpha1_all.deb (with props) release/cassandra/debian/pool/main/c/cassandra/cassandra_4.0~alpha1.diff.gz (with props) release/cassandra/debian/pool/main/c/cassandra/cassandra_4.0~alpha1.dsc release/cassandra/debian/pool/main/c/cassandra/cassandra_4.0~alpha1.orig.tar.gz (with props) release/cassandra/debian/pool/main/c/cassandra/cassandra_4.0~alpha1.orig.tar.gz.asc release/cassandra/debian/pool/main/c/cassandra/cassandra_4.0~alpha1_all.deb (with props) release/cassandra/redhat/40x/ release/cassandra/redhat/40x/cassandra-4.0~alpha1-1.noarch.rpm (with props) release/cassandra/redhat/40x/cassandra-4.0~alpha1-1.src.rpm (with props) release/cassandra/redhat/40x/cassandra-tools-4.0~alpha1-1.noarch.rpm (with props) release/cassandra/redhat/40x/repodata/ release/cassandra/redhat/40x/repodata/1242c3bfa50a2b85dd2d9e5131f01ee26d25ab17996fc567d7831165968d4729-primary.sqlite.bz2 (with props) release/cassandra/redhat/40x/repodata/14506f12ac7ad5fc3d4557614c9875aebeceb837490a4fbdf7b328b776adbb6a-other.sqlite.bz2 (with props) release/cassandra/redhat/40x/repodata/3050ea04fc75cb064094dfbb9fb3bb7def05ab0e7dedb5e5300bfc55c8973c18-primary.xml.gz (with props) release/cassandra/redhat/40x/repodata/3f31851c790201318c71fdcf467f6e67715348201ca8ed8b75f23d569b564600-other.xml.gz (with props) release/cassandra/redhat/40x/repodata/6180e4eaf4374e34e11851bd082cfc8dc60d17185f7ab82b34bf258ccb1d338b-filelists.sqlite.bz2 (with props) release/cassandra/redhat/40x/repodata/fe6ee138a57e93c7ac62a0e145d135c947e3ff975a35adf55c980e5476a7d755-filelists.xml.gz (with props) release/cassandra/redhat/40x/repodata/repomd.xml release/cassandra/redhat/40x/repodata/repomd.xml.asc Added: release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz == Binary file - no diff available. Propchange: release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz -- svn:mime-type = application/octet-stream Added: release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc == --- release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc (added) +++ release/cassandra/4.0-alpha1/apache-cassandra-4.0-alpha1-bin.tar.gz.asc Sun Sep 8 17:15:33 2019 @@ -0,0 +1,16 @@ +-BEGIN PGP SIGNATURE- + +iQIzBAABCAAdFiEEom5SiycfGbnl2OGeoni3gf5LK9oFAl1xiwEACgkQoni3gf5L +K9puyg/+I/GfV0WwJu0/fU0pcWl+rWF5keLyF5YdxPjt2X50KsmQR2BNsQE590Ck +4IphnAKqybxYJpJORMD0GHQfMlda3MiGvSJWPj1tH9hUQFZTjAZDUMkf9NqvMjhM +N5+qrEq85YNvsYAK2SEKNoPG7W5vU9rk9jykL8DZrblkq1rcdfxBl3xRkucIr0fw +ikorNUtAwhhJugndxnruZGfDFkZOL
[cassandra] annotated tag cassandra-4.0-alpha1 created (now 5c71d94)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to annotated tag cassandra-4.0-alpha1 in repository https://gitbox.apache.org/repos/asf/cassandra.git. at 5c71d94 (tag) tagging fc4381ca89ab39a82c9018e5171975285cc3bfe7 (commit) replaces cassandra-3.11.4 by Michael Shuler on Sun Sep 8 09:15:06 2019 -0700 - Log - Apache Cassandra 4.0-alpha1 release --- No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] tag 4.0-alpha1-tentative deleted (was fc4381c)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to tag 4.0-alpha1-tentative in repository https://gitbox.apache.org/repos/asf/cassandra.git. *** WARNING: tag 4.0-alpha1-tentative was deleted! *** was fc4381c Add auditlogviewer and fqltool to rpm spec The revisions that were on this tag are still contained in other references; therefore, this change does not discard any commits from the repository. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] annotated tag cassandra-4.0-alpha1 deleted (was 61cefd0)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to annotated tag cassandra-4.0-alpha1 in repository https://gitbox.apache.org/repos/asf/cassandra.git. *** WARNING: tag cassandra-4.0-alpha1 was deleted! *** tag was 61cefd0 The revisions that were on this annotated tag are still contained in other references; therefore, this change does not discard any commits from the repository. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] tag 4.0-alpha1-tentative created (now fc4381c)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to tag 4.0-alpha1-tentative in repository https://gitbox.apache.org/repos/asf/cassandra.git. at fc4381c (commit) No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] annotated tag cassandra-4.0-alpha1 created (now 61cefd0)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to annotated tag cassandra-4.0-alpha1 in repository https://gitbox.apache.org/repos/asf/cassandra.git. at 61cefd0 (tag) tagging fc4381ca89ab39a82c9018e5171975285cc3bfe7 (commit) replaces cassandra-3.11.4 by Michael Shuler on Sun Sep 8 09:05:24 2019 -0700 - Log - Apache Cassandra 4.0-alpha1 release --- No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] tag 4.0-alpha1-tentative deleted (was fc4381c)
This is an automated email from the ASF dual-hosted git repository. mshuler pushed a change to tag 4.0-alpha1-tentative in repository https://gitbox.apache.org/repos/asf/cassandra.git. *** WARNING: tag 4.0-alpha1-tentative was deleted! *** was fc4381c Add auditlogviewer and fqltool to rpm spec The revisions that were on this tag are still contained in other references; therefore, this change does not discard any commits from the repository. - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15317) CAST AS function vulnerable to integer overflow
Avi Kivity created CASSANDRA-15317: -- Summary: CAST AS function vulnerable to integer overflow Key: CASSANDRA-15317 URL: https://issues.apache.org/jira/browse/CASSANDRA-15317 Project: Cassandra Issue Type: Bug Components: CQL/Interpreter Reporter: Avi Kivity {noformat} cqlsh:ks1> create table bigdec (k decimal primary key); cqlsh:ks1> insert into bigdec (k) values (100); cqlsh:ks1> select * from bigdec; k - 100 (1 rows) cqlsh:ks1> select cast(k as int) from bigdec; cast(k as int) 276447232{noformat} This overflow is unexpected for the user and can lead to incorrect results. Better to refuse to execute the query. -- This message was sent by Atlassian Jira (v8.3.2#803003) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org