[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178136#comment-17178136
 ] 

David Capwell commented on CASSANDRA-15946:
---

Looks like the issue is that 
org.apache.cassandra.net.Message.Serializer#deserializePayloadPre40 returns 
null when the payloadSize is 0, so won't call the deserializer to return empty

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178126#comment-17178126
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/15/20, 2:01 AM:
-

Ok been testing this out, this is what I see

1) when using mock networking, this patch is required because jvm-dtest filters 
will serialize all messages; this fails without the patch
2) with networking and gossip are used this still fails, but with a different 
error

{code}
DEBUG [node1_GossipStage:1] node1 2020-08-14 18:49:18,033 
MigrationManager.java:96 - Not pulling schema because release version in Gossip 
is not major version 3, it is 4.0-beta2-SNAPSHOT
ERROR 01:49:18 Exception in thread Thread[node2_GossipStage:1,5,node2]
java.lang.IllegalArgumentException: null
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:40)
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:36)
at 
org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
at 
org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
at 
org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:263)
at 
org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:236)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:44)
at 
org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:890)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:748)
{code}

In a debugger I see the verb that fails is ECHO_REQ, this is caused by the fact 
the payload is null rather than NoPayload

{code}
ECHO_REQ   (31,  P0, rpcTimeout,  GOSSIP,() -> 
NoPayload.serializer, () -> EchoVerbHandler.instance,   
 ECHO_RSP),
{code}


was (Author: dcapwell):
Ok been testing this out, this is what I see

1) when using mock networking, this patch is required because jvm-dtest filters 
will serialize all messages; this fails without the patch
2) with networking and gossip are used this still fails, but with a different 
error

{code}
DEBUG [node1_GossipStage:1] node1 2020-08-14 18:49:18,033 
MigrationManager.java:96 - Not pulling schema because release version in Gossip 
is not major version 3, it is 4.0-beta2-SNAPSHOT
ERROR 01:49:18 Exception in thread Thread[node2_GossipStage:1,5,node2]
java.lang.IllegalArgumentException: null
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:40)
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:36)
at 
org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
at 
org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
at 
org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:263)
at 
org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:236)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:44)
at 
org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:890)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:748)
{code}

In a debugger I see the verb that fails is ECHO_REQ

{code}
ECHO_REQ   (31,  P0, rpcTimeout,  GOSSIP,() -> 
NoPayload.serializer, () -> EchoVerbHandler.instance,   
 ECHO_RSP),
{code}

> NPE when sending 

[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178126#comment-17178126
 ] 

David Capwell commented on CASSANDRA-15946:
---

Ok been testing this out, this is what I see

1) when using mock networking, this patch is required because jvm-dtest filters 
will serialize all messages; this fails without the patch
2) with networking and gossip are used this still fails, but with a different 
error

{code}
DEBUG [node1_GossipStage:1] node1 2020-08-14 18:49:18,033 
MigrationManager.java:96 - Not pulling schema because release version in Gossip 
is not major version 3, it is 4.0-beta2-SNAPSHOT
ERROR 01:49:18 Exception in thread Thread[node2_GossipStage:1,5,node2]
java.lang.IllegalArgumentException: null
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:40)
at org.apache.cassandra.net.NoPayload$1.serialize(NoPayload.java:36)
at 
org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
at 
org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
at 
org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:263)
at 
org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:236)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
at 
org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:44)
at 
org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:890)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:748)
{code}

In a debugger I see the verb that fails is ECHO_REQ

{code}
ECHO_REQ   (31,  P0, rpcTimeout,  GOSSIP,() -> 
NoPayload.serializer, () -> EchoVerbHandler.instance,   
 ECHO_RSP),
{code}

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at 

[jira] [Commented] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure

2020-08-14 Thread Caleb Rackliffe (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178123#comment-17178123
 ] 

Caleb Rackliffe commented on CASSANDRA-15861:
-

[~jasonstack] [~benedict] I [made a 
pass|https://github.com/maedhroz/cassandra/tree/CASSANDRA-15861-final-summary] 
at a builder-based approach (to making more of {{SSTableReader}}'s fields 
{{final}}) based on the [current 
branch/PR|https://github.com/apache/cassandra/pull/642#discussion_r470073171]. 
There are some loose ends to tie up, but it does at least make progress toward 
consolidating the logic that builds the components necessary for reader 
creation. (It also seems to pass the tests in this PR and things like 
{{FailingRepairTest}} and {{SSTableReaderTest}} without trouble.)

> Mutating sstable component may race with entire-sstable-streaming(ZCS) 
> causing checksum validation failure
> --
>
> Key: CASSANDRA-15861
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15861
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair, Consistency/Streaming, 
> Local/Compaction
>Reporter: ZhaoYang
>Assignee: ZhaoYang
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Flaky dtest: [test_dead_sync_initiator - 
> repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/]
> {code:java|title=stacktrace}
> Unexpected error found in node logs (see stdout for full details). Errors: 
> [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 
> CassandraEntireSSTableStreamReader.java:145 - [Stream 
> 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream 
> for table = keyspace1.standard1
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db
>   at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219)
>   at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198)
>   at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129)
>   at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:49)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:36)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:49)
>   at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:181)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.IOException: Checksums do not match for 
> /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db
> {code}
>  
> In the above test, it executes "nodetool repair" on node1 and kills node2 
> during repair. At the end, node3 reports checksum validation failure on 
> sstable transferred from node1.
> {code:java|title=what happened}
> 1. When repair started on node1, it performs anti-compaction which modifies 
> sstable's repairAt to 0 and pending repair id to session-id.
> 2. Then node1 creates {{ComponentManifest}} which contains file lengths to be 
> transferred to node3.
> 3. Before node1 actually sends the files to node3, node2 is killed and node1 
> starts to broadcast repair-failure-message to all participants in 
> {{CoordinatorSession#fail}}
> 4. Node1 receives its own repair-failure-message and fails its local repair 
> sessions at {{LocalSessions#failSession}} which triggers async background 
> compaction.
> 5. Node1's background compaction will mutate sstable's repairAt to 0 and 
> pending repair id to null via  
> {{PendingRepairManager#getNextRepairFinishedTask}}, as there is no more 
> in-progress repair.
> 6. Node1 actually sends the sstable to node3 where the sstable's STATS 
> component size is different from 

[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178054#comment-17178054
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/15/20, 1:12 AM:
-

Very simple to replicate with

{code}
@Test
public void test() throws Exception
{
Versions versions = Versions.find();
try (Cluster cluster = init(builder()
.withNodes(2)

.withVersion(versions.getLatest(Versions.Major.v30))
.withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
.start()))
{
cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int 
primary key)"));

cluster.get(2).shutdown().get();
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(2).startup();

System.err.println("Doing insert");
cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), ConsistencyLevel.ALL, 0);
cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), 1);
System.err.println("Doing query");
SimpleQueryResult qr = 
cluster.coordinator(2).executeWithResult(withKeyspace("SELECT * FROM %s.tbl"), 
ConsistencyLevel.ALL);
AssertUtils.assertRows(qr, QueryResults.builder()
   .row(1)
   .row(0)
   .build());
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.


was (Author: dcapwell):
Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws Exception
{
Versions versions = Versions.find();
try (Cluster cluster = init(builder()
.withNodes(2)

.withVersion(versions.getLatest(Versions.Major.v30))
.withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
.start()))
{
cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int 
primary key)"));

cluster.get(2).shutdown().get();
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(2).startup();

System.err.println("Doing insert");
cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), ConsistencyLevel.ALL, 0);
cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), 1);
System.err.println("Doing query");
SimpleQueryResult qr = 
cluster.coordinator(2).executeWithResult(withKeyspace("SELECT * FROM %s.tbl"), 
ConsistencyLevel.ALL);
AssertUtils.assertRows(qr, QueryResults.builder()
   .row(0)
   .row(1)
   .build());
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> 

[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178054#comment-17178054
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/15/20, 12:38 AM:
--

Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws Exception
{
Versions versions = Versions.find();
try (Cluster cluster = init(builder()
.withNodes(2)

.withVersion(versions.getLatest(Versions.Major.v30))
.withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
.start()))
{
cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int 
primary key)"));

cluster.get(2).shutdown().get();
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(2).startup();

System.err.println("Doing insert");
cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), ConsistencyLevel.ALL, 0);
cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), 1);
System.err.println("Doing query");
SimpleQueryResult qr = 
cluster.coordinator(2).executeWithResult(withKeyspace("SELECT * FROM %s.tbl"), 
ConsistencyLevel.ALL);
AssertUtils.assertRows(qr, QueryResults.builder()
   .row(0)
   .row(1)
   .build());
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.


was (Author: dcapwell):
Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws IOException, InterruptedException, 
ExecutionException
{
Versions versions = Versions.find();
try (Cluster cluster = init(builder()
.withNodes(2)

.withVersion(versions.getLatest(Versions.Major.v30))
.withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
.start()))
{
cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int 
primary key)"));

cluster.get(2).shutdown().get();
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(2).startup();

System.err.println("Doing insert");
cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), ConsistencyLevel.ALL, 0L);
System.err.println("Doing query");
SimpleQueryResult qr = 
cluster.coordinator(2).executeWithResult(withKeyspace("SELECT * FROM %s.tbl"), 
ConsistencyLevel.ALL);
System.err.println("Query results: " + qr);

for (int i = 0; i < 100; i++)
TimeUnit.SECONDS.sleep(1);
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> 

[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178066#comment-17178066
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/15/20, 12:07 AM:
--

disabling filters (comment out in code) fixes most other messages

In short, I think this is a much bigger issue.  jvm dtest doesn't work properly 
in mixed mode

Given that, I don't think this patch is enough


was (Author: dcapwell):
disabling filters (comment out in code) fixes most other messages

In short, I think this is a much bigger issue.  jvm dtest doesn't work properly 
in mixed mode and there might be mixed mode bugs as well (not sure if I can 
attribute this to jvm dtest or not)...

Given that, I don't think this patch is enough

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178066#comment-17178066
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/15/20, 12:05 AM:
--

disabling filters (comment out in code) fixes most other messages

In short, I think this is a much bigger issue.  jvm dtest doesn't work properly 
in mixed mode and there might be mixed mode bugs as well (not sure if I can 
attribute this to jvm dtest or not)...

Given that, I don't think this patch is enough


was (Author: dcapwell):
Ok, now that I disabled filters I added read/write queries and getting

{code}
Caused by: org.apache.cassandra.exceptions.InvalidRequestException: Expected 4 
or 0 byte int (8)
at 
org.apache.cassandra.cql3.Constants$Marker.bindAndGet(Constants.java:303)
at 
org.apache.cassandra.cql3.restrictions.SingleColumnRestriction$EQRestriction.appendTo(SingleColumnRestriction.java:160)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.appendTo(PrimaryKeyRestrictionSet.java:171)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.valuesAsClustering(PrimaryKeyRestrictionSet.java:163)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.values(PrimaryKeyRestrictionSet.java:223)
at 
org.apache.cassandra.cql3.restrictions.StatementRestrictions.getPartitionKeys(StatementRestrictions.java:648)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.buildPartitionKeyNames(ModificationStatement.java:296)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.addUpdates(ModificationStatement.java:643)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.getMutations(ModificationStatement.java:632)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:432)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:420)
at 
org.apache.cassandra.distributed.impl.Coordinator.executeInternal(Coordinator.java:100)
at 
org.apache.cassandra.distributed.impl.Coordinator.lambda$executeWithResult$0(Coordinator.java:62)
{code}

In short, I think this is a much bigger issue.  jvm dtest doesn't work properly 
in mixed mode and there might be mixed mode bugs as well (not sure if I can 
attribute this to jvm dtest or not)...

Given that, I don't think this patch solves the real problem.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
> 

[jira] [Commented] (CASSANDRA-15854) Truncation should fail any ongoing repairs

2020-08-14 Thread Caleb Rackliffe (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178070#comment-17178070
 ] 

Caleb Rackliffe commented on CASSANDRA-15854:
-

[~marcuse] Thanks!

Left a few more very [minor 
nits|https://github.com/krummas/cassandra/commit/0f81a451bba4e2d40babe6fdacfb9565abe42775],
 but otherwise LGTM ;)

> Truncation should fail any ongoing repairs
> --
>
> Key: CASSANDRA-15854
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15854
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Truncation may race with ongoing repairs, making it possible to clear data on 
> one node but then stream data its truncation would have deleted from another 
> node. We should abort any ongoing preview repairs if we get a truncation 
> request.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178066#comment-17178066
 ] 

David Capwell commented on CASSANDRA-15946:
---

Ok, now that I disabled filters I added read/write queries and getting

{code}
Caused by: org.apache.cassandra.exceptions.InvalidRequestException: Expected 4 
or 0 byte int (8)
at 
org.apache.cassandra.cql3.Constants$Marker.bindAndGet(Constants.java:303)
at 
org.apache.cassandra.cql3.restrictions.SingleColumnRestriction$EQRestriction.appendTo(SingleColumnRestriction.java:160)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.appendTo(PrimaryKeyRestrictionSet.java:171)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.valuesAsClustering(PrimaryKeyRestrictionSet.java:163)
at 
org.apache.cassandra.cql3.restrictions.PrimaryKeyRestrictionSet.values(PrimaryKeyRestrictionSet.java:223)
at 
org.apache.cassandra.cql3.restrictions.StatementRestrictions.getPartitionKeys(StatementRestrictions.java:648)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.buildPartitionKeyNames(ModificationStatement.java:296)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.addUpdates(ModificationStatement.java:643)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.getMutations(ModificationStatement.java:632)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:432)
at 
org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:420)
at 
org.apache.cassandra.distributed.impl.Coordinator.executeInternal(Coordinator.java:100)
at 
org.apache.cassandra.distributed.impl.Coordinator.lambda$executeWithResult$0(Coordinator.java:62)
{code}

In short, I think this is a much bigger issue.  jvm dtest doesn't work properly 
in mixed mode and there might be mixed mode bugs as well (not sure if I can 
attribute this to jvm dtest or not)...

Given that, I don't think this patch solves the real problem.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: 

[jira] [Comment Edited] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178054#comment-17178054
 ] 

David Capwell edited comment on CASSANDRA-15946 at 8/14/20, 9:14 PM:
-

Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws IOException, InterruptedException, 
ExecutionException
{
Versions versions = Versions.find();
try (Cluster cluster = init(builder()
.withNodes(2)

.withVersion(versions.getLatest(Versions.Major.v30))
.withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
.start()))
{
cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int 
primary key)"));

cluster.get(2).shutdown().get();
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(2).startup();

System.err.println("Doing insert");
cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl 
(pk) VALUES (?)"), ConsistencyLevel.ALL, 0L);
System.err.println("Doing query");
SimpleQueryResult qr = 
cluster.coordinator(2).executeWithResult(withKeyspace("SELECT * FROM %s.tbl"), 
ConsistencyLevel.ALL);
System.err.println("Query results: " + qr);

for (int i = 0; i < 100; i++)
TimeUnit.SECONDS.sleep(1);
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.


was (Author: dcapwell):
Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws IOException, InterruptedException
{
Versions versions = Versions.find();
try (Cluster cluster = builder()
   .withNodes(2)
   .withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
   .createWithoutStarting())
{
((IUpgradeableInstance) 
cluster.get(1)).setVersion(versions.getLatest(Versions.Major.v30));
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(1).startup();
cluster.get(2).startup();
 init(cluster);
 for (int i = 0; i < 100; i++)
 TimeUnit.SECONDS.sleep(1);
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> 

[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178061#comment-17178061
 ] 

David Capwell commented on CASSANDRA-15946:
---

I did a quick test to disable filter logic in both versions and reran; no 
issues.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178054#comment-17178054
 ] 

David Capwell commented on CASSANDRA-15946:
---

Very simple to replicate with

{code}
public class MixModeLongLiveTest extends TestBaseImpl
{
@Test
public void test() throws IOException, InterruptedException
{
Versions versions = Versions.find();
try (Cluster cluster = builder()
   .withNodes(2)
   .withConfig(c -> c.with(Feature.GOSSIP, 
Feature.NATIVE_PROTOCOL, Feature.NETWORK))
   .createWithoutStarting())
{
((IUpgradeableInstance) 
cluster.get(1)).setVersion(versions.getLatest(Versions.Major.v30));
((IUpgradeableInstance) 
cluster.get(2)).setVersion(versions.getLatest(Versions.Major.v4));
cluster.get(1).startup();
cluster.get(2).startup();
 init(cluster);
 for (int i = 0; i < 100; i++)
 TimeUnit.SECONDS.sleep(1);
}
}
}
{code}

I don't do any writes, so this patch doesn't fix the messages being sent.

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15946) NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests

2020-08-14 Thread David Capwell (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178053#comment-17178053
 ] 

David Capwell commented on CASSANDRA-15946:
---

I got curious so did a simple test where one node was v3.0 and the other was 
v4.0 and just let it run; this replicated the issue super quickly.  What I see 
is that the issue is with 
org.apache.cassandra.net.EndpointMessagingVersions#get(org.apache.cassandra.locator.InetAddressAndPort).
  When we don't know the version we assume it is the current version... so next 
question is... why don't we know the version from v3.0?

> NPE when sending REQUEST_RSP from 3.0 to 4.0 in in-jvm dtests
> -
>
> Key: CASSANDRA-15946
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15946
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> There is a communication problem when testing upgrades using in-JVM dtest 
> between Cassandra 3 and 4. 
> In a method {{registerInboundFilter}} of {{Instance}}, we get a message which 
> was just received and we prepare it for filtering as part of which, we 
> serialize the payload again. This is fine when dealing with incoming 
> Cassandra 4 message, because we can serialize it. However when we get the 
> Cassandra 3 message, which uses a different protocol, and we get something 
> like {{REQUEST_RSP}}, we can surely deserialize it through some special 
> deserialization path, but we cannot serialize the payload for it as there is 
> no serializer defined for {{REQUEST_RSP}} - no wonder, why would Cassandra 
> 4.0 need to be able to serialize Cassandra 3.0 payloads?
> {code}
> java.lang.NullPointerException: null
>   at 
> org.apache.cassandra.net.Message$Serializer.serializePost40(Message.java:760)
>   at 
> org.apache.cassandra.net.Message$Serializer.serialize(Message.java:618)
>   at 
> org.apache.cassandra.distributed.impl.Instance.serializeMessage(Instance.java:267)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$registerInboundFilter$4(Instance.java:234)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:62)
>   at 
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:49)
>   at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
>   at 
> org.apache.cassandra.distributed.impl.Instance.lambda$null$6(Instance.java:305)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:119)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-15854) Truncation should fail any ongoing repairs

2020-08-14 Thread Caleb Rackliffe (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-15854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Caleb Rackliffe updated CASSANDRA-15854:

Description: Truncation may race with ongoing repairs, making it possible 
to clear data on one node but then stream data its truncation would have 
deleted from another node. We should abort any ongoing preview repairs if we 
get a truncation request.  (was: Truncation may race with preview repairs, 
validating different data on different nodes, we should abort any ongoing 
preview repairs if we get a truncation request)

> Truncation should fail any ongoing repairs
> --
>
> Key: CASSANDRA-15854
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15854
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Truncation may race with ongoing repairs, making it possible to clear data on 
> one node but then stream data its truncation would have deleted from another 
> node. We should abort any ongoing preview repairs if we get a truncation 
> request.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16046) BLOG - Post about Cassandra & Kubernetes - SIG Update

2020-08-14 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated CASSANDRA-16046:
---
Labels: pull-request-available  (was: )

> BLOG - Post about Cassandra & Kubernetes - SIG Update
> -
>
> Key: CASSANDRA-16046
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16046
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Documentation/Blog
>Reporter: Rahul Singh
>Assignee: Rahul Singh
>Priority: Normal
>  Labels: pull-request-available
>
> We're getting a blog post out before/coinciding with Kubecon 2020. Content 
> code is mostly ready 
>  
>  
> [https://github.com/Anant/cassandra-website/blob/master/src/_posts/2020-08-17-cassandra-and-kubernetes-sig-update.markdown]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15909) Make Table/Keyspace Metric Names Consistent With Each Other

2020-08-14 Thread Adam Holmberg (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17178017#comment-17178017
 ] 

Adam Holmberg commented on CASSANDRA-15909:
---

[~spmallette] should this be "Patch Available" ?

> Make Table/Keyspace Metric Names Consistent With Each Other
> ---
>
> Key: CASSANDRA-15909
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15909
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Observability/Metrics
>Reporter: Stephen Mallette
>Assignee: Stephen Mallette
>Priority: Normal
> Fix For: 4.0-beta
>
>
> As part of CASSANDRA-15821 it became apparent that certain metric names found 
> in keyspace and tables had different names but were in fact the same metric - 
> they are as follows:
> * Table.SyncTime == Keyspace.RepairSyncTime
> * Table.RepairedDataTrackingOverreadRows == Keyspace.RepairedOverreadRows
> * Table.RepairedDataTrackingOverreadTime == Keyspace.RepairedOverreadTime
> * Table.AllMemtablesHeapSize == Keyspace.AllMemtablesOnHeapDataSize
> * Table.AllMemtablesOffHeapSize == Keyspace.AllMemtablesOffHeapDataSize
> * Table.MemtableOnHeapSize == Keyspace.MemtableOnHeapDataSize
> * Table.MemtableOffHeapSize == Keyspace.MemtableOffHeapDataSize
> Also, client metrics are the only metrics to start with a lower case letter. 
> Change those to upper case to match all the other metrics.
> Unifying this naming would help make metrics more consistent as part of 
> CASSANDRA-15582



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15828) Remove jackson-mapper-asl-1.9.13 to address CVE

2020-08-14 Thread Kevin Eveker (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15828?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177988#comment-17177988
 ] 

Kevin Eveker commented on CASSANDRA-15828:
--

[~mdenihan] Since the security scanners found this jar that has a vulnerability 
and is no longer supported, it has to be removed to pass our security scans. Is 
there a supported jar that can be used to replace it? We are working to get an 
ATO with Cassandra as part of our solution which motivated this ticket. I wish 
I understood this baseline better and could offer some help from me or a member 
of my team. 

> Remove jackson-mapper-asl-1.9.13 to address CVE
> ---
>
> Key: CASSANDRA-15828
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15828
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Kevin Eveker
>Priority: Normal
>
> Recent scan results identified the following CVE that require this upgrade to 
> address
> [https://nvd.nist.gov/vuln/detail/CVE-2019-10172]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15939) Remove previously deprecated HintedHandoffManager

2020-08-14 Thread Adam Holmberg (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15939?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177987#comment-17177987
 ] 

Adam Holmberg commented on CASSANDRA-15939:
---

[~spmallette] is it safe to assume you do not intend to address this? Just 
asking because you're assigned.

> Remove previously deprecated HintedHandoffManager
> -
>
> Key: CASSANDRA-15939
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15939
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Consistency/Hints
>Reporter: Stephen Mallette
>Assignee: Stephen Mallette
>Priority: Normal
> Fix For: 4.0-beta
>
>
> {{HintedHandOffManager}} was deprecated about 6 years ago:
> https://github.com/apache/cassandra/commit/96d41f0e0e44d9b3114a5d80dedf12053d36a76b
> with the TODO to remove it for 4.0. While looking at CASSANDRA-15582 and 
> examining the published JMX elements it seemed sensible to remove this MBean 
> to help clean up a bit as it appears to just be a proxy to the 
> {{HintService}} at this point. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16051) sstablemetadata line 44: : command not found

2020-08-14 Thread Brandon Williams (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-16051:
-
 Bug Category: Parent values: Correctness(12982)
   Complexity: Normal
Discovered By: User Report
Fix Version/s: 4.0-beta2
 Severity: Low
   Status: Open  (was: Triage Needed)

[~snazy] CASSANDRA-9608 removed this code, can you take a look?

> sstablemetadata line 44: : command not found
> 
>
> Key: CASSANDRA-16051
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16051
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Arvinder Singh
>Priority: Normal
> Fix For: 4.0-beta2
>
>
> sstablemetadata shipped with 
> [4.0-beta|https://issues.apache.org/jira/issues/?jql=project+%3D+CASSANDRA+AND+fixVersion+%3D+4.0-beta]
>  is missing the code "#Use JAVA_HOME if set, otherwise look for java in PATH" 
> and report error:
> sstablemetadata: line 44: : command not found
>  
> The fix is to merge below code from 3.x:
> ^# Use JAVA_HOME if set, otherwise look for java in PATH^
> ^if [ -x "$JAVA_HOME/bin/java" ]; then^
>  ^JAVA="$JAVA_HOME/bin/java"^
> ^else^
>  ^JAVA="`which java`"^
> ^fi^
> ^if [ "x$JAVA" = "x" ]; then^
>  ^echo "Java executable not found (hint: set JAVA_HOME)" >&2^
>  ^exit 1^
> ^fi^



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13701) Lower default num_tokens

2020-08-14 Thread Alexander Dejanovski (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177965#comment-17177965
 ] 

Alexander Dejanovski commented on CASSANDRA-13701:
--

I've identified several issues today:
 * ccm uses a hardcoded 30s timeout when waiting for events (like nodes to 
start) which doesn't work with the additional wait times that come with the new 
token allocation algorithm. Fix is 
[here|https://github.com/riptano/ccm/commit/8a91a5aa49473211863a1fb7a980206e5222ce5d].
 * ccm starts all nodes at the same time when cluster.start() is invoked, which 
creates clashes when the new token allocation algorithm is used and makes some 
tests flaky. Starting them sequentially using [this 
fix|https://github.com/riptano/ccm/commit/e6e4abcff375debde8195104c5cffd1cecb8d6cf],
 allowed all the bootstrap dtests to pass.
* [~jeromatron]'s branch is missing some commits in the current trunk that fix 
other failing dtests. Rebasing it over trunk is necessary to get them all to 
pass
* Adding a few seconds of sleep in 
[bootstrap_test.py::TestBootstrap::test_simultaneous_bootstrap|https://github.com/adejanovski/cassandra-dtest/blob/master/bootstrap_test.py#L769-L771]
 allows the test to pass.

I'm currently rerunning all dtests with the various fixes to see if I still get 
failures. I'll follow up on monday and hopefully push PRs to ccm and 
cassandra-dtests that will allow the patch to be applied (there are conflicts 
though so a rebase will be necessary).

A follow up discussion and ticket will probably be necessary because the new 
token allocation algorithm and concurrent bootstraps aren't working nicely 
together.

> Lower default num_tokens
> 
>
> Key: CASSANDRA-13701
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13701
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Local/Config
>Reporter: Chris Lohfink
>Assignee: Alexander Dejanovski
>Priority: Low
> Fix For: 4.0-alpha
>
>
> For reasons highlighted in CASSANDRA-7032, the high number of vnodes is not 
> necessary. It is very expensive for operations processes and scanning. Its 
> come up a lot and its pretty standard and known now to always reduce the 
> num_tokens within the community. We should just lower the defaults.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:15 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI run results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins: 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI run results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: 

[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:14 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI run results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: 

[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:14 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI run results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI run results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: 

[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:14 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results after rebase and the addition of your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results after rebase and adding your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: 

[jira] [Created] (CASSANDRA-16051) sstablemetadata line 44: : command not found

2020-08-14 Thread Arvinder Singh (Jira)
Arvinder Singh created CASSANDRA-16051:
--

 Summary: sstablemetadata line 44: : command not found
 Key: CASSANDRA-16051
 URL: https://issues.apache.org/jira/browse/CASSANDRA-16051
 Project: Cassandra
  Issue Type: Bug
  Components: Tool/sstable
Reporter: Arvinder Singh


sstablemetadata shipped with 
[4.0-beta|https://issues.apache.org/jira/issues/?jql=project+%3D+CASSANDRA+AND+fixVersion+%3D+4.0-beta]
 is missing the code "#Use JAVA_HOME if set, otherwise look for java in PATH" 
and report error:

sstablemetadata: line 44: : command not found

 

The fix is to merge below code from 3.x:

^# Use JAVA_HOME if set, otherwise look for java in PATH^
^if [ -x "$JAVA_HOME/bin/java" ]; then^
 ^JAVA="$JAVA_HOME/bin/java"^
^else^
 ^JAVA="`which java`"^
^fi^

^if [ "x$JAVA" = "x" ]; then^
 ^echo "Java executable not found (hint: set JAVA_HOME)" >&2^
 ^exit 1^
^fi^



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:13 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]

 

 

 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom circle config removed [here | 
https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]

 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: 

[jira] [Comment Edited] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova edited comment on CASSANDRA-13994 at 8/14/20, 5:13 PM:
---

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results after rebase and adding your commit look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]
 

 

 

 


was (Author: e.dimitrova):
Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom CircleCI config file removed [here 
|https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]

 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional 

[jira] [Commented] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177949#comment-17177949
 ] 

Ekaterina Dimitrova commented on CASSANDRA-13994:
-

Thank you for the review and the additional suggestions which I cherry-picked 
to the patch [~slebresne]

CI results look good to me:

[JAVA8|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3]
 
[JAVA11|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/f60d6eca-8bd1-4d68-add7-0a13c35f4695]
 * stMessagePurging - org.apache.cassandra.net.ConnectionTest - 
CASSANDRA-15958; failing both for Java 8 and Java 11
 * test_simple_parallel_repair - repair_tests.repair_test.TestRepair - not 
related, I will add it to be checked with CASSANDRA-15986; Java 8
 * j11_cqlsh-dtests-py38-with-vnodes show they failed but they passed, there 
was at the end some circle resource issue
 * test_resumable_rebuild - rebuild_test.TestRebuild - failing in java 11; 
CASSANDRA-15892
 * test_dead_sync_initiator - repair_tests.repair_test.TestRepair - 
CASSANDRA-15963
 * test_authentication_handle_unavailable - auth_test.TestAuthUnavailable - 
java 11 - not related; have to check whether a ticket is needed
 * The upgrade tests that are failing for reason different than No Host 
Available(test_static_columns_with_2i and upgrade_udtfix_test) were already 
failing on trunk in Jenkins:
 * 
[https://ci-cassandra.apache.org/job/Cassandra-trunk-dtest-upgrade/1/#showFailuresLink]

 

Commits squashed and custom circle config removed [here | 
https://github.com/ekaterinadimitrova2/cassandra/commit/e7de05c5ca70fa59aab622f2df41529971e63fb6]

 

 

 

 

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-15903) Doc update: stream-entire-sstable supports all compaction strategies and internode encryption

2020-08-14 Thread Adam Holmberg (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-15903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Adam Holmberg updated CASSANDRA-15903:
--
Change Category: Operability
 Complexity: Normal
Component/s: Documentation/Website
 Status: Open  (was: Triage Needed)

> Doc update: stream-entire-sstable supports all compaction strategies and 
> internode encryption
> -
>
> Key: CASSANDRA-15903
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15903
> Project: Cassandra
>  Issue Type: Task
>  Components: Documentation/Website
>Reporter: ZhaoYang
>Priority: Normal
> Fix For: 4.0
>
>
> As [~mck] point out, doc needs to be updated for CASSANDRA-15657  and 
> CASSANDRA-15740.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15249) Add documentation on release lifecycle

2020-08-14 Thread Adam Holmberg (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177870#comment-17177870
 ] 

Adam Holmberg commented on CASSANDRA-15249:
---

Is there anything left to do here? I see the doc is deprecated and its content 
is captured in the wiki, along with the outstanding questions.

> Add documentation on release lifecycle
> --
>
> Key: CASSANDRA-15249
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15249
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Documentation/Website
>Reporter: Sumanth Pasupuleti
>Assignee: Sumanth Pasupuleti
>Priority: Normal
> Fix For: 4.0
>
> Attachments: release_lifecycle.patch
>
>
> Relevant dev list mail thread: 
> https://lists.apache.org/thread.html/1a768d057d1af5a0f373c4c399a23e65cb04c61bbfff612634b9437c@%3Cdev.cassandra.apache.org%3E
> Cassandra wiki on release lifecycle - 
> https://cwiki.apache.org/confluence/display/CASSANDRA/Release+Lifecycle
> [Legacy - this google doc content is now moved to above cwiki; keeping google 
> doc link here for preserving comments history] Google doc with community 
> collaboration on documenting release lifecycle 
> https://docs.google.com/document/d/1bS6sr-HSrHFjZb0welife6Qx7u3ZDgRiAoENMLYlfz8/edit.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15828) Remove jackson-mapper-asl-1.9.13 to address CVE

2020-08-14 Thread Mark Denihan (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15828?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177857#comment-17177857
 ] 

Mark Denihan commented on CASSANDRA-15828:
--

[~c3-keveker] Is this jar being removed to stop CVE-2019-10172 from being 
reported at all or is jackson-mapper-asl used anywhere else other than compile? 
It appears to me that this CVE doesn't actually apply to the runtime of 
Cassandra, but I'd appreciate your thoughts!

> Remove jackson-mapper-asl-1.9.13 to address CVE
> ---
>
> Key: CASSANDRA-15828
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15828
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Kevin Eveker
>Priority: Normal
>
> Recent scan results identified the following CVE that require this upgrade to 
> address
> [https://nvd.nist.gov/vuln/detail/CVE-2019-10172]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15986) Repair tests flakiness

2020-08-14 Thread Ekaterina Dimitrova (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177850#comment-17177850
 ] 

Ekaterina Dimitrova commented on CASSANDRA-15986:
-

Thank you for the update [~Bereng], I will try to look at this next week. Have 
a nice vacation.

 

I am adding also this failure of test_simple_parallel_repair to be checked as I 
feel those might be related.

https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/296/workflows/953f79e3-7416-4375-82cf-3dcfeda61ab3/jobs/1816

> Repair tests flakiness
> --
>
> Key: CASSANDRA-15986
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15986
> Project: Cassandra
>  Issue Type: Task
>  Components: Test/dtest
>Reporter: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Repair tests come up in test failure reports every now and then. I have tried 
> to repro the 
> [latest|https://ci-cassandra.apache.org/job/Cassandra-trunk/241/testReport/junit/dtest-novnode.repair_tests.repair_test/TestRepair/test_simple_sequential_repair/]
>  locally 100 times with no luck.
> Still from experience from fixing other flaky tests I have some intuition 
> where the problems may lie. The proposed fix should add no harm if merged. We 
> can reopen the ticket if repair tests keep failing.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16050) Handle difference in timestamp precision between java8 and java11 in LogFIle.java

2020-08-14 Thread Marcus Eriksson (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16050?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Eriksson updated CASSANDRA-16050:

 Bug Category: Parent values: Degradation(12984)Level 1 values: Other 
Exception(12998)
   Complexity: Low Hanging Fruit
  Component/s: Local/Startup and Shutdown
   Local/SSTable
Discovered By: Code Inspection
Fix Version/s: 3.11.x
   3.0.x
   4.0
 Severity: Normal
   Status: Open  (was: Triage Needed)

> Handle difference in timestamp precision between java8 and java11 in 
> LogFIle.java
> -
>
> Key: CASSANDRA-16050
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16050
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable, Local/Startup and Shutdown
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Normal
> Fix For: 4.0, 3.0.x, 3.11.x
>
>
> https://bugs.openjdk.java.net/browse/JDK-8177809 made File.lastModified 
> include milliseconds while java8 always has them as 000.
> This causes problems in LogFile.java where we compare the timestamps with 
> what we have in the logfile: 
> [LogFile.java|https://github.com/apache/cassandra/blob/ffc8e407e087e942c4e5c40605743fe3b32d8fd5/src/java/org/apache/cassandra/db/lifecycle/LogFile.java#L234].
>  This means that we can throw errors when changing between java11 and 8



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-16050) Handle difference in timestamp precision between java8 and java11 in LogFIle.java

2020-08-14 Thread Marcus Eriksson (Jira)
Marcus Eriksson created CASSANDRA-16050:
---

 Summary: Handle difference in timestamp precision between java8 
and java11 in LogFIle.java
 Key: CASSANDRA-16050
 URL: https://issues.apache.org/jira/browse/CASSANDRA-16050
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson


https://bugs.openjdk.java.net/browse/JDK-8177809 made File.lastModified include 
milliseconds while java8 always has them as 000.

This causes problems in LogFile.java where we compare the timestamps with what 
we have in the logfile: 
[LogFile.java|https://github.com/apache/cassandra/blob/ffc8e407e087e942c4e5c40605743fe3b32d8fd5/src/java/org/apache/cassandra/db/lifecycle/LogFile.java#L234].
 This means that we can throw errors when changing between java11 and 8



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14853) Change default timestamp format to output only milliseconds, not microseconds

2020-08-14 Thread Brandon Williams (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-14853:
-
  Fix Version/s: 4.0-beta2
Source Control Link: 
https://github.com/apache/cassandra/commit/139ad44e8b53ed194ba4b47fbfae3484015034f9
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

I agree this should be fixed on the server side later.  Committed doc changes 
for now, thanks.

> Change default timestamp format to output only milliseconds, not microseconds
> -
>
> Key: CASSANDRA-14853
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14853
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Tool/cqlsh
> Environment: Reproduced in trunk
>Reporter: Alex Ott
>Priority: Normal
>  Labels: cqlsh
> Fix For: 4.0-beta2
>
>
> By default cqlsh outputs the timestamp column with microseconds precision, 
> like this:
> {noformat}
> cqlsh:test> create table t1(tm timestamp primary key, t text);
> cqlsh:test> insert into t1(tm, t) values(toTimestamp(now()), 't');
> cqlsh:test> insert into t1(tm, t) values(toTimestamp(now()), 't2');
> cqlsh:test> SELECT * from t1;
>  tm  | t
> -+
>  2018-10-27 18:01:54.738000+ | t2
>  2018-10-27 18:01:52.599000+ |  t
> (2 rows)
> {noformat}
> But if I want to use the value that is output on the screen in my query, I 
> get an error:
> {noformat}
> cqlsh:test> select * from t1 where tm = '2018-10-27 18:01:54.738000+';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="Unable 
> to coerce '2018-10-27 18:01:54.738000+' to a formatted date (long)"
> {noformat}
> But if I manually round it to milliseconds, then everything works:
> {noformat}
> cqlsh:test> select * from t1 where tm = '2018-10-27 18:01:54.738+';
>  tm  | t
> -+
>  2018-10-27 18:01:54.738000+ | t2
> (1 rows)
> {noformat}
> It would be much easier user's experience if we use the same format for 
> output & input data, because right now this leads to errors, that often not 
> really understandable by novice users.
> P.S. I know about cqlshrc, but not every user has it configured.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14853) Change default timestamp format to output only milliseconds, not microseconds

2020-08-14 Thread Brandon Williams (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-14853:
-
Status: Ready to Commit  (was: Review In Progress)

> Change default timestamp format to output only milliseconds, not microseconds
> -
>
> Key: CASSANDRA-14853
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14853
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Tool/cqlsh
> Environment: Reproduced in trunk
>Reporter: Alex Ott
>Priority: Normal
>  Labels: cqlsh
>
> By default cqlsh outputs the timestamp column with microseconds precision, 
> like this:
> {noformat}
> cqlsh:test> create table t1(tm timestamp primary key, t text);
> cqlsh:test> insert into t1(tm, t) values(toTimestamp(now()), 't');
> cqlsh:test> insert into t1(tm, t) values(toTimestamp(now()), 't2');
> cqlsh:test> SELECT * from t1;
>  tm  | t
> -+
>  2018-10-27 18:01:54.738000+ | t2
>  2018-10-27 18:01:52.599000+ |  t
> (2 rows)
> {noformat}
> But if I want to use the value that is output on the screen in my query, I 
> get an error:
> {noformat}
> cqlsh:test> select * from t1 where tm = '2018-10-27 18:01:54.738000+';
> InvalidRequest: Error from server: code=2200 [Invalid query] message="Unable 
> to coerce '2018-10-27 18:01:54.738000+' to a formatted date (long)"
> {noformat}
> But if I manually round it to milliseconds, then everything works:
> {noformat}
> cqlsh:test> select * from t1 where tm = '2018-10-27 18:01:54.738+';
>  tm  | t
> -+
>  2018-10-27 18:01:54.738000+ | t2
> (1 rows)
> {noformat}
> It would be much easier user's experience if we use the same format for 
> output & input data, because right now this leads to errors, that often not 
> really understandable by novice users.
> P.S. I know about cqlshrc, but not every user has it configured.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[cassandra] branch trunk updated: Update docs regarding timestamp precision

2020-08-14 Thread brandonwilliams
This is an automated email from the ASF dual-hosted git repository.

brandonwilliams 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 139ad44  Update docs regarding timestamp precision
139ad44 is described below

commit 139ad44e8b53ed194ba4b47fbfae3484015034f9
Author: Adam Holmberg 
AuthorDate: Thu Aug 13 14:05:18 2020 -0500

Update docs regarding timestamp precision

Patch by Adam Holmberg, reviewed by brandonwilliams for CASSANDRA-14853
---
 doc/source/cql/types.rst   | 3 +++
 doc/source/development/testing.rst | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)

diff --git a/doc/source/cql/types.rst b/doc/source/cql/types.rst
index 509a756..783f3d1 100644
--- a/doc/source/cql/types.rst
+++ b/doc/source/cql/types.rst
@@ -153,6 +153,9 @@ The time of day may also be omitted (``'2011-02-03'`` or 
``'2011-02-03+'``),
 default to 00:00:00 in the specified or default time zone. However, if only 
the date part is relevant, consider using
 the :ref:`date ` type.
 
+Note: while Cassandra will parse and accept time literals with a greater 
number of digits, the value stored will be truncated to 
+millisecond precision.
+
 .. _dates:
 
 Working with dates
diff --git a/doc/source/development/testing.rst 
b/doc/source/development/testing.rst
index 7f38fe5..1f32e12 100644
--- a/doc/source/development/testing.rst
+++ b/doc/source/development/testing.rst
@@ -70,7 +70,7 @@ DTests
 
 One way of doing integration or system testing at larger scale is by using 
`dtest `_, which stands for 
“Cassandra Distributed Tests”. The idea is to automatically setup Cassandra 
clusters using various configurations and simulate certain use cases you want 
to test. This is done using Python scripts and ``ccmlib`` from the `ccm 
`_ project. Dtests will setup clusters using 
this library just as you do running ad-hoc  [...]
 
-Using dtests helps us to prevent regression bugs by continually executing 
tests on the `CI server `_ against new patches. 
Committers will be able to set up build branches there and your reviewer may 
use the CI environment to run tests for your patch. Read more on the motivation 
behind continuous integration `here 
`_.
+Using dtests helps us to prevent regression bugs by continually executing 
tests on the `CI server `_ against new patches. 
Committers will be able to set up build branches there and your reviewer may 
use the CI environment to run tests for your patch.
 
 The best way to learn how to write dtests is probably by reading the 
introduction "`How to Write a Dtest 
`_" and by looking at 
existing, recently updated tests in the project. New tests must follow certain 
`style conventions 
`_ that 
are being checked before accepting contributions. In contrast to Cassandra, 
dtest issues and pull-requests are managed on github, the [...]
 


-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ekaterina Dimitrova updated CASSANDRA-13994:

Reviewers: Dinesh Joshi, Sylvain Lebresne  (was: Dinesh Joshi, Ekaterina 
Dimitrova, Sylvain Lebresne)

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-13994) Remove dead compact storage code before 4.0 release

2020-08-14 Thread Ekaterina Dimitrova (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ekaterina Dimitrova updated CASSANDRA-13994:

Reviewers: Dinesh Joshi, Sylvain Lebresne, Ekaterina Dimitrova  (was: 
Dinesh Joshi, Ekaterina Dimitrova, Sylvain Lebresne)
   Dinesh Joshi, Sylvain Lebresne, Ekaterina Dimitrova  (was: 
Dinesh Joshi, Sylvain Lebresne)
   Status: Review In Progress  (was: Patch Available)

> Remove dead compact storage code before 4.0 release
> ---
>
> Key: CASSANDRA-13994
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13994
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Legacy/Local Write-Read Paths
>Reporter: Alex Petrov
>Assignee: Ekaterina Dimitrova
>Priority: Low
> Fix For: 4.0, 4.0-beta
>
>
> 4.0 comes without thrift (after [CASSANDRA-5]) and COMPACT STORAGE (after 
> [CASSANDRA-10857]), and since Compact Storage flags are now disabled, all of 
> the related functionality is useless.
> There are still some things to consider:
> 1. One of the system tables (built indexes) was compact. For now, we just 
> added {{value}} column to it to make sure it's backwards-compatible, but we 
> might want to make sure it's just a "normal" table and doesn't have redundant 
> columns.
> 2. Compact Tables were building indexes in {{KEYS}} mode. Removing it is 
> trivial, but this would mean that all built indexes will be defunct. We could 
> log a warning for now and ask users to migrate off those for now and 
> completely remove it from future releases. It's just a couple of classes 
> though.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-15854) Truncation should fail any ongoing repairs

2020-08-14 Thread Marcus Eriksson (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-15854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Eriksson updated CASSANDRA-15854:

Summary: Truncation should fail any ongoing repairs  (was: Truncation 
should fail any ongoing preview repairs)

> Truncation should fail any ongoing repairs
> --
>
> Key: CASSANDRA-15854
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15854
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Truncation may race with preview repairs, validating different data on 
> different nodes, we should abort any ongoing preview repairs if we get a 
> truncation request



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15854) Truncation should fail any ongoing preview repairs

2020-08-14 Thread Marcus Eriksson (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177782#comment-17177782
 ] 

Marcus Eriksson commented on CASSANDRA-15854:
-

Sorry for the delay [~maedhroz] - I've pushed a few commits addressing your 
comments - and the test was horrible, I've pushed a simpler version which makes 
it clearer what is going on

> Truncation should fail any ongoing preview repairs
> --
>
> Key: CASSANDRA-15854
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15854
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Truncation may race with preview repairs, validating different data on 
> different nodes, we should abort any ongoing preview repairs if we get a 
> truncation request



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15986) Repair tests flakiness

2020-08-14 Thread Berenguer Blasi (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-15986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177701#comment-17177701
 ] 

Berenguer Blasi commented on CASSANDRA-15986:
-

So I have tried with a VM on diff memory settings down to 3GB and even capping 
CPU to 10% only. That thing is solid as a rock. I can't repro. On too low specs 
it starts to fail on legit timeouts on other areas. So no way to repro this one 
for me.

On the other hand I tried to repro the Tracing test failures and I did manage 
to. So I will try that once I am back from my 1w break #justfyi

> Repair tests flakiness
> --
>
> Key: CASSANDRA-15986
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15986
> Project: Cassandra
>  Issue Type: Task
>  Components: Test/dtest
>Reporter: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0-beta
>
>
> Repair tests come up in test failure reports every now and then. I have tried 
> to repro the 
> [latest|https://ci-cassandra.apache.org/job/Cassandra-trunk/241/testReport/junit/dtest-novnode.repair_tests.repair_test/TestRepair/test_simple_sequential_repair/]
>  locally 100 times with no luck.
> Still from experience from fixing other flaky tests I have some intuition 
> where the problems may lie. The proposed fix should add no harm if merged. We 
> can reopen the ticket if repair tests keep failing.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org